georgew5656 commented on code in PR #14156:
URL: https://github.com/apache/druid/pull/14156#discussion_r1177969905


##########
extensions-contrib/kubernetes-overlord-extensions/pom.xml:
##########
@@ -157,6 +157,18 @@
       <version>5.8.2</version>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-core</artifactId>

Review Comment:
   is this actually needed? it looks like its deprecated and I don't see it used



##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java:
##########
@@ -89,174 +69,128 @@
 
 /**
  * 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<>();
+  protected final ConcurrentHashMap<String, KubernetesWorkItem> tasks = new 
ConcurrentHashMap<>();
   protected final TaskAdapter adapter;
-  protected final KubernetesPeonClient client;
 
-  private final ObjectMapper mapper;
-  private final KubernetesTaskRunnerConfig k8sConfig;
-  private final TaskQueueConfig taskQueueConfig;
-  private final TaskLogs taskLogs;
+  private final KubernetesPeonClient client;
+  private final KubernetesTaskRunnerConfig config;
   private final ListeningExecutorService exec;
   private final HttpClient httpClient;
+  private final PeonLifecycleFactory peonLifecycleFactory;
 
 
   public KubernetesTaskRunner(
-      ObjectMapper mapper,
       TaskAdapter adapter,
-      KubernetesTaskRunnerConfig k8sConfig,
-      TaskQueueConfig taskQueueConfig,
-      TaskLogs taskLogs,
+      KubernetesTaskRunnerConfig config,
       KubernetesPeonClient client,
-      HttpClient httpClient
+      HttpClient httpClient,
+      PeonLifecycleFactory peonLifecycleFactory
   )
   {
-    this.mapper = mapper;
     this.adapter = adapter;
-    this.k8sConfig = k8sConfig;
-    this.taskQueueConfig = taskQueueConfig;
-    this.taskLogs = taskLogs;
+    this.config = config;
     this.client = client;
     this.httpClient = httpClient;
+    this.peonLifecycleFactory = peonLifecycleFactory;
     this.cleanupExecutor = Executors.newScheduledThreadPool(1);
     this.exec = MoreExecutors.listeningDecorator(
-        Execs.multiThreaded(taskQueueConfig.getMaxSize(), "k8s-task-runner-%d")
-    );
-    Preconditions.checkArgument(

Review Comment:
   we probably still want to check that capacity is less than the max queue size



##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java:
##########
@@ -89,174 +69,128 @@
 
 /**
  * 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

Review Comment:
   it seems to me this comment still applies, since the running k8s jobs will 
still be rehydrated into the KubernetesTaskRunner by the restore() call



##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java:
##########
@@ -19,37 +19,244 @@
 
 package org.apache.druid.k8s.overlord.common;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Optional;
 import io.fabric8.kubernetes.api.model.Pod;
 import io.fabric8.kubernetes.api.model.batch.v1.Job;
+import io.fabric8.kubernetes.client.KubernetesClient;
+import io.fabric8.kubernetes.client.dsl.LogWatch;
+import org.apache.druid.java.util.common.RetryUtils;
+import org.apache.druid.java.util.emitter.EmittingLogger;
 
 import java.io.InputStream;
+import java.sql.Timestamp;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 
-/**
- * A Kubernetes client wrapper to assist with peon task managment.
- * It provides a high level api to retreive jobs, launch jobs, delete jobs and 
various other
- * tasks like getting task logs, listing all active tasks.
- */
-public interface KubernetesPeonClient
+public class KubernetesPeonClient
 {
+  private static final EmittingLogger log = new 
EmittingLogger(KubernetesPeonClient.class);
+
+  private final KubernetesClientApi clientApi;
+  private final String namespace;
+  private final boolean debugJobs;
+
+  public KubernetesPeonClient(KubernetesClientApi clientApi, String namespace, 
boolean debugJobs)
+  {
+    this.clientApi = clientApi;
+    this.namespace = namespace;
+    this.debugJobs = debugJobs;
+  }
+
+  public Optional<Job> getPeonJob(K8sTaskId taskId)
+  {
+    return clientApi.executeRequest(
+        client -> {
+          return Optional.fromNullable(
+              client.batch()
+                    .v1()
+                    .jobs()
+                    .inNamespace(namespace)
+                    .withName(taskId.getK8sTaskId())
+                    .get());
+        }
+    );
+  }
+
+  public Pod launchPeonJobAndWaitForStart(Job job, long howLong, TimeUnit 
timeUnit)
+  {
+    long start = System.currentTimeMillis();
+    // launch job
+    return clientApi.executeRequest(client -> {
+      client.batch().v1().jobs().inNamespace(namespace).resource(job).create();
+      K8sTaskId taskId = new K8sTaskId(job.getMetadata().getName());
+      log.info("Successfully submitted job: %s ... waiting for job to launch", 
taskId);
+      // wait until the pod is running or complete or failed, any of those is 
fine
+      // TODO: I believe we can do the following instead and remove 
getPeonPodWithRetries
+      // Pod result = client.pods()
+      //     .inNamespace(namespace)
+      //     .withLabel("job-name", job.getMetadata().getName())
+      //     .waitUntilCondition(pod -> {
+      //       if (pod == null || pod.getStatus() == null) {
+      //         return false;
+      //       }
+      //       return pod.getStatus().getPodIP() != null;
+      //     }, howLong, timeUnit);
+      Pod mainPod = getPeonPodWithRetries(taskId);
+      Pod result = 
client.pods().inNamespace(namespace).withName(mainPod.getMetadata().getName())

Review Comment:
   this is unrelated but i think there might be a race condition here still 
(i've never encountered it) where a task is shutdown during this logic and it 
gets stuck, but I don't think we should worry about it until it happens



##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/PeonLifecycleFactory.java:
##########
@@ -0,0 +1,27 @@
+/*
+ * 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 org.apache.druid.indexing.common.task.Task;
+
+public interface PeonLifecycleFactory
+{
+  KubernetesPeonLifecycle build(Task task);

Review Comment:
   is there a reason we need this interface right now considering its in the 
k8s overlord extensions directory? feels pretty similar to the 
KubernetesPeonClient interface that was removed in this PR



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