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


##########
indexing-service/src/main/java/org/apache/druid/indexing/scheduledbatch/ScheduledBatchTaskManager.java:
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.client.broker.BrokerClient;
+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.query.http.ClientSqlQuery;
+import org.apache.druid.query.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;
+
+/**
+ * Responsible for task management for the scheduled batch supervisor.
+ */
+public class ScheduledBatchTaskManager
+{
+  private static final Logger log = new 
EmittingLogger(ScheduledBatchTaskManager.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 schedule jobs that is shared across scheduled 
batch supervisors.
+   */
+  private final ScheduledExecutorService jobsExecutor;
+
+  private final ConcurrentHashMap<String, SchedulerManager> supervisorToManager
+      = new ConcurrentHashMap<>();
+
+  @Inject
+  public ScheduledBatchTaskManager(
+      final TaskMaster taskMaster,
+      final ScheduledExecutorFactory executorFactory,
+      final BrokerClient brokerClient,
+      final ServiceEmitter emitter,
+      final ScheduledBatchStatusTracker statusTracker
+  )
+  {
+    this.taskMaster = taskMaster;
+    this.brokerClient = brokerClient;
+    this.emitter = emitter;
+    this.statusTracker = statusTracker;
+
+    this.jobsExecutor = executorFactory.create(1, 
"ScheduledBatchJobsExecutor-%s");
+
+    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);
+        }
+      }
+    };
+  }
+
+  /**
+   * Initializes and starts scheduled ingestion for the specified {@code 
supervisorId}, scheduling tasks
+   * using the provided {@code spec} and {@code cronSchedulerConfig}.
+   */
+  public void startScheduledIngestion(
+      final String supervisorId,
+      final String dataSource,
+      final CronSchedulerConfig cronSchedulerConfig,
+      final ClientSqlQuery spec
+  )
+  {
+    log.info(
+        "Starting scheduled batch ingestion for supervisorId[%s] with 
datasource[%s], cronSchedule[%s] and spec[%s].",
+        supervisorId, dataSource, cronSchedulerConfig, spec
+    );
+    final SchedulerManager manager = new SchedulerManager(supervisorId, 
dataSource, cronSchedulerConfig, spec);
+    manager.startScheduling();
+    supervisorToManager.put(supervisorId, manager);
+  }
+
+  /**
+   * Stops the scheduler for the specified {@code supervisorId}, pausing task 
submissions
+   * while retaining the ability to track the supervisor's status.
+   */
+  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()

Review Comment:
   Good point on tracking the totals and subset. I've made that change by 
tracking the total counts and only the recent tasks (similar to the compaction 
supervisor).
   
   >Since these are going to be MSQ tasks, they would have randomly generated 
group IDs. I wonder if there could be a way to find all the tasks for a given 
batch supervisor. But at least we can always filter by data source and task 
type.
   
   Yeah, I initially explored setting the batch supervisor ID as the group ID 
for MSQ tasks via the query context so users can filter by the supervisor ID. I 
don’t exactly recall why that didn’t work, but it’s probably worth exploring 
sometime in the future...



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