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


##########
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:
   No longer relevant since it's a shared executor



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