You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2022/10/28 16:36:54 UTC

[GitHub] [druid] a2l007 commented on a diff in pull request #13156: Support for middle manager less druid, tasks launch as k8s jobs

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


##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java:
##########
@@ -0,0 +1,552 @@
+/*
+ * 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.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"
+    );
+  }
+
+
+  @Override
+  public Optional<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    return client.getPeonLogs(new K8sTaskId(taskid));
+  }
+
+  @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()) {
+                    PeonCommandContext context = new PeonCommandContext(
+                        generateCommand(task),
+                        javaOpts(task),
+                        taskConfig.getTaskDir(task.getId()),
+                        node.isEnableTlsPort()
+                    );
+                    Job job = adapter.fromTask(task, context);
+                    log.info("Job created %s and ready to launch", k8sTaskId);
+                    Pod peonPod = client.launchJobAndWaitForStart(
+                        job,
+                        KubernetesTaskRunnerConfig.toMilliseconds(k8sConfig.k8sjobLaunchTimeout),
+                        TimeUnit.MILLISECONDS
+                    );
+                    log.info("Job %s launched in k8s", k8sTaskId);
+                    completedPhase = monitorJob(peonPod, k8sTaskId);
+                  } else {
+                    Job job = existingJob.get();
+                    if (job.getStatus().getActive() == null) {
+                      if (job.getStatus().getSucceeded() != null) {
+                        completedPhase = new JobResponse(job, PeonPhase.SUCCEEDED);
+                      } else {
+                        completedPhase = new JobResponse(job, PeonPhase.FAILED);
+                      }
+                    } else {
+                      // the job is active lets monitor it
+                      completedPhase = monitorJob(k8sTaskId);
+                    }
+                  }
+                  TaskStatus status;
+                  if (PeonPhase.SUCCEEDED.equals(completedPhase.getPhase())) {
+                    status = TaskStatus.success(task.getId());
+                  } else {
+                    status = TaskStatus.failure(
+                        task.getId(),
+                        "Task failed %s: " + k8sTaskId
+                    );
+                  }
+                  if (completedPhase.getJobDuration().isPresent()) {
+                    status = status.withDuration(completedPhase.getJobDuration().get());
+                  }
+                  updateStatus(task, status);
+                  return status;
+                }
+                catch (Exception e) {
+                  log.error(e, "Error with task: %s", k8sTaskId);
+                  throw e;
+                }
+                finally {
+                  // publish task logs
+                  Path log = Files.createTempFile(task.getId(), "log");
+                  try {
+                    FileUtils.write(
+                        log.toFile(),
+                        client.getJobLogs(new K8sTaskId(task.getId())),
+                        StandardCharsets.UTF_8
+                    );
+                    taskLogPusher.pushTaskLog(task.getId(), log.toFile());
+                  }
+                  finally {
+                    Files.deleteIfExists(log);
+                  }
+                  client.cleanUpJob(new K8sTaskId(task.getId()));
+                  synchronized (tasks) {
+                    tasks.remove(task.getId());
+                  }
+                }
+              })
+          ));
+      return tasks.get(task.getId()).getResult();
+    }
+  }
+
+  JobResponse monitorJob(K8sTaskId k8sTaskId)
+  {
+    return monitorJob(client.getMainJobPod(k8sTaskId), k8sTaskId);
+  }
+
+  JobResponse monitorJob(Pod peonPod, K8sTaskId k8sTaskId)
+  {
+    if (peonPod == null) {
+      throw new ISE("Error in k8s launching peon pod for task %s", k8sTaskId);
+    }
+    return client.waitForJobCompletion(
+        k8sTaskId,
+        KubernetesTaskRunnerConfig.toMilliseconds(k8sConfig.maxTaskDuration),
+        TimeUnit.MILLISECONDS
+    );
+  }
+
+  @Override
+  public void updateStatus(Task task, TaskStatus status)
+  {
+    TaskRunnerUtils.notifyStatusChanged(listeners, task.getId(), status);
+  }
+
+  @Override
+  public void updateLocation(Task task, TaskLocation location)
+  {
+    TaskRunnerUtils.notifyLocationChanged(listeners, task.getId(), location);
+  }
+
+  @Override
+  public void shutdown(String taskid, String reason)
+  {
+    client.cleanUpJob(new K8sTaskId(taskid));
+  }
+
+
+  @Override
+  public Optional<InputStream> streamTaskReports(String taskid)
+  {
+    return Optional.absent();
+  }
+
+  @Override
+  public List<Pair<Task, ListenableFuture<TaskStatus>>> restore()
+  {
+    return ImmutableList.of();
+  }
+
+  @Override
+  public void start()
+  {
+    log.info("Starting cleanup executor for jobs older than 1 day....");
+    cleanupExecutor.scheduleAtFixedRate(
+        () ->
+            client.cleanCompletedJobsOlderThan(
+                KubernetesTaskRunnerConfig.toMilliseconds(k8sConfig.taskCleanupDelay),
+                TimeUnit.MILLISECONDS
+            ),
+        1,
+        KubernetesTaskRunnerConfig.toMilliseconds(k8sConfig.taskCleanupInterval),
+        TimeUnit.MILLISECONDS
+    );
+    log.debug("Started cleanup executor for jobs older than 1 day....");
+  }
+
+
+  @Override
+  public void stop()
+  {
+    log.info("Stopping...");
+    cleanupExecutor.shutdownNow();
+    log.info("Stopped...");
+  }
+
+  @Override
+  public Map<String, Long> getTotalTaskSlotCount()
+  {
+    return ImmutableMap.of("taskQueue", (long) taskQueueConfig.getMaxSize());
+  }
+
+  private List<String> javaOpts(Task task)
+  {
+    final List<String> javaOpts = new ArrayList<>();
+    Iterables.addAll(javaOpts, k8sConfig.javaOptsArray);
+
+    // Override task specific javaOpts
+    Object taskJavaOpts = task.getContextValue(
+        ForkingTaskRunnerConfig.JAVA_OPTS_PROPERTY
+    );
+    if (taskJavaOpts != null) {
+      Iterables.addAll(
+          javaOpts,
+          new QuotableWhiteSpaceSplitter((String) taskJavaOpts)
+      );
+    }
+
+    javaOpts.add(StringUtils.format("-Ddruid.port=%d", DruidK8sConstants.PORT));
+    javaOpts.add(StringUtils.format("-Ddruid.plaintextPort=%d", DruidK8sConstants.PORT));
+    javaOpts.add(StringUtils.format("-Ddruid.tlsPort=%d", node.isEnableTlsPort() ? DruidK8sConstants.TLS_PORT : -1));
+    javaOpts.add(StringUtils.format(
+        "-Ddruid.task.executor.tlsPort=%d",
+        node.isEnableTlsPort() ? DruidK8sConstants.TLS_PORT : -1
+    ));
+    javaOpts.add(StringUtils.format("-Ddruid.task.executor.enableTlsPort=%s", node.isEnableTlsPort())
+    );
+    return javaOpts;
+  }
+
+  private List<String> generateCommand(Task task)
+  {
+    final List<String> command = new ArrayList<>();
+    // must use deep storage shuffle now with k8s peon tasks.

Review Comment:
   Do we need this comment here?



##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/K8sTaskAdapter.java:
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.common;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.ContainerPort;
+import io.fabric8.kubernetes.api.model.EnvVar;
+import io.fabric8.kubernetes.api.model.EnvVarBuilder;
+import io.fabric8.kubernetes.api.model.EnvVarSourceBuilder;
+import io.fabric8.kubernetes.api.model.ObjectFieldSelector;
+import io.fabric8.kubernetes.api.model.ObjectMeta;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.PodSpec;
+import io.fabric8.kubernetes.api.model.PodTemplateSpec;
+import io.fabric8.kubernetes.api.model.Quantity;
+import io.fabric8.kubernetes.api.model.ResourceRequirementsBuilder;
+import io.fabric8.kubernetes.api.model.batch.v1.Job;
+import io.fabric8.kubernetes.api.model.batch.v1.JobBuilder;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.druid.indexing.common.task.Task;
+import org.apache.druid.java.util.common.HumanReadableBytes;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * This class transforms tasks to pods, and pods to tasks to assist with creating the job spec for a
+ * peon task.
+ * The two subclasses of this class are the SingleContainerTaskAdapter and the MultiContainerTaskAdapter
+ * This class runs on the overlord, to convert a task into a job, it will take its own podSpec (the current running overlord)
+ * keep volumees, secrets, env variables, config maps, etc. and add some additional information as well as provide a new
+ * command for running the task.
+ * The SingleContainerTaskAdapter only runs a task in a single container (no sidecars)
+ * The MultiContainerTaskAdapter runs with all the sidecars the current running overlord runs with.  Thus, it needs
+ * to add some extra coordination to shut down sidecar containers when the main pod exits.
+ */
+
+public abstract class K8sTaskAdapter implements TaskAdapter<Pod, Job>
+{
+
+  private static final EmittingLogger log = new EmittingLogger(K8sTaskAdapter.class);
+
+  protected final KubernetesClientApi client;
+  protected final KubernetesTaskRunnerConfig config;
+  protected final ObjectMapper mapper;
+
+  public K8sTaskAdapter(
+      KubernetesClientApi client,
+      KubernetesTaskRunnerConfig config,
+      ObjectMapper mapper
+  )
+  {
+    this.client = client;
+    this.config = config;
+    this.mapper = mapper;
+  }
+
+  @Override
+  public Job fromTask(Task task, PeonCommandContext context) throws IOException
+  {
+    String myPodName = System.getenv("HOSTNAME");
+    Pod pod = client.executeRequest(client -> client.pods().inNamespace(config.namespace).withName(myPodName).get());
+    return createJobFromPodSpec(pod.getSpec(), task, context);
+  }
+
+  @Override
+  public Task toTask(Pod from) throws IOException
+  {
+    List<EnvVar> envVars = from.getSpec().getContainers().get(0).getEnv();
+    Optional<EnvVar> taskJson = envVars.stream().filter(x -> "TASK_JSON".equals(x.getName())).findFirst();
+    String contents = taskJson.map(envVar -> taskJson.get().getValue()).orElse(null);
+    if (contents == null) {
+      throw new IOException("No TASK_JSON environment variable found in pod: " + from.getMetadata().getName());
+    }
+    return mapper.readValue(Base64Compression.decompressBase64(contents), Task.class);
+  }
+
+  @VisibleForTesting
+  public abstract Job createJobFromPodSpec(PodSpec podSpec, Task task, PeonCommandContext context) throws IOException;
+
+  protected Job buildJob(
+      K8sTaskId k8sTaskId,
+      Map<String, String> labels,
+      Map<String, String> annotations, PodTemplateSpec podTemplate
+  )
+  {
+    return new JobBuilder()
+        .withNewMetadata()
+        .withName(k8sTaskId.getK8sTaskId())
+        .addToLabels(labels)
+        .addToAnnotations(annotations)
+        .endMetadata()
+        .withNewSpec()
+        .withTemplate(podTemplate)
+        .withActiveDeadlineSeconds(config.maxTaskDuration.toStandardDuration().getStandardSeconds())
+        .withBackoffLimit(0)
+        .withTtlSecondsAfterFinished((int) config.taskCleanupDelay.toStandardDuration().getStandardSeconds())
+        .endSpec()
+        .build();
+  }
+
+  @VisibleForTesting
+  static Optional<Long> getJavaOptValueBytes(String qualifier, List<String> commands)
+  {
+    Long result = null;
+    Optional<String> lastHeapValue = commands.stream().filter(x -> x.startsWith(qualifier)).reduce((x, y) -> y);
+    if (lastHeapValue.isPresent()) {
+      result = HumanReadableBytes.parse(StringUtils.removeStart(lastHeapValue.get(), qualifier));
+    }
+    return Optional.ofNullable(result);
+  }
+
+  @VisibleForTesting
+  static long getContainerMemory(PeonCommandContext context)

Review Comment:
   Comments might be useful for this method on how the container memory is estimated.



##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java:
##########
@@ -0,0 +1,552 @@
+/*
+ * 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.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"
+    );
+  }
+
+
+  @Override
+  public Optional<InputStream> streamTaskLog(String taskid, long offset)
+  {
+    return client.getPeonLogs(new K8sTaskId(taskid));
+  }
+
+  @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()) {
+                    PeonCommandContext context = new PeonCommandContext(
+                        generateCommand(task),
+                        javaOpts(task),
+                        taskConfig.getTaskDir(task.getId()),
+                        node.isEnableTlsPort()
+                    );
+                    Job job = adapter.fromTask(task, context);
+                    log.info("Job created %s and ready to launch", k8sTaskId);
+                    Pod peonPod = client.launchJobAndWaitForStart(
+                        job,
+                        KubernetesTaskRunnerConfig.toMilliseconds(k8sConfig.k8sjobLaunchTimeout),
+                        TimeUnit.MILLISECONDS
+                    );
+                    log.info("Job %s launched in k8s", k8sTaskId);
+                    completedPhase = monitorJob(peonPod, k8sTaskId);
+                  } else {
+                    Job job = existingJob.get();
+                    if (job.getStatus().getActive() == null) {
+                      if (job.getStatus().getSucceeded() != null) {
+                        completedPhase = new JobResponse(job, PeonPhase.SUCCEEDED);
+                      } else {
+                        completedPhase = new JobResponse(job, PeonPhase.FAILED);
+                      }
+                    } else {
+                      // the job is active lets monitor it
+                      completedPhase = monitorJob(k8sTaskId);
+                    }
+                  }
+                  TaskStatus status;
+                  if (PeonPhase.SUCCEEDED.equals(completedPhase.getPhase())) {
+                    status = TaskStatus.success(task.getId());
+                  } else {
+                    status = TaskStatus.failure(
+                        task.getId(),
+                        "Task failed %s: " + k8sTaskId
+                    );
+                  }
+                  if (completedPhase.getJobDuration().isPresent()) {
+                    status = status.withDuration(completedPhase.getJobDuration().get());
+                  }
+                  updateStatus(task, status);
+                  return status;
+                }
+                catch (Exception e) {
+                  log.error(e, "Error with task: %s", k8sTaskId);
+                  throw e;
+                }
+                finally {
+                  // publish task logs
+                  Path log = Files.createTempFile(task.getId(), "log");
+                  try {
+                    FileUtils.write(
+                        log.toFile(),
+                        client.getJobLogs(new K8sTaskId(task.getId())),
+                        StandardCharsets.UTF_8
+                    );
+                    taskLogPusher.pushTaskLog(task.getId(), log.toFile());
+                  }
+                  finally {
+                    Files.deleteIfExists(log);
+                  }
+                  client.cleanUpJob(new K8sTaskId(task.getId()));
+                  synchronized (tasks) {
+                    tasks.remove(task.getId());
+                  }
+                }
+              })
+          ));
+      return tasks.get(task.getId()).getResult();
+    }
+  }
+
+  JobResponse monitorJob(K8sTaskId k8sTaskId)
+  {
+    return monitorJob(client.getMainJobPod(k8sTaskId), k8sTaskId);
+  }
+
+  JobResponse monitorJob(Pod peonPod, K8sTaskId k8sTaskId)
+  {
+    if (peonPod == null) {
+      throw new ISE("Error in k8s launching peon pod for task %s", k8sTaskId);
+    }
+    return client.waitForJobCompletion(
+        k8sTaskId,
+        KubernetesTaskRunnerConfig.toMilliseconds(k8sConfig.maxTaskDuration),
+        TimeUnit.MILLISECONDS
+    );
+  }
+
+  @Override
+  public void updateStatus(Task task, TaskStatus status)
+  {
+    TaskRunnerUtils.notifyStatusChanged(listeners, task.getId(), status);
+  }
+
+  @Override
+  public void updateLocation(Task task, TaskLocation location)
+  {
+    TaskRunnerUtils.notifyLocationChanged(listeners, task.getId(), location);
+  }
+
+  @Override
+  public void shutdown(String taskid, String reason)
+  {
+    client.cleanUpJob(new K8sTaskId(taskid));
+  }
+
+
+  @Override
+  public Optional<InputStream> streamTaskReports(String taskid)
+  {
+    return Optional.absent();
+  }
+
+  @Override
+  public List<Pair<Task, ListenableFuture<TaskStatus>>> restore()
+  {
+    return ImmutableList.of();
+  }
+
+  @Override
+  public void start()
+  {
+    log.info("Starting cleanup executor for jobs older than 1 day....");
+    cleanupExecutor.scheduleAtFixedRate(
+        () ->
+            client.cleanCompletedJobsOlderThan(
+                KubernetesTaskRunnerConfig.toMilliseconds(k8sConfig.taskCleanupDelay),
+                TimeUnit.MILLISECONDS
+            ),
+        1,
+        KubernetesTaskRunnerConfig.toMilliseconds(k8sConfig.taskCleanupInterval),
+        TimeUnit.MILLISECONDS
+    );
+    log.debug("Started cleanup executor for jobs older than 1 day....");
+  }
+
+
+  @Override
+  public void stop()
+  {
+    log.info("Stopping...");

Review Comment:
   Can these log lines be more descriptive?



##########
indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java:
##########
@@ -927,6 +932,23 @@ public Response apply(TaskRunner taskRunner)
           {
             if (taskRunner instanceof WorkerTaskRunner) {
               return Response.ok(((WorkerTaskRunner) taskRunner).getWorkers()).build();
+            } else if (taskRunner.isK8sTaskRunner()) {

Review Comment:
   Can we add a comment here describing that k8staskrunner specific handling was needed since it doesn't have the concept of workers and a dummy worker is being returned here?



##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesClientApi.java:
##########
@@ -0,0 +1,25 @@
+/*
+ * 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.common;
+
+public interface KubernetesClientApi

Review Comment:
   Can we add some javadocs here?



##########
indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractTask.java:
##########
@@ -125,6 +139,69 @@ protected AbstractTask(
     this(id, groupId, taskResource, dataSource, context, IngestionMode.NONE);
   }
 
+  @Nullable
+  public String setup(TaskToolbox toolbox) throws Exception

Review Comment:
   Can we add tests for this and the `cleanup` method? That should help with the Travis branch coverage failures.



##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.common;
+
+import com.google.common.base.Optional;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.batch.v1.Job;
+
+import java.io.InputStream;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+public interface KubernetesPeonClient

Review Comment:
   Can we add some docs here as well?



##########
distribution/docker/peon.sh:
##########
@@ -0,0 +1,153 @@
+#!/bin/sh
+
+#
+# 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.
+#
+
+# NOTE: this is a 'run' script for the stock tarball
+# It takes 1 required argument (the name of the service,
+# e.g. 'broker', 'historical' etc). Any additional arguments
+# are passed to that service.

Review Comment:
   Can we add a comment here specifying that this is similar to druid.sh and is used exclusively for the kubernetes-overlord-extension?



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org