churromorales commented on code in PR #13156:
URL: https://github.com/apache/druid/pull/13156#discussion_r1007171468


##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java:
##########
@@ -0,0 +1,571 @@
+/*
+ * 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.k8s.overlord;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.batch.v1.Job;
+import io.netty.util.SuppressForbidden;
+import org.apache.commons.io.FileUtils;
+import org.apache.druid.indexer.RunnerTaskState;
+import org.apache.druid.indexer.TaskLocation;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.indexing.common.config.TaskConfig;
+import org.apache.druid.indexing.common.task.Task;
+import org.apache.druid.indexing.overlord.ForkingTaskRunner;
+import org.apache.druid.indexing.overlord.QuotableWhiteSpaceSplitter;
+import org.apache.druid.indexing.overlord.TaskRunner;
+import org.apache.druid.indexing.overlord.TaskRunnerListener;
+import org.apache.druid.indexing.overlord.TaskRunnerUtils;
+import org.apache.druid.indexing.overlord.TaskRunnerWorkItem;
+import org.apache.druid.indexing.overlord.autoscaling.ScalingStats;
+import org.apache.druid.indexing.overlord.config.ForkingTaskRunnerConfig;
+import org.apache.druid.indexing.overlord.config.TaskQueueConfig;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.k8s.overlord.common.DruidK8sConstants;
+import org.apache.druid.k8s.overlord.common.JobResponse;
+import org.apache.druid.k8s.overlord.common.K8sTaskId;
+import org.apache.druid.k8s.overlord.common.KubernetesPeonClient;
+import org.apache.druid.k8s.overlord.common.PeonCommandContext;
+import org.apache.druid.k8s.overlord.common.PeonPhase;
+import org.apache.druid.k8s.overlord.common.TaskAdapter;
+import org.apache.druid.server.DruidNode;
+import org.apache.druid.server.log.StartupLoggingConfig;
+import org.apache.druid.tasklogs.TaskLogPusher;
+import org.apache.druid.tasklogs.TaskLogStreamer;
+import org.joda.time.DateTime;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Runs tasks as k8s jobs using the "internal peon" verb.
+ * One additional feature of this class is that kubernetes is the source of 
truth, so if you launch a task
+ * shutdown druid, bring up druid, the task will keep running and the state 
will be updated when the cluster
+ * comes back.  Thus while no tasks are technically restorable, all tasks once 
launched will run in isolation to the
+ * extent possible without requiring the overlord consistently up during their 
lifetime.
+ */
+
+public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner
+{
+
+  private static final EmittingLogger log = new 
EmittingLogger(KubernetesTaskRunner.class);
+  private final CopyOnWriteArrayList<Pair<TaskRunnerListener, Executor>> 
listeners = new CopyOnWriteArrayList<>();
+
+  // to cleanup old jobs that might not have been deleted.
+  private final ScheduledExecutorService cleanupExecutor;
+
+  protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new 
ConcurrentHashMap<>();
+  private final TaskConfig taskConfig;
+  private final StartupLoggingConfig startupLoggingConfig;
+  private final TaskAdapter<Pod, Job> adapter;
+
+  private final KubernetesTaskRunnerConfig k8sConfig;
+  private final TaskQueueConfig taskQueueConfig;
+  private final TaskLogPusher taskLogPusher;
+  private final ListeningExecutorService exec;
+  private final KubernetesPeonClient client;
+  private final DruidNode node;
+
+
+  public KubernetesTaskRunner(
+      TaskConfig taskConfig,
+      StartupLoggingConfig startupLoggingConfig,
+      TaskAdapter<Pod, Job> adapter,
+      KubernetesTaskRunnerConfig k8sConfig,
+      TaskQueueConfig taskQueueConfig,
+      TaskLogPusher taskLogPusher,
+      KubernetesPeonClient client,
+      DruidNode node
+  )
+  {
+    this.taskConfig = taskConfig;
+    this.startupLoggingConfig = startupLoggingConfig;
+    this.adapter = adapter;
+    this.k8sConfig = k8sConfig;
+    this.taskQueueConfig = taskQueueConfig;
+    this.taskLogPusher = taskLogPusher;
+    this.client = client;
+    this.node = node;
+    this.cleanupExecutor = Executors.newScheduledThreadPool(1);
+    this.exec = MoreExecutors.listeningDecorator(
+        Execs.multiThreaded(taskQueueConfig.getMaxSize(), "k8s-task-runner-%d")
+    );
+    Preconditions.checkArgument(
+        taskQueueConfig.getMaxSize() < Integer.MAX_VALUE,
+        "The task queue bounds how many concurrent k8s tasks you can have, set 
it to a resonable value"
+    );
+  }
+
+
+  @Override
+  public Optional<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    Optional<InputStream> peonLogs = client.getPeonLogs(new K8sTaskId(taskid));
+    if (!peonLogs.isPresent()) {
+      return Optional.absent();
+    }
+    return Optional.of(peonLogs.get());
+  }
+
+  @Override
+  public ListenableFuture<TaskStatus> run(Task task)
+  {
+    synchronized (tasks) {
+      tasks.computeIfAbsent(
+          task.getId(), k -> new K8sWorkItem(
+              task,
+              exec.submit(() -> {
+                K8sTaskId k8sTaskId = new K8sTaskId(task);
+                try {
+                  JobResponse completedPhase;
+                  Optional<Job> existingJob = client.jobExists(k8sTaskId);
+                  if (!existingJob.isPresent()) {

Review Comment:
   this is from the comment earlier.  This is the scenario this happens.
   1. overlord launches task.  
   2. overlord goes down
   3. overlord comes up, calls get knownTasks() which calls run()
   4. Now the task doesn't need to be launched, it already exists, just track 
it. 



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