kfaraz commented on code in PR #17353:
URL: https://github.com/apache/druid/pull/17353#discussion_r1856011761


##########
indexing-service/src/main/java/org/apache/druid/indexing/scheduledbatch/ScheduledBatchScheduler.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.indexing.scheduledbatch;
+
+import com.google.common.base.Optional;
+import com.google.inject.Inject;
+import org.apache.druid.common.guava.FutureUtils;
+import org.apache.druid.indexer.TaskLocation;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.indexing.overlord.TaskMaster;
+import org.apache.druid.indexing.overlord.TaskRunner;
+import org.apache.druid.indexing.overlord.TaskRunnerListener;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.apache.druid.sql.client.BrokerClient;
+import org.apache.druid.sql.http.SqlQuery;
+import org.apache.druid.sql.http.SqlTaskStatus;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+
+import javax.annotation.Nullable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+public class ScheduledBatchScheduler
+{
+  private static final Logger log = new 
EmittingLogger(ScheduledBatchScheduler.class);
+
+  private final TaskRunnerListener taskRunnerListener;
+  private final TaskMaster taskMaster;
+  private final ScheduledBatchStatusTracker statusTracker;
+  private final BrokerClient brokerClient;
+  private final ServiceEmitter emitter;
+
+  /**
+   * Single-threaded executor to process the cron jobs queue.
+   */
+  private final ScheduledExecutorService cronExecutor;
+
+  private final ConcurrentHashMap<String, SchedulerManager> supervisorToManager
+      = new ConcurrentHashMap<>();
+  private final ScheduledExecutorFactory executorFactory;
+
+  @Inject
+  public ScheduledBatchScheduler(
+      final TaskMaster taskMaster,
+      final ScheduledExecutorFactory executorFactory,
+      final BrokerClient brokerClient,
+      final ServiceEmitter emitter,
+      final ScheduledBatchStatusTracker statusTracker
+  )
+  {
+    this.taskMaster = taskMaster;
+    this.executorFactory = executorFactory;
+    this.cronExecutor = executorFactory.create(1, 
"ScheduledBatchScheduler-%s");
+    this.brokerClient = brokerClient;
+    this.emitter = emitter;
+    this.statusTracker = statusTracker;
+
+    this.taskRunnerListener = new TaskRunnerListener()
+    {
+      @Override
+      public String getListenerId()
+      {
+        return "ScheduledBatchScheduler";
+      }
+
+      @Override
+      public void locationChanged(String taskId, TaskLocation newLocation)
+      {
+        // Do nothing
+      }
+
+      @Override
+      public void statusChanged(final String taskId, final TaskStatus 
taskStatus)
+      {
+        if (taskStatus.isComplete()) {
+          statusTracker.onTaskCompleted(taskId, taskStatus);
+        }
+      }
+    };
+  }
+
+  public void startScheduledIngestion(
+      final String supervisorId,
+      final CronSchedulerConfig cronSchedulerConfig,
+      final SqlQuery spec
+  )
+  {
+    log.info(
+        "Starting scheduled batch ingestion for supervisorId[%s] with 
cronSchedule[%s] and spec[%s].",

Review Comment:
   Do we have the datasource name at this point? Would be nice to log that too.



##########
indexing-service/pom.xml:
##########
@@ -195,6 +195,22 @@
             <artifactId>commons-collections4</artifactId>
             <scope>provided</scope>
         </dependency>
+        <dependency>
+            <groupId>com.cronutils</groupId>
+            <artifactId>cron-utils</artifactId>
+            <version>9.2.0</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.druid</groupId>
+            <artifactId>druid-sql</artifactId>

Review Comment:
   Rather than depending on druid-sql, we should consider moving the required 
classes like `BrokerClient`, `SqlTaskStatus` etc to an existing upstream module 
like `druid-processing` or `druid-server` or something. It feels a little weird 
for the indexing-service to depend on the whole of the druid-sql module.



##########
indexing-service/src/main/java/org/apache/druid/indexing/scheduledbatch/QuartzCronSchedulerConfig.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.indexing.scheduledbatch;
+
+import com.cronutils.model.CronType;
+import com.cronutils.model.definition.CronDefinitionBuilder;
+import com.cronutils.model.time.ExecutionTime;
+import com.cronutils.parser.CronParser;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+
+import javax.annotation.Nullable;
+import java.util.Objects;
+
+public class QuartzCronSchedulerConfig implements CronSchedulerConfig
+{
+  public static final String TYPE = "quartz";
+
+  private static final CronParser QUARTZ_PARSER =
+      new 
CronParser(CronDefinitionBuilder.instanceDefinitionFor(CronType.QUARTZ));
+
+  @JsonProperty
+  private final String schedule;
+
+  private final ExecutionTime executionTime;
+
+  @JsonCreator
+  public QuartzCronSchedulerConfig(@JsonProperty("schedule") final String 
schedule)
+  {
+    this.executionTime = ExecutionTime.forCron(QUARTZ_PARSER.parse(schedule));

Review Comment:
   If the schedule string is malformed, what is the exception currently thrown? 
We should throw a nice `DruidException` with a proper message, unless we are 
already doing that.



##########
indexing-service/src/main/java/org/apache/druid/indexing/scheduledbatch/ScheduledBatchSupervisor.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.druid.indexing.scheduledbatch;
+
+import org.apache.druid.indexing.overlord.DataSourceMetadata;
+import org.apache.druid.indexing.overlord.supervisor.Supervisor;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorReport;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.logger.Logger;
+
+import javax.annotation.Nullable;
+
+public class ScheduledBatchSupervisor implements Supervisor
+{
+  private static final Logger log = new Logger(ScheduledBatchSupervisor.class);
+  private final ScheduledBatchSupervisorSpec supervisorSpec;
+  private final ScheduledBatchScheduler scheduler;
+
+  public ScheduledBatchSupervisor(
+      final ScheduledBatchSupervisorSpec supervisorSpec,
+      final ScheduledBatchScheduler scheduler
+  )
+  {
+    this.supervisorSpec = supervisorSpec;
+    this.scheduler = scheduler;
+  }
+
+  @Override
+  public void start()
+  {
+    if (supervisorSpec.isSuspended()) {
+      log.info("Suspending the scheduled batch supervisor[%s].", 
supervisorSpec.getId());

Review Comment:
   Nit: These logs might be redundant since the 
`scheduler.stopScheduledIngestion()` would log this info anyway.



##########
indexing-service/src/main/java/org/apache/druid/indexing/scheduledbatch/ScheduledBatchSupervisorSnapshot.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.indexing.scheduledbatch;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.indexer.TaskStatus;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+
+import javax.annotation.Nullable;
+import java.util.Map;
+
+public class ScheduledBatchSupervisorSnapshot
+{
+  public enum BatchSupervisorStatus
+  {
+    SCHEDULER_RUNNING,
+    SCHEDULER_SHUTDOWN,
+    SCHEDULER_ERROR
+  }

Review Comment:
   I wonder if this status should be de-duplicated with 
`SchedulerBatchSupervisor.State`.
   I think the `AutoCompactionSnapshot` also has a similar duplication, but I 
will try to get rid of it in time.



##########
indexing-service/src/main/java/org/apache/druid/indexing/scheduledbatch/ScheduledBatchScheduler.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.indexing.scheduledbatch;
+
+import com.google.common.base.Optional;
+import com.google.inject.Inject;
+import org.apache.druid.common.guava.FutureUtils;
+import org.apache.druid.indexer.TaskLocation;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.indexing.overlord.TaskMaster;
+import org.apache.druid.indexing.overlord.TaskRunner;
+import org.apache.druid.indexing.overlord.TaskRunnerListener;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.apache.druid.sql.client.BrokerClient;
+import org.apache.druid.sql.http.SqlQuery;
+import org.apache.druid.sql.http.SqlTaskStatus;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+
+import javax.annotation.Nullable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+public class ScheduledBatchScheduler
+{
+  private static final Logger log = new 
EmittingLogger(ScheduledBatchScheduler.class);
+
+  private final TaskRunnerListener taskRunnerListener;
+  private final TaskMaster taskMaster;
+  private final ScheduledBatchStatusTracker statusTracker;
+  private final BrokerClient brokerClient;
+  private final ServiceEmitter emitter;
+
+  /**
+   * Single-threaded executor to process the cron jobs queue.
+   */
+  private final ScheduledExecutorService cronExecutor;
+
+  private final ConcurrentHashMap<String, SchedulerManager> supervisorToManager
+      = new ConcurrentHashMap<>();
+  private final ScheduledExecutorFactory executorFactory;
+
+  @Inject
+  public ScheduledBatchScheduler(
+      final TaskMaster taskMaster,
+      final ScheduledExecutorFactory executorFactory,
+      final BrokerClient brokerClient,
+      final ServiceEmitter emitter,
+      final ScheduledBatchStatusTracker statusTracker
+  )
+  {
+    this.taskMaster = taskMaster;
+    this.executorFactory = executorFactory;
+    this.cronExecutor = executorFactory.create(1, 
"ScheduledBatchScheduler-%s");
+    this.brokerClient = brokerClient;
+    this.emitter = emitter;
+    this.statusTracker = statusTracker;
+
+    this.taskRunnerListener = new TaskRunnerListener()
+    {
+      @Override
+      public String getListenerId()
+      {
+        return "ScheduledBatchScheduler";
+      }
+
+      @Override
+      public void locationChanged(String taskId, TaskLocation newLocation)
+      {
+        // Do nothing
+      }
+
+      @Override
+      public void statusChanged(final String taskId, final TaskStatus 
taskStatus)
+      {
+        if (taskStatus.isComplete()) {
+          statusTracker.onTaskCompleted(taskId, taskStatus);
+        }
+      }
+    };
+  }
+
+  public void startScheduledIngestion(
+      final String supervisorId,
+      final CronSchedulerConfig cronSchedulerConfig,
+      final SqlQuery spec
+  )
+  {
+    log.info(
+        "Starting scheduled batch ingestion for supervisorId[%s] with 
cronSchedule[%s] and spec[%s].",
+        supervisorId, cronSchedulerConfig, spec
+    );
+    final SchedulerManager manager = new SchedulerManager(supervisorId, 
cronSchedulerConfig, spec, executorFactory);
+    manager.startScheduling();
+    supervisorToManager.put(supervisorId, manager);
+  }
+
+  public void stopScheduledIngestion(final String supervisorId)
+  {
+    log.info("Stopping scheduled batch ingestion for supervisorId[%s].", 
supervisorId);
+    final SchedulerManager manager = supervisorToManager.get(supervisorId);
+    if (manager != null) {
+      // Don't remove the supervisorId from supervisorToManager. We want to be 
able to track the status
+      // for suspended supervisors to track any in-flight tasks. stop() will 
clean up all state completely.
+      manager.stopScheduling();
+    }
+  }
+
+  public void start()
+  {
+    log.info("Starting scheduled batch scheduler.");
+    final Optional<TaskRunner> taskRunnerOptional = taskMaster.getTaskRunner();
+    if (taskRunnerOptional.isPresent()) {
+      taskRunnerOptional.get().registerListener(taskRunnerListener, 
Execs.directExecutor());
+    } else {
+      log.warn("Task runner not registered for scheduled batch scheduler.");
+    }
+  }
+
+  public void stop()
+  {
+    log.info("Stopping scheduled batch scheduler.");
+    final Optional<TaskRunner> taskRunnerOptional = taskMaster.getTaskRunner();
+    if (taskRunnerOptional.isPresent()) {
+      
taskRunnerOptional.get().unregisterListener(taskRunnerListener.getListenerId());
+    }
+
+    supervisorToManager.clear();
+  }
+
+  @Nullable
+  public ScheduledBatchSupervisorSnapshot getSchedulerSnapshot(final String 
supervisorId)
+  {
+    final SchedulerManager manager = supervisorToManager.get(supervisorId);
+    if (manager == null) {
+      return null;
+    }
+
+    final ScheduledBatchStatusTracker.BatchSupervisorTaskStatus tasks = 
statusTracker.getSupervisorTasks(supervisorId);
+    return new ScheduledBatchSupervisorSnapshot(
+        supervisorId,
+        manager.getSchedulerStatus(),
+        manager.getLastTaskSubmittedTime(),
+        manager.getNextTaskSubmissionTime(),
+        manager.getTimeUntilNextTaskSubmission(),
+        tasks.getSubmittedTasks(),
+        tasks.getCompletedTasks()
+    );
+  }
+
+  private void enqueueTask(final Runnable runnable)
+  {
+    cronExecutor.submit(runnable);
+  }
+
+  private void submitSqlTask(final String supervisorId, final SqlQuery spec)
+      throws ExecutionException, InterruptedException
+  {
+    log.info("Submitting a new task with spec[%s] for supervisor[%s].", spec, 
supervisorId);
+    final SqlTaskStatus taskStatus = 
FutureUtils.get(brokerClient.submitSqlTask(spec), true);
+    statusTracker.onTaskSubmitted(supervisorId, taskStatus);
+  }
+
+  private class SchedulerManager
+  {
+    private final String supervisorId;
+    private final SqlQuery spec;
+    private final ScheduledExecutorService managerExecutor;
+    private final CronSchedulerConfig cronSchedulerConfig;
+
+    private ScheduledBatchSupervisorSnapshot.BatchSupervisorStatus status;
+
+    /**
+     * Note that the last task submitted per supervisor should eventually be 
persisted in the metadata store,
+     * along with any other scheduler state, so that the batch supervisor can 
recover gracefully and
+     * avoid missing task submissions during rolling restarts, etc.
+     */
+    private DateTime lastTaskSubmittedTime;
+
+    private SchedulerManager(
+        final String supervisorId,
+        final CronSchedulerConfig cronSchedulerConfig,
+        final SqlQuery spec,
+        final ScheduledExecutorFactory executorFactory
+    )
+    {
+      this.supervisorId = supervisorId;
+      this.cronSchedulerConfig = cronSchedulerConfig;
+      this.spec = spec;
+      this.managerExecutor = executorFactory.create(1, "scheduler-" + 
supervisorId + "-%d");

Review Comment:
   Do we really need separate executors for each supervisor? I think for now it 
should be okay to schedule all the jobs on a single executor. It is only 
responsible for submitting tasks which would be handled internally by the 
`ServiceClientImpl` in `BrokerClient`. So most of the work would be async 
anyway.



##########
indexing-service/src/main/java/org/apache/druid/indexing/scheduledbatch/ScheduledBatchStatusTracker.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.indexing.scheduledbatch;
+
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.sql.http.SqlTaskStatus;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+
+public class ScheduledBatchStatusTracker
+{
+  // Track supervisor -> tasks.

Review Comment:
   convert to javadocs?



##########
indexing-service/src/main/java/org/apache/druid/indexing/scheduledbatch/CronSchedulerConfig.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.indexing.scheduledbatch;
+
+import com.fasterxml.jackson.annotation.JsonSubTypes;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+
+import javax.annotation.Nullable;
+
+/**
+ * Interface representing a configuration for scheduling tasks based on cron 
expressions.
+ */
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
+@JsonSubTypes(value = {
+    @JsonSubTypes.Type(name = UnixCronSchedulerConfig.TYPE, value = 
UnixCronSchedulerConfig.class),
+    @JsonSubTypes.Type(name = QuartzCronSchedulerConfig.TYPE, value = 
QuartzCronSchedulerConfig.class),
+})
+public interface CronSchedulerConfig
+{
+  /**
+   * Gets the next task submission time after the specified reference time 
{@code dateTime}, or {@code null}
+   * if no future submission time can be determined (e.g., if the schedule is 
in the past).
+   */
+  @Nullable
+  DateTime getNextTaskSubmissionTime(DateTime dateTime);

Review Comment:
   In the javadoc, please include a small description about what the 
`referenceTime` represents. Same comment for the other method in this interface.
   
   ```suggestion
     DateTime getNextTaskSubmissionTime(DateTime referenceTime);
   ```
   
   I guess you could even just add the suffix `...After` for clarity and 
replace `Submission` with `Start` for simplicity: `getNextTaskStartTimeAfter()`.



##########
indexing-service/src/main/java/org/apache/druid/indexing/scheduledbatch/ScheduledBatchScheduler.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.indexing.scheduledbatch;
+
+import com.google.common.base.Optional;
+import com.google.inject.Inject;
+import org.apache.druid.common.guava.FutureUtils;
+import org.apache.druid.indexer.TaskLocation;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.indexing.overlord.TaskMaster;
+import org.apache.druid.indexing.overlord.TaskRunner;
+import org.apache.druid.indexing.overlord.TaskRunnerListener;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.apache.druid.sql.client.BrokerClient;
+import org.apache.druid.sql.http.SqlQuery;
+import org.apache.druid.sql.http.SqlTaskStatus;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+
+import javax.annotation.Nullable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+public class ScheduledBatchScheduler

Review Comment:
   Nit: This seems like a weird name due to the double `Schedule`. Can we use 
something else like `ScheduledBatchTaskManager` or `ScheduledBatchTaskExecutor` 
or `SchedulerBatchSupervisorManager`?
   
   Please add a short javadoc too.



##########
services/src/main/java/org/apache/druid/cli/CliOverlord.java:
##########
@@ -457,7 +460,8 @@ private void configureOverlordHelpers(Binder binder)
         new InputSourceModule(),
         new SupervisorModule(),
         new LookupSerdeModule(),
-        new SamplerModule()
+        new SamplerModule(),
+        new BrokerServiceModule()

Review Comment:
   We should try to bring in only the required dependencies rather than 
importing all the broker service modules.



##########
indexing-service/src/main/java/org/apache/druid/indexing/scheduledbatch/ScheduledBatchSupervisorSpec.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.indexing.scheduledbatch;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.druid.common.config.Configs;
+import org.apache.druid.error.DruidException;
+import org.apache.druid.error.InvalidInput;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.sql.calcite.planner.ExplainAttributes;
+import org.apache.druid.sql.client.BrokerClient;
+import org.apache.druid.sql.http.ExplainPlan;
+import org.apache.druid.sql.http.SqlQuery;
+
+import javax.annotation.Nullable;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+
+public class ScheduledBatchSupervisorSpec implements SupervisorSpec
+{
+  public static final String TYPE = "scheduled_batch";
+  public static final String ID_PREFIX = "scheduled_batch__";
+
+  private static final Logger log = new Logger(ScheduledBatchSupervisor.class);
+
+  @JsonProperty
+  private final SqlQuery spec;
+  @JsonProperty
+  private final boolean suspended;
+  @JsonProperty
+  private final CronSchedulerConfig schedulerConfig;
+
+  /**
+   * Note that both {@link #dataSource} and {@link #id} are optional JSON 
fields present in the spec.
+   * They are only used internally because we use and persist the user-facing 
spec in the metadata store. So these
+   * additional fields are required for jackson deserialization.
+   * It would be better to have separate user-facing and domain-specific DTOs 
for this purpose and map them, but
+   * that'll entail a larger change.
+   */
+  @JsonProperty
+  private final String dataSource;
+  @JsonProperty
+  private final String id;
+
+  private final ObjectMapper objectMapper;
+  private final ScheduledBatchScheduler batchScheduler;
+  private final BrokerClient brokerClient;
+
+  @JsonCreator
+  public ScheduledBatchSupervisorSpec(
+      @JsonProperty("spec") final SqlQuery spec,
+      @JsonProperty("schedulerConfig") final CronSchedulerConfig 
schedulerConfig,
+      @JsonProperty("suspended") @Nullable Boolean suspended,
+      @JsonProperty("id") @Nullable final String id,
+      @JsonProperty("dataSource") @Nullable final String dataSource,
+      @JacksonInject ObjectMapper objectMapper,
+      @JacksonInject ScheduledBatchScheduler batchScheduler,
+      @JacksonInject BrokerClient brokerClient
+  )
+  {
+    this.spec = spec;
+    this.schedulerConfig = schedulerConfig;
+    this.suspended = Configs.valueOrDefault(suspended, false);
+    this.objectMapper = objectMapper;
+    this.batchScheduler = batchScheduler;
+    this.brokerClient = brokerClient;
+
+    this.dataSource = dataSource != null ? dataSource : 
getDatasourceFromQuery();

Review Comment:
   Instead of invoking the broker client, should we just ensure that datasource 
is never passed as null? It would be in line with the other supervisors.



##########
indexing-service/src/main/java/org/apache/druid/indexing/scheduledbatch/ScheduledBatchScheduler.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.indexing.scheduledbatch;
+
+import com.google.common.base.Optional;
+import com.google.inject.Inject;
+import org.apache.druid.common.guava.FutureUtils;
+import org.apache.druid.indexer.TaskLocation;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.indexing.overlord.TaskMaster;
+import org.apache.druid.indexing.overlord.TaskRunner;
+import org.apache.druid.indexing.overlord.TaskRunnerListener;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.apache.druid.sql.client.BrokerClient;
+import org.apache.druid.sql.http.SqlQuery;
+import org.apache.druid.sql.http.SqlTaskStatus;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+
+import javax.annotation.Nullable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+public class ScheduledBatchScheduler
+{
+  private static final Logger log = new 
EmittingLogger(ScheduledBatchScheduler.class);
+
+  private final TaskRunnerListener taskRunnerListener;
+  private final TaskMaster taskMaster;
+  private final ScheduledBatchStatusTracker statusTracker;
+  private final BrokerClient brokerClient;
+  private final ServiceEmitter emitter;
+
+  /**
+   * Single-threaded executor to process the cron jobs queue.
+   */
+  private final ScheduledExecutorService cronExecutor;
+
+  private final ConcurrentHashMap<String, SchedulerManager> supervisorToManager
+      = new ConcurrentHashMap<>();
+  private final ScheduledExecutorFactory executorFactory;
+
+  @Inject
+  public ScheduledBatchScheduler(
+      final TaskMaster taskMaster,
+      final ScheduledExecutorFactory executorFactory,
+      final BrokerClient brokerClient,
+      final ServiceEmitter emitter,
+      final ScheduledBatchStatusTracker statusTracker
+  )
+  {
+    this.taskMaster = taskMaster;
+    this.executorFactory = executorFactory;
+    this.cronExecutor = executorFactory.create(1, 
"ScheduledBatchScheduler-%s");
+    this.brokerClient = brokerClient;
+    this.emitter = emitter;
+    this.statusTracker = statusTracker;
+
+    this.taskRunnerListener = new TaskRunnerListener()
+    {
+      @Override
+      public String getListenerId()
+      {
+        return "ScheduledBatchScheduler";
+      }
+
+      @Override
+      public void locationChanged(String taskId, TaskLocation newLocation)
+      {
+        // Do nothing
+      }
+
+      @Override
+      public void statusChanged(final String taskId, final TaskStatus 
taskStatus)
+      {
+        if (taskStatus.isComplete()) {
+          statusTracker.onTaskCompleted(taskId, taskStatus);
+        }
+      }
+    };
+  }
+
+  public void startScheduledIngestion(

Review Comment:
   Please add short javadocs for these methods as it is a little unclear 
whether these methods are used to just create the schedule or launch the tasks 
themselves.



##########
indexing-service/src/main/java/org/apache/druid/indexing/scheduledbatch/ScheduledBatchSupervisor.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.druid.indexing.scheduledbatch;
+
+import org.apache.druid.indexing.overlord.DataSourceMetadata;
+import org.apache.druid.indexing.overlord.supervisor.Supervisor;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorReport;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.logger.Logger;
+
+import javax.annotation.Nullable;
+
+public class ScheduledBatchSupervisor implements Supervisor
+{
+  private static final Logger log = new Logger(ScheduledBatchSupervisor.class);
+  private final ScheduledBatchSupervisorSpec supervisorSpec;
+  private final ScheduledBatchScheduler scheduler;
+
+  public ScheduledBatchSupervisor(
+      final ScheduledBatchSupervisorSpec supervisorSpec,
+      final ScheduledBatchScheduler scheduler
+  )
+  {
+    this.supervisorSpec = supervisorSpec;
+    this.scheduler = scheduler;
+  }
+
+  @Override
+  public void start()
+  {
+    if (supervisorSpec.isSuspended()) {
+      log.info("Suspending the scheduled batch supervisor[%s].", 
supervisorSpec.getId());
+      scheduler.stopScheduledIngestion(supervisorSpec.getId());
+    } else {
+      scheduler.startScheduledIngestion(
+          supervisorSpec.getId(),
+          supervisorSpec.getSchedulerConfig(),
+          supervisorSpec.getSpec()
+      );
+      log.info("Starting the scheduled batch supervisor[%s].", 
supervisorSpec.getId());
+    }
+  }
+
+  @Override
+  public void stop(boolean stopGracefully)
+  {
+    log.info("Stopping the scheduled batch supervisor[%s]", 
supervisorSpec.getId());
+    scheduler.stopScheduledIngestion(supervisorSpec.getId());
+  }
+
+  @Override
+  public SupervisorReport<ScheduledBatchSupervisorSnapshot> getStatus()
+  {
+    return new SupervisorReport<>(
+        supervisorSpec.getId(),
+        DateTimes.nowUtc(),
+        scheduler.getSchedulerSnapshot(supervisorSpec.getId())
+    );
+  }
+
+  @Override
+  public SupervisorStateManager.State getState()
+  {
+    if (supervisorSpec.isSuspended()) {
+      return State.SUSPENDED;
+    } else {
+      return State.RUNNING;
+    }
+  }
+
+  @Override
+  public void reset(@Nullable DataSourceMetadata dataSourceMetadata)
+  {
+    // do nothing
+  }
+
+  public enum State implements SupervisorStateManager.State
+  {
+    SCHEDULER_STOPPED(true),
+    RUNNING(true),
+    SUSPENDED(true),
+    UNHEALTHY(false);

Review Comment:
   We never seem to return the `SCHEDULER_STOPPED` and `UNHEALTHY` states in 
the `getState()` method. I guess the `SCHEDULER_STOPPED` might not even be 
needed since the batch supervisor scheduler is always running if the Overlord 
is leader, right?



##########
indexing-service/src/main/java/org/apache/druid/indexing/scheduledbatch/ScheduledBatchScheduler.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.indexing.scheduledbatch;
+
+import com.google.common.base.Optional;
+import com.google.inject.Inject;
+import org.apache.druid.common.guava.FutureUtils;
+import org.apache.druid.indexer.TaskLocation;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.indexing.overlord.TaskMaster;
+import org.apache.druid.indexing.overlord.TaskRunner;
+import org.apache.druid.indexing.overlord.TaskRunnerListener;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.apache.druid.sql.client.BrokerClient;
+import org.apache.druid.sql.http.SqlQuery;
+import org.apache.druid.sql.http.SqlTaskStatus;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+
+import javax.annotation.Nullable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+public class ScheduledBatchScheduler
+{
+  private static final Logger log = new 
EmittingLogger(ScheduledBatchScheduler.class);
+
+  private final TaskRunnerListener taskRunnerListener;
+  private final TaskMaster taskMaster;
+  private final ScheduledBatchStatusTracker statusTracker;
+  private final BrokerClient brokerClient;
+  private final ServiceEmitter emitter;
+
+  /**
+   * Single-threaded executor to process the cron jobs queue.
+   */
+  private final ScheduledExecutorService cronExecutor;
+
+  private final ConcurrentHashMap<String, SchedulerManager> supervisorToManager
+      = new ConcurrentHashMap<>();
+  private final ScheduledExecutorFactory executorFactory;
+
+  @Inject
+  public ScheduledBatchScheduler(
+      final TaskMaster taskMaster,
+      final ScheduledExecutorFactory executorFactory,
+      final BrokerClient brokerClient,
+      final ServiceEmitter emitter,
+      final ScheduledBatchStatusTracker statusTracker
+  )
+  {
+    this.taskMaster = taskMaster;
+    this.executorFactory = executorFactory;
+    this.cronExecutor = executorFactory.create(1, 
"ScheduledBatchScheduler-%s");
+    this.brokerClient = brokerClient;
+    this.emitter = emitter;
+    this.statusTracker = statusTracker;
+
+    this.taskRunnerListener = new TaskRunnerListener()
+    {
+      @Override
+      public String getListenerId()
+      {
+        return "ScheduledBatchScheduler";
+      }
+
+      @Override
+      public void locationChanged(String taskId, TaskLocation newLocation)
+      {
+        // Do nothing
+      }
+
+      @Override
+      public void statusChanged(final String taskId, final TaskStatus 
taskStatus)
+      {
+        if (taskStatus.isComplete()) {
+          statusTracker.onTaskCompleted(taskId, taskStatus);
+        }
+      }
+    };
+  }
+
+  public void startScheduledIngestion(
+      final String supervisorId,
+      final CronSchedulerConfig cronSchedulerConfig,
+      final SqlQuery spec
+  )
+  {
+    log.info(
+        "Starting scheduled batch ingestion for supervisorId[%s] with 
cronSchedule[%s] and spec[%s].",
+        supervisorId, cronSchedulerConfig, spec
+    );
+    final SchedulerManager manager = new SchedulerManager(supervisorId, 
cronSchedulerConfig, spec, executorFactory);
+    manager.startScheduling();
+    supervisorToManager.put(supervisorId, manager);
+  }
+
+  public void stopScheduledIngestion(final String supervisorId)
+  {
+    log.info("Stopping scheduled batch ingestion for supervisorId[%s].", 
supervisorId);
+    final SchedulerManager manager = supervisorToManager.get(supervisorId);
+    if (manager != null) {
+      // Don't remove the supervisorId from supervisorToManager. We want to be 
able to track the status
+      // for suspended supervisors to track any in-flight tasks. stop() will 
clean up all state completely.
+      manager.stopScheduling();
+    }
+  }
+
+  public void start()
+  {
+    log.info("Starting scheduled batch scheduler.");
+    final Optional<TaskRunner> taskRunnerOptional = taskMaster.getTaskRunner();
+    if (taskRunnerOptional.isPresent()) {
+      taskRunnerOptional.get().registerListener(taskRunnerListener, 
Execs.directExecutor());
+    } else {
+      log.warn("Task runner not registered for scheduled batch scheduler.");
+    }
+  }
+
+  public void stop()
+  {
+    log.info("Stopping scheduled batch scheduler.");
+    final Optional<TaskRunner> taskRunnerOptional = taskMaster.getTaskRunner();
+    if (taskRunnerOptional.isPresent()) {
+      
taskRunnerOptional.get().unregisterListener(taskRunnerListener.getListenerId());
+    }
+
+    supervisorToManager.clear();
+  }
+
+  @Nullable
+  public ScheduledBatchSupervisorSnapshot getSchedulerSnapshot(final String 
supervisorId)
+  {
+    final SchedulerManager manager = supervisorToManager.get(supervisorId);
+    if (manager == null) {
+      return null;
+    }
+
+    final ScheduledBatchStatusTracker.BatchSupervisorTaskStatus tasks = 
statusTracker.getSupervisorTasks(supervisorId);
+    return new ScheduledBatchSupervisorSnapshot(
+        supervisorId,
+        manager.getSchedulerStatus(),
+        manager.getLastTaskSubmittedTime(),
+        manager.getNextTaskSubmissionTime(),
+        manager.getTimeUntilNextTaskSubmission(),
+        tasks.getSubmittedTasks(),
+        tasks.getCompletedTasks()
+    );
+  }
+
+  private void enqueueTask(final Runnable runnable)
+  {
+    cronExecutor.submit(runnable);
+  }
+
+  private void submitSqlTask(final String supervisorId, final SqlQuery spec)
+      throws ExecutionException, InterruptedException
+  {
+    log.info("Submitting a new task with spec[%s] for supervisor[%s].", spec, 
supervisorId);

Review Comment:
   Can this be debug?



##########
indexing-service/src/main/java/org/apache/druid/indexing/scheduledbatch/ScheduledBatchSupervisorSpec.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.indexing.scheduledbatch;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.druid.common.config.Configs;
+import org.apache.druid.error.DruidException;
+import org.apache.druid.error.InvalidInput;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.sql.calcite.planner.ExplainAttributes;
+import org.apache.druid.sql.client.BrokerClient;
+import org.apache.druid.sql.http.ExplainPlan;
+import org.apache.druid.sql.http.SqlQuery;
+
+import javax.annotation.Nullable;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+
+public class ScheduledBatchSupervisorSpec implements SupervisorSpec
+{
+  public static final String TYPE = "scheduled_batch";
+  public static final String ID_PREFIX = "scheduled_batch__";
+
+  private static final Logger log = new Logger(ScheduledBatchSupervisor.class);
+
+  @JsonProperty
+  private final SqlQuery spec;
+  @JsonProperty
+  private final boolean suspended;
+  @JsonProperty
+  private final CronSchedulerConfig schedulerConfig;
+
+  /**
+   * Note that both {@link #dataSource} and {@link #id} are optional JSON 
fields present in the spec.
+   * They are only used internally because we use and persist the user-facing 
spec in the metadata store. So these
+   * additional fields are required for jackson deserialization.
+   * It would be better to have separate user-facing and domain-specific DTOs 
for this purpose and map them, but
+   * that'll entail a larger change.
+   */
+  @JsonProperty
+  private final String dataSource;
+  @JsonProperty
+  private final String id;
+
+  private final ObjectMapper objectMapper;
+  private final ScheduledBatchScheduler batchScheduler;
+  private final BrokerClient brokerClient;
+
+  @JsonCreator
+  public ScheduledBatchSupervisorSpec(
+      @JsonProperty("spec") final SqlQuery spec,
+      @JsonProperty("schedulerConfig") final CronSchedulerConfig 
schedulerConfig,
+      @JsonProperty("suspended") @Nullable Boolean suspended,
+      @JsonProperty("id") @Nullable final String id,
+      @JsonProperty("dataSource") @Nullable final String dataSource,
+      @JacksonInject ObjectMapper objectMapper,
+      @JacksonInject ScheduledBatchScheduler batchScheduler,
+      @JacksonInject BrokerClient brokerClient
+  )
+  {
+    this.spec = spec;
+    this.schedulerConfig = schedulerConfig;
+    this.suspended = Configs.valueOrDefault(suspended, false);
+    this.objectMapper = objectMapper;
+    this.batchScheduler = batchScheduler;
+    this.brokerClient = brokerClient;
+
+    this.dataSource = dataSource != null ? dataSource : 
getDatasourceFromQuery();
+    this.id = id != null ? id : ID_PREFIX + this.dataSource + "__" + 
UUID.randomUUID();
+  }
+
+  private String getDatasourceFromQuery()
+  {
+    final List<ExplainPlan> explainPlanInfos;
+    final ListenableFuture<List<ExplainPlan>> explainPlanFuture = 
brokerClient.fetchExplainPlan(spec);
+    try {
+      explainPlanInfos = explainPlanFuture.get();
+    }
+    catch (Exception e) {
+      throw InvalidInput.exception("Error getting datasource from query[%s]: 
[%s]", spec, e);
+    }
+
+    if (explainPlanInfos.size() != 1) {
+      throw DruidException.defensive(
+          "Received an invalid EXPLAIN PLAN response for query[%s]. Expected a 
single plan information, but received[%d]: [%s].",
+          spec.getQuery(),
+          explainPlanInfos.size(),
+          explainPlanInfos
+      );
+    }
+
+    final ExplainAttributes explainAttributes = 
explainPlanInfos.get(0).getAttributes();
+
+    if ("SELECT".equalsIgnoreCase(explainAttributes.getStatementType())) {
+      throw InvalidInput.exception(
+          "SELECT queries are not supported by the [%s] supervisor. "
+          + "Only INSERT or REPLACE ingest queries are allowed.", getType());
+    }
+
+    return explainAttributes.getTargetDataSource();
+  }
+
+  @Override
+  public String getId()
+  {
+    return id;
+  }
+
+  @Override
+  public ScheduledBatchSupervisor createSupervisor()
+  {
+    return new ScheduledBatchSupervisor(this, batchScheduler);
+  }
+
+  @Override
+  public List<String> getDataSources()
+  {
+    return Collections.singletonList(dataSource);
+  }
+
+  @Override
+  public ScheduledBatchSupervisorSpec createSuspendedSpec()
+  {
+    return new ScheduledBatchSupervisorSpec(
+        spec,
+        schedulerConfig,
+        true,
+        id, dataSource,

Review Comment:
   ```suggestion
           id,
           dataSource,
   ```



##########
indexing-service/src/main/java/org/apache/druid/indexing/scheduledbatch/ScheduledBatchScheduler.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.indexing.scheduledbatch;
+
+import com.google.common.base.Optional;
+import com.google.inject.Inject;
+import org.apache.druid.common.guava.FutureUtils;
+import org.apache.druid.indexer.TaskLocation;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.indexing.overlord.TaskMaster;
+import org.apache.druid.indexing.overlord.TaskRunner;
+import org.apache.druid.indexing.overlord.TaskRunnerListener;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.apache.druid.sql.client.BrokerClient;
+import org.apache.druid.sql.http.SqlQuery;
+import org.apache.druid.sql.http.SqlTaskStatus;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+
+import javax.annotation.Nullable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+public class ScheduledBatchScheduler
+{
+  private static final Logger log = new 
EmittingLogger(ScheduledBatchScheduler.class);
+
+  private final TaskRunnerListener taskRunnerListener;
+  private final TaskMaster taskMaster;
+  private final ScheduledBatchStatusTracker statusTracker;
+  private final BrokerClient brokerClient;
+  private final ServiceEmitter emitter;
+
+  /**
+   * Single-threaded executor to process the cron jobs queue.
+   */
+  private final ScheduledExecutorService cronExecutor;
+
+  private final ConcurrentHashMap<String, SchedulerManager> supervisorToManager
+      = new ConcurrentHashMap<>();
+  private final ScheduledExecutorFactory executorFactory;
+
+  @Inject
+  public ScheduledBatchScheduler(
+      final TaskMaster taskMaster,
+      final ScheduledExecutorFactory executorFactory,
+      final BrokerClient brokerClient,
+      final ServiceEmitter emitter,
+      final ScheduledBatchStatusTracker statusTracker
+  )
+  {
+    this.taskMaster = taskMaster;
+    this.executorFactory = executorFactory;
+    this.cronExecutor = executorFactory.create(1, 
"ScheduledBatchScheduler-%s");
+    this.brokerClient = brokerClient;
+    this.emitter = emitter;
+    this.statusTracker = statusTracker;
+
+    this.taskRunnerListener = new TaskRunnerListener()
+    {
+      @Override
+      public String getListenerId()
+      {
+        return "ScheduledBatchScheduler";
+      }
+
+      @Override
+      public void locationChanged(String taskId, TaskLocation newLocation)
+      {
+        // Do nothing
+      }
+
+      @Override
+      public void statusChanged(final String taskId, final TaskStatus 
taskStatus)
+      {
+        if (taskStatus.isComplete()) {
+          statusTracker.onTaskCompleted(taskId, taskStatus);
+        }
+      }
+    };
+  }
+
+  public void startScheduledIngestion(
+      final String supervisorId,
+      final CronSchedulerConfig cronSchedulerConfig,
+      final SqlQuery spec
+  )
+  {
+    log.info(
+        "Starting scheduled batch ingestion for supervisorId[%s] with 
cronSchedule[%s] and spec[%s].",
+        supervisorId, cronSchedulerConfig, spec
+    );
+    final SchedulerManager manager = new SchedulerManager(supervisorId, 
cronSchedulerConfig, spec, executorFactory);
+    manager.startScheduling();
+    supervisorToManager.put(supervisorId, manager);
+  }
+
+  public void stopScheduledIngestion(final String supervisorId)
+  {
+    log.info("Stopping scheduled batch ingestion for supervisorId[%s].", 
supervisorId);
+    final SchedulerManager manager = supervisorToManager.get(supervisorId);
+    if (manager != null) {
+      // Don't remove the supervisorId from supervisorToManager. We want to be 
able to track the status
+      // for suspended supervisors to track any in-flight tasks. stop() will 
clean up all state completely.
+      manager.stopScheduling();
+    }
+  }
+
+  public void start()
+  {
+    log.info("Starting scheduled batch scheduler.");
+    final Optional<TaskRunner> taskRunnerOptional = taskMaster.getTaskRunner();
+    if (taskRunnerOptional.isPresent()) {
+      taskRunnerOptional.get().registerListener(taskRunnerListener, 
Execs.directExecutor());
+    } else {
+      log.warn("Task runner not registered for scheduled batch scheduler.");
+    }
+  }
+
+  public void stop()
+  {
+    log.info("Stopping scheduled batch scheduler.");
+    final Optional<TaskRunner> taskRunnerOptional = taskMaster.getTaskRunner();
+    if (taskRunnerOptional.isPresent()) {
+      
taskRunnerOptional.get().unregisterListener(taskRunnerListener.getListenerId());
+    }
+
+    supervisorToManager.clear();
+  }
+
+  @Nullable
+  public ScheduledBatchSupervisorSnapshot getSchedulerSnapshot(final String 
supervisorId)
+  {
+    final SchedulerManager manager = supervisorToManager.get(supervisorId);
+    if (manager == null) {
+      return null;
+    }
+
+    final ScheduledBatchStatusTracker.BatchSupervisorTaskStatus tasks = 
statusTracker.getSupervisorTasks(supervisorId);
+    return new ScheduledBatchSupervisorSnapshot(
+        supervisorId,
+        manager.getSchedulerStatus(),
+        manager.getLastTaskSubmittedTime(),
+        manager.getNextTaskSubmissionTime(),
+        manager.getTimeUntilNextTaskSubmission(),
+        tasks.getSubmittedTasks(),
+        tasks.getCompletedTasks()
+    );
+  }
+
+  private void enqueueTask(final Runnable runnable)
+  {
+    cronExecutor.submit(runnable);
+  }
+
+  private void submitSqlTask(final String supervisorId, final SqlQuery spec)
+      throws ExecutionException, InterruptedException
+  {
+    log.info("Submitting a new task with spec[%s] for supervisor[%s].", spec, 
supervisorId);
+    final SqlTaskStatus taskStatus = 
FutureUtils.get(brokerClient.submitSqlTask(spec), true);
+    statusTracker.onTaskSubmitted(supervisorId, taskStatus);
+  }
+
+  private class SchedulerManager
+  {
+    private final String supervisorId;
+    private final SqlQuery spec;
+    private final ScheduledExecutorService managerExecutor;
+    private final CronSchedulerConfig cronSchedulerConfig;
+
+    private ScheduledBatchSupervisorSnapshot.BatchSupervisorStatus status;
+
+    /**
+     * Note that the last task submitted per supervisor should eventually be 
persisted in the metadata store,

Review Comment:
   We should clarify that this is still a TODO.



##########
indexing-service/src/main/java/org/apache/druid/indexing/scheduledbatch/ScheduledBatchScheduler.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.indexing.scheduledbatch;
+
+import com.google.common.base.Optional;
+import com.google.inject.Inject;
+import org.apache.druid.common.guava.FutureUtils;
+import org.apache.druid.indexer.TaskLocation;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.indexing.overlord.TaskMaster;
+import org.apache.druid.indexing.overlord.TaskRunner;
+import org.apache.druid.indexing.overlord.TaskRunnerListener;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.apache.druid.sql.client.BrokerClient;
+import org.apache.druid.sql.http.SqlQuery;
+import org.apache.druid.sql.http.SqlTaskStatus;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+
+import javax.annotation.Nullable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+public class ScheduledBatchScheduler
+{
+  private static final Logger log = new 
EmittingLogger(ScheduledBatchScheduler.class);
+
+  private final TaskRunnerListener taskRunnerListener;
+  private final TaskMaster taskMaster;
+  private final ScheduledBatchStatusTracker statusTracker;
+  private final BrokerClient brokerClient;
+  private final ServiceEmitter emitter;
+
+  /**
+   * Single-threaded executor to process the cron jobs queue.
+   */
+  private final ScheduledExecutorService cronExecutor;
+
+  private final ConcurrentHashMap<String, SchedulerManager> supervisorToManager
+      = new ConcurrentHashMap<>();
+  private final ScheduledExecutorFactory executorFactory;
+
+  @Inject
+  public ScheduledBatchScheduler(
+      final TaskMaster taskMaster,
+      final ScheduledExecutorFactory executorFactory,
+      final BrokerClient brokerClient,
+      final ServiceEmitter emitter,
+      final ScheduledBatchStatusTracker statusTracker
+  )
+  {
+    this.taskMaster = taskMaster;
+    this.executorFactory = executorFactory;
+    this.cronExecutor = executorFactory.create(1, 
"ScheduledBatchScheduler-%s");
+    this.brokerClient = brokerClient;
+    this.emitter = emitter;
+    this.statusTracker = statusTracker;
+
+    this.taskRunnerListener = new TaskRunnerListener()
+    {
+      @Override
+      public String getListenerId()
+      {
+        return "ScheduledBatchScheduler";
+      }
+
+      @Override
+      public void locationChanged(String taskId, TaskLocation newLocation)
+      {
+        // Do nothing
+      }
+
+      @Override
+      public void statusChanged(final String taskId, final TaskStatus 
taskStatus)
+      {
+        if (taskStatus.isComplete()) {
+          statusTracker.onTaskCompleted(taskId, taskStatus);
+        }
+      }
+    };
+  }
+
+  public void startScheduledIngestion(
+      final String supervisorId,
+      final CronSchedulerConfig cronSchedulerConfig,
+      final SqlQuery spec
+  )
+  {
+    log.info(
+        "Starting scheduled batch ingestion for supervisorId[%s] with 
cronSchedule[%s] and spec[%s].",
+        supervisorId, cronSchedulerConfig, spec
+    );
+    final SchedulerManager manager = new SchedulerManager(supervisorId, 
cronSchedulerConfig, spec, executorFactory);
+    manager.startScheduling();
+    supervisorToManager.put(supervisorId, manager);
+  }
+
+  public void stopScheduledIngestion(final String supervisorId)
+  {
+    log.info("Stopping scheduled batch ingestion for supervisorId[%s].", 
supervisorId);
+    final SchedulerManager manager = supervisorToManager.get(supervisorId);
+    if (manager != null) {
+      // Don't remove the supervisorId from supervisorToManager. We want to be 
able to track the status
+      // for suspended supervisors to track any in-flight tasks. stop() will 
clean up all state completely.
+      manager.stopScheduling();
+    }
+  }
+
+  public void start()
+  {
+    log.info("Starting scheduled batch scheduler.");
+    final Optional<TaskRunner> taskRunnerOptional = taskMaster.getTaskRunner();
+    if (taskRunnerOptional.isPresent()) {
+      taskRunnerOptional.get().registerListener(taskRunnerListener, 
Execs.directExecutor());
+    } else {
+      log.warn("Task runner not registered for scheduled batch scheduler.");
+    }
+  }
+
+  public void stop()
+  {
+    log.info("Stopping scheduled batch scheduler.");
+    final Optional<TaskRunner> taskRunnerOptional = taskMaster.getTaskRunner();
+    if (taskRunnerOptional.isPresent()) {
+      
taskRunnerOptional.get().unregisterListener(taskRunnerListener.getListenerId());
+    }
+
+    supervisorToManager.clear();
+  }
+
+  @Nullable
+  public ScheduledBatchSupervisorSnapshot getSchedulerSnapshot(final String 
supervisorId)
+  {
+    final SchedulerManager manager = supervisorToManager.get(supervisorId);
+    if (manager == null) {
+      return null;
+    }
+
+    final ScheduledBatchStatusTracker.BatchSupervisorTaskStatus tasks = 
statusTracker.getSupervisorTasks(supervisorId);
+    return new ScheduledBatchSupervisorSnapshot(
+        supervisorId,
+        manager.getSchedulerStatus(),
+        manager.getLastTaskSubmittedTime(),
+        manager.getNextTaskSubmissionTime(),
+        manager.getTimeUntilNextTaskSubmission(),
+        tasks.getSubmittedTasks(),
+        tasks.getCompletedTasks()
+    );
+  }
+
+  private void enqueueTask(final Runnable runnable)
+  {
+    cronExecutor.submit(runnable);
+  }
+
+  private void submitSqlTask(final String supervisorId, final SqlQuery spec)
+      throws ExecutionException, InterruptedException
+  {
+    log.info("Submitting a new task with spec[%s] for supervisor[%s].", spec, 
supervisorId);
+    final SqlTaskStatus taskStatus = 
FutureUtils.get(brokerClient.submitSqlTask(spec), true);
+    statusTracker.onTaskSubmitted(supervisorId, taskStatus);
+  }
+
+  private class SchedulerManager
+  {
+    private final String supervisorId;
+    private final SqlQuery spec;
+    private final ScheduledExecutorService managerExecutor;
+    private final CronSchedulerConfig cronSchedulerConfig;
+
+    private ScheduledBatchSupervisorSnapshot.BatchSupervisorStatus status;
+
+    /**
+     * Note that the last task submitted per supervisor should eventually be 
persisted in the metadata store,
+     * along with any other scheduler state, so that the batch supervisor can 
recover gracefully and
+     * avoid missing task submissions during rolling restarts, etc.
+     */
+    private DateTime lastTaskSubmittedTime;
+
+    private SchedulerManager(
+        final String supervisorId,
+        final CronSchedulerConfig cronSchedulerConfig,
+        final SqlQuery spec,
+        final ScheduledExecutorFactory executorFactory
+    )
+    {
+      this.supervisorId = supervisorId;
+      this.cronSchedulerConfig = cronSchedulerConfig;
+      this.spec = spec;
+      this.managerExecutor = executorFactory.create(1, "scheduler-" + 
supervisorId + "-%d");
+    }
+
+    private synchronized void startScheduling()
+    {
+      if (managerExecutor.isTerminated() || managerExecutor.isShutdown()) {
+        return;
+      }
+
+      status = 
ScheduledBatchSupervisorSnapshot.BatchSupervisorStatus.SCHEDULER_RUNNING;
+      final Duration timeUntilNextSubmission = 
getTimeUntilNextTaskSubmission();
+      if (timeUntilNextSubmission == null) {
+        log.info("No more tasks will be submitted for supervisor[%s].", 
supervisorId);
+        return;
+      }
+
+      managerExecutor.schedule(
+          () -> {
+            enqueueTask(() -> {
+              try {
+                lastTaskSubmittedTime = DateTimes.nowUtc();
+                submitSqlTask(supervisorId, spec);
+                emitMetric("batchSupervisor/tasks/submit/success", 1);
+              }
+              catch (Exception e) {
+                emitMetric("batchSupervisor/tasks/submit/failed", 1);
+                log.error(e, "Error submitting task for supervisor[%s]. 
Continuing schedule.", supervisorId);
+              }
+            });
+            startScheduling();
+          },
+          timeUntilNextSubmission.getMillis(),
+          TimeUnit.MILLISECONDS
+      );
+    }
+
+    private synchronized void stopScheduling()
+    {
+      managerExecutor.shutdown();
+      try {
+        if (!managerExecutor.awaitTermination(2, TimeUnit.SECONDS)) {
+          managerExecutor.shutdownNow();
+        }
+      }
+      catch (InterruptedException e) {
+        log.error(e, "Forcing shutdown of executor service for 
supervisor[%s].", supervisorId);
+        managerExecutor.shutdownNow();
+        Thread.currentThread().interrupt();
+      }
+      status = 
ScheduledBatchSupervisorSnapshot.BatchSupervisorStatus.SCHEDULER_SHUTDOWN;
+    }
+
+    private void emitMetric(final String metricName, final int value)
+    {
+      emitter.emit(
+          ServiceMetricEvent.builder()
+                            .setDimension("supervisorId", supervisorId)

Review Comment:
   datasource should be added as a dimension here too.



##########
indexing-service/src/main/java/org/apache/druid/indexing/scheduledbatch/ScheduledBatchScheduler.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.indexing.scheduledbatch;
+
+import com.google.common.base.Optional;
+import com.google.inject.Inject;
+import org.apache.druid.common.guava.FutureUtils;
+import org.apache.druid.indexer.TaskLocation;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.indexing.overlord.TaskMaster;
+import org.apache.druid.indexing.overlord.TaskRunner;
+import org.apache.druid.indexing.overlord.TaskRunnerListener;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.apache.druid.sql.client.BrokerClient;
+import org.apache.druid.sql.http.SqlQuery;
+import org.apache.druid.sql.http.SqlTaskStatus;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+
+import javax.annotation.Nullable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+public class ScheduledBatchScheduler
+{
+  private static final Logger log = new 
EmittingLogger(ScheduledBatchScheduler.class);
+
+  private final TaskRunnerListener taskRunnerListener;
+  private final TaskMaster taskMaster;
+  private final ScheduledBatchStatusTracker statusTracker;
+  private final BrokerClient brokerClient;
+  private final ServiceEmitter emitter;
+
+  /**
+   * Single-threaded executor to process the cron jobs queue.
+   */
+  private final ScheduledExecutorService cronExecutor;
+
+  private final ConcurrentHashMap<String, SchedulerManager> supervisorToManager
+      = new ConcurrentHashMap<>();
+  private final ScheduledExecutorFactory executorFactory;
+
+  @Inject
+  public ScheduledBatchScheduler(
+      final TaskMaster taskMaster,
+      final ScheduledExecutorFactory executorFactory,
+      final BrokerClient brokerClient,
+      final ServiceEmitter emitter,
+      final ScheduledBatchStatusTracker statusTracker
+  )
+  {
+    this.taskMaster = taskMaster;
+    this.executorFactory = executorFactory;
+    this.cronExecutor = executorFactory.create(1, 
"ScheduledBatchScheduler-%s");
+    this.brokerClient = brokerClient;
+    this.emitter = emitter;
+    this.statusTracker = statusTracker;
+
+    this.taskRunnerListener = new TaskRunnerListener()
+    {
+      @Override
+      public String getListenerId()
+      {
+        return "ScheduledBatchScheduler";
+      }
+
+      @Override
+      public void locationChanged(String taskId, TaskLocation newLocation)
+      {
+        // Do nothing
+      }
+
+      @Override
+      public void statusChanged(final String taskId, final TaskStatus 
taskStatus)
+      {
+        if (taskStatus.isComplete()) {
+          statusTracker.onTaskCompleted(taskId, taskStatus);
+        }
+      }
+    };
+  }
+
+  public void startScheduledIngestion(
+      final String supervisorId,
+      final CronSchedulerConfig cronSchedulerConfig,
+      final SqlQuery spec
+  )
+  {
+    log.info(
+        "Starting scheduled batch ingestion for supervisorId[%s] with 
cronSchedule[%s] and spec[%s].",
+        supervisorId, cronSchedulerConfig, spec
+    );
+    final SchedulerManager manager = new SchedulerManager(supervisorId, 
cronSchedulerConfig, spec, executorFactory);
+    manager.startScheduling();
+    supervisorToManager.put(supervisorId, manager);
+  }
+
+  public void stopScheduledIngestion(final String supervisorId)
+  {
+    log.info("Stopping scheduled batch ingestion for supervisorId[%s].", 
supervisorId);
+    final SchedulerManager manager = supervisorToManager.get(supervisorId);
+    if (manager != null) {
+      // Don't remove the supervisorId from supervisorToManager. We want to be 
able to track the status
+      // for suspended supervisors to track any in-flight tasks. stop() will 
clean up all state completely.
+      manager.stopScheduling();
+    }
+  }
+
+  public void start()
+  {
+    log.info("Starting scheduled batch scheduler.");
+    final Optional<TaskRunner> taskRunnerOptional = taskMaster.getTaskRunner();
+    if (taskRunnerOptional.isPresent()) {
+      taskRunnerOptional.get().registerListener(taskRunnerListener, 
Execs.directExecutor());
+    } else {
+      log.warn("Task runner not registered for scheduled batch scheduler.");
+    }
+  }
+
+  public void stop()
+  {
+    log.info("Stopping scheduled batch scheduler.");
+    final Optional<TaskRunner> taskRunnerOptional = taskMaster.getTaskRunner();
+    if (taskRunnerOptional.isPresent()) {
+      
taskRunnerOptional.get().unregisterListener(taskRunnerListener.getListenerId());
+    }
+
+    supervisorToManager.clear();
+  }
+
+  @Nullable
+  public ScheduledBatchSupervisorSnapshot getSchedulerSnapshot(final String 
supervisorId)
+  {
+    final SchedulerManager manager = supervisorToManager.get(supervisorId);
+    if (manager == null) {
+      return null;
+    }
+
+    final ScheduledBatchStatusTracker.BatchSupervisorTaskStatus tasks = 
statusTracker.getSupervisorTasks(supervisorId);
+    return new ScheduledBatchSupervisorSnapshot(
+        supervisorId,
+        manager.getSchedulerStatus(),
+        manager.getLastTaskSubmittedTime(),
+        manager.getNextTaskSubmissionTime(),
+        manager.getTimeUntilNextTaskSubmission(),
+        tasks.getSubmittedTasks(),
+        tasks.getCompletedTasks()
+    );
+  }
+
+  private void enqueueTask(final Runnable runnable)
+  {
+    cronExecutor.submit(runnable);
+  }
+
+  private void submitSqlTask(final String supervisorId, final SqlQuery spec)
+      throws ExecutionException, InterruptedException
+  {
+    log.info("Submitting a new task with spec[%s] for supervisor[%s].", spec, 
supervisorId);
+    final SqlTaskStatus taskStatus = 
FutureUtils.get(brokerClient.submitSqlTask(spec), true);
+    statusTracker.onTaskSubmitted(supervisorId, taskStatus);
+  }
+
+  private class SchedulerManager
+  {
+    private final String supervisorId;
+    private final SqlQuery spec;
+    private final ScheduledExecutorService managerExecutor;
+    private final CronSchedulerConfig cronSchedulerConfig;
+
+    private ScheduledBatchSupervisorSnapshot.BatchSupervisorStatus status;
+
+    /**
+     * Note that the last task submitted per supervisor should eventually be 
persisted in the metadata store,
+     * along with any other scheduler state, so that the batch supervisor can 
recover gracefully and
+     * avoid missing task submissions during rolling restarts, etc.
+     */
+    private DateTime lastTaskSubmittedTime;
+
+    private SchedulerManager(
+        final String supervisorId,
+        final CronSchedulerConfig cronSchedulerConfig,
+        final SqlQuery spec,
+        final ScheduledExecutorFactory executorFactory
+    )
+    {
+      this.supervisorId = supervisorId;
+      this.cronSchedulerConfig = cronSchedulerConfig;
+      this.spec = spec;
+      this.managerExecutor = executorFactory.create(1, "scheduler-" + 
supervisorId + "-%d");
+    }
+
+    private synchronized void startScheduling()
+    {
+      if (managerExecutor.isTerminated() || managerExecutor.isShutdown()) {
+        return;
+      }
+
+      status = 
ScheduledBatchSupervisorSnapshot.BatchSupervisorStatus.SCHEDULER_RUNNING;
+      final Duration timeUntilNextSubmission = 
getTimeUntilNextTaskSubmission();
+      if (timeUntilNextSubmission == null) {
+        log.info("No more tasks will be submitted for supervisor[%s].", 
supervisorId);
+        return;
+      }
+
+      managerExecutor.schedule(
+          () -> {
+            enqueueTask(() -> {
+              try {
+                lastTaskSubmittedTime = DateTimes.nowUtc();
+                submitSqlTask(supervisorId, spec);
+                emitMetric("batchSupervisor/tasks/submit/success", 1);
+              }
+              catch (Exception e) {
+                emitMetric("batchSupervisor/tasks/submit/failed", 1);
+                log.error(e, "Error submitting task for supervisor[%s]. 
Continuing schedule.", supervisorId);
+              }
+            });
+            startScheduling();
+          },
+          timeUntilNextSubmission.getMillis(),
+          TimeUnit.MILLISECONDS
+      );
+    }
+
+    private synchronized void stopScheduling()
+    {
+      managerExecutor.shutdown();
+      try {
+        if (!managerExecutor.awaitTermination(2, TimeUnit.SECONDS)) {
+          managerExecutor.shutdownNow();
+        }
+      }
+      catch (InterruptedException e) {
+        log.error(e, "Forcing shutdown of executor service for 
supervisor[%s].", supervisorId);
+        managerExecutor.shutdownNow();
+        Thread.currentThread().interrupt();

Review Comment:
   How does interrupting the calling thread help?



##########
indexing-service/src/main/java/org/apache/druid/indexing/scheduledbatch/ScheduledBatchScheduler.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.indexing.scheduledbatch;
+
+import com.google.common.base.Optional;
+import com.google.inject.Inject;
+import org.apache.druid.common.guava.FutureUtils;
+import org.apache.druid.indexer.TaskLocation;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.indexing.overlord.TaskMaster;
+import org.apache.druid.indexing.overlord.TaskRunner;
+import org.apache.druid.indexing.overlord.TaskRunnerListener;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.apache.druid.sql.client.BrokerClient;
+import org.apache.druid.sql.http.SqlQuery;
+import org.apache.druid.sql.http.SqlTaskStatus;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+
+import javax.annotation.Nullable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+public class ScheduledBatchScheduler
+{
+  private static final Logger log = new 
EmittingLogger(ScheduledBatchScheduler.class);
+
+  private final TaskRunnerListener taskRunnerListener;
+  private final TaskMaster taskMaster;
+  private final ScheduledBatchStatusTracker statusTracker;
+  private final BrokerClient brokerClient;
+  private final ServiceEmitter emitter;
+
+  /**
+   * Single-threaded executor to process the cron jobs queue.
+   */
+  private final ScheduledExecutorService cronExecutor;
+
+  private final ConcurrentHashMap<String, SchedulerManager> supervisorToManager
+      = new ConcurrentHashMap<>();
+  private final ScheduledExecutorFactory executorFactory;
+
+  @Inject
+  public ScheduledBatchScheduler(
+      final TaskMaster taskMaster,
+      final ScheduledExecutorFactory executorFactory,
+      final BrokerClient brokerClient,
+      final ServiceEmitter emitter,
+      final ScheduledBatchStatusTracker statusTracker
+  )
+  {
+    this.taskMaster = taskMaster;
+    this.executorFactory = executorFactory;
+    this.cronExecutor = executorFactory.create(1, 
"ScheduledBatchScheduler-%s");
+    this.brokerClient = brokerClient;
+    this.emitter = emitter;
+    this.statusTracker = statusTracker;
+
+    this.taskRunnerListener = new TaskRunnerListener()
+    {
+      @Override
+      public String getListenerId()
+      {
+        return "ScheduledBatchScheduler";
+      }
+
+      @Override
+      public void locationChanged(String taskId, TaskLocation newLocation)
+      {
+        // Do nothing
+      }
+
+      @Override
+      public void statusChanged(final String taskId, final TaskStatus 
taskStatus)
+      {
+        if (taskStatus.isComplete()) {
+          statusTracker.onTaskCompleted(taskId, taskStatus);
+        }
+      }
+    };
+  }
+
+  public void startScheduledIngestion(
+      final String supervisorId,
+      final CronSchedulerConfig cronSchedulerConfig,
+      final SqlQuery spec
+  )
+  {
+    log.info(
+        "Starting scheduled batch ingestion for supervisorId[%s] with 
cronSchedule[%s] and spec[%s].",
+        supervisorId, cronSchedulerConfig, spec
+    );
+    final SchedulerManager manager = new SchedulerManager(supervisorId, 
cronSchedulerConfig, spec, executorFactory);
+    manager.startScheduling();
+    supervisorToManager.put(supervisorId, manager);
+  }
+
+  public void stopScheduledIngestion(final String supervisorId)
+  {
+    log.info("Stopping scheduled batch ingestion for supervisorId[%s].", 
supervisorId);
+    final SchedulerManager manager = supervisorToManager.get(supervisorId);
+    if (manager != null) {
+      // Don't remove the supervisorId from supervisorToManager. We want to be 
able to track the status
+      // for suspended supervisors to track any in-flight tasks. stop() will 
clean up all state completely.
+      manager.stopScheduling();
+    }
+  }
+
+  public void start()
+  {
+    log.info("Starting scheduled batch scheduler.");
+    final Optional<TaskRunner> taskRunnerOptional = taskMaster.getTaskRunner();
+    if (taskRunnerOptional.isPresent()) {
+      taskRunnerOptional.get().registerListener(taskRunnerListener, 
Execs.directExecutor());
+    } else {
+      log.warn("Task runner not registered for scheduled batch scheduler.");
+    }
+  }
+
+  public void stop()
+  {
+    log.info("Stopping scheduled batch scheduler.");
+    final Optional<TaskRunner> taskRunnerOptional = taskMaster.getTaskRunner();
+    if (taskRunnerOptional.isPresent()) {
+      
taskRunnerOptional.get().unregisterListener(taskRunnerListener.getListenerId());
+    }
+
+    supervisorToManager.clear();
+  }
+
+  @Nullable
+  public ScheduledBatchSupervisorSnapshot getSchedulerSnapshot(final String 
supervisorId)
+  {
+    final SchedulerManager manager = supervisorToManager.get(supervisorId);
+    if (manager == null) {
+      return null;
+    }
+
+    final ScheduledBatchStatusTracker.BatchSupervisorTaskStatus tasks = 
statusTracker.getSupervisorTasks(supervisorId);
+    return new ScheduledBatchSupervisorSnapshot(
+        supervisorId,
+        manager.getSchedulerStatus(),
+        manager.getLastTaskSubmittedTime(),
+        manager.getNextTaskSubmissionTime(),
+        manager.getTimeUntilNextTaskSubmission(),
+        tasks.getSubmittedTasks(),
+        tasks.getCompletedTasks()
+    );
+  }
+
+  private void enqueueTask(final Runnable runnable)
+  {
+    cronExecutor.submit(runnable);
+  }
+
+  private void submitSqlTask(final String supervisorId, final SqlQuery spec)
+      throws ExecutionException, InterruptedException
+  {
+    log.info("Submitting a new task with spec[%s] for supervisor[%s].", spec, 
supervisorId);
+    final SqlTaskStatus taskStatus = 
FutureUtils.get(brokerClient.submitSqlTask(spec), true);
+    statusTracker.onTaskSubmitted(supervisorId, taskStatus);
+  }
+
+  private class SchedulerManager
+  {
+    private final String supervisorId;
+    private final SqlQuery spec;
+    private final ScheduledExecutorService managerExecutor;
+    private final CronSchedulerConfig cronSchedulerConfig;
+
+    private ScheduledBatchSupervisorSnapshot.BatchSupervisorStatus status;
+
+    /**
+     * Note that the last task submitted per supervisor should eventually be 
persisted in the metadata store,
+     * along with any other scheduler state, so that the batch supervisor can 
recover gracefully and
+     * avoid missing task submissions during rolling restarts, etc.
+     */
+    private DateTime lastTaskSubmittedTime;

Review Comment:
   Perhaps keep an atomic reference instead or make it volatile.



-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]


Reply via email to