diff --git a/extensions-contrib/kubernetes-overlord-extensions/pom.xml b/extensions-contrib/kubernetes-overlord-extensions/pom.xml
index 2c377398cff..3ebcbfbf8d7 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/pom.xml
+++ b/extensions-contrib/kubernetes-overlord-extensions/pom.xml
@@ -157,6 +157,18 @@
5.8.2
test
+
+ org.hamcrest
+ hamcrest-core
+ 2.2
+ test
+
+
+ org.hamcrest
+ hamcrest
+ 2.2
+ test
+
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/K8sOverlordModule.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java
similarity index 98%
rename from extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/K8sOverlordModule.java
rename to extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java
index c176330b15a..ee7ae1b3292 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/K8sOverlordModule.java
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java
@@ -40,7 +40,7 @@ import org.apache.druid.tasklogs.TaskLogPusher;
import org.apache.druid.tasklogs.TaskLogs;
@LoadScope(roles = NodeRole.OVERLORD_JSON_NAME)
-public class K8sOverlordModule implements DruidModule
+public class KubernetesOverlordModule implements DruidModule
{
@Override
public void configure(Binder binder)
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java
new file mode 100644
index 00000000000..9825469f678
--- /dev/null
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java
@@ -0,0 +1,292 @@
+/*
+ * 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.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.PodStatus;
+import io.fabric8.kubernetes.api.model.batch.v1.Job;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
+import org.apache.druid.indexer.TaskLocation;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.indexing.common.task.Task;
+import org.apache.druid.java.util.common.StringUtils;
+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.tasklogs.TaskLogs;
+
+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.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * This class is a wrapper per Druid task responsible for managing the task lifecycle
+ * once it has been transformed into a K8s Job and submitted by the KubernetesTaskRunner.
+ *
+ * This includes actually submitting the waiting Job to K8s,
+ * waiting for the Job to report completion,
+ * joining an already running Job and waiting for it to report completion,
+ * shutting down a Job, including queued jobs that have not been submitted to K8s,
+ * streaming task logs for a running job
+ */
+public class KubernetesPeonLifecycle
+{
+ private static final EmittingLogger log = new EmittingLogger(KubernetesPeonLifecycle.class);
+
+ protected enum State
+ {
+ /** Lifecycle's state before {@link #run(Job, long, long)} or {@link #join(long)} is called. */
+ NOT_STARTED,
+ /** Lifecycle's state since {@link #run(Job, long, long)} is called. */
+ PENDING,
+ /** Lifecycle's state since {@link #join(long)} is called. */
+ RUNNING,
+ /** Lifecycle's state since the task has completed. */
+ STOPPED
+ }
+
+ private final AtomicReference state = new AtomicReference<>(State.NOT_STARTED);
+ private final K8sTaskId taskId;
+ private final TaskLogs taskLogs;
+ private final KubernetesPeonClient kubernetesClient;
+ private final ObjectMapper mapper;
+
+ protected KubernetesPeonLifecycle(
+ Task task,
+ KubernetesPeonClient kubernetesClient,
+ TaskLogs taskLogs,
+ ObjectMapper mapper
+ )
+ {
+ this.taskId = new K8sTaskId(task);
+ this.kubernetesClient = kubernetesClient;
+ this.taskLogs = taskLogs;
+ this.mapper = mapper;
+ }
+
+ /**
+ * Run a Kubernetes Job
+ *
+ * @param job
+ * @param launchTimeout
+ * @param timeout
+ * @return
+ * @throws IllegalStateException
+ */
+ protected synchronized TaskStatus run(Job job, long launchTimeout, long timeout) throws IllegalStateException
+ {
+ try {
+ Preconditions.checkState(
+ state.compareAndSet(State.NOT_STARTED, State.PENDING),
+ "Task [%s] failed to run: invalid peon lifecycle state transition [%s]->[%s]",
+ taskId.getOriginalTaskId(),
+ state.get(),
+ State.PENDING
+ );
+
+ kubernetesClient.launchPeonJobAndWaitForStart(
+ job,
+ launchTimeout,
+ TimeUnit.MILLISECONDS
+ );
+
+ return join(timeout);
+ }
+ finally {
+ state.set(State.STOPPED);
+ }
+ }
+
+ /**
+ * Join existing Kubernetes Job
+ *
+ * @param timeout
+ * @return
+ * @throws IllegalStateException
+ */
+ protected synchronized TaskStatus join(long timeout) throws IllegalStateException
+ {
+ try {
+ Preconditions.checkState(
+ (
+ state.compareAndSet(State.NOT_STARTED, State.RUNNING) ||
+ state.compareAndSet(State.PENDING, State.RUNNING)
+ ),
+ "Task [%s] failed to join: invalid peon lifecycle state transition [%s]->[%s]",
+ taskId.getOriginalTaskId(),
+ state.get(),
+ State.RUNNING
+ );
+
+ JobResponse jobResponse = kubernetesClient.waitForPeonJobCompletion(
+ taskId,
+ timeout,
+ TimeUnit.MILLISECONDS
+ );
+
+ saveLogs();
+
+ return getTaskStatus(jobResponse.getJobDuration());
+ }
+ finally {
+ try {
+ shutdown();
+ }
+ catch (Exception e) {
+ log.warn(e, "Task [%s] shutdown failed", taskId);
+ }
+
+ state.set(State.STOPPED);
+ }
+ }
+
+ /**
+ * Shutdown Kubernetes job and associated pods
+ *
+ * Behavior: Deletes Kubernetes job which a kill signal to the containers running in
+ * the job's associated pod.
+ *
+ * Task state will be set by the thread running the run(...) or join(...) commands
+ */
+ protected void shutdown()
+ {
+ if (State.PENDING.equals(state.get()) || State.RUNNING.equals(state.get())) {
+ kubernetesClient.deletePeonJob(taskId);
+ }
+ }
+
+ /**
+ * Stream logs from the Kubernetes pod running the peon process
+ *
+ * @return
+ */
+ protected Optional streamLogs()
+ {
+ if (!State.RUNNING.equals(state.get())) {
+ return Optional.absent();
+ }
+ return kubernetesClient.getPeonLogs(taskId);
+ }
+
+ /**
+ * Get peon lifecycle state
+ *
+ * @return
+ */
+ protected State getState()
+ {
+ return state.get();
+ }
+
+ /**
+ * Get task location for the Kubernetes pod running the peon process
+ *
+ * @return
+ */
+ protected TaskLocation getTaskLocation()
+ {
+ if (!State.RUNNING.equals(state.get())) {
+ log.debug("Can't get task location for non-running job. [%s]", taskId.getOriginalTaskId());
+ return TaskLocation.unknown();
+ }
+
+ Optional maybePod = kubernetesClient.getPeonPod(taskId);
+ if (!maybePod.isPresent()) {
+ return TaskLocation.unknown();
+ }
+
+ Pod pod = maybePod.get();
+ PodStatus podStatus = pod.getStatus();
+
+ if (podStatus == null || podStatus.getPodIP() == null) {
+ return TaskLocation.unknown();
+ }
+
+ return TaskLocation.create(
+ podStatus.getPodIP(),
+ DruidK8sConstants.PORT,
+ DruidK8sConstants.TLS_PORT,
+ Boolean.parseBoolean(pod.getMetadata()
+ .getAnnotations()
+ .getOrDefault(DruidK8sConstants.TLS_ENABLED, "false")
+ )
+ );
+ }
+
+ private TaskStatus getTaskStatus(long duration)
+ {
+ TaskStatus taskStatus;
+ try {
+ Optional maybeTaskStatusStream = taskLogs.streamTaskStatus(taskId.getOriginalTaskId());
+ if (maybeTaskStatusStream.isPresent()) {
+ taskStatus = mapper.readValue(
+ IOUtils.toString(maybeTaskStatusStream.get(), StandardCharsets.UTF_8),
+ TaskStatus.class
+ );
+ } else {
+ taskStatus = TaskStatus.failure(taskId.getOriginalTaskId(), "task status not found");
+ }
+ }
+ catch (IOException e) {
+ log.error(e, "Failed to load task status for task [%s]", taskId.getOriginalTaskId());
+ taskStatus = TaskStatus.failure(
+ taskId.getOriginalTaskId(),
+ StringUtils.format("error loading status: %s", e.getMessage())
+ );
+ }
+
+ return taskStatus.withDuration(duration);
+ }
+
+ private void saveLogs()
+ {
+ try {
+ Path file = Files.createTempFile(taskId.getOriginalTaskId(), "log");
+ try {
+ Optional maybeLogStream = streamLogs();
+ if (maybeLogStream.isPresent()) {
+ FileUtils.copyInputStreamToFile(maybeLogStream.get(), file.toFile());
+ } else {
+ log.debug("Log stream not found for %s", taskId.getOriginalTaskId());
+ }
+ taskLogs.pushTaskLog(taskId.getOriginalTaskId(), file.toFile());
+ }
+ catch (IOException e) {
+ log.error(e, "Failed to stream logs for task [%s]", taskId.getOriginalTaskId());
+ }
+ finally {
+ Files.deleteIfExists(file);
+ }
+ }
+ catch (IOException e) {
+ log.warn(e, "Failed to manage temporary log file for task [%s]", taskId.getOriginalTaskId());
+ }
+ }
+}
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java
new file mode 100644
index 00000000000..2f2375789be
--- /dev/null
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java
@@ -0,0 +1,54 @@
+/*
+ * 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.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.druid.indexing.common.task.Task;
+import org.apache.druid.k8s.overlord.common.KubernetesPeonClient;
+import org.apache.druid.tasklogs.TaskLogs;
+
+public class KubernetesPeonLifecycleFactory implements PeonLifecycleFactory
+{
+ private final KubernetesPeonClient client;
+ private final TaskLogs taskLogs;
+ private final ObjectMapper mapper;
+
+ public KubernetesPeonLifecycleFactory(
+ KubernetesPeonClient client,
+ TaskLogs taskLogs,
+ ObjectMapper mapper
+ )
+ {
+ this.client = client;
+ this.taskLogs = taskLogs;
+ this.mapper = mapper;
+ }
+
+ @Override
+ public KubernetesPeonLifecycle build(Task task)
+ {
+ return new KubernetesPeonLifecycle(
+ task,
+ client,
+ taskLogs,
+ mapper
+ );
+ }
+}
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java
index a360dc09f06..fe2f4be3711 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java
@@ -19,20 +19,15 @@
package org.apache.druid.k8s.overlord;
-import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.api.client.util.Lists;
+import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
-import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
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.commons.io.IOUtils;
import org.apache.druid.indexer.RunnerTaskState;
import org.apache.druid.indexer.TaskLocation;
import org.apache.druid.indexer.TaskStatus;
@@ -42,42 +37,27 @@ 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.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.java.util.http.client.HttpClient;
import org.apache.druid.java.util.http.client.Request;
import org.apache.druid.java.util.http.client.response.InputStreamResponseHandler;
-import org.apache.druid.k8s.overlord.common.DruidK8sConstants;
-import org.apache.druid.k8s.overlord.common.JobResponse;
-import org.apache.druid.k8s.overlord.common.JobStatus;
-import org.apache.druid.k8s.overlord.common.K8sTaskId;
import org.apache.druid.k8s.overlord.common.KubernetesPeonClient;
-import org.apache.druid.k8s.overlord.common.KubernetesResourceNotFoundException;
-import org.apache.druid.k8s.overlord.common.PeonPhase;
-import org.apache.druid.k8s.overlord.common.TaskAdapter;
+import org.apache.druid.k8s.overlord.taskadapter.TaskAdapter;
import org.apache.druid.tasklogs.TaskLogStreamer;
-import org.apache.druid.tasklogs.TaskLogs;
import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.joda.time.DateTime;
import javax.annotation.Nullable;
import java.io.IOException;
import java.io.InputStream;
import java.net.URL;
-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.Objects;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.ExecutionException;
@@ -89,174 +69,142 @@ import java.util.stream.Collectors;
/**
* 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.
+ * The KubernetesTaskRunner runs tasks by transforming the task spec into a K8s Job spec based
+ * on the TaskAdapter it is configured with. The KubernetesTaskRunner has a pool of threads
+ * (configurable with the capacity configuration) to track the jobs (1 thread tracks 1 job).
+ *
+ * Each thread calls down to the KubernetesPeonLifecycle class to submit the Job to K8s and then
+ * waits for the lifecycle class to report back with the Job's status (success/failure).
+ *
+ * If there are not enough threads in the thread pool to execute and wait for a job, then the
+ * task is put in a queue and left in WAITING state until another task completes.
+ *
+ * When the KubernetesTaskRunner comes up it attempts to restore its internal mapping of tasks
+ * from Kubernetes by listing running jobs and calling join on each job, which spawns a thread to
+ * wait for the fabric8 client library to report back, similar to what happens when a new
+ * job is run.
*/
-
public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner
{
-
private static final EmittingLogger log = new EmittingLogger(KubernetesTaskRunner.class);
private final CopyOnWriteArrayList> listeners = new CopyOnWriteArrayList<>();
// to cleanup old jobs that might not have been deleted.
private final ScheduledExecutorService cleanupExecutor;
- protected final ConcurrentHashMap tasks = new ConcurrentHashMap<>();
+ protected final ConcurrentHashMap 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(
- taskQueueConfig.getMaxSize() < Integer.MAX_VALUE,
- "The task queue bounds how many concurrent k8s tasks you can have"
+ Execs.multiThreaded(config.getCapacity(), "k8s-task-runner-%d")
);
}
-
@Override
public Optional streamTaskLog(String taskid, long offset)
{
- return client.getPeonLogs(new K8sTaskId(taskid));
+ KubernetesWorkItem workItem = tasks.get(taskid);
+ if (workItem == null) {
+ return Optional.absent();
+ }
+ return workItem.streamTaskLogs();
}
@Override
public ListenableFuture run(Task task)
{
synchronized (tasks) {
- tasks.computeIfAbsent(
- task.getId(), k -> new K8sWorkItem(
- client,
- task,
- exec.submit(() -> {
- K8sTaskId k8sTaskId = new K8sTaskId(task);
- try {
- JobResponse completedPhase;
- Optional existingJob = client.jobExists(k8sTaskId);
- if (!existingJob.isPresent()) {
- Job job = adapter.fromTask(task);
- 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 = getTaskStatus(k8sTaskId, completedPhase);
- 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 {
- Optional logStream = client.getPeonLogs(new K8sTaskId(task.getId()));
- if (logStream.isPresent()) {
- FileUtils.copyInputStreamToFile(logStream.get(), log.toFile());
- }
- taskLogs.pushTaskLog(task.getId(), log.toFile());
- }
- finally {
- Files.deleteIfExists(log);
- }
- client.cleanUpJob(new K8sTaskId(task.getId()));
- synchronized (tasks) {
- tasks.remove(task.getId());
- }
- }
- })
- ));
+ tasks.computeIfAbsent(task.getId(), k -> new KubernetesWorkItem(task, exec.submit(() -> runTask(task))));
return tasks.get(task.getId()).getResult();
}
}
- JobResponse monitorJob(K8sTaskId k8sTaskId)
+ protected ListenableFuture joinAsync(Task task)
{
- 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);
+ synchronized (tasks) {
+ tasks.computeIfAbsent(task.getId(), k -> new KubernetesWorkItem(task, exec.submit(() -> joinTask(task))));
+ return tasks.get(task.getId()).getResult();
}
- return client.waitForJobCompletion(
- k8sTaskId,
- KubernetesTaskRunnerConfig.toMilliseconds(k8sConfig.maxTaskDuration),
- TimeUnit.MILLISECONDS
- );
}
- private TaskStatus getTaskStatus(K8sTaskId task, JobResponse jobResponse) throws IOException
+ private TaskStatus runTask(Task task)
{
- Optional maybeTaskStatusStream = taskLogs.streamTaskStatus(task.getOriginalTaskId());
- if (maybeTaskStatusStream.isPresent()) {
- String taskStatus = IOUtils.toString(maybeTaskStatusStream.get(), StandardCharsets.UTF_8);
- return mapper.readValue(taskStatus, TaskStatus.class);
- } else if (PeonPhase.SUCCEEDED.equals(jobResponse.getPhase())) {
- // fallback to behavior before the introduction of task status streaming for backwards compatibility
- return TaskStatus.success(task.getOriginalTaskId());
- } else if (Objects.isNull(jobResponse.getJob())) {
- return TaskStatus.failure(
- task.getOriginalTaskId(),
- StringUtils.format("Task [%s] failed kubernetes job disappeared before completion", task.getOriginalTaskId())
- );
- } else {
- return TaskStatus.failure(
- task.getOriginalTaskId(),
- StringUtils.format("Task [%s] failed", task.getOriginalTaskId())
- );
+ return doTask(task, true);
+ }
+
+ private TaskStatus joinTask(Task task)
+ {
+ return doTask(task, false);
+ }
+
+ @VisibleForTesting
+ protected TaskStatus doTask(Task task, boolean run)
+ {
+ KubernetesPeonLifecycle peonLifecycle = peonLifecycleFactory.build(task);
+
+ synchronized (tasks) {
+ KubernetesWorkItem workItem = tasks.get(task.getId());
+
+ if (workItem == null) {
+ throw new ISE("Task [%s] disappeared", task.getId());
+ }
+
+ if (workItem.isShutdownRequested()) {
+ throw new ISE("Task [%s] has been shut down", task.getId());
+ }
+
+ workItem.setKubernetesPeonLifecycle(peonLifecycle);
+ }
+
+ try {
+ TaskStatus taskStatus;
+ if (run) {
+ taskStatus = peonLifecycle.run(
+ adapter.fromTask(task),
+ config.getTaskLaunchTimeout().toStandardDuration().getMillis(),
+ config.getTaskTimeout().toStandardDuration().getMillis()
+ );
+ } else {
+ taskStatus = peonLifecycle.join(
+ config.getTaskTimeout().toStandardDuration().getMillis()
+ );
+ }
+
+ updateStatus(task, taskStatus);
+
+ return taskStatus;
+ }
+
+ catch (Exception e) {
+ log.error(e, "Task [%s] execution caught an exception", task.getId());
+ throw new RuntimeException(e);
+ }
+
+ finally {
+ synchronized (tasks) {
+ tasks.remove(task.getId());
+ }
}
}
@@ -275,13 +223,22 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner
@Override
public void shutdown(String taskid, String reason)
{
- client.cleanUpJob(new K8sTaskId(taskid));
+ log.info("Shutdown [%s] because [%s]", taskid, reason);
+
+ KubernetesWorkItem workItem = tasks.get(taskid);
+
+ if (workItem == null) {
+ log.info("Ignoring request to cancel unknown task [%s]", taskid);
+ return;
+ }
+
+ workItem.shutdown();
}
@Override
public Optional streamTaskReports(String taskid) throws IOException
{
- final K8sWorkItem workItem = tasks.get(taskid);
+ final KubernetesWorkItem workItem = tasks.get(taskid);
if (workItem == null) {
return Optional.absent();
}
@@ -318,7 +275,17 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner
@Override
public List>> restore()
{
- return ImmutableList.of();
+ List>> tasks = new ArrayList<>();
+ for (Job job : client.getPeonJobs()) {
+ try {
+ Task task = adapter.toTask(job);
+ tasks.add(Pair.of(task, joinAsync(task)));
+ }
+ catch (IOException e) {
+ log.error(e, "Error deserializing task from job [%s]", job.getMetadata().getName());
+ }
+ }
+ return tasks;
}
@Override
@@ -326,53 +293,38 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner
{
cleanupExecutor.scheduleAtFixedRate(
() ->
- client.cleanCompletedJobsOlderThan(
- KubernetesTaskRunnerConfig.toMilliseconds(k8sConfig.taskCleanupDelay),
+ client.deleteCompletedPeonJobsOlderThan(
+ config.getTaskCleanupDelay().toStandardDuration().getMillis(),
TimeUnit.MILLISECONDS
),
1,
- KubernetesTaskRunnerConfig.toMilliseconds(k8sConfig.taskCleanupInterval),
+ config.getTaskCleanupInterval().toStandardDuration().getMillis(),
TimeUnit.MILLISECONDS
);
- log.debug("Started cleanup executor for jobs older than %s....", k8sConfig.taskCleanupDelay);
+ log.debug("Started cleanup executor for jobs older than %s...", config.getTaskCleanupDelay());
}
@Override
public void stop()
{
- log.info("Stopping KubernetesTaskRunner");
+ log.debug("Stopping KubernetesTaskRunner");
cleanupExecutor.shutdownNow();
- log.info("Stopped KubernetesTaskRunner");
+ log.debug("Stopped KubernetesTaskRunner");
}
@Override
public Map getTotalTaskSlotCount()
{
- return ImmutableMap.of("taskQueue", (long) taskQueueConfig.getMaxSize());
+ return ImmutableMap.of("taskQueue", (long) config.getCapacity());
}
@Override
public Collection extends TaskRunnerWorkItem> getKnownTasks()
{
- List result = new ArrayList<>();
- for (Job existingTask : client.listAllPeonJobs()) {
- try {
- Task task = adapter.toTask(existingTask);
- ListenableFuture future = run(task);
- result.add(new K8sWorkItem(
- client,
- task,
- future,
- DateTimes.of(existingTask.getMetadata().getCreationTimestamp())
- ));
- }
- catch (IOException e) {
- log.error("Error deserializing task from pod: " + existingTask.getMetadata().getName());
- }
-
+ synchronized (tasks) {
+ return Lists.newArrayList(tasks.values());
}
- return result;
}
@@ -439,121 +391,36 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner
}
@Override
- @SuppressForbidden(reason = "Sets#newHashSet")
public Collection getRunningTasks()
{
- List result = new ArrayList<>();
- for (Job existingTask : client.listAllPeonJobs().stream().filter(JobStatus::isActive).collect(Collectors.toSet())) {
- try {
- Task task = adapter.toTask(existingTask);
- ListenableFuture future = run(task);
- result.add(new K8sWorkItem(
- client,
- task,
- future,
- DateTime.parse(existingTask.getMetadata().getCreationTimestamp())
- ));
- }
- catch (IOException e) {
- log.error("Error deserializing task from pod: " + existingTask.getMetadata().getName());
- }
+ synchronized (tasks) {
+ return tasks.values()
+ .stream()
+ .filter(KubernetesWorkItem::isRunning)
+ .collect(Collectors.toList());
}
- return result;
}
@Override
public Collection getPendingTasks()
{
- // the task queue limits concurrent tasks, we fire off to k8s right away
- // thus nothing is really "pending"
- return new ArrayList<>();
+ synchronized (tasks) {
+ return tasks.values()
+ .stream()
+ .filter(KubernetesWorkItem::isPending)
+ .collect(Collectors.toList());
+ }
}
@Nullable
@Override
public RunnerTaskState getRunnerTaskState(String taskId)
{
- Pod item = client.getMainJobPod(new K8sTaskId(taskId));
- if (item == null) {
+ KubernetesWorkItem workItem = tasks.get(taskId);
+ if (workItem == null) {
return null;
- } else {
- PeonPhase phase = PeonPhase.getPhaseFor(item);
- switch (phase) {
- case PENDING:
- return RunnerTaskState.PENDING;
- case RUNNING:
- return RunnerTaskState.RUNNING;
- default:
- return RunnerTaskState.WAITING;
- }
}
+
+ return workItem.getRunnerTaskState();
}
-
- public static class K8sWorkItem extends TaskRunnerWorkItem
- {
- private final Task task;
- private KubernetesPeonClient client;
-
- public K8sWorkItem(KubernetesPeonClient client, Task task, ListenableFuture statusFuture)
- {
- super(task.getId(), statusFuture);
- this.task = task;
- this.client = client;
- }
-
- public K8sWorkItem(
- KubernetesPeonClient client,
- Task task,
- ListenableFuture statusFuture,
- DateTime createdTime
- )
- {
- super(task.getId(), statusFuture, createdTime, createdTime);
- this.task = task;
- this.client = client;
- }
-
- @Override
- public TaskLocation getLocation()
- {
- K8sTaskId taskId = new K8sTaskId(task.getId());
- try {
- Pod mainPod = client.getMainJobPod(new K8sTaskId(task.getId()));
- if (mainPod.getStatus() == null || mainPod.getStatus().getPodIP() == null) {
- return TaskLocation.unknown();
- }
- boolean tlsEnabled = Boolean.parseBoolean(
- mainPod.getMetadata()
- .getAnnotations()
- .getOrDefault(DruidK8sConstants.TLS_ENABLED, "false"));
- return TaskLocation.create(
- mainPod.getStatus().getPodIP(),
- DruidK8sConstants.PORT,
- DruidK8sConstants.TLS_PORT,
- tlsEnabled
- );
- }
- catch (KubernetesResourceNotFoundException e) {
- log.debug(e, "Error getting task location for task %s", taskId);
- return TaskLocation.unknown();
- }
- catch (Exception e) {
- log.error(e, "Error getting task location for task %s", taskId);
- return TaskLocation.unknown();
- }
- }
-
- @Override
- public String getTaskType()
- {
- return task.getType();
- }
-
- @Override
- public String getDataSource()
- {
- return task.getDataSource();
- }
- }
-
}
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java
index d83b81b1596..0d67c55b30a 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java
@@ -20,24 +20,26 @@
package org.apache.druid.k8s.overlord;
import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.collect.Lists;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.commons.lang3.ObjectUtils;
import org.joda.time.Period;
+import javax.annotation.Nonnull;
+import javax.validation.constraints.Max;
+import javax.validation.constraints.Min;
import javax.validation.constraints.NotNull;
-import java.util.ArrayList;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
public class KubernetesTaskRunnerConfig
{
-
@JsonProperty
@NotNull
- public String namespace;
+ private String namespace;
@JsonProperty
- public boolean debugJobs = false;
+ private boolean debugJobs = false;
/**
* Deprecated, please specify adapter type runtime property instead
@@ -46,86 +48,381 @@ public class KubernetesTaskRunnerConfig
*/
@Deprecated
@JsonProperty
- public boolean sidecarSupport = false;
+ private boolean sidecarSupport = false;
@JsonProperty
// if this is not set, then the first container in your pod spec is assumed to be the overlord container.
// usually this is fine, but when you are dynamically adding sidecars like istio, the service mesh could
// in fact place the istio-proxy container as the first container. Thus you would specify this value to
// the name of your primary container. eg) druid-overlord
- public String primaryContainerName = null;
+ private String primaryContainerName = null;
@JsonProperty
// for multi-container jobs, we need this image to shut down sidecars after the main container
// has completed
- public String kubexitImage = "karlkfi/kubexit:v0.3.2";
+ private String kubexitImage = "karlkfi/kubexit:v0.3.2";
// how much time to wait for preStop hooks to execute
// lower number speeds up pod termination time to release locks
// faster, defaults to your k8s setup, usually 30 seconds.
- public Long graceTerminationPeriodSeconds = null;
+ private Long graceTerminationPeriodSeconds = null;
@JsonProperty
// disable using http / https proxy environment variables
- public boolean disableClientProxy;
+ private boolean disableClientProxy;
@JsonProperty
@NotNull
- public Period maxTaskDuration = new Period("PT4H");
+ private Period maxTaskDuration = new Period("PT4H");
@JsonProperty
@NotNull
// how long to wait for the jobs to be cleaned up.
- public Period taskCleanupDelay = new Period("P2D");
+ private Period taskCleanupDelay = new Period("P2D");
@JsonProperty
@NotNull
// interval for k8s job cleanup to run
- public Period taskCleanupInterval = new Period("PT10m");
+ private Period taskCleanupInterval = new Period("PT10m");
@JsonProperty
@NotNull
// how long to wait for the peon k8s job to launch
- public Period k8sjobLaunchTimeout = new Period("PT1H");
+ private Period k8sjobLaunchTimeout = new Period("PT1H");
@JsonProperty
// ForkingTaskRunner inherits the monitors from the MM, in k8s mode
// the peon inherits the monitors from the overlord, so if someone specifies
// a TaskCountStatsMonitor in the overlord for example, the peon process
// fails because it can not inject this monitor in the peon process.
- public List peonMonitors = new ArrayList<>();
+ private List peonMonitors = ImmutableList.of();
@JsonProperty
@NotNull
- public List javaOptsArray = new ArrayList<>();
+ private List javaOptsArray = ImmutableList.of();
@JsonProperty
@NotNull
- public String classpath = System.getProperty("java.class.path");
+ private Map labels = ImmutableMap.of();
@JsonProperty
@NotNull
- public Map labels = new HashMap<>();
+ private Map annotations = ImmutableMap.of();
@JsonProperty
+ @Min(1)
+ @Max(Integer.MAX_VALUE)
@NotNull
- public Map annotations = new HashMap<>();
+ private Integer capacity = Integer.MAX_VALUE;
- @JsonProperty
- @NotNull
- List allowedPrefixes = Lists.newArrayList(
- "com.metamx",
- "druid",
- "org.apache.druid",
- "user.timezone",
- "file.encoding",
- "java.io.tmpdir",
- "hadoop"
- );
-
- public static long toMilliseconds(Period period)
+ public KubernetesTaskRunnerConfig()
{
- return period.toStandardDuration().getMillis();
}
+ private KubernetesTaskRunnerConfig(
+ @Nonnull String namespace,
+ boolean debugJobs,
+ boolean sidecarSupport,
+ String primaryContainerName,
+ String kubexitImage,
+ Long graceTerminationPeriodSeconds,
+ boolean disableClientProxy,
+ Period maxTaskDuration,
+ Period taskCleanupDelay,
+ Period taskCleanupInterval,
+ Period k8sjobLaunchTimeout,
+ List peonMonitors,
+ List javaOptsArray,
+ Map labels,
+ Map annotations,
+ Integer capacity
+ )
+ {
+ this.namespace = namespace;
+ this.debugJobs = ObjectUtils.defaultIfNull(
+ debugJobs,
+ this.debugJobs
+ );
+ this.sidecarSupport = ObjectUtils.defaultIfNull(
+ sidecarSupport,
+ this.sidecarSupport
+ );
+ this.primaryContainerName = ObjectUtils.defaultIfNull(
+ primaryContainerName,
+ this.primaryContainerName
+ );
+ this.kubexitImage = ObjectUtils.defaultIfNull(
+ kubexitImage,
+ this.kubexitImage
+ );
+ this.graceTerminationPeriodSeconds = ObjectUtils.defaultIfNull(
+ graceTerminationPeriodSeconds,
+ this.graceTerminationPeriodSeconds
+ );
+ this.disableClientProxy = disableClientProxy;
+ this.maxTaskDuration = ObjectUtils.defaultIfNull(
+ maxTaskDuration,
+ this.maxTaskDuration
+ );
+ this.taskCleanupDelay = ObjectUtils.defaultIfNull(
+ taskCleanupDelay,
+ this.taskCleanupDelay
+ );
+ this.taskCleanupInterval = ObjectUtils.defaultIfNull(
+ taskCleanupInterval,
+ this.taskCleanupInterval
+ );
+ this.k8sjobLaunchTimeout = ObjectUtils.defaultIfNull(
+ k8sjobLaunchTimeout,
+ this.k8sjobLaunchTimeout
+ );
+ this.peonMonitors = ObjectUtils.defaultIfNull(
+ peonMonitors,
+ this.peonMonitors
+ );
+ this.javaOptsArray = ObjectUtils.defaultIfNull(
+ javaOptsArray,
+ this.javaOptsArray
+ );
+ this.labels = ObjectUtils.defaultIfNull(
+ labels,
+ this.labels
+ );
+ this.annotations = ObjectUtils.defaultIfNull(
+ annotations,
+ this.annotations
+ );
+ this.capacity = ObjectUtils.defaultIfNull(
+ capacity,
+ this.capacity
+ );
+ }
+
+ public String getNamespace()
+ {
+ return namespace;
+ }
+
+ public boolean isDebugJobs()
+ {
+ return debugJobs;
+ }
+
+ @Deprecated
+ public boolean isSidecarSupport()
+ {
+ return sidecarSupport;
+ }
+
+ public String getPrimaryContainerName()
+ {
+ return primaryContainerName;
+ }
+
+ public String getKubexitImage()
+ {
+ return kubexitImage;
+ }
+
+ public Long getGraceTerminationPeriodSeconds()
+ {
+ return graceTerminationPeriodSeconds;
+ }
+
+ public boolean isDisableClientProxy()
+ {
+ return disableClientProxy;
+ }
+
+ public Period getTaskTimeout()
+ {
+ return maxTaskDuration;
+ }
+
+ public Period getTaskCleanupDelay()
+ {
+ return taskCleanupDelay;
+ }
+
+ public Period getTaskCleanupInterval()
+ {
+ return taskCleanupInterval;
+ }
+
+ public Period getTaskLaunchTimeout()
+ {
+ return k8sjobLaunchTimeout;
+ }
+
+ public List getPeonMonitors()
+ {
+ return peonMonitors;
+ }
+
+ public List getJavaOptsArray()
+ {
+ return javaOptsArray;
+ }
+
+ public Map getLabels()
+ {
+ return labels;
+ }
+
+ public Map getAnnotations()
+ {
+ return annotations;
+ }
+
+ public Integer getCapacity()
+ {
+ return capacity;
+ }
+
+ public static Builder builder()
+ {
+ return new Builder();
+ }
+
+ public static class Builder
+ {
+ private String namespace;
+ private boolean debugJob;
+ private boolean sidecarSupport;
+ private String primaryContainerName;
+ private String kubexitImage;
+ private Long graceTerminationPeriodSeconds;
+ private boolean disableClientProxy;
+ private Period maxTaskDuration;
+ private Period taskCleanupDelay;
+ private Period taskCleanupInterval;
+ private Period k8sjobLaunchTimeout;
+ private List peonMonitors;
+ private List javaOptsArray;
+ private Map labels;
+ private Map annotations;
+ private Integer capacity;
+
+ public Builder()
+ {
+ }
+
+ public Builder withNamespace(String namespace)
+ {
+ this.namespace = namespace;
+ return this;
+ }
+
+ public Builder withDebugJob(boolean debugJob)
+ {
+ this.debugJob = debugJob;
+ return this;
+ }
+
+ public Builder withSidecarSupport(boolean sidecarSupport)
+ {
+ this.sidecarSupport = sidecarSupport;
+ return this;
+ }
+
+ public Builder withPrimaryContainerName(String primaryContainerName)
+ {
+ this.primaryContainerName = primaryContainerName;
+ return this;
+ }
+
+ public Builder withKubexitImage(String kubexitImage)
+ {
+ this.kubexitImage = kubexitImage;
+ return this;
+ }
+
+ public Builder withGraceTerminationPeriodSeconds(Long graceTerminationPeriodSeconds)
+ {
+ this.graceTerminationPeriodSeconds = graceTerminationPeriodSeconds;
+ return this;
+ }
+
+ public Builder withDisableClientProxy(boolean disableClientProxy)
+ {
+ this.disableClientProxy = disableClientProxy;
+ return this;
+ }
+
+ public Builder withTaskTimeout(Period taskTimeout)
+ {
+ this.maxTaskDuration = taskTimeout;
+ return this;
+ }
+
+ public Builder withTaskCleanupDelay(Period taskCleanupDelay)
+ {
+ this.taskCleanupDelay = taskCleanupDelay;
+ return this;
+ }
+
+ public Builder withTaskCleanupInterval(Period taskCleanupInterval)
+ {
+ this.taskCleanupInterval = taskCleanupInterval;
+ return this;
+ }
+
+ public Builder withK8sJobLaunchTimeout(Period k8sjobLaunchTimeout)
+ {
+ this.k8sjobLaunchTimeout = k8sjobLaunchTimeout;
+ return this;
+ }
+
+ public Builder withPeonMonitors(List peonMonitors)
+ {
+ this.peonMonitors = peonMonitors;
+ return this;
+ }
+
+ public Builder withJavaOptsArray(List javaOptsArray)
+ {
+ this.javaOptsArray = javaOptsArray;
+ return this;
+ }
+
+ public Builder withLabels(Map labels)
+ {
+ this.labels = labels;
+ return this;
+ }
+
+ public Builder withAnnotations(Map annotations)
+ {
+ this.annotations = annotations;
+ return this;
+ }
+
+ public Builder withCapacity(@Min(0) @Max(Integer.MAX_VALUE) Integer capacity)
+ {
+ this.capacity = capacity;
+ return this;
+ }
+
+ public KubernetesTaskRunnerConfig build()
+ {
+ return new KubernetesTaskRunnerConfig(
+ this.namespace,
+ this.debugJob,
+ this.sidecarSupport,
+ this.primaryContainerName,
+ this.kubexitImage,
+ this.graceTerminationPeriodSeconds,
+ this.disableClientProxy,
+ this.maxTaskDuration,
+ this.taskCleanupDelay,
+ this.taskCleanupInterval,
+ this.k8sjobLaunchTimeout,
+ this.peonMonitors,
+ this.javaOptsArray,
+ this.labels,
+ this.annotations,
+ this.capacity
+ );
+ }
+ }
}
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java
index fdee5702c57..8dbfe68a49a 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java
@@ -19,7 +19,6 @@
package org.apache.druid.k8s.overlord;
-import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.inject.Inject;
import io.fabric8.kubernetes.client.Config;
@@ -30,15 +29,14 @@ import org.apache.druid.guice.annotations.Self;
import org.apache.druid.guice.annotations.Smile;
import org.apache.druid.indexing.common.config.TaskConfig;
import org.apache.druid.indexing.overlord.TaskRunnerFactory;
-import org.apache.druid.indexing.overlord.config.TaskQueueConfig;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.http.client.HttpClient;
import org.apache.druid.k8s.overlord.common.DruidKubernetesClient;
-import org.apache.druid.k8s.overlord.common.DruidKubernetesPeonClient;
-import org.apache.druid.k8s.overlord.common.MultiContainerTaskAdapter;
-import org.apache.druid.k8s.overlord.common.PodTemplateTaskAdapter;
-import org.apache.druid.k8s.overlord.common.SingleContainerTaskAdapter;
-import org.apache.druid.k8s.overlord.common.TaskAdapter;
+import org.apache.druid.k8s.overlord.common.KubernetesPeonClient;
+import org.apache.druid.k8s.overlord.taskadapter.MultiContainerTaskAdapter;
+import org.apache.druid.k8s.overlord.taskadapter.PodTemplateTaskAdapter;
+import org.apache.druid.k8s.overlord.taskadapter.SingleContainerTaskAdapter;
+import org.apache.druid.k8s.overlord.taskadapter.TaskAdapter;
import org.apache.druid.server.DruidNode;
import org.apache.druid.server.log.StartupLoggingConfig;
import org.apache.druid.tasklogs.TaskLogs;
@@ -53,7 +51,6 @@ public class KubernetesTaskRunnerFactory implements TaskRunnerFactory statusFuture)
+ {
+ super(task.getId(), statusFuture);
+ this.task = task;
+ }
+
+ protected synchronized void setKubernetesPeonLifecycle(KubernetesPeonLifecycle kubernetesPeonLifecycle)
+ {
+ Preconditions.checkState(this.kubernetesPeonLifecycle == null);
+ this.kubernetesPeonLifecycle = kubernetesPeonLifecycle;
+ }
+
+ protected synchronized void shutdown()
+ {
+ this.shutdownRequested.set(true);
+
+ if (this.kubernetesPeonLifecycle != null) {
+ this.kubernetesPeonLifecycle.shutdown();
+ }
+ }
+
+ protected boolean isShutdownRequested()
+ {
+ return shutdownRequested.get();
+ }
+
+ protected boolean isPending()
+ {
+ return RunnerTaskState.PENDING.equals(getRunnerTaskState());
+ }
+
+ protected boolean isRunning()
+ {
+ return RunnerTaskState.RUNNING.equals(getRunnerTaskState());
+ }
+
+ protected RunnerTaskState getRunnerTaskState()
+ {
+ if (kubernetesPeonLifecycle == null) {
+ return RunnerTaskState.PENDING;
+ }
+
+ switch (kubernetesPeonLifecycle.getState()) {
+ case NOT_STARTED:
+ case PENDING:
+ return RunnerTaskState.PENDING;
+ case RUNNING:
+ return RunnerTaskState.RUNNING;
+ case STOPPED:
+ return RunnerTaskState.NONE;
+ default:
+ throw new ISE("Peon lifecycle in unknown state");
+ }
+ }
+
+ protected Optional streamTaskLogs()
+ {
+ if (kubernetesPeonLifecycle == null) {
+ return Optional.absent();
+ }
+ return kubernetesPeonLifecycle.streamLogs();
+ }
+
+ @Override
+ public TaskLocation getLocation()
+ {
+ if (kubernetesPeonLifecycle == null) {
+ return TaskLocation.unknown();
+ }
+ return kubernetesPeonLifecycle.getTaskLocation();
+ }
+
+ @Override
+ public String getTaskType()
+ {
+ return task.getType();
+ }
+
+ @Override
+ public String getDataSource()
+ {
+ return task.getDataSource();
+ }
+}
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/PeonLifecycleFactory.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/PeonLifecycleFactory.java
new file mode 100644
index 00000000000..2587069d756
--- /dev/null
+++ b/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);
+}
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java
index 0a0beba7daa..56adfab11b2 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java
@@ -37,7 +37,7 @@ public class DruidK8sConstants
public static final String JAVA_OPTS = "JAVA_OPTS";
public static final String DRUID_HOST_ENV = "druid_host";
public static final String DRUID_HOSTNAME_ENV = "HOSTNAME";
+ public static final String LABEL_KEY = "druid.k8s.peons";
public static final String DRUID_LABEL_PREFIX = "druid.";
- static final String LABEL_KEY = "druid.k8s.peons";
static final Predicate IS_TRANSIENT = e -> e instanceof KubernetesResourceNotFoundException;
}
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesPeonClient.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesPeonClient.java
deleted file mode 100644
index 336c9910158..00000000000
--- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesPeonClient.java
+++ /dev/null
@@ -1,248 +0,0 @@
-/*
- * 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.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
-import io.fabric8.kubernetes.api.model.Pod;
-import io.fabric8.kubernetes.api.model.PodList;
-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;
-
-public class DruidKubernetesPeonClient implements KubernetesPeonClient
-{
-
- private static final EmittingLogger log = new EmittingLogger(DruidKubernetesPeonClient.class);
-
- private final KubernetesClientApi clientApi;
- private final String namespace;
- private final boolean debugJobs;
-
- public DruidKubernetesPeonClient(KubernetesClientApi clientApi, String namespace, boolean debugJobs)
- {
- this.clientApi = clientApi;
- this.namespace = namespace;
- this.debugJobs = debugJobs;
- }
-
- @Override
- public Optional jobExists(K8sTaskId taskId)
- {
- return clientApi.executeRequest(
- client -> {
- return Optional.fromNullable(
- client.batch()
- .v1()
- .jobs()
- .inNamespace(namespace)
- .withName(taskId.getK8sTaskId())
- .get());
- }
- );
- }
-
- @Override
- public Pod launchJobAndWaitForStart(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
- Pod mainPod = getMainJobPod(taskId);
- Pod result = client.pods().inNamespace(namespace).withName(mainPod.getMetadata().getName())
- .waitUntilCondition(pod -> {
- if (pod == null) {
- return false;
- }
- return pod.getStatus() != null && pod.getStatus().getPodIP() != null;
- }, howLong, timeUnit);
- long duration = System.currentTimeMillis() - start;
- log.info("Took task %s %d ms for pod to startup", taskId, duration);
- return result;
- });
- }
-
- @Override
- public JobResponse waitForJobCompletion(K8sTaskId taskId, long howLong, TimeUnit unit)
- {
- return clientApi.executeRequest(client -> {
- Job job = client.batch()
- .v1()
- .jobs()
- .inNamespace(namespace)
- .withName(taskId.getK8sTaskId())
- .waitUntilCondition(
- x -> (x == null) || (x.getStatus() != null && x.getStatus().getActive() == null
- && (x.getStatus().getFailed() != null || x.getStatus().getSucceeded() != null)),
- howLong,
- unit
- );
- if (job == null) {
- log.info("K8s job for the task [%s] was not found. It can happen if the task was canceled", taskId);
- return new JobResponse(null, PeonPhase.FAILED);
- }
- if (job.getStatus().getSucceeded() != null) {
- return new JobResponse(job, PeonPhase.SUCCEEDED);
- }
- log.warn("Task %s failed with status %s", taskId, job.getStatus());
- return new JobResponse(job, PeonPhase.FAILED);
- });
- }
-
- @Override
- public boolean cleanUpJob(K8sTaskId taskId)
- {
- if (!debugJobs) {
- Boolean result = clientApi.executeRequest(client -> !client.batch()
- .v1()
- .jobs()
- .inNamespace(namespace)
- .withName(taskId.getK8sTaskId())
- .delete().isEmpty());
- if (result) {
- log.info("Cleaned up k8s task: %s", taskId);
- } else {
- log.info("K8s task does not exist: %s", taskId);
- }
- return result;
- } else {
- log.info("Not cleaning up task %s due to flag: debugJobs=true", taskId);
- return true;
- }
-
- }
-
- @Override
- public Optional getPeonLogs(K8sTaskId taskId)
- {
- KubernetesClient k8sClient = clientApi.getClient();
- try {
- LogWatch logWatch = k8sClient.batch()
- .v1()
- .jobs()
- .inNamespace(namespace)
- .withName(taskId.getK8sTaskId())
- .inContainer("main")
- .watchLog();
- if (logWatch == null) {
- k8sClient.close();
- return Optional.absent();
- }
- return Optional.of(new LogWatchInputStream(k8sClient, logWatch));
- }
- catch (Exception e) {
- log.error(e, "Error streaming logs from task: %s", taskId);
- k8sClient.close();
- return Optional.absent();
- }
- }
-
- @Override
- public List listAllPeonJobs()
- {
- return clientApi.executeRequest(client -> client.batch()
- .v1()
- .jobs()
- .inNamespace(namespace)
- .withLabel(DruidK8sConstants.LABEL_KEY)
- .list()
- .getItems());
- }
-
- @Override
- public int cleanCompletedJobsOlderThan(long howFarBack, TimeUnit timeUnit)
- {
- AtomicInteger numDeleted = new AtomicInteger();
- return clientApi.executeRequest(client -> {
- List jobs = getJobsToCleanup(listAllPeonJobs(), howFarBack, timeUnit);
- jobs.forEach(x -> {
- if (!client.batch()
- .v1()
- .jobs()
- .inNamespace(namespace)
- .withName(x.getMetadata().getName())
- .delete().isEmpty()) {
- numDeleted.incrementAndGet();
- }
- });
- return numDeleted.get();
- });
- }
-
- @Override
- public Pod getMainJobPod(K8sTaskId taskId)
- {
- return clientApi.executeRequest(client -> getMainJobPod(client, taskId));
- }
-
-
- @VisibleForTesting
- List getJobsToCleanup(List candidates, long howFarBack, TimeUnit timeUnit)
- {
- List toDelete = new ArrayList<>();
- long cutOff = System.currentTimeMillis() - timeUnit.toMillis(howFarBack);
- candidates.forEach(x -> {
- // jobs that are complete
- if (x.getStatus().getActive() == null) {
- Timestamp timestamp = Timestamp.valueOf(x.getStatus().getCompletionTime());
- if (timestamp.before(new Timestamp(cutOff))) {
- toDelete.add(x);
- }
- }
- });
- return toDelete;
- }
-
- Pod getMainJobPod(KubernetesClient client, K8sTaskId taskId)
- {
- String k8sTaskId = taskId.getK8sTaskId();
- try {
- return RetryUtils.retry(
- () -> {
- PodList list = client.pods().inNamespace(namespace).withLabel("job-name", k8sTaskId).list();
- if (list.getItems().size() > 0) {
- return list.getItems().get(0);
- }
- throw new KubernetesResourceNotFoundException(
- "K8s pod with label: job-name="
- + k8sTaskId
- + " not found");
- },
- DruidK8sConstants.IS_TRANSIENT, 5, RetryUtils.DEFAULT_MAX_TRIES
- );
- }
- catch (Exception e) {
- throw new KubernetesResourceNotFoundException("K8s pod with label: job-name=" + k8sTaskId + " not found");
- }
- }
-}
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/JobResponse.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/JobResponse.java
index 9254f34225a..a7a8156468f 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/JobResponse.java
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/JobResponse.java
@@ -19,7 +19,6 @@
package org.apache.druid.k8s.overlord.common;
-import com.google.common.base.Optional;
import io.fabric8.kubernetes.api.model.batch.v1.Job;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.emitter.EmittingLogger;
@@ -28,10 +27,8 @@ import org.joda.time.PeriodType;
import javax.annotation.Nullable;
-
public class JobResponse
{
-
private static final EmittingLogger LOGGER = new EmittingLogger(JobResponse.class);
private final Job job;
@@ -53,29 +50,24 @@ public class JobResponse
return phase;
}
- public Optional getJobDuration()
+ public long getJobDuration()
{
- Optional duration = Optional.absent();
+ long duration = -1L;
String jobName = job != null && job.getMetadata() != null ? job.getMetadata().getName() : "";
try {
if (job != null && job.getStatus() != null
&& job.getStatus().getStartTime() != null
&& job.getStatus().getCompletionTime() != null) {
- duration = Optional.of((long) new Period(
+ duration = new Period(
DateTimes.of(job.getStatus().getStartTime()),
DateTimes.of(job.getStatus().getCompletionTime()),
PeriodType.millis()
- ).getMillis());
+ ).getMillis();
}
}
catch (Exception e) {
LOGGER.error(e, "Error calculating duration for job: %s", jobName);
}
- if (duration.isPresent()) {
- LOGGER.info("Duration for Job: %s was %d seconds", jobName, duration.get());
- } else {
- LOGGER.info("Unable to calcuate duration for Job: %s", jobName);
- }
return duration;
}
}
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java
index f9e402f94e2..0b80515a148 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java
@@ -19,37 +19,220 @@
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);
- Optional jobExists(K8sTaskId taskId);
+ private final KubernetesClientApi clientApi;
+ private final String namespace;
+ private final boolean debugJobs;
- Pod launchJobAndWaitForStart(Job job, long howLong, TimeUnit timeUnit);
+ public KubernetesPeonClient(KubernetesClientApi clientApi, String namespace, boolean debugJobs)
+ {
+ this.clientApi = clientApi;
+ this.namespace = namespace;
+ this.debugJobs = debugJobs;
+ }
- JobResponse waitForJobCompletion(K8sTaskId taskId, long howLong, TimeUnit timeUnit);
+ 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
+ Pod mainPod = getPeonPodWithRetries(taskId);
+ Pod result = client.pods().inNamespace(namespace).withName(mainPod.getMetadata().getName())
+ .waitUntilCondition(pod -> {
+ if (pod == null) {
+ return false;
+ }
+ return pod.getStatus() != null && pod.getStatus().getPodIP() != null;
+ }, howLong, timeUnit);
+ long duration = System.currentTimeMillis() - start;
+ log.info("Took task %s %d ms for pod to startup", taskId, duration);
+ return result;
+ });
+ }
- boolean cleanUpJob(K8sTaskId taskId);
+ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, TimeUnit unit)
+ {
+ return clientApi.executeRequest(client -> {
+ Job job = client.batch()
+ .v1()
+ .jobs()
+ .inNamespace(namespace)
+ .withName(taskId.getK8sTaskId())
+ .waitUntilCondition(
+ x -> (x == null) || (x.getStatus() != null && x.getStatus().getActive() == null
+ && (x.getStatus().getFailed() != null || x.getStatus().getSucceeded() != null)),
+ howLong,
+ unit
+ );
+ if (job == null) {
+ log.info("K8s job for the task [%s] was not found. It can happen if the task was canceled", taskId);
+ return new JobResponse(null, PeonPhase.FAILED);
+ }
+ if (job.getStatus().getSucceeded() != null) {
+ return new JobResponse(job, PeonPhase.SUCCEEDED);
+ }
+ log.warn("Task %s failed with status %s", taskId, job.getStatus());
+ return new JobResponse(job, PeonPhase.FAILED);
+ });
+ }
- Optional getPeonLogs(K8sTaskId taskId);
+ public boolean deletePeonJob(K8sTaskId taskId)
+ {
+ if (!debugJobs) {
+ Boolean result = clientApi.executeRequest(client -> !client.batch()
+ .v1()
+ .jobs()
+ .inNamespace(namespace)
+ .withName(taskId.getK8sTaskId())
+ .delete().isEmpty());
+ if (result) {
+ log.info("Cleaned up k8s task: %s", taskId);
+ } else {
+ log.info("K8s task does not exist: %s", taskId);
+ }
+ return result;
+ } else {
+ log.info("Not cleaning up task %s due to flag: debugJobs=true", taskId);
+ return true;
+ }
+ }
- List listAllPeonJobs();
+ public Optional getPeonLogs(K8sTaskId taskId)
+ {
+ KubernetesClient k8sClient = clientApi.getClient();
+ try {
+ LogWatch logWatch = k8sClient.batch()
+ .v1()
+ .jobs()
+ .inNamespace(namespace)
+ .withName(taskId.getK8sTaskId())
+ .inContainer("main")
+ .watchLog();
+ if (logWatch == null) {
+ k8sClient.close();
+ return Optional.absent();
+ }
+ return Optional.of(new LogWatchInputStream(k8sClient, logWatch));
+ }
+ catch (Exception e) {
+ log.error(e, "Error streaming logs from task: %s", taskId);
+ k8sClient.close();
+ return Optional.absent();
+ }
+ }
- int cleanCompletedJobsOlderThan(long howFarBack, TimeUnit timeUnit);
+ public List getPeonJobs()
+ {
+ return clientApi.executeRequest(client -> client.batch()
+ .v1()
+ .jobs()
+ .inNamespace(namespace)
+ .withLabel(DruidK8sConstants.LABEL_KEY)
+ .list()
+ .getItems());
+ }
- Pod getMainJobPod(K8sTaskId taskId);
+ public int deleteCompletedPeonJobsOlderThan(long howFarBack, TimeUnit timeUnit)
+ {
+ AtomicInteger numDeleted = new AtomicInteger();
+ return clientApi.executeRequest(client -> {
+ List jobs = getJobsToCleanup(getPeonJobs(), howFarBack, timeUnit);
+ jobs.forEach(x -> {
+ if (!client.batch()
+ .v1()
+ .jobs()
+ .inNamespace(namespace)
+ .withName(x.getMetadata().getName())
+ .delete().isEmpty()) {
+ numDeleted.incrementAndGet();
+ } else {
+ log.error("Failed to delete job %s", x.getMetadata().getName());
+ }
+ });
+ return numDeleted.get();
+ });
+ }
+ private List getJobsToCleanup(List candidates, long howFarBack, TimeUnit timeUnit)
+ {
+ List toDelete = new ArrayList<>();
+ long cutOff = System.currentTimeMillis() - timeUnit.toMillis(howFarBack);
+ candidates.forEach(x -> {
+ // jobs that are complete
+ if (x.getStatus().getActive() == null) {
+ Timestamp timestamp = Timestamp.valueOf(x.getStatus().getCompletionTime());
+ if (timestamp.before(new Timestamp(cutOff))) {
+ toDelete.add(x);
+ }
+ }
+ });
+ return toDelete;
+ }
+ public Optional getPeonPod(K8sTaskId taskId)
+ {
+ return clientApi.executeRequest(client -> getPeonPod(client, taskId));
+ }
+
+ private Optional getPeonPod(KubernetesClient client, K8sTaskId taskId)
+ {
+ List pods = client.pods()
+ .inNamespace(namespace)
+ .withLabel("job-name", taskId.getK8sTaskId())
+ .list()
+ .getItems();
+ return pods.isEmpty() ? Optional.absent() : Optional.of(pods.get(0));
+ }
+
+ public Pod getPeonPodWithRetries(K8sTaskId taskId)
+ {
+ return clientApi.executeRequest(client -> getPeonPodWithRetries(client, taskId, 5, RetryUtils.DEFAULT_MAX_TRIES));
+ }
+
+ @VisibleForTesting
+ Pod getPeonPodWithRetries(KubernetesClient client, K8sTaskId taskId, int quietTries, int maxTries)
+ {
+ String k8sTaskId = taskId.getK8sTaskId();
+ try {
+ return RetryUtils.retry(
+ () -> {
+ Optional maybePod = getPeonPod(client, taskId);
+ if (maybePod.isPresent()) {
+ return maybePod.get();
+ }
+ throw new KubernetesResourceNotFoundException(
+ "K8s pod with label: job-name="
+ + k8sTaskId
+ + " not found");
+ },
+ DruidK8sConstants.IS_TRANSIENT, quietTries, maxTries
+ );
+ }
+ catch (Exception e) {
+ throw new KubernetesResourceNotFoundException("K8s pod with label: job-name=" + k8sTaskId + " not found");
+ }
+ }
}
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/K8sTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java
similarity index 90%
rename from extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/K8sTaskAdapter.java
rename to extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java
index e9b923fab73..bb41dcae14a 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/K8sTaskAdapter.java
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java
@@ -17,12 +17,13 @@
* under the License.
*/
-package org.apache.druid.k8s.overlord.common;
+package org.apache.druid.k8s.overlord.taskadapter;
import com.fasterxml.jackson.core.JsonProcessingException;
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;
@@ -49,6 +50,11 @@ import org.apache.druid.indexing.overlord.config.ForkingTaskRunnerConfig;
import org.apache.druid.java.util.common.HumanReadableBytes;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig;
+import org.apache.druid.k8s.overlord.common.Base64Compression;
+import org.apache.druid.k8s.overlord.common.DruidK8sConstants;
+import org.apache.druid.k8s.overlord.common.K8sTaskId;
+import org.apache.druid.k8s.overlord.common.KubernetesClientApi;
+import org.apache.druid.k8s.overlord.common.PeonCommandContext;
import org.apache.druid.server.DruidNode;
import org.apache.druid.server.log.StartupLoggingConfig;
@@ -105,10 +111,7 @@ public abstract class K8sTaskAdapter implements TaskAdapter
public Job fromTask(Task task) throws IOException
{
String myPodName = System.getenv("HOSTNAME");
- Pod pod = client.executeRequest(client -> client.pods()
- .inNamespace(taskRunnerConfig.namespace)
- .withName(myPodName)
- .get());
+ Pod pod = client.executeRequest(client -> client.pods().inNamespace(taskRunnerConfig.getNamespace()).withName(myPodName).get());
PeonCommandContext context = new PeonCommandContext(
generateCommand(task),
javaOpts(task),
@@ -116,7 +119,7 @@ public abstract class K8sTaskAdapter implements TaskAdapter
node.isEnableTlsPort()
);
PodSpec podSpec = pod.getSpec();
- massageSpec(podSpec, taskRunnerConfig.primaryContainerName);
+ massageSpec(podSpec, taskRunnerConfig.getPrimaryContainerName());
return createJobFromPodSpec(podSpec, task, context);
}
@@ -152,9 +155,9 @@ public abstract class K8sTaskAdapter implements TaskAdapter
.endMetadata()
.withNewSpec()
.withTemplate(podTemplate)
- .withActiveDeadlineSeconds(taskRunnerConfig.maxTaskDuration.toStandardDuration().getStandardSeconds())
+ .withActiveDeadlineSeconds(taskRunnerConfig.getTaskTimeout().toStandardDuration().getStandardSeconds())
.withBackoffLimit(0)
- .withTtlSecondsAfterFinished((int) taskRunnerConfig.taskCleanupDelay.toStandardDuration().getStandardSeconds())
+ .withTtlSecondsAfterFinished((int) taskRunnerConfig.getTaskCleanupDelay().toStandardDuration().getStandardSeconds())
.endSpec()
.build();
}
@@ -208,11 +211,11 @@ public abstract class K8sTaskAdapter implements TaskAdapter
throws JsonProcessingException
{
// if the peon monitors are set, override the overlord's monitors (if set) with the peon monitors
- if (!taskRunnerConfig.peonMonitors.isEmpty()) {
+ if (!taskRunnerConfig.getPeonMonitors().isEmpty()) {
mainContainer.getEnv().removeIf(x -> "druid_monitoring_monitors".equals(x.getName()));
mainContainer.getEnv().add(new EnvVarBuilder()
.withName("druid_monitoring_monitors")
- .withValue(mapper.writeValueAsString(taskRunnerConfig.peonMonitors))
+ .withValue(mapper.writeValueAsString(taskRunnerConfig.getPeonMonitors()))
.build());
}
@@ -281,17 +284,19 @@ public abstract class K8sTaskAdapter implements TaskAdapter
protected Map addJobSpecificAnnotations(PeonCommandContext context, K8sTaskId k8sTaskId)
{
- Map annotations = taskRunnerConfig.annotations;
- annotations.put(DruidK8sConstants.TASK_ID, k8sTaskId.getOriginalTaskId());
- annotations.put(DruidK8sConstants.TLS_ENABLED, String.valueOf(context.isEnableTls()));
- return annotations;
+ return ImmutableMap.builder()
+ .putAll(taskRunnerConfig.getAnnotations())
+ .put(DruidK8sConstants.TASK_ID, k8sTaskId.getOriginalTaskId())
+ .put(DruidK8sConstants.TLS_ENABLED, String.valueOf(context.isEnableTls()))
+ .build();
}
protected Map addJobSpecificLabels()
{
- Map labels = taskRunnerConfig.labels;
- labels.put(DruidK8sConstants.LABEL_KEY, "true");
- return labels;
+ return ImmutableMap.builder()
+ .putAll(taskRunnerConfig.getLabels())
+ .put(DruidK8sConstants.LABEL_KEY, "true")
+ .build();
}
protected PodTemplateSpec createTemplateFromSpec(
@@ -305,7 +310,7 @@ public abstract class K8sTaskAdapter implements TaskAdapter
podSpec.setNodeName(null);
podSpec.setRestartPolicy("Never");
podSpec.setHostname(k8sTaskId.getK8sTaskId());
- podSpec.setTerminationGracePeriodSeconds(taskRunnerConfig.graceTerminationPeriodSeconds);
+ podSpec.setTerminationGracePeriodSeconds(taskRunnerConfig.getGraceTerminationPeriodSeconds());
PodTemplateSpec podTemplate = new PodTemplateSpec();
ObjectMeta objectMeta = new ObjectMeta();
@@ -343,7 +348,7 @@ public abstract class K8sTaskAdapter implements TaskAdapter
private List javaOpts(Task task)
{
final List javaOpts = new ArrayList<>();
- Iterables.addAll(javaOpts, taskRunnerConfig.javaOptsArray);
+ Iterables.addAll(javaOpts, taskRunnerConfig.getJavaOptsArray());
// Override task specific javaOpts
Object taskJavaOpts = task.getContextValue(
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/MultiContainerTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapter.java
similarity index 94%
rename from extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/MultiContainerTaskAdapter.java
rename to extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapter.java
index 601a480f900..9cda8f86488 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/MultiContainerTaskAdapter.java
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapter.java
@@ -17,7 +17,7 @@
* under the License.
*/
-package org.apache.druid.k8s.overlord.common;
+package org.apache.druid.k8s.overlord.taskadapter;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.annotations.VisibleForTesting;
@@ -37,6 +37,10 @@ import org.apache.commons.lang.StringEscapeUtils;
import org.apache.druid.indexing.common.config.TaskConfig;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig;
+import org.apache.druid.k8s.overlord.common.Base64Compression;
+import org.apache.druid.k8s.overlord.common.K8sTaskId;
+import org.apache.druid.k8s.overlord.common.KubernetesClientApi;
+import org.apache.druid.k8s.overlord.common.PeonCommandContext;
import org.apache.druid.server.DruidNode;
import org.apache.druid.server.log.StartupLoggingConfig;
@@ -95,7 +99,7 @@ public class MultiContainerTaskAdapter extends K8sTaskAdapter
{
return new ContainerBuilder()
.withName("kubexit")
- .withImage(taskRunnerConfig.kubexitImage)
+ .withImage(taskRunnerConfig.getKubexitImage())
.withCommand("cp", "/bin/kubexit", "/kubexit/kubexit")
.withVolumeMounts(new VolumeMountBuilder().withMountPath("/kubexit").withName("kubexit").build())
.build();
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/PodTemplateTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java
similarity index 95%
rename from extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/PodTemplateTaskAdapter.java
rename to extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java
index ed671106c37..11f7c4a4119 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/PodTemplateTaskAdapter.java
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java
@@ -17,7 +17,7 @@
* under the License.
*/
-package org.apache.druid.k8s.overlord.common;
+package org.apache.druid.k8s.overlord.taskadapter;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.cfg.MapperConfig;
@@ -44,6 +44,9 @@ import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig;
+import org.apache.druid.k8s.overlord.common.Base64Compression;
+import org.apache.druid.k8s.overlord.common.DruidK8sConstants;
+import org.apache.druid.k8s.overlord.common.K8sTaskId;
import org.apache.druid.server.DruidNode;
import java.io.File;
@@ -137,9 +140,9 @@ public class PodTemplateTaskAdapter implements TaskAdapter
.endContainer()
.endSpec()
.endTemplate()
- .withActiveDeadlineSeconds(taskRunnerConfig.maxTaskDuration.toStandardDuration().getStandardSeconds())
+ .withActiveDeadlineSeconds(taskRunnerConfig.getTaskTimeout().toStandardDuration().getStandardSeconds())
.withBackoffLimit(0) // druid does not support an external system retrying failed tasks
- .withTtlSecondsAfterFinished((int) taskRunnerConfig.taskCleanupDelay.toStandardDuration().getStandardSeconds())
+ .withTtlSecondsAfterFinished((int) taskRunnerConfig.getTaskCleanupDelay().toStandardDuration().getStandardSeconds())
.endSpec()
.build();
}
@@ -244,7 +247,7 @@ public class PodTemplateTaskAdapter implements TaskAdapter
private Map getJobLabels(KubernetesTaskRunnerConfig config, Task task)
{
return ImmutableMap.builder()
- .putAll(config.labels)
+ .putAll(config.getLabels())
.put(DruidK8sConstants.LABEL_KEY, "true")
.put(getDruidLabel(DruidK8sConstants.TASK_ID), task.getId())
.put(getDruidLabel(DruidK8sConstants.TASK_TYPE), task.getType())
@@ -256,7 +259,7 @@ public class PodTemplateTaskAdapter implements TaskAdapter
private Map getJobAnnotations(KubernetesTaskRunnerConfig config, Task task)
{
return ImmutableMap.builder()
- .putAll(config.annotations)
+ .putAll(config.getAnnotations())
.put(DruidK8sConstants.TASK_ID, task.getId())
.put(DruidK8sConstants.TASK_TYPE, task.getType())
.put(DruidK8sConstants.TASK_GROUP_ID, task.getGroupId())
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/SingleContainerTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapter.java
similarity index 90%
rename from extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/SingleContainerTaskAdapter.java
rename to extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapter.java
index 8f8ba05213b..c64acd15310 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/SingleContainerTaskAdapter.java
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapter.java
@@ -17,7 +17,7 @@
* under the License.
*/
-package org.apache.druid.k8s.overlord.common;
+package org.apache.druid.k8s.overlord.taskadapter;
import com.fasterxml.jackson.databind.ObjectMapper;
import io.fabric8.kubernetes.api.model.Container;
@@ -26,6 +26,10 @@ import io.fabric8.kubernetes.api.model.batch.v1.Job;
import org.apache.druid.indexing.common.config.TaskConfig;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig;
+import org.apache.druid.k8s.overlord.common.Base64Compression;
+import org.apache.druid.k8s.overlord.common.K8sTaskId;
+import org.apache.druid.k8s.overlord.common.KubernetesClientApi;
+import org.apache.druid.k8s.overlord.common.PeonCommandContext;
import org.apache.druid.server.DruidNode;
import org.apache.druid.server.log.StartupLoggingConfig;
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/TaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/TaskAdapter.java
similarity index 95%
rename from extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/TaskAdapter.java
rename to extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/TaskAdapter.java
index a58240e40ae..05933604f2b 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/TaskAdapter.java
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/TaskAdapter.java
@@ -17,7 +17,7 @@
* under the License.
*/
-package org.apache.druid.k8s.overlord.common;
+package org.apache.druid.k8s.overlord.taskadapter;
import io.fabric8.kubernetes.api.model.batch.v1.Job;
import org.apache.druid.indexing.common.task.Task;
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/kubernetes-overlord-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
index fd4d4454083..927a4349a2e 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
@@ -13,4 +13,4 @@
# See the License for the specific language governing permissions and
# limitations under the License.
-org.apache.druid.k8s.overlord.K8sOverlordModule
+org.apache.druid.k8s.overlord.KubernetesOverlordModule
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java
new file mode 100644
index 00000000000..09ad1e10051
--- /dev/null
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java
@@ -0,0 +1,644 @@
+/*
+ * 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.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Optional;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.PodBuilder;
+import io.fabric8.kubernetes.api.model.batch.v1.Job;
+import io.fabric8.kubernetes.api.model.batch.v1.JobBuilder;
+import org.apache.commons.io.IOUtils;
+import org.apache.druid.indexer.TaskLocation;
+import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.indexing.common.TestUtils;
+import org.apache.druid.indexing.common.task.NoopTask;
+import org.apache.druid.indexing.common.task.Task;
+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.PeonPhase;
+import org.apache.druid.tasklogs.TaskLogs;
+import org.easymock.EasyMock;
+import org.easymock.EasyMockRunner;
+import org.easymock.EasyMockSupport;
+import org.easymock.Mock;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+@RunWith(EasyMockRunner.class)
+public class KubernetesPeonLifecycleTest extends EasyMockSupport
+{
+ private static final String ID = "id";
+ private static final TaskStatus SUCCESS = TaskStatus.success(ID);
+
+ @Mock KubernetesPeonClient kubernetesClient;
+ @Mock TaskLogs taskLogs;
+
+ private ObjectMapper mapper;
+ private Task task;
+ private K8sTaskId k8sTaskId;
+
+ @Before
+ public void setup()
+ {
+ mapper = new TestUtils().getTestObjectMapper();
+ task = NoopTask.create(ID, 0);
+ k8sTaskId = new K8sTaskId(task);
+ }
+
+ @Test
+ public void test_run()
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(task, kubernetesClient, taskLogs, mapper) {
+ @Override
+ protected synchronized TaskStatus join(long timeout)
+ {
+ return TaskStatus.success(ID);
+ }
+ };
+
+ Job job = new JobBuilder().withNewMetadata().withName(ID).endMetadata().build();
+
+ EasyMock.expect(kubernetesClient.launchPeonJobAndWaitForStart(
+ EasyMock.eq(job),
+ EasyMock.anyLong(),
+ EasyMock.eq(TimeUnit.MILLISECONDS)
+ )).andReturn(null);
+
+ Assert.assertEquals(KubernetesPeonLifecycle.State.NOT_STARTED, peonLifecycle.getState());
+
+ replayAll();
+
+ TaskStatus taskStatus = peonLifecycle.run(job, 0L, 0L);
+
+ verifyAll();
+
+ Assert.assertTrue(taskStatus.isSuccess());
+ Assert.assertEquals(ID, taskStatus.getId());
+ Assert.assertEquals(KubernetesPeonLifecycle.State.STOPPED, peonLifecycle.getState());
+ }
+
+ @Test
+ public void test_run_whenCalledMultipleTimes_raisesIllegalStateException()
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(task, kubernetesClient, taskLogs, mapper) {
+ @Override
+ protected synchronized TaskStatus join(long timeout)
+ {
+ return TaskStatus.success(ID);
+ }
+ };
+
+ Job job = new JobBuilder().withNewMetadata().withName(ID).endMetadata().build();
+
+ EasyMock.expect(kubernetesClient.launchPeonJobAndWaitForStart(
+ EasyMock.eq(job),
+ EasyMock.anyLong(),
+ EasyMock.eq(TimeUnit.MILLISECONDS)
+ )).andReturn(null);
+
+ Assert.assertEquals(KubernetesPeonLifecycle.State.NOT_STARTED, peonLifecycle.getState());
+
+ replayAll();
+
+ peonLifecycle.run(job, 0L, 0L);
+
+ Assert.assertThrows(
+ "Task [id] failed to run: invalid peon lifecycle state transition [STOPPED]->[PENDING]",
+ IllegalStateException.class,
+ () -> peonLifecycle.run(job, 0L, 0L)
+ );
+
+ verifyAll();
+
+ Assert.assertEquals(KubernetesPeonLifecycle.State.STOPPED, peonLifecycle.getState());
+ }
+
+ @Test
+ public void test_run_whenExceptionRaised_setsRunnerTaskStateToNone()
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(task, kubernetesClient, taskLogs, mapper) {
+ @Override
+ protected synchronized TaskStatus join(long timeout)
+ {
+ throw new IllegalStateException();
+ }
+ };
+
+ Job job = new JobBuilder().withNewMetadata().withName(ID).endMetadata().build();
+
+ EasyMock.expect(kubernetesClient.launchPeonJobAndWaitForStart(
+ EasyMock.eq(job),
+ EasyMock.anyLong(),
+ EasyMock.eq(TimeUnit.MILLISECONDS)
+ )).andReturn(null);
+
+ Assert.assertEquals(KubernetesPeonLifecycle.State.NOT_STARTED, peonLifecycle.getState());
+
+ replayAll();
+
+ Assert.assertThrows(
+ Exception.class,
+ () -> peonLifecycle.run(job, 0L, 0L)
+ );
+
+ verifyAll();
+
+ Assert.assertEquals(KubernetesPeonLifecycle.State.STOPPED, peonLifecycle.getState());
+ }
+
+ @Test
+ public void test_join_withoutJob_returnsFailedTaskStatus() throws IOException
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(task, kubernetesClient, taskLogs, mapper);
+
+ EasyMock.expect(kubernetesClient.waitForPeonJobCompletion(
+ EasyMock.eq(k8sTaskId),
+ EasyMock.anyLong(),
+ EasyMock.eq(TimeUnit.MILLISECONDS)
+ )).andReturn(new JobResponse(null, PeonPhase.FAILED));
+ EasyMock.expect(kubernetesClient.getPeonLogs(k8sTaskId)).andReturn(Optional.of(
+ IOUtils.toInputStream("", StandardCharsets.UTF_8)
+ ));
+ EasyMock.expect(taskLogs.streamTaskStatus(ID)).andReturn(Optional.absent());
+ taskLogs.pushTaskLog(EasyMock.eq(ID), EasyMock.anyObject(File.class));
+ EasyMock.expectLastCall();
+ EasyMock.expect(kubernetesClient.deletePeonJob(k8sTaskId)).andReturn(true);
+
+ replayAll();
+
+ TaskStatus taskStatus = peonLifecycle.join(0L);
+
+ verifyAll();
+
+ Assert.assertTrue(taskStatus.isFailure());
+ Assert.assertEquals(ID, taskStatus.getId());
+ Assert.assertEquals("task status not found", taskStatus.getErrorMsg());
+ Assert.assertEquals(KubernetesPeonLifecycle.State.STOPPED, peonLifecycle.getState());
+ }
+
+ @Test
+ public void test_join() throws IOException
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(task, kubernetesClient, taskLogs, mapper);
+
+ Job job = new JobBuilder()
+ .withNewMetadata()
+ .withName(ID)
+ .endMetadata()
+ .withNewStatus()
+ .withSucceeded(1)
+ .withStartTime("2022-09-19T23:31:50Z")
+ .withCompletionTime("2022-09-19T23:32:48Z")
+ .endStatus()
+ .build();
+
+ EasyMock.expect(kubernetesClient.waitForPeonJobCompletion(
+ EasyMock.eq(k8sTaskId),
+ EasyMock.anyLong(),
+ EasyMock.eq(TimeUnit.MILLISECONDS)
+ )).andReturn(new JobResponse(job, PeonPhase.SUCCEEDED));
+ EasyMock.expect(kubernetesClient.getPeonLogs(k8sTaskId)).andReturn(Optional.of(
+ IOUtils.toInputStream("", StandardCharsets.UTF_8)
+ ));
+ EasyMock.expect(taskLogs.streamTaskStatus(ID)).andReturn(Optional.of(
+ IOUtils.toInputStream(mapper.writeValueAsString(SUCCESS), StandardCharsets.UTF_8)
+ ));
+ taskLogs.pushTaskLog(EasyMock.eq(ID), EasyMock.anyObject(File.class));
+ EasyMock.expectLastCall();
+ EasyMock.expect(kubernetesClient.deletePeonJob(k8sTaskId)).andReturn(true);
+
+ Assert.assertEquals(KubernetesPeonLifecycle.State.NOT_STARTED, peonLifecycle.getState());
+
+ replayAll();
+
+ TaskStatus taskStatus = peonLifecycle.join(0L);
+
+ verifyAll();
+
+ Assert.assertEquals(SUCCESS.withDuration(58000), taskStatus);
+ Assert.assertEquals(KubernetesPeonLifecycle.State.STOPPED, peonLifecycle.getState());
+ }
+
+ @Test
+ public void test_join_whenCalledMultipleTimes_raisesIllegalStateException() throws IOException
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(task, kubernetesClient, taskLogs, mapper);
+
+ Job job = new JobBuilder()
+ .withNewMetadata()
+ .withName(ID)
+ .endMetadata()
+ .withNewStatus()
+ .withSucceeded(1)
+ .endStatus()
+ .build();
+
+ EasyMock.expect(kubernetesClient.waitForPeonJobCompletion(
+ EasyMock.eq(k8sTaskId),
+ EasyMock.anyLong(),
+ EasyMock.eq(TimeUnit.MILLISECONDS)
+ )).andReturn(new JobResponse(job, PeonPhase.SUCCEEDED));
+ EasyMock.expect(kubernetesClient.getPeonLogs(k8sTaskId)).andReturn(
+ Optional.of(IOUtils.toInputStream("", StandardCharsets.UTF_8))
+ );
+ EasyMock.expect(taskLogs.streamTaskStatus(ID)).andReturn(
+ Optional.of(IOUtils.toInputStream(mapper.writeValueAsString(SUCCESS), StandardCharsets.UTF_8))
+ );
+ taskLogs.pushTaskLog(EasyMock.eq(ID), EasyMock.anyObject(File.class));
+ EasyMock.expectLastCall();
+ EasyMock.expect(kubernetesClient.deletePeonJob(k8sTaskId)).andReturn(true);
+
+ Assert.assertEquals(KubernetesPeonLifecycle.State.NOT_STARTED, peonLifecycle.getState());
+
+ replayAll();
+
+ TaskStatus taskStatus = peonLifecycle.join(0L);
+
+ Assert.assertThrows(
+ "Task [id] failed to join: invalid peon lifecycle state transition [STOPPED]->[PENDING]",
+ IllegalStateException.class,
+ () -> peonLifecycle.join(0L)
+ );
+
+ verifyAll();
+
+ Assert.assertEquals(SUCCESS, taskStatus);
+ Assert.assertEquals(KubernetesPeonLifecycle.State.STOPPED, peonLifecycle.getState());
+ }
+
+ @Test
+ public void test_join_withoutTaskStatus_returnsFailedTaskStatus() throws IOException
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(task, kubernetesClient, taskLogs, mapper);
+
+ Job job = new JobBuilder()
+ .withNewMetadata()
+ .withName(ID)
+ .endMetadata()
+ .withNewStatus()
+ .withSucceeded(1)
+ .endStatus()
+ .build();
+
+ EasyMock.expect(kubernetesClient.waitForPeonJobCompletion(
+ EasyMock.eq(k8sTaskId),
+ EasyMock.anyLong(),
+ EasyMock.eq(TimeUnit.MILLISECONDS)
+ )).andReturn(new JobResponse(job, PeonPhase.SUCCEEDED));
+ EasyMock.expect(kubernetesClient.getPeonLogs(k8sTaskId)).andReturn(
+ Optional.of(IOUtils.toInputStream("", StandardCharsets.UTF_8))
+ );
+ EasyMock.expect(taskLogs.streamTaskStatus(ID)).andReturn(Optional.absent());
+ taskLogs.pushTaskLog(EasyMock.eq(ID), EasyMock.anyObject(File.class));
+ EasyMock.expectLastCall();
+ EasyMock.expect(kubernetesClient.deletePeonJob(k8sTaskId)).andReturn(true);
+
+ Assert.assertEquals(KubernetesPeonLifecycle.State.NOT_STARTED, peonLifecycle.getState());
+
+ replayAll();
+
+ TaskStatus taskStatus = peonLifecycle.join(0L);
+
+ verifyAll();
+
+ Assert.assertTrue(taskStatus.isFailure());
+ Assert.assertEquals(ID, taskStatus.getId());
+ Assert.assertEquals("task status not found", taskStatus.getErrorMsg());
+ Assert.assertEquals(KubernetesPeonLifecycle.State.STOPPED, peonLifecycle.getState());
+ }
+
+ @Test
+ public void test_join_whenIOExceptionThrownWhileStreamingTaskStatus_returnsFailedTaskStatus() throws IOException
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(task, kubernetesClient, taskLogs, mapper);
+
+ Job job = new JobBuilder()
+ .withNewMetadata()
+ .withName(ID)
+ .endMetadata()
+ .withNewStatus()
+ .withSucceeded(1)
+ .endStatus()
+ .build();
+
+ EasyMock.expect(kubernetesClient.waitForPeonJobCompletion(
+ EasyMock.eq(k8sTaskId),
+ EasyMock.anyLong(),
+ EasyMock.eq(TimeUnit.MILLISECONDS)
+ )).andReturn(new JobResponse(job, PeonPhase.SUCCEEDED));
+ EasyMock.expect(kubernetesClient.getPeonLogs(k8sTaskId)).andReturn(
+ Optional.of(IOUtils.toInputStream("", StandardCharsets.UTF_8))
+ );
+ EasyMock.expect(taskLogs.streamTaskStatus(ID)).andThrow(new IOException());
+ taskLogs.pushTaskLog(EasyMock.eq(ID), EasyMock.anyObject(File.class));
+ EasyMock.expectLastCall();
+ EasyMock.expect(kubernetesClient.deletePeonJob(k8sTaskId)).andReturn(true);
+
+ Assert.assertEquals(KubernetesPeonLifecycle.State.NOT_STARTED, peonLifecycle.getState());
+
+ replayAll();
+
+ TaskStatus taskStatus = peonLifecycle.join(0L);
+
+ verifyAll();
+
+ Assert.assertTrue(taskStatus.isFailure());
+ Assert.assertEquals(ID, taskStatus.getId());
+ Assert.assertEquals("error loading status: null", taskStatus.getErrorMsg());
+ Assert.assertEquals(KubernetesPeonLifecycle.State.STOPPED, peonLifecycle.getState());
+ }
+
+ @Test
+ public void test_join_whenIOExceptionThrownWhileStreamingTaskLogs_isIgnored() throws IOException
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(task, kubernetesClient, taskLogs, mapper);
+
+ Job job = new JobBuilder()
+ .withNewMetadata()
+ .withName(ID)
+ .endMetadata()
+ .withNewStatus()
+ .withSucceeded(1)
+ .endStatus()
+ .build();
+
+ EasyMock.expect(kubernetesClient.waitForPeonJobCompletion(
+ EasyMock.eq(k8sTaskId),
+ EasyMock.anyLong(),
+ EasyMock.eq(TimeUnit.MILLISECONDS)
+ )).andReturn(new JobResponse(job, PeonPhase.SUCCEEDED));
+ EasyMock.expect(kubernetesClient.getPeonLogs(k8sTaskId)).andReturn(
+ Optional.of(IOUtils.toInputStream("", StandardCharsets.UTF_8))
+ );
+ EasyMock.expect(taskLogs.streamTaskStatus(ID)).andReturn(
+ Optional.of(IOUtils.toInputStream(mapper.writeValueAsString(SUCCESS), StandardCharsets.UTF_8))
+ );
+ taskLogs.pushTaskLog(EasyMock.eq(ID), EasyMock.anyObject(File.class));
+ EasyMock.expectLastCall().andThrow(new IOException());
+ EasyMock.expect(kubernetesClient.deletePeonJob(k8sTaskId)).andReturn(true);
+
+ Assert.assertEquals(KubernetesPeonLifecycle.State.NOT_STARTED, peonLifecycle.getState());
+
+ replayAll();
+
+ TaskStatus taskStatus = peonLifecycle.join(0L);
+
+ verifyAll();
+
+ Assert.assertEquals(SUCCESS, taskStatus);
+ Assert.assertEquals(KubernetesPeonLifecycle.State.STOPPED, peonLifecycle.getState());
+ }
+
+ @Test
+ public void test_shutdown_withNotStartedTaskState()
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(task, kubernetesClient, taskLogs, mapper);
+ peonLifecycle.shutdown();
+ }
+
+ @Test
+ public void test_shutdown_withPendingTaskState() throws NoSuchFieldException, IllegalAccessException
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(task, kubernetesClient, taskLogs, mapper);
+ setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.PENDING);
+
+ EasyMock.expect(kubernetesClient.deletePeonJob(k8sTaskId)).andReturn(true);
+
+ replayAll();
+
+ peonLifecycle.shutdown();
+
+ verifyAll();
+ }
+
+ @Test
+ public void test_shutdown_withRunningTaskState() throws NoSuchFieldException, IllegalAccessException
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(task, kubernetesClient, taskLogs, mapper);
+ setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING);
+
+ EasyMock.expect(kubernetesClient.deletePeonJob(k8sTaskId)).andReturn(true);
+
+ replayAll();
+
+ peonLifecycle.shutdown();
+
+ verifyAll();
+ }
+
+ @Test
+ public void test_shutdown_withStoppedTaskState() throws NoSuchFieldException, IllegalAccessException
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(task, kubernetesClient, taskLogs, mapper);
+ setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.STOPPED);
+
+ peonLifecycle.shutdown();
+ }
+
+ @Test
+ public void test_streamLogs_withNotStartedTaskState() throws NoSuchFieldException, IllegalAccessException
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(task, kubernetesClient, taskLogs, mapper);
+ setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.NOT_STARTED);
+
+ peonLifecycle.streamLogs();
+ }
+
+ @Test
+ public void test_streamLogs_withPendingTaskState() throws NoSuchFieldException, IllegalAccessException
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(task, kubernetesClient, taskLogs, mapper);
+ setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.PENDING);
+
+ peonLifecycle.streamLogs();
+ }
+
+ @Test
+ public void test_streamLogs_withRunningTaskState() throws NoSuchFieldException, IllegalAccessException
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(task, kubernetesClient, taskLogs, mapper);
+ setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING);
+
+ EasyMock.expect(kubernetesClient.getPeonLogs(k8sTaskId)).andReturn(
+ Optional.of(IOUtils.toInputStream("", StandardCharsets.UTF_8))
+ );
+
+ replayAll();
+
+ peonLifecycle.streamLogs();
+
+ verifyAll();
+ }
+
+ @Test
+ public void test_streamLogs_withStoppedTaskState() throws NoSuchFieldException, IllegalAccessException
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(task, kubernetesClient, taskLogs, mapper);
+ setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.STOPPED);
+
+ peonLifecycle.streamLogs();
+ }
+
+ @Test
+ public void test_getTaskLocation_withNotStartedTaskState_returnsUnknown()
+ throws NoSuchFieldException, IllegalAccessException
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(task, kubernetesClient, taskLogs, mapper);
+ setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.NOT_STARTED);
+
+ Assert.assertEquals(TaskLocation.unknown(), peonLifecycle.getTaskLocation());
+ }
+
+ @Test
+ public void test_getTaskLocation_withPendingTaskState_returnsUnknown()
+ throws NoSuchFieldException, IllegalAccessException
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(task, kubernetesClient, taskLogs, mapper);
+ setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.PENDING);
+
+ Assert.assertEquals(TaskLocation.unknown(), peonLifecycle.getTaskLocation());
+ }
+
+ @Test
+ public void test_getTaskLocation_withRunningTaskState_withoutPeonPod_returnsUnknown()
+ throws NoSuchFieldException, IllegalAccessException
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(task, kubernetesClient, taskLogs, mapper);
+ setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING);
+
+ EasyMock.expect(kubernetesClient.getPeonPod(k8sTaskId)).andReturn(Optional.absent());
+
+ replayAll();
+
+ Assert.assertEquals(TaskLocation.unknown(), peonLifecycle.getTaskLocation());
+
+ verifyAll();
+ }
+
+ @Test
+ public void test_getTaskLocation_withRunningTaskState_withPeonPodWithoutStatus_returnsUnknown()
+ throws NoSuchFieldException, IllegalAccessException
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(task, kubernetesClient, taskLogs, mapper);
+ setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING);
+
+ Pod pod = new PodBuilder()
+ .withNewMetadata()
+ .withName(ID)
+ .endMetadata()
+ .build();
+
+ EasyMock.expect(kubernetesClient.getPeonPod(k8sTaskId)).andReturn(Optional.of(pod));
+
+ replayAll();
+
+ Assert.assertEquals(TaskLocation.unknown(), peonLifecycle.getTaskLocation());
+
+ verifyAll();
+ }
+
+ @Test
+ public void test_getTaskLocation_withRunningTaskState_withPeonPodWithStatus_returnsLocation()
+ throws NoSuchFieldException, IllegalAccessException
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(task, kubernetesClient, taskLogs, mapper);
+ setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING);
+
+ Pod pod = new PodBuilder()
+ .withNewMetadata()
+ .withName(ID)
+ .endMetadata()
+ .withNewStatus()
+ .withPodIP("ip")
+ .endStatus()
+ .build();
+
+ EasyMock.expect(kubernetesClient.getPeonPod(k8sTaskId)).andReturn(Optional.of(pod));
+
+ replayAll();
+
+ TaskLocation location = peonLifecycle.getTaskLocation();
+
+ Assert.assertEquals("ip", location.getHost());
+ Assert.assertEquals(8100, location.getPort());
+ Assert.assertEquals(-1, location.getTlsPort());
+
+ verifyAll();
+ }
+
+ @Test
+ public void test_getTaskLocation_withRunningTaskState_withPeonPodWithStatusWithTLSAnnotation_returnsLocation()
+ throws NoSuchFieldException, IllegalAccessException
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(task, kubernetesClient, taskLogs, mapper);
+ setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING);
+
+ Pod pod = new PodBuilder()
+ .withNewMetadata()
+ .withName(ID)
+ .addToAnnotations("tls.enabled", "true")
+ .endMetadata()
+ .withNewStatus()
+ .withPodIP("ip")
+ .endStatus()
+ .build();
+
+ EasyMock.expect(kubernetesClient.getPeonPod(k8sTaskId)).andReturn(Optional.of(pod));
+
+ replayAll();
+
+ TaskLocation location = peonLifecycle.getTaskLocation();
+
+ Assert.assertEquals("ip", location.getHost());
+ Assert.assertEquals(-1, location.getPort());
+ Assert.assertEquals(8091, location.getTlsPort());
+
+ verifyAll();
+ }
+
+ @Test
+ public void test_getTaskLocation_withStoppedTaskState_returnsUnknown()
+ throws NoSuchFieldException, IllegalAccessException
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(task, kubernetesClient, taskLogs, mapper);
+ setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.STOPPED);
+
+ Assert.assertEquals(TaskLocation.unknown(), peonLifecycle.getTaskLocation());
+ }
+
+ private void setPeonLifecycleState(KubernetesPeonLifecycle peonLifecycle, KubernetesPeonLifecycle.State state)
+ throws NoSuchFieldException, IllegalAccessException
+ {
+ Field stateField = peonLifecycle.getClass().getDeclaredField("state");
+ stateField.setAccessible(true);
+ stateField.set(peonLifecycle, new AtomicReference<>(state));
+ }
+}
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfigTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfigTest.java
new file mode 100644
index 00000000000..1f4a7281f64
--- /dev/null
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfigTest.java
@@ -0,0 +1,123 @@
+/*
+ * 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.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.joda.time.Period;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+
+public class KubernetesTaskRunnerConfigTest
+{
+ @Test
+ public void test_deserializable() throws IOException
+ {
+ ObjectMapper mapper = new DefaultObjectMapper();
+ KubernetesTaskRunnerConfig config = mapper.readValue(
+ this.getClass().getClassLoader().getResource("kubernetesTaskRunnerConfig.json"),
+ KubernetesTaskRunnerConfig.class
+ );
+
+ Assert.assertEquals("namespace", config.getNamespace());
+ Assert.assertFalse(config.isDebugJobs());
+ Assert.assertEquals("name", config.getPrimaryContainerName());
+ Assert.assertEquals("karlkfi/kubexit:v0.3.2", config.getKubexitImage());
+ Assert.assertNull(config.getGraceTerminationPeriodSeconds());
+ Assert.assertTrue(config.isDisableClientProxy());
+ Assert.assertEquals(new Period("PT4H"), config.getTaskTimeout());
+ Assert.assertEquals(new Period("P2D"), config.getTaskCleanupDelay());
+ Assert.assertEquals(new Period("PT10m"), config.getTaskCleanupInterval());
+ Assert.assertEquals(new Period("PT1H"), config.getTaskLaunchTimeout());
+ Assert.assertEquals(ImmutableList.of(), config.getPeonMonitors());
+ Assert.assertEquals(ImmutableList.of(), config.getJavaOptsArray());
+ Assert.assertEquals(ImmutableMap.of(), config.getLabels());
+ Assert.assertEquals(ImmutableMap.of(), config.getAnnotations());
+ Assert.assertEquals(Integer.valueOf(Integer.MAX_VALUE), config.getCapacity());
+ }
+
+ @Test
+ public void test_builder_preservesDefaults()
+ {
+ KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder()
+ .withNamespace("namespace")
+ .withDisableClientProxy(true)
+ .build();
+
+ Assert.assertEquals("namespace", config.getNamespace());
+ Assert.assertFalse(config.isDebugJobs());
+ Assert.assertNull(config.getPrimaryContainerName());
+ Assert.assertEquals("karlkfi/kubexit:v0.3.2", config.getKubexitImage());
+ Assert.assertNull(config.getGraceTerminationPeriodSeconds());
+ Assert.assertTrue(config.isDisableClientProxy());
+ Assert.assertEquals(new Period("PT4H"), config.getTaskTimeout());
+ Assert.assertEquals(new Period("P2D"), config.getTaskCleanupDelay());
+ Assert.assertEquals(new Period("PT10m"), config.getTaskCleanupInterval());
+ Assert.assertEquals(new Period("PT1H"), config.getTaskLaunchTimeout());
+ Assert.assertEquals(ImmutableList.of(), config.getPeonMonitors());
+ Assert.assertEquals(ImmutableList.of(), config.getJavaOptsArray());
+ Assert.assertEquals(ImmutableMap.of(), config.getLabels());
+ Assert.assertEquals(ImmutableMap.of(), config.getAnnotations());
+ Assert.assertEquals(Integer.valueOf(Integer.MAX_VALUE), config.getCapacity());
+ }
+
+ @Test
+ public void test_builder()
+ {
+ KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder()
+ .withNamespace("namespace")
+ .withDebugJob(true)
+ .withSidecarSupport(true)
+ .withPrimaryContainerName("primary")
+ .withKubexitImage("image")
+ .withGraceTerminationPeriodSeconds(0L)
+ .withDisableClientProxy(true)
+ .withTaskTimeout(new Period("PT2H"))
+ .withTaskCleanupDelay(new Period("P1D"))
+ .withTaskCleanupInterval(new Period("PT1H"))
+ .withK8sJobLaunchTimeout(new Period("PT1H"))
+ .withPeonMonitors(ImmutableList.of("monitor"))
+ .withJavaOptsArray(ImmutableList.of("option"))
+ .withLabels(ImmutableMap.of("key", "value"))
+ .withAnnotations(ImmutableMap.of("key", "value"))
+ .withCapacity(1)
+ .build();
+
+ Assert.assertEquals("namespace", config.getNamespace());
+ Assert.assertTrue(config.isDebugJobs());
+ Assert.assertEquals("primary", config.getPrimaryContainerName());
+ Assert.assertEquals("image", config.getKubexitImage());
+ Assert.assertEquals(Long.valueOf(0), config.getGraceTerminationPeriodSeconds());
+ Assert.assertTrue(config.isDisableClientProxy());
+ Assert.assertEquals(new Period("PT2H"), config.getTaskTimeout());
+ Assert.assertEquals(new Period("P1D"), config.getTaskCleanupDelay());
+ Assert.assertEquals(new Period("PT1H"), config.getTaskCleanupInterval());
+ Assert.assertEquals(new Period("PT1H"), config.getTaskLaunchTimeout());
+ Assert.assertEquals(ImmutableList.of("monitor"), config.getPeonMonitors());
+ Assert.assertEquals(ImmutableList.of("option"), config.getJavaOptsArray());
+ Assert.assertEquals(ImmutableMap.of("key", "value"), config.getLabels());
+ Assert.assertEquals(ImmutableMap.of("key", "value"), config.getAnnotations());
+ Assert.assertEquals(Integer.valueOf(1), config.getCapacity());
+ }
+}
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java
index 8badaafd3a4..c79ad753e35 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java
@@ -23,11 +23,10 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.druid.indexing.common.TestUtils;
import org.apache.druid.indexing.common.config.TaskConfig;
import org.apache.druid.indexing.common.config.TaskConfigBuilder;
-import org.apache.druid.indexing.overlord.config.TaskQueueConfig;
import org.apache.druid.java.util.common.IAE;
-import org.apache.druid.k8s.overlord.common.MultiContainerTaskAdapter;
-import org.apache.druid.k8s.overlord.common.PodTemplateTaskAdapter;
-import org.apache.druid.k8s.overlord.common.SingleContainerTaskAdapter;
+import org.apache.druid.k8s.overlord.taskadapter.MultiContainerTaskAdapter;
+import org.apache.druid.k8s.overlord.taskadapter.PodTemplateTaskAdapter;
+import org.apache.druid.k8s.overlord.taskadapter.SingleContainerTaskAdapter;
import org.apache.druid.server.DruidNode;
import org.apache.druid.server.log.StartupLoggingConfig;
import org.apache.druid.tasklogs.NoopTaskLogs;
@@ -44,7 +43,6 @@ public class KubernetesTaskRunnerFactoryTest
private ObjectMapper objectMapper;
private KubernetesTaskRunnerConfig kubernetesTaskRunnerConfig;
private StartupLoggingConfig startupLoggingConfig;
- private TaskQueueConfig taskQueueConfig;
private TaskLogs taskLogs;
private DruidNode druidNode;
private TaskConfig taskConfig;
@@ -54,14 +52,10 @@ public class KubernetesTaskRunnerFactoryTest
public void setup()
{
objectMapper = new TestUtils().getTestObjectMapper();
- kubernetesTaskRunnerConfig = new KubernetesTaskRunnerConfig();
+ kubernetesTaskRunnerConfig = KubernetesTaskRunnerConfig.builder()
+ .withCapacity(1)
+ .build();
startupLoggingConfig = new StartupLoggingConfig();
- taskQueueConfig = new TaskQueueConfig(
- 1,
- null,
- null,
- null
- );
taskLogs = new NoopTaskLogs();
druidNode = new DruidNode(
"test",
@@ -84,7 +78,6 @@ public class KubernetesTaskRunnerFactoryTest
null,
kubernetesTaskRunnerConfig,
startupLoggingConfig,
- taskQueueConfig,
taskLogs,
druidNode,
taskConfig,
@@ -105,7 +98,6 @@ public class KubernetesTaskRunnerFactoryTest
null,
kubernetesTaskRunnerConfig,
startupLoggingConfig,
- taskQueueConfig,
taskLogs,
druidNode,
taskConfig,
@@ -121,14 +113,16 @@ public class KubernetesTaskRunnerFactoryTest
@Test
public void test_build_withSidecarSupport_returnsKubernetesTaskRunnerWithMultiContainerTaskAdapter()
{
- kubernetesTaskRunnerConfig.sidecarSupport = true;
+ kubernetesTaskRunnerConfig = KubernetesTaskRunnerConfig.builder()
+ .withCapacity(1)
+ .withSidecarSupport(true)
+ .build();
KubernetesTaskRunnerFactory factory = new KubernetesTaskRunnerFactory(
objectMapper,
null,
kubernetesTaskRunnerConfig,
startupLoggingConfig,
- taskQueueConfig,
taskLogs,
druidNode,
taskConfig,
@@ -152,7 +146,6 @@ public class KubernetesTaskRunnerFactoryTest
null,
kubernetesTaskRunnerConfig,
startupLoggingConfig,
- taskQueueConfig,
taskLogs,
druidNode,
taskConfig,
@@ -168,7 +161,10 @@ public class KubernetesTaskRunnerFactoryTest
@Test
public void test_build_withSingleContainerAdapterTypeAndSidecarSupport_throwsIAE()
{
- kubernetesTaskRunnerConfig.sidecarSupport = true;
+ kubernetesTaskRunnerConfig = KubernetesTaskRunnerConfig.builder()
+ .withCapacity(1)
+ .withSidecarSupport(true)
+ .build();
Properties props = new Properties();
props.setProperty("druid.indexer.runner.k8s.adapter.type", "overlordSingleContainer");
@@ -178,7 +174,6 @@ public class KubernetesTaskRunnerFactoryTest
null,
kubernetesTaskRunnerConfig,
startupLoggingConfig,
- taskQueueConfig,
taskLogs,
druidNode,
taskConfig,
@@ -195,7 +190,10 @@ public class KubernetesTaskRunnerFactoryTest
@Test
public void test_build_withMultiContainerAdapterType_returnsKubernetesTaskRunnerWithMultiContainerTaskAdapter()
{
- kubernetesTaskRunnerConfig.sidecarSupport = true;
+ kubernetesTaskRunnerConfig = KubernetesTaskRunnerConfig.builder()
+ .withCapacity(1)
+ .withSidecarSupport(true)
+ .build();
Properties props = new Properties();
props.setProperty("druid.indexer.runner.k8s.adapter.type", "overlordMultiContainer");
@@ -205,7 +203,6 @@ public class KubernetesTaskRunnerFactoryTest
null,
kubernetesTaskRunnerConfig,
startupLoggingConfig,
- taskQueueConfig,
taskLogs,
druidNode,
taskConfig,
@@ -229,7 +226,6 @@ public class KubernetesTaskRunnerFactoryTest
null,
kubernetesTaskRunnerConfig,
startupLoggingConfig,
- taskQueueConfig,
taskLogs,
druidNode,
taskConfig,
@@ -256,7 +252,6 @@ public class KubernetesTaskRunnerFactoryTest
null,
kubernetesTaskRunnerConfig,
startupLoggingConfig,
- taskQueueConfig,
taskLogs,
druidNode,
taskConfig,
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java
index c31a2a42f86..4a3dd322a73 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java
@@ -19,950 +19,585 @@
package org.apache.druid.k8s.overlord;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.databind.Module;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.google.common.base.Optional;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Iterables;
+import com.google.common.collect.ImmutableList;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
-import io.fabric8.kubernetes.api.model.ObjectMeta;
-import io.fabric8.kubernetes.api.model.Pod;
-import io.fabric8.kubernetes.api.model.PodStatus;
import io.fabric8.kubernetes.api.model.batch.v1.Job;
-import io.fabric8.kubernetes.api.model.batch.v1.JobStatus;
+import io.fabric8.kubernetes.api.model.batch.v1.JobBuilder;
import org.apache.commons.io.IOUtils;
-import org.apache.druid.data.input.FirehoseFactory;
-import org.apache.druid.guice.FirehoseModule;
import org.apache.druid.indexer.RunnerTaskState;
import org.apache.druid.indexer.TaskLocation;
-import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexer.TaskStatus;
-import org.apache.druid.indexing.common.TestUtils;
-import org.apache.druid.indexing.common.task.IndexTask;
import org.apache.druid.indexing.common.task.NoopTask;
import org.apache.druid.indexing.common.task.Task;
-import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig;
import org.apache.druid.indexing.overlord.TaskRunnerWorkItem;
-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.StringUtils;
+import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.http.client.HttpClient;
import org.apache.druid.java.util.http.client.Request;
import org.apache.druid.java.util.http.client.response.InputStreamResponseHandler;
-import org.apache.druid.k8s.overlord.common.DruidK8sConstants;
-import org.apache.druid.k8s.overlord.common.DruidKubernetesPeonClient;
-import org.apache.druid.k8s.overlord.common.JobResponse;
-import org.apache.druid.k8s.overlord.common.K8sTaskAdapter;
-import org.apache.druid.k8s.overlord.common.K8sTaskId;
import org.apache.druid.k8s.overlord.common.KubernetesPeonClient;
-import org.apache.druid.k8s.overlord.common.KubernetesResourceNotFoundException;
-import org.apache.druid.k8s.overlord.common.PeonPhase;
-import org.apache.druid.server.DruidNode;
-import org.apache.druid.server.log.StartupLoggingConfig;
-import org.apache.druid.tasklogs.TaskLogs;
-import org.joda.time.Period;
+import org.apache.druid.k8s.overlord.taskadapter.TaskAdapter;
+import org.easymock.EasyMock;
+import org.easymock.EasyMockRunner;
+import org.easymock.EasyMockSupport;
+import org.easymock.Mock;
+import org.hamcrest.MatcherAssert;
+import org.hamcrest.Matchers;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
+import org.junit.runner.RunWith;
+import java.io.IOException;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
import java.util.Collection;
import java.util.Collections;
+import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
import java.util.concurrent.TimeUnit;
import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertThrows;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.anyLong;
-import static org.mockito.ArgumentMatchers.eq;
-import static org.mockito.ArgumentMatchers.isA;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
-public class KubernetesTaskRunnerTest
+@RunWith(EasyMockRunner.class)
+public class KubernetesTaskRunnerTest extends EasyMockSupport
{
- private TaskQueueConfig taskQueueConfig;
- private StartupLoggingConfig startupLoggingConfig;
- private ObjectMapper jsonMapper;
- private KubernetesTaskRunnerConfig kubernetesTaskRunnerConfig;
- private TaskLogs taskLogs;
- private DruidNode druidNode;
+ private static final String ID = "id";
+
+ @Mock private HttpClient httpClient;
+ @Mock private TaskAdapter taskAdapter;
+ @Mock private KubernetesPeonClient peonClient;
+ @Mock private KubernetesPeonLifecycle kubernetesPeonLifecycle;
+
+ private KubernetesTaskRunnerConfig config;
+ private KubernetesTaskRunner runner;
+ private Task task;
@Before
- public void setUp()
+ public void setup()
{
- TestUtils utils = new TestUtils();
- jsonMapper = utils.getTestObjectMapper();
- for (Module jacksonModule : new FirehoseModule().getJacksonModules()) {
- jsonMapper.registerModule(jacksonModule);
- }
- jsonMapper.registerSubtypes(
- new NamedType(ParallelIndexTuningConfig.class, "index_parallel"),
- new NamedType(IndexTask.IndexTuningConfig.class, "index")
+ config = KubernetesTaskRunnerConfig.builder()
+ .withCapacity(1)
+ .build();
+
+ task = NoopTask.create(ID, 0);
+
+ runner = new KubernetesTaskRunner(
+ taskAdapter,
+ config,
+ peonClient,
+ httpClient,
+ new TestPeonLifecycleFactory(kubernetesPeonLifecycle)
);
- kubernetesTaskRunnerConfig = new KubernetesTaskRunnerConfig();
- kubernetesTaskRunnerConfig.namespace = "test";
- kubernetesTaskRunnerConfig.javaOptsArray = Collections.singletonList("-Xmx2g");
- taskQueueConfig = new TaskQueueConfig(1, Period.millis(1), Period.millis(1), Period.millis(1));
- startupLoggingConfig = new StartupLoggingConfig();
- taskLogs = mock(TaskLogs.class);
- druidNode = mock(DruidNode.class);
- when(druidNode.isEnableTlsPort()).thenReturn(false);
}
@Test
- public void testAlreadyRunningJobInK8s() throws Exception
+ public void test_streamTaskLog_withoutExistingTask_returnsEmptyOptional()
{
- Task task = makeTask();
- K8sTaskId k8sTaskId = new K8sTaskId(task.getId());
+ Optional maybeInputStream = runner.streamTaskLog(task.getId(), 0L);
+ assertFalse(maybeInputStream.isPresent());
+ }
- ObjectMeta metadata = mock(ObjectMeta.class);
- when(metadata.getName()).thenReturn("jobName");
+ @Test
+ public void test_streamTaskLog_withExistingTask() throws IOException
+ {
+ KubernetesWorkItem workItem = new KubernetesWorkItem(task, null)
+ {
+ @Override
+ protected Optional streamTaskLogs()
+ {
+ return Optional.of(IOUtils.toInputStream("", StandardCharsets.UTF_8));
+ }
+ };
- Job job = mock(Job.class);
- JobStatus status = mock(JobStatus.class);
- when(job.getMetadata()).thenReturn(metadata);
- when(status.getActive()).thenReturn(1).thenReturn(null);
- when(job.getStatus()).thenReturn(status);
+ runner.tasks.put(task.getId(), workItem);
- Pod peonPod = mock(Pod.class);
- when(peonPod.getMetadata()).thenReturn(metadata);
- PodStatus podStatus = mock(PodStatus.class);
- when(podStatus.getPodIP()).thenReturn("SomeIP");
- when(peonPod.getStatus()).thenReturn(podStatus);
+ Optional maybeInputStream = runner.streamTaskLog(task.getId(), 0L);
- K8sTaskAdapter adapter = mock(K8sTaskAdapter.class);
- when(adapter.fromTask(eq(task))).thenReturn(job);
+ assertTrue(maybeInputStream.isPresent());
+ assertEquals("", IOUtils.toString(maybeInputStream.get(), StandardCharsets.UTF_8));
+ }
- DruidKubernetesPeonClient peonClient = mock(DruidKubernetesPeonClient.class);
-
- when(peonClient.jobExists(eq(k8sTaskId))).thenReturn(Optional.of(job));
- when(peonClient.getMainJobPod(eq(k8sTaskId))).thenReturn(peonPod);
- when(peonClient.waitForJobCompletion(eq(k8sTaskId), anyLong(), isA(TimeUnit.class))).thenReturn(new JobResponse(
- job,
- PeonPhase.SUCCEEDED
- ));
- when(peonClient.getPeonLogs(eq(k8sTaskId))).thenReturn(Optional.absent());
- when(peonClient.cleanUpJob(eq(k8sTaskId))).thenReturn(true);
+ @Test
+ public void test_run_withoutExistingTask() throws IOException, ExecutionException, InterruptedException
+ {
+ Job job = new JobBuilder()
+ .withNewMetadata()
+ .withName(ID)
+ .endMetadata()
+ .build();
TaskStatus taskStatus = TaskStatus.success(task.getId());
- when(taskLogs.streamTaskStatus(eq(task.getId()))).thenReturn(Optional.of(IOUtils.toInputStream(
- jsonMapper.writeValueAsString(taskStatus),
- StandardCharsets.UTF_8))
- );
- KubernetesTaskRunner taskRunner = new KubernetesTaskRunner(
- jsonMapper,
- adapter,
- kubernetesTaskRunnerConfig,
- taskQueueConfig,
- taskLogs,
- peonClient,
- null
- );
- KubernetesTaskRunner spyRunner = spy(taskRunner);
+ EasyMock.expect(taskAdapter.fromTask(task)).andReturn(job);
+ EasyMock.expect(kubernetesPeonLifecycle.run(
+ EasyMock.eq(job),
+ EasyMock.anyLong(),
+ EasyMock.anyLong()
+ )).andReturn(taskStatus);
- ListenableFuture future = spyRunner.run(task);
- TaskStatus actualTaskStatus = future.get();
- Assert.assertEquals(taskStatus, actualTaskStatus);
- // we should never launch the job here, one exists
- verify(peonClient, never()).launchJobAndWaitForStart(isA(Job.class), anyLong(), isA(TimeUnit.class));
- verify(peonClient, times(1)).cleanUpJob(eq(k8sTaskId));
- verify(spyRunner, times(1)).updateStatus(eq(task), eq(taskStatus));
+ replayAll();
+
+ ListenableFuture future = runner.run(task);
+ Assert.assertEquals(taskStatus, future.get());
+
+ verifyAll();
+
+ Assert.assertFalse(runner.tasks.containsKey(task.getId()));
}
@Test
- public void testJobNeedsToLaunchInK8s() throws Exception
+ public void test_run_withExistingTask_returnsExistingWorkItem()
{
- Task task = makeTask();
- K8sTaskId k8sTaskId = new K8sTaskId(task.getId());
+ KubernetesWorkItem workItem = new KubernetesWorkItem(task, null);
+ runner.tasks.put(task.getId(), workItem);
- Job job = mock(Job.class);
- ObjectMeta jobMetadata = mock(ObjectMeta.class);
- when(jobMetadata.getName()).thenReturn(k8sTaskId.getK8sTaskId());
- JobStatus status = mock(JobStatus.class);
- when(status.getActive()).thenReturn(1).thenReturn(null);
- when(job.getStatus()).thenReturn(status);
- when(job.getMetadata()).thenReturn(jobMetadata);
+ ListenableFuture future = runner.run(task);
- Pod peonPod = mock(Pod.class);
- ObjectMeta metadata = mock(ObjectMeta.class);
- when(metadata.getName()).thenReturn("peonPodName");
- when(peonPod.getMetadata()).thenReturn(metadata);
- PodStatus podStatus = mock(PodStatus.class);
- when(podStatus.getPodIP()).thenReturn("SomeIP");
- when(peonPod.getStatus()).thenReturn(podStatus);
+ assertEquals(workItem.getResult(), future);
+ }
- K8sTaskAdapter adapter = mock(K8sTaskAdapter.class);
- when(adapter.fromTask(eq(task))).thenReturn(job);
+ @Test
+ public void test_run_whenExceptionThrown_throwsRuntimeException() throws IOException
+ {
+ Job job = new JobBuilder()
+ .withNewMetadata()
+ .withName(ID)
+ .endMetadata()
+ .build();
- DruidKubernetesPeonClient peonClient = mock(DruidKubernetesPeonClient.class);
+ EasyMock.expect(taskAdapter.fromTask(task)).andReturn(job);
+ EasyMock.expect(kubernetesPeonLifecycle.run(
+ EasyMock.eq(job),
+ EasyMock.anyLong(),
+ EasyMock.anyLong()
+ )).andThrow(new IllegalStateException());
- when(peonClient.jobExists(eq(k8sTaskId))).thenReturn(Optional.fromNullable(null));
- when(peonClient.launchJobAndWaitForStart(isA(Job.class), anyLong(), isA(TimeUnit.class))).thenReturn(peonPod);
- when(peonClient.getMainJobPod(eq(k8sTaskId))).thenReturn(peonPod);
- when(peonClient.waitForJobCompletion(eq(k8sTaskId), anyLong(), isA(TimeUnit.class))).thenReturn(new JobResponse(
- job,
- PeonPhase.SUCCEEDED
- ));
+ replayAll();
+ ListenableFuture future = runner.run(task);
+
+ Exception e = Assert.assertThrows(ExecutionException.class, future::get);
+ Assert.assertTrue(e.getCause() instanceof RuntimeException);
+
+ verifyAll();
+
+ Assert.assertFalse(runner.tasks.containsKey(task.getId()));
+ }
+
+ @Test
+ public void test_join_withoutExistingTask() throws ExecutionException, InterruptedException
+ {
TaskStatus taskStatus = TaskStatus.success(task.getId());
- when(taskLogs.streamTaskStatus(eq(task.getId()))).thenReturn(Optional.of(IOUtils.toInputStream(
- jsonMapper.writeValueAsString(taskStatus),
- StandardCharsets.UTF_8))
- );
- when(peonClient.getPeonLogs(eq(k8sTaskId))).thenReturn(Optional.absent());
- when(peonClient.cleanUpJob(eq(k8sTaskId))).thenReturn(true);
+ EasyMock.expect(kubernetesPeonLifecycle.join(EasyMock.anyLong())).andReturn(taskStatus);
- KubernetesTaskRunner taskRunner = new KubernetesTaskRunner(
- jsonMapper,
- adapter,
- kubernetesTaskRunnerConfig,
- taskQueueConfig,
- taskLogs,
- peonClient,
- null
- );
- KubernetesTaskRunner spyRunner = spy(taskRunner);
+ replayAll();
- ListenableFuture future = spyRunner.run(task);
- TaskStatus actualTaskStatus = future.get();
- Assert.assertEquals(taskStatus, actualTaskStatus);
- // we should never launch the job here, one exists
- verify(peonClient, times(1)).launchJobAndWaitForStart(isA(Job.class), anyLong(), isA(TimeUnit.class));
- verify(peonClient, times(1)).cleanUpJob(eq(k8sTaskId));
- verify(spyRunner, times(1)).updateStatus(eq(task), eq(taskStatus));
+ ListenableFuture future = runner.joinAsync(task);
+ Assert.assertEquals(taskStatus, future.get());
+
+ verifyAll();
+
+ Assert.assertFalse(runner.tasks.containsKey(task.getId()));
}
@Test
- public void test_run_withSuccessfulJobAndWithoutStatusFile_returnsSucessfulTask() throws Exception
+ public void test_join_withExistingTask_returnsExistingWorkItem()
{
- Task task = makeTask();
- K8sTaskId k8sTaskId = new K8sTaskId(task.getId());
+ KubernetesWorkItem workItem = new KubernetesWorkItem(task, null);
+ runner.tasks.put(task.getId(), workItem);
- Job job = mock(Job.class);
- ObjectMeta jobMetadata = mock(ObjectMeta.class);
- when(jobMetadata.getName()).thenReturn(k8sTaskId.getK8sTaskId());
- JobStatus status = mock(JobStatus.class);
- when(status.getActive()).thenReturn(1).thenReturn(null);
- when(job.getStatus()).thenReturn(status);
- when(job.getMetadata()).thenReturn(jobMetadata);
+ ListenableFuture future = runner.run(task);
- Pod peonPod = mock(Pod.class);
- ObjectMeta metadata = mock(ObjectMeta.class);
- when(metadata.getName()).thenReturn("peonPodName");
- when(peonPod.getMetadata()).thenReturn(metadata);
- PodStatus podStatus = mock(PodStatus.class);
- when(podStatus.getPodIP()).thenReturn("SomeIP");
- when(peonPod.getStatus()).thenReturn(podStatus);
-
- K8sTaskAdapter adapter = mock(K8sTaskAdapter.class);
- when(adapter.fromTask(eq(task))).thenReturn(job);
-
- DruidKubernetesPeonClient peonClient = mock(DruidKubernetesPeonClient.class);
-
- when(peonClient.jobExists(eq(k8sTaskId))).thenReturn(Optional.fromNullable(null));
- when(peonClient.launchJobAndWaitForStart(isA(Job.class), anyLong(), isA(TimeUnit.class))).thenReturn(peonPod);
- when(peonClient.getMainJobPod(eq(k8sTaskId))).thenReturn(peonPod);
- when(peonClient.waitForJobCompletion(eq(k8sTaskId), anyLong(), isA(TimeUnit.class))).thenReturn(new JobResponse(
- job,
- PeonPhase.SUCCEEDED
- ));
-
- when(taskLogs.streamTaskStatus(eq(task.getId()))).thenReturn(Optional.absent());
-
- when(peonClient.getPeonLogs(eq(k8sTaskId))).thenReturn(Optional.absent());
- when(peonClient.cleanUpJob(eq(k8sTaskId))).thenReturn(true);
-
- KubernetesTaskRunner taskRunner = new KubernetesTaskRunner(
- jsonMapper,
- adapter,
- kubernetesTaskRunnerConfig,
- taskQueueConfig,
- taskLogs,
- peonClient,
- null
- );
- KubernetesTaskRunner spyRunner = spy(taskRunner);
-
- ListenableFuture future = spyRunner.run(task);
- TaskStatus actualTaskStatus = future.get();
- Assert.assertTrue(actualTaskStatus.isSuccess());
-
- // we should never launch the job here, one exists
- verify(peonClient, times(1)).launchJobAndWaitForStart(isA(Job.class), anyLong(), isA(TimeUnit.class));
- verify(peonClient, times(1)).cleanUpJob(eq(k8sTaskId));
- verify(spyRunner, times(1)).updateStatus(eq(task), eq(actualTaskStatus));
+ assertEquals(workItem.getResult(), future);
}
@Test
- public void test_run_withFailedJob_returnsFailedTask() throws Exception
+ public void test_join_whenExceptionThrown_throwsRuntimeException()
{
- Task task = makeTask();
- K8sTaskId k8sTaskId = new K8sTaskId(task.getId());
+ EasyMock.expect(kubernetesPeonLifecycle.join(EasyMock.anyLong())).andThrow(new IllegalStateException());
- Job job = mock(Job.class);
- ObjectMeta jobMetadata = mock(ObjectMeta.class);
- when(jobMetadata.getName()).thenReturn(k8sTaskId.getK8sTaskId());
- JobStatus status = mock(JobStatus.class);
- when(status.getActive()).thenReturn(1).thenReturn(null);
- when(job.getStatus()).thenReturn(status);
- when(job.getMetadata()).thenReturn(jobMetadata);
+ replayAll();
- Pod peonPod = mock(Pod.class);
- ObjectMeta metadata = mock(ObjectMeta.class);
- when(metadata.getName()).thenReturn("peonPodName");
- when(peonPod.getMetadata()).thenReturn(metadata);
- PodStatus podStatus = mock(PodStatus.class);
- when(podStatus.getPodIP()).thenReturn("SomeIP");
- when(peonPod.getStatus()).thenReturn(podStatus);
+ ListenableFuture future = runner.joinAsync(task);
- K8sTaskAdapter adapter = mock(K8sTaskAdapter.class);
- when(adapter.fromTask(eq(task))).thenReturn(job);
+ Exception e = Assert.assertThrows(ExecutionException.class, future::get);
+ Assert.assertTrue(e.getCause() instanceof RuntimeException);
- DruidKubernetesPeonClient peonClient = mock(DruidKubernetesPeonClient.class);
+ verifyAll();
- when(peonClient.jobExists(eq(k8sTaskId))).thenReturn(Optional.fromNullable(null));
- when(peonClient.launchJobAndWaitForStart(isA(Job.class), anyLong(), isA(TimeUnit.class))).thenReturn(peonPod);
- when(peonClient.getMainJobPod(eq(k8sTaskId))).thenReturn(peonPod);
- when(peonClient.waitForJobCompletion(eq(k8sTaskId), anyLong(), isA(TimeUnit.class))).thenReturn(new JobResponse(
- job,
- PeonPhase.FAILED
- ));
-
- when(taskLogs.streamTaskStatus(eq(task.getId()))).thenReturn(Optional.absent());
-
- when(peonClient.getPeonLogs(eq(k8sTaskId))).thenReturn(Optional.absent());
- when(peonClient.cleanUpJob(eq(k8sTaskId))).thenReturn(true);
-
- KubernetesTaskRunner taskRunner = new KubernetesTaskRunner(
- jsonMapper,
- adapter,
- kubernetesTaskRunnerConfig,
- taskQueueConfig,
- taskLogs,
- peonClient,
- null
- );
- KubernetesTaskRunner spyRunner = spy(taskRunner);
-
- ListenableFuture future = spyRunner.run(task);
- TaskStatus actualTaskStatus = future.get();
- Assert.assertTrue(actualTaskStatus.isFailure());
- Assert.assertEquals(
- StringUtils.format("Task [%s] failed", task.getId()),
- actualTaskStatus.getErrorMsg()
- );
-
- // we should never launch the job here, one exists
- verify(peonClient, times(1)).launchJobAndWaitForStart(isA(Job.class), anyLong(), isA(TimeUnit.class));
- verify(peonClient, times(1)).cleanUpJob(eq(k8sTaskId));
- TaskLocation expectedTaskLocation = TaskLocation.create(
- peonPod.getStatus().getPodIP(),
- DruidK8sConstants.PORT,
- DruidK8sConstants.TLS_PORT,
- druidNode.isEnableTlsPort()
- );
- verify(spyRunner, times(1)).updateStatus(eq(task), eq(actualTaskStatus));
+ Assert.assertFalse(runner.tasks.containsKey(task.getId()));
}
@Test
- public void testTheK8sRestartState() throws Exception
+ public void test_doTask_withoutWorkItem_throwsISE()
{
- // we have a shutdown, now we start-up the overlord, it should catch and deal with all the peon k8s tasks in-flight
- Task task = makeTask();
- K8sTaskId k8sTaskId = new K8sTaskId(task.getId());
-
- Job job = mock(Job.class);
- ObjectMeta jobMetadata = mock(ObjectMeta.class);
- when(jobMetadata.getName()).thenReturn(k8sTaskId.getK8sTaskId());
- JobStatus status = mock(JobStatus.class);
- when(status.getActive()).thenReturn(1).thenReturn(null);
- when(job.getStatus()).thenReturn(status);
- when(job.getMetadata()).thenReturn(jobMetadata);
-
- Pod peonPod = mock(Pod.class);
- ObjectMeta metadata = mock(ObjectMeta.class);
- when(metadata.getName()).thenReturn("peonPodName");
- when(metadata.getCreationTimestamp()).thenReturn(DateTimes.nowUtc().toString());
- when(peonPod.getMetadata()).thenReturn(metadata);
- PodStatus podStatus = mock(PodStatus.class);
- when(podStatus.getPodIP()).thenReturn("SomeIP");
- when(peonPod.getStatus()).thenReturn(podStatus);
-
- K8sTaskAdapter adapter = mock(K8sTaskAdapter.class);
- when(adapter.fromTask(eq(task))).thenReturn(job);
- when(adapter.toTask(eq(job))).thenReturn(task);
-
- DruidKubernetesPeonClient peonClient = mock(DruidKubernetesPeonClient.class);
-
- when(peonClient.listAllPeonJobs()).thenReturn(Collections.singletonList(job));
- when(peonClient.jobExists(eq(k8sTaskId))).thenReturn(Optional.of(job));
- when(peonClient.launchJobAndWaitForStart(isA(Job.class), anyLong(), isA(TimeUnit.class))).thenReturn(peonPod);
- when(peonClient.getMainJobPod(eq(k8sTaskId))).thenReturn(peonPod);
- when(peonClient.waitForJobCompletion(eq(k8sTaskId), anyLong(), isA(TimeUnit.class))).thenReturn(new JobResponse(
- job,
- PeonPhase.SUCCEEDED
- ));
-
- TaskStatus taskStatus = TaskStatus.success(task.getId());
- when(taskLogs.streamTaskStatus(eq(task.getId()))).thenReturn(Optional.of(IOUtils.toInputStream(
- jsonMapper.writeValueAsString(taskStatus),
- StandardCharsets.UTF_8))
+ Assert.assertThrows(
+ "Task [id] disappeared",
+ ISE.class,
+ () -> runner.doTask(task, true)
);
-
- when(peonClient.getPeonLogs(eq(k8sTaskId))).thenReturn(Optional.absent());
- when(peonClient.cleanUpJob(eq(k8sTaskId))).thenReturn(true);
-
- KubernetesTaskRunner taskRunner = new KubernetesTaskRunner(
- jsonMapper,
- adapter,
- kubernetesTaskRunnerConfig,
- taskQueueConfig,
- taskLogs,
- peonClient,
- null
- );
- KubernetesTaskRunner spyRunner = spy(taskRunner);
- Collection extends TaskRunnerWorkItem> workItems = spyRunner.getKnownTasks();
- assertEquals(1, workItems.size());
- TaskRunnerWorkItem item = Iterables.getOnlyElement(workItems);
- item.getResult().get();
-
- // we should never launch the job here, one exists
- verify(peonClient, never()).launchJobAndWaitForStart(isA(Job.class), anyLong(), isA(TimeUnit.class));
- verify(peonClient, times(1)).cleanUpJob(eq(k8sTaskId));
- verify(spyRunner, times(1)).updateStatus(eq(task), eq(TaskStatus.success(task.getId())));
- verify(spyRunner, times(1)).run(eq(task));
}
@Test
- public void testTheK8sRestartStateAndHandleJobsThatAlreadyCompletedWhileDown() throws Exception
+ public void test_doTask_whenShutdownRequested_throwsISE()
{
- // we have a shutdown, now we start-up the overlord, it should monitor k8s jobs that finished.
- Task task = makeTask();
- K8sTaskId k8sTaskId = new K8sTaskId(task.getId());
+ KubernetesWorkItem workItem = new KubernetesWorkItem(task, null);
+ workItem.shutdown();
- Job job = mock(Job.class);
- ObjectMeta jobMetadata = mock(ObjectMeta.class);
- when(jobMetadata.getName()).thenReturn(k8sTaskId.getK8sTaskId());
- JobStatus status = mock(JobStatus.class);
- when(status.getActive()).thenReturn(null);
- when(job.getStatus()).thenReturn(status);
- when(job.getStatus().getSucceeded()).thenReturn(1);
- when(job.getMetadata()).thenReturn(jobMetadata);
+ runner.tasks.put(task.getId(), workItem);
- Pod peonPod = mock(Pod.class);
- ObjectMeta metadata = mock(ObjectMeta.class);
- when(metadata.getName()).thenReturn("peonPodName");
- when(metadata.getCreationTimestamp()).thenReturn(DateTimes.nowUtc().toString());
- when(peonPod.getMetadata()).thenReturn(metadata);
- PodStatus podStatus = mock(PodStatus.class);
- when(podStatus.getPodIP()).thenReturn("SomeIP");
- when(peonPod.getStatus()).thenReturn(podStatus);
-
- K8sTaskAdapter adapter = mock(K8sTaskAdapter.class);
- when(adapter.fromTask(eq(task))).thenReturn(job);
- when(adapter.toTask(eq(job))).thenReturn(task);
-
- DruidKubernetesPeonClient peonClient = mock(DruidKubernetesPeonClient.class);
-
- when(peonClient.listAllPeonJobs()).thenReturn(Collections.singletonList(job));
- when(peonClient.jobExists(eq(k8sTaskId))).thenReturn(Optional.of(job));
- when(peonClient.launchJobAndWaitForStart(isA(Job.class), anyLong(), isA(TimeUnit.class))).thenReturn(peonPod);
- when(peonClient.getMainJobPod(eq(k8sTaskId))).thenReturn(peonPod);
- when(peonClient.waitForJobCompletion(eq(k8sTaskId), anyLong(), isA(TimeUnit.class))).thenReturn(new JobResponse(
- job,
- PeonPhase.SUCCEEDED
- ));
-
- TaskStatus taskStatus = TaskStatus.success(task.getId());
- when(taskLogs.streamTaskStatus(eq(task.getId()))).thenReturn(Optional.of(IOUtils.toInputStream(
- jsonMapper.writeValueAsString(taskStatus),
- StandardCharsets.UTF_8))
+ Assert.assertThrows(
+ "Task [id] has been shut down",
+ ISE.class,
+ () -> runner.doTask(task, true)
);
-
- when(peonClient.getPeonLogs(eq(k8sTaskId))).thenReturn(Optional.absent());
- when(peonClient.cleanUpJob(eq(k8sTaskId))).thenReturn(true);
-
- KubernetesTaskRunner taskRunner = new KubernetesTaskRunner(
- jsonMapper,
- adapter,
- kubernetesTaskRunnerConfig,
- taskQueueConfig,
- taskLogs,
- peonClient,
- null
- );
- KubernetesTaskRunner spyRunner = spy(taskRunner);
- Collection extends TaskRunnerWorkItem> workItems = spyRunner.getKnownTasks();
- assertEquals(1, workItems.size());
- TaskRunnerWorkItem item = Iterables.getOnlyElement(workItems);
- item.getResult().get();
-
- // we should never launch the job here, one exists
- verify(peonClient, never()).launchJobAndWaitForStart(isA(Job.class), anyLong(), isA(TimeUnit.class));
- verify(peonClient, times(1)).cleanUpJob(eq(k8sTaskId));
- TaskLocation expectedTaskLocation = TaskLocation.create(
- peonPod.getStatus().getPodIP(),
- DruidK8sConstants.PORT,
- DruidK8sConstants.TLS_PORT
- );
- // don't need to update the location, the one in the db was correct when it launched,
- verify(spyRunner, never()).updateLocation(eq(task), eq(expectedTaskLocation));
- // the state is still running, as it was before the overlord went down.
- verify(spyRunner, never()).updateStatus(eq(task), eq(TaskStatus.running(task.getId())));
- verify(spyRunner, times(1)).updateStatus(eq(task), eq(TaskStatus.success(task.getId(), expectedTaskLocation)));
- verify(spyRunner, times(1)).run(eq(task));
}
@Test
- public void testStreamTaskReports() throws Exception
+ public void test_shutdown_withoutExistingTask()
{
- Task task = makeTask();
- K8sTaskId k8sTaskId = new K8sTaskId(task.getId());
+ runner.shutdown(task.getId(), "");
+ }
- Job job = mock(Job.class);
- ObjectMeta jobMetadata = mock(ObjectMeta.class);
- when(jobMetadata.getName()).thenReturn(k8sTaskId.getK8sTaskId());
- JobStatus status = mock(JobStatus.class);
- when(status.getActive()).thenReturn(1).thenReturn(null);
- when(job.getStatus()).thenReturn(status);
- when(job.getMetadata()).thenReturn(jobMetadata);
+ @Test
+ public void test_shutdown_withExistingTask()
+ {
+ KubernetesWorkItem workItem = new KubernetesWorkItem(task, null) {
+ @Override
+ protected synchronized void shutdown()
+ {
+ }
+ };
- Pod peonPod = mock(Pod.class);
- ObjectMeta metadata = mock(ObjectMeta.class);
- when(metadata.getName()).thenReturn("peonPodName");
- when(peonPod.getMetadata()).thenReturn(metadata);
- PodStatus podStatus = mock(PodStatus.class);
- when(podStatus.getPodIP()).thenReturn("SomeIP");
- when(peonPod.getStatus()).thenReturn(podStatus);
+ runner.tasks.put(task.getId(), workItem);
- K8sTaskAdapter adapter = mock(K8sTaskAdapter.class);
- when(adapter.fromTask(eq(task))).thenReturn(job);
+ runner.shutdown(task.getId(), "");
+ }
- DruidKubernetesPeonClient peonClient = mock(DruidKubernetesPeonClient.class);
-
- when(peonClient.jobExists(eq(k8sTaskId))).thenReturn(Optional.fromNullable(null));
- when(peonClient.launchJobAndWaitForStart(isA(Job.class), anyLong(), isA(TimeUnit.class))).thenReturn(peonPod);
- when(peonClient.getMainJobPod(eq(k8sTaskId))).thenReturn(peonPod);
- when(peonClient.waitForJobCompletion(eq(k8sTaskId), anyLong(), isA(TimeUnit.class))).thenReturn(new JobResponse(
- job,
- PeonPhase.SUCCEEDED
- ));
-
- HttpClient httpClient = mock(HttpClient.class);
- when(httpClient.go(isA(Request.class), isA(InputStreamResponseHandler.class))).thenReturn(
- Futures.immediateFuture(IOUtils.toInputStream("{}", StandardCharsets.UTF_8))
- );
-
- KubernetesTaskRunner taskRunner = new KubernetesTaskRunner(
- jsonMapper,
- null,
- kubernetesTaskRunnerConfig,
- taskQueueConfig,
- taskLogs,
+ @Test
+ public void test_restore_withExistingJobs() throws IOException
+ {
+ KubernetesTaskRunner runner = new KubernetesTaskRunner(
+ taskAdapter,
+ config,
peonClient,
- httpClient
- );
- taskRunner.run(task);
- Optional maybeInputStream = taskRunner.streamTaskReports(task.getId());
+ httpClient,
+ new TestPeonLifecycleFactory(kubernetesPeonLifecycle)
+ ) {
+ @Override
+ protected ListenableFuture joinAsync(Task task)
+ {
+ return new KubernetesWorkItem(task, null).getResult();
+ }
+ };
+
+ Job job = new JobBuilder()
+ .withNewMetadata()
+ .withName(ID)
+ .endMetadata()
+ .build();
+
+ EasyMock.expect(peonClient.getPeonJobs()).andReturn(ImmutableList.of(job));
+ EasyMock.expect(taskAdapter.toTask(job)).andReturn(task);
+
+ replayAll();
+
+ List>> tasks = runner.restore();
+
+ verifyAll();
+
+ Assert.assertNotNull(tasks);
+ Assert.assertEquals(1, tasks.size());
+ }
+
+ @Test
+ public void test_restore_whenDeserializationExceptionThrown_isIgnored() throws IOException
+ {
+ KubernetesTaskRunner runner = new KubernetesTaskRunner(
+ taskAdapter,
+ config,
+ peonClient,
+ httpClient,
+ new TestPeonLifecycleFactory(kubernetesPeonLifecycle)
+ ) {
+ @Override
+ protected ListenableFuture joinAsync(Task task)
+ {
+ return new KubernetesWorkItem(task, null).getResult();
+ }
+ };
+
+ Job job = new JobBuilder()
+ .withNewMetadata()
+ .withName(ID)
+ .endMetadata()
+ .build();
+
+ EasyMock.expect(peonClient.getPeonJobs()).andReturn(ImmutableList.of(job));
+ EasyMock.expect(taskAdapter.toTask(job)).andThrow(new IOException());
+
+ replayAll();
+
+ List>> tasks = runner.restore();
+
+ verifyAll();
+
+ Assert.assertNotNull(tasks);
+ Assert.assertEquals(0, tasks.size());
+ }
+
+ @Test
+ public void test_getTotalTaskSlotCount()
+ {
+ Map slotCount = runner.getTotalTaskSlotCount();
+
+ MatcherAssert.assertThat(slotCount, Matchers.allOf(
+ Matchers.aMapWithSize(1),
+ Matchers.hasEntry(
+ Matchers.equalTo("taskQueue"),
+ Matchers.equalTo(1L)
+ )
+ ));
+ }
+
+ @Test
+ public void test_getKnownTasks()
+ {
+ KubernetesWorkItem workItem = new KubernetesWorkItem(task, null);
+
+ runner.tasks.put(task.getId(), workItem);
+
+ Collection extends TaskRunnerWorkItem> tasks = runner.getKnownTasks();
+
+ assertEquals(1, tasks.size());
+ assertEquals(Collections.singletonList(workItem), runner.getKnownTasks());
+ }
+
+ @Test
+ public void test_getRunningTasks()
+ {
+ Task pendingTask = NoopTask.create("pending-id", 0);
+ KubernetesWorkItem pendingWorkItem = new KubernetesWorkItem(pendingTask, null) {
+ @Override
+ protected RunnerTaskState getRunnerTaskState()
+ {
+ return RunnerTaskState.PENDING;
+ }
+ };
+ runner.tasks.put(pendingTask.getId(), pendingWorkItem);
+
+ Task runningTask = NoopTask.create("running-id", 0);
+ KubernetesWorkItem runningWorkItem = new KubernetesWorkItem(runningTask, null) {
+ @Override
+ protected RunnerTaskState getRunnerTaskState()
+ {
+ return RunnerTaskState.RUNNING;
+ }
+ };
+ runner.tasks.put(runningTask.getId(), runningWorkItem);
+
+ Collection extends TaskRunnerWorkItem> tasks = runner.getRunningTasks();
+
+ assertEquals(1, tasks.size());
+ assertEquals(Collections.singletonList(runningWorkItem), tasks);
+ }
+
+ @Test
+ public void test_getPendingTasks()
+ {
+ Task pendingTask = NoopTask.create("pending-id", 0);
+ KubernetesWorkItem pendingWorkItem = new KubernetesWorkItem(pendingTask, null) {
+ @Override
+ protected RunnerTaskState getRunnerTaskState()
+ {
+ return RunnerTaskState.PENDING;
+ }
+ };
+ runner.tasks.put(pendingTask.getId(), pendingWorkItem);
+
+ Task runningTask = NoopTask.create("running-id", 0);
+ KubernetesWorkItem runningWorkItem = new KubernetesWorkItem(runningTask, null) {
+ @Override
+ protected RunnerTaskState getRunnerTaskState()
+ {
+ return RunnerTaskState.RUNNING;
+ }
+ };
+ runner.tasks.put(runningTask.getId(), runningWorkItem);
+
+ Collection extends TaskRunnerWorkItem> tasks = runner.getPendingTasks();
+
+ assertEquals(1, tasks.size());
+ assertEquals(Collections.singletonList(pendingWorkItem), tasks);
+ }
+
+ @Test
+ public void test_getRunnerTaskState_withoutExistingTask_returnsNull()
+ {
+ Assert.assertNull(runner.getRunnerTaskState(task.getId()));
+ }
+
+ @Test
+ public void test_getRunnerTaskState_withExistingTask()
+ {
+ KubernetesWorkItem workItem = new KubernetesWorkItem(task, null) {
+ @Override
+ protected RunnerTaskState getRunnerTaskState()
+ {
+ return RunnerTaskState.NONE;
+ }
+ };
+ runner.tasks.put(task.getId(), workItem);
+
+ Assert.assertEquals(RunnerTaskState.NONE, runner.getRunnerTaskState(task.getId()));
+ }
+
+ @Test
+ public void test_streamTaskReports_withExistingTask() throws Exception
+ {
+ KubernetesWorkItem workItem = new KubernetesWorkItem(task, null) {
+ @Override
+ public TaskLocation getLocation()
+ {
+ return TaskLocation.create("host", 0, 1, false);
+ }
+ };
+
+ runner.tasks.put(task.getId(), workItem);
+
+ EasyMock.expect(httpClient.go(
+ EasyMock.anyObject(Request.class),
+ EasyMock.anyObject(InputStreamResponseHandler.class))
+ ).andReturn(Futures.immediateFuture(IOUtils.toInputStream("{}", StandardCharsets.UTF_8)));
+
+ replayAll();
+
+ Optional maybeInputStream = runner.streamTaskReports(task.getId());
+
+ verifyAll();
Assert.assertTrue(maybeInputStream.isPresent());
- InputStream report = maybeInputStream.get();
- Assert.assertEquals("{}", IOUtils.toString(report, StandardCharsets.UTF_8));
+ Assert.assertEquals("{}", IOUtils.toString(maybeInputStream.get(), StandardCharsets.UTF_8));
}
@Test
- public void testStreamTaskReports_whereJobDoesNotExist_returnsEmptyOptional() throws Exception
+ public void test_streamTaskReports_withoutExistingTask_returnsEmptyOptional() throws Exception
{
- Task task = makeTask();
-
- KubernetesTaskRunner taskRunner = new KubernetesTaskRunner(
- jsonMapper,
- mock(K8sTaskAdapter.class),
- kubernetesTaskRunnerConfig,
- taskQueueConfig,
- taskLogs,
- mock(DruidKubernetesPeonClient.class),
- mock(HttpClient.class)
- );
-
- Optional maybeInputStream = taskRunner.streamTaskReports(task.getId());
-
+ Optional maybeInputStream = runner.streamTaskReports(task.getId());
Assert.assertFalse(maybeInputStream.isPresent());
}
@Test
- public void testStreamTaskReports_withoutEmptyLocation_returnsEmptyOptional() throws Exception
+ public void test_streamTaskReports_withUnknownTaskLocation_returnsEmptyOptional() throws Exception
{
- Task task = makeTask();
- K8sTaskId k8sTaskId = new K8sTaskId(task.getId());
+ KubernetesWorkItem workItem = new KubernetesWorkItem(task, null) {
+ @Override
+ public TaskLocation getLocation()
+ {
+ return TaskLocation.unknown();
+ }
+ };
- Job job = mock(Job.class);
- ObjectMeta jobMetadata = mock(ObjectMeta.class);
- when(jobMetadata.getName()).thenReturn(k8sTaskId.getK8sTaskId());
- JobStatus status = mock(JobStatus.class);
- when(status.getActive()).thenReturn(1).thenReturn(null);
- when(job.getStatus()).thenReturn(status);
- when(job.getMetadata()).thenReturn(jobMetadata);
+ runner.tasks.put(task.getId(), workItem);
- Pod peonPod = mock(Pod.class);
- ObjectMeta metadata = mock(ObjectMeta.class);
- when(metadata.getName()).thenReturn("peonPodName");
- when(peonPod.getMetadata()).thenReturn(metadata);
-
- K8sTaskAdapter adapter = mock(K8sTaskAdapter.class);
- when(adapter.fromTask(eq(task))).thenReturn(job);
-
- DruidKubernetesPeonClient peonClient = mock(DruidKubernetesPeonClient.class);
-
- when(peonClient.jobExists(eq(k8sTaskId))).thenReturn(Optional.fromNullable(null));
- when(peonClient.launchJobAndWaitForStart(isA(Job.class), anyLong(), isA(TimeUnit.class))).thenReturn(peonPod);
- when(peonClient.getMainJobPod(eq(k8sTaskId))).thenReturn(peonPod);
- when(peonClient.waitForJobCompletion(eq(k8sTaskId), anyLong(), isA(TimeUnit.class))).thenReturn(new JobResponse(
- job,
- PeonPhase.SUCCEEDED
- ));
-
- HttpClient httpClient = mock(HttpClient.class);
- ListenableFuture future = mock(ListenableFuture.class);
-
- when(httpClient.go(isA(Request.class), isA(InputStreamResponseHandler.class))).thenReturn(future);
- when(future.get()).thenThrow(InterruptedException.class);
-
- KubernetesTaskRunner taskRunner = new KubernetesTaskRunner(
- jsonMapper,
- null,
- kubernetesTaskRunnerConfig,
- taskQueueConfig,
- taskLogs,
- peonClient,
- httpClient
- );
-
- taskRunner.run(task);
-
- Assert.assertFalse(taskRunner.streamTaskReports(task.getId()).isPresent());
+ Optional maybeInputStream = runner.streamTaskReports(task.getId());
+ Assert.assertFalse(maybeInputStream.isPresent());
}
@Test
- public void testStreamTaskReports_getInputStreamThrowsInterruptedException_throwsRuntimeException() throws Exception
+ public void test_streamTaskReports_whenInterruptedExceptionThrown_throwsRuntimeException()
{
- Task task = makeTask();
- K8sTaskId k8sTaskId = new K8sTaskId(task.getId());
+ KubernetesWorkItem workItem = new KubernetesWorkItem(task, null) {
+ @Override
+ public TaskLocation getLocation()
+ {
+ return TaskLocation.create("host", 0, 1, false);
+ }
+ };
- Job job = mock(Job.class);
- ObjectMeta jobMetadata = mock(ObjectMeta.class);
- when(jobMetadata.getName()).thenReturn(k8sTaskId.getK8sTaskId());
- JobStatus status = mock(JobStatus.class);
- when(status.getActive()).thenReturn(1).thenReturn(null);
- when(job.getStatus()).thenReturn(status);
- when(job.getMetadata()).thenReturn(jobMetadata);
+ runner.tasks.put(task.getId(), workItem);
- Pod peonPod = mock(Pod.class);
- ObjectMeta metadata = mock(ObjectMeta.class);
- when(metadata.getName()).thenReturn("peonPodName");
- when(peonPod.getMetadata()).thenReturn(metadata);
- PodStatus podStatus = mock(PodStatus.class);
- when(podStatus.getPodIP()).thenReturn("SomeIP");
- when(peonPod.getStatus()).thenReturn(podStatus);
-
- K8sTaskAdapter adapter = mock(K8sTaskAdapter.class);
- when(adapter.fromTask(eq(task))).thenReturn(job);
-
- DruidKubernetesPeonClient peonClient = mock(DruidKubernetesPeonClient.class);
-
- when(peonClient.jobExists(eq(k8sTaskId))).thenReturn(Optional.fromNullable(null));
- when(peonClient.launchJobAndWaitForStart(isA(Job.class), anyLong(), isA(TimeUnit.class))).thenReturn(peonPod);
- when(peonClient.getMainJobPod(eq(k8sTaskId))).thenReturn(peonPod);
- when(peonClient.waitForJobCompletion(eq(k8sTaskId), anyLong(), isA(TimeUnit.class))).thenReturn(new JobResponse(
- job,
- PeonPhase.SUCCEEDED
- ));
-
- HttpClient httpClient = mock(HttpClient.class);
- ListenableFuture future = mock(ListenableFuture.class);
-
- when(httpClient.go(isA(Request.class), isA(InputStreamResponseHandler.class))).thenReturn(future);
- when(future.get()).thenThrow(InterruptedException.class);
-
- KubernetesTaskRunner taskRunner = new KubernetesTaskRunner(
- jsonMapper,
- null,
- kubernetesTaskRunnerConfig,
- taskQueueConfig,
- taskLogs,
- peonClient,
- httpClient
- );
-
- taskRunner.run(task);
-
- Assert.assertThrows(RuntimeException.class, () -> taskRunner.streamTaskReports(task.getId()));
- }
-
- @Test
- public void testStreamTaskReports_getInputStreamThrowsExecutionException_throwsRuntimeException() throws Exception
- {
- Task task = makeTask();
- K8sTaskId k8sTaskId = new K8sTaskId(task.getId());
-
- Job job = mock(Job.class);
- ObjectMeta jobMetadata = mock(ObjectMeta.class);
- when(jobMetadata.getName()).thenReturn(k8sTaskId.getK8sTaskId());
- JobStatus status = mock(JobStatus.class);
- when(status.getActive()).thenReturn(1).thenReturn(null);
- when(job.getStatus()).thenReturn(status);
- when(job.getMetadata()).thenReturn(jobMetadata);
-
- Pod peonPod = mock(Pod.class);
- ObjectMeta metadata = mock(ObjectMeta.class);
- when(metadata.getName()).thenReturn("peonPodName");
- when(peonPod.getMetadata()).thenReturn(metadata);
- PodStatus podStatus = mock(PodStatus.class);
- when(podStatus.getPodIP()).thenReturn("SomeIP");
- when(peonPod.getStatus()).thenReturn(podStatus);
-
- K8sTaskAdapter adapter = mock(K8sTaskAdapter.class);
- when(adapter.fromTask(eq(task))).thenReturn(job);
-
- DruidKubernetesPeonClient peonClient = mock(DruidKubernetesPeonClient.class);
-
- when(peonClient.jobExists(eq(k8sTaskId))).thenReturn(Optional.fromNullable(null));
- when(peonClient.launchJobAndWaitForStart(isA(Job.class), anyLong(), isA(TimeUnit.class))).thenReturn(peonPod);
- when(peonClient.getMainJobPod(eq(k8sTaskId))).thenReturn(peonPod);
- when(peonClient.waitForJobCompletion(eq(k8sTaskId), anyLong(), isA(TimeUnit.class))).thenReturn(new JobResponse(
- job,
- PeonPhase.SUCCEEDED
- ));
-
- HttpClient httpClient = mock(HttpClient.class);
- ListenableFuture future = mock(ListenableFuture.class);
-
- when(httpClient.go(isA(Request.class), isA(InputStreamResponseHandler.class))).thenReturn(future);
- when(future.get()).thenThrow(ExecutionException.class);
-
- KubernetesTaskRunner taskRunner = new KubernetesTaskRunner(
- jsonMapper,
- null,
- kubernetesTaskRunnerConfig,
- taskQueueConfig,
- taskLogs,
- peonClient,
- httpClient
- );
-
- taskRunner.run(task);
-
- Assert.assertThrows(RuntimeException.class, () -> taskRunner.streamTaskReports(task.getId()));
- }
-
- @Test
- public void testMakingCodeCoverageHappy()
- {
- // have to test multiple branches of code for code-coverage, avoiding doing a lot of repetetive setup.
- DruidKubernetesPeonClient peonClient = mock(DruidKubernetesPeonClient.class);
- Pod pod = mock(Pod.class);
- PodStatus status = mock(PodStatus.class);
- when(status.getPhase()).thenReturn(PeonPhase.PENDING.getPhase()).thenReturn(PeonPhase.FAILED.getPhase());
- when(pod.getStatus()).thenReturn(status);
- when(peonClient.getMainJobPod(any())).thenReturn(null).thenReturn(pod);
-
- KubernetesTaskRunner taskRunner = new KubernetesTaskRunner(
- jsonMapper,
- mock(K8sTaskAdapter.class),
- kubernetesTaskRunnerConfig,
- taskQueueConfig,
- taskLogs,
- peonClient,
- null
- );
-
- RunnerTaskState state = taskRunner.getRunnerTaskState("foo");
- assertNull(state);
- assertEquals(RunnerTaskState.PENDING, taskRunner.getRunnerTaskState("bar"));
- assertEquals(RunnerTaskState.WAITING, taskRunner.getRunnerTaskState("baz"));
-
- assertThrows(ISE.class, () -> {
- taskRunner.monitorJob(null, new K8sTaskId("foo"));
- });
- }
-
- @Test
- public void testMaxQueueSizeIsEnforced()
- {
- TaskQueueConfig taskQueueConfig = new TaskQueueConfig(
- Integer.MAX_VALUE,
- Period.millis(1),
- Period.millis(1),
- Period.millis(1)
- );
- assertThrows(IllegalArgumentException.class, () -> new KubernetesTaskRunner(
- jsonMapper,
- mock(K8sTaskAdapter.class),
- kubernetesTaskRunnerConfig,
- taskQueueConfig,
- taskLogs,
- mock(DruidKubernetesPeonClient.class),
- null
- ));
- }
-
- @Test
- public void testWorkItemGetLocation()
- {
- KubernetesPeonClient client = mock(KubernetesPeonClient.class);
- Pod pod = mock(Pod.class);
- PodStatus status = mock(PodStatus.class);
- when(status.getPodIP()).thenReturn(null).thenReturn("tweak");
- when(pod.getStatus()).thenReturn(status);
-
- ObjectMeta metadata = mock(ObjectMeta.class);
- when(metadata.getAnnotations()).thenReturn(ImmutableMap.of(DruidK8sConstants.TLS_ENABLED, "false"));
-
- when(pod.getMetadata()).thenReturn(metadata);
- when(client.getMainJobPod(any())).thenReturn(pod);
-
- Task task = mock(Task.class);
- when(task.getId()).thenReturn("butters");
- KubernetesTaskRunner.K8sWorkItem k8sWorkItem = new KubernetesTaskRunner.K8sWorkItem(client, task, null);
- TaskLocation location = k8sWorkItem.getLocation();
- assertEquals(TaskLocation.unknown(), location);
-
- TaskLocation realLocation = k8sWorkItem.getLocation();
- assertEquals(TaskLocation.create("tweak", DruidK8sConstants.PORT, DruidK8sConstants.TLS_PORT, false), realLocation);
- }
-
- @Test
- public void testWorkItemGetLocation_withKubernetesResourceNotFoundException_returnsUnknownLocation()
- {
- KubernetesPeonClient client = mock(KubernetesPeonClient.class);
-
- when(client.getMainJobPod(any())).thenThrow(KubernetesResourceNotFoundException.class);
-
- Task task = mock(Task.class);
- when(task.getId()).thenReturn("id");
- KubernetesTaskRunner.K8sWorkItem k8sWorkItem = new KubernetesTaskRunner.K8sWorkItem(client, task, null);
- TaskLocation location = k8sWorkItem.getLocation();
- assertEquals(TaskLocation.unknown(), location);
- }
-
- @Test
- public void testK8sJobManualShutdown() throws Exception
- {
- Task task = makeTask();
- K8sTaskId k8sTaskId = new K8sTaskId(task.getId());
-
- Job job = mock(Job.class);
- ObjectMeta jobMetadata = mock(ObjectMeta.class);
- when(jobMetadata.getName()).thenReturn(k8sTaskId.getK8sTaskId());
- JobStatus status = mock(JobStatus.class);
- when(status.getActive()).thenReturn(1);
- when(job.getStatus()).thenReturn(status);
- when(job.getMetadata()).thenReturn(jobMetadata);
-
- Pod peonPod = mock(Pod.class);
- ObjectMeta metadata = mock(ObjectMeta.class);
- when(metadata.getName()).thenReturn("peonPodName");
- when(metadata.getCreationTimestamp()).thenReturn(DateTimes.nowUtc().toString());
- when(peonPod.getMetadata()).thenReturn(metadata);
- PodStatus podStatus = mock(PodStatus.class);
- when(podStatus.getPodIP()).thenReturn("SomeIP");
- when(peonPod.getStatus()).thenReturn(podStatus);
-
- K8sTaskAdapter adapter = mock(K8sTaskAdapter.class);
- when(adapter.fromTask(eq(task))).thenReturn(job);
- when(adapter.toTask(eq(job))).thenReturn(task);
-
- DruidKubernetesPeonClient peonClient = mock(DruidKubernetesPeonClient.class);
-
- when(taskLogs.streamTaskStatus(eq(task.getId()))).thenReturn(Optional.absent());
-
- when(peonClient.jobExists(eq(k8sTaskId))).thenReturn(Optional.of(job));
- when(peonClient.getMainJobPod(eq(k8sTaskId))).thenReturn(peonPod);
-
- // Client returns a null job if the job has been deleted
- when(peonClient.waitForJobCompletion(eq(k8sTaskId), anyLong(), isA(TimeUnit.class))).thenReturn(new JobResponse(
- null,
- PeonPhase.FAILED
- ));
- when(peonClient.getPeonLogs(eq(k8sTaskId))).thenReturn(Optional.absent());
- when(peonClient.cleanUpJob(eq(k8sTaskId))).thenReturn(true);
-
- KubernetesTaskRunner taskRunner = new KubernetesTaskRunner(
- jsonMapper,
- adapter,
- kubernetesTaskRunnerConfig,
- taskQueueConfig,
- taskLogs,
- peonClient,
- null
- );
- KubernetesTaskRunner spyRunner = spy(taskRunner);
- ListenableFuture future = spyRunner.run(task);
- TaskStatus taskStatusResponse = future.get();
- Assert.assertEquals(TaskState.FAILED, taskStatusResponse.getStatusCode());
- Assert.assertEquals("Task [k8sTaskId] failed kubernetes job disappeared before completion", taskStatusResponse.getErrorMsg());
-
- }
-
- private Task makeTask()
- {
- return new TestableNoopTask(
- "k8sTaskId",
- null,
- null,
- 0,
- 0,
- null,
- null,
- ImmutableMap.of("druid.indexer.runner.javaOpts", "abc",
- "druid.indexer.fork.property.druid.processing.buffer.sizeBytes", "2048",
- "druid.peon.pod.cpu", "1",
- "druid.peon.pod.memory", "2G"
- )
- );
- }
- private static class TestableNoopTask extends NoopTask
- {
- TestableNoopTask(
- @JsonProperty("id") String id,
- @JsonProperty("groupId") String groupId,
- @JsonProperty("dataSource") String dataSource,
- @JsonProperty("runTime") long runTime,
- @JsonProperty("isReadyTime") long isReadyTime,
- @JsonProperty("isReadyResult") String isReadyResult,
- @JsonProperty("firehose") FirehoseFactory firehoseFactory,
- @JsonProperty("context") Map context
- )
+ ListenableFuture future = new ListenableFuture()
{
- super(id, groupId, dataSource, runTime, isReadyTime, isReadyResult, firehoseFactory, context);
- }
+ @Override
+ public void addListener(Runnable runnable, Executor executor)
+ {
+ }
- @Override
- public String getNodeType()
- {
- return "ForkNodeType";
- }
+ @Override
+ public boolean cancel(boolean mayInterruptIfRunning)
+ {
+ return false;
+ }
- @Override
- public boolean supportsQueries()
- {
- return true;
- }
+ @Override
+ public boolean isCancelled()
+ {
+ return false;
+ }
+
+ @Override
+ public boolean isDone()
+ {
+ return false;
+ }
+
+ @Override
+ public InputStream get() throws InterruptedException
+ {
+ throw new InterruptedException();
+ }
+
+ @Override
+ public InputStream get(long timeout, TimeUnit unit) throws InterruptedException
+ {
+ throw new InterruptedException();
+ }
+ };
+
+ EasyMock.expect(httpClient.go(
+ EasyMock.anyObject(Request.class),
+ EasyMock.anyObject(InputStreamResponseHandler.class))
+ ).andReturn(future);
+
+ replayAll();
+
+ Exception e = Assert.assertThrows(RuntimeException.class, () -> runner.streamTaskReports(task.getId()));
+ Assert.assertTrue(e.getCause() instanceof InterruptedException);
+
+ verifyAll();
+ }
+
+ @Test
+ public void test_streamTaskReports_whenExecutionExceptionThrown_throwsRuntimeException()
+ {
+ KubernetesWorkItem workItem = new KubernetesWorkItem(task, null) {
+ @Override
+ public TaskLocation getLocation()
+ {
+ return TaskLocation.create("host", 0, 1, false);
+ }
+ };
+
+ runner.tasks.put(task.getId(), workItem);
+
+ EasyMock.expect(httpClient.go(
+ EasyMock.anyObject(Request.class),
+ EasyMock.anyObject(InputStreamResponseHandler.class))
+ ).andReturn(Futures.immediateFailedFuture(new Exception()));
+
+ replayAll();
+
+ Assert.assertThrows(RuntimeException.class, () -> runner.streamTaskReports(task.getId()));
+
+ verifyAll();
}
}
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesWorkItemTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesWorkItemTest.java
new file mode 100644
index 00000000000..1cca2fd89cb
--- /dev/null
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesWorkItemTest.java
@@ -0,0 +1,273 @@
+/*
+ * 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.indexer.RunnerTaskState;
+import org.apache.druid.indexer.TaskLocation;
+import org.apache.druid.indexing.common.task.NoopTask;
+import org.apache.druid.indexing.common.task.Task;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class KubernetesWorkItemTest
+{
+ private KubernetesWorkItem workItem;
+ private Task task;
+
+ @Before
+ public void setup()
+ {
+ task = NoopTask.create("id", 0);
+ workItem = new KubernetesWorkItem(task, null);
+ }
+
+ @Test
+ public void test_setKubernetesPeonLifecycleTwice_throwsIllegalStateException()
+ {
+ workItem.setKubernetesPeonLifecycle(new KubernetesPeonLifecycle(
+ task,
+ null,
+ null,
+ null
+ ));
+
+ Assert.assertThrows(
+ IllegalStateException.class,
+ () -> workItem.setKubernetesPeonLifecycle(new KubernetesPeonLifecycle(
+ task,
+ null,
+ null,
+ null
+ ))
+ );
+ }
+
+ @Test
+ public void test_shutdown_withoutKubernetesPeonLifecycle()
+ {
+ workItem.shutdown();
+ Assert.assertTrue(workItem.isShutdownRequested());
+ }
+
+ @Test
+ public void test_shutdown_withKubernetesPeonLifecycle()
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(
+ task,
+ null,
+ null,
+ null
+ ) {
+ @Override
+ protected synchronized void shutdown()
+ {
+ }
+ };
+
+ workItem.setKubernetesPeonLifecycle(peonLifecycle);
+
+ workItem.shutdown();
+ Assert.assertTrue(workItem.isShutdownRequested());
+ }
+
+ @Test
+ public void test_isPending_withTaskStateWaiting_returnsFalse()
+ {
+ workItem = new KubernetesWorkItem(task, null) {
+ @Override
+ protected RunnerTaskState getRunnerTaskState()
+ {
+ return RunnerTaskState.WAITING;
+ }
+ };
+ Assert.assertFalse(workItem.isPending());
+ }
+
+ @Test
+ public void test_isPending_withTaskStatePending_returnsTrue()
+ {
+ workItem = new KubernetesWorkItem(task, null) {
+ @Override
+ protected RunnerTaskState getRunnerTaskState()
+ {
+ return RunnerTaskState.PENDING;
+ }
+ };
+ Assert.assertTrue(workItem.isPending());
+ }
+
+ @Test
+ public void test_isRunning_withTaskStateWaiting_returnsFalse()
+ {
+ workItem = new KubernetesWorkItem(task, null) {
+ @Override
+ protected RunnerTaskState getRunnerTaskState()
+ {
+ return RunnerTaskState.WAITING;
+ }
+ };
+ Assert.assertFalse(workItem.isRunning());
+ }
+
+ @Test
+ public void test_isRunning_withTaskStatePending_returnsTrue()
+ {
+ workItem = new KubernetesWorkItem(task, null) {
+ @Override
+ protected RunnerTaskState getRunnerTaskState()
+ {
+ return RunnerTaskState.RUNNING;
+ }
+ };
+ Assert.assertTrue(workItem.isRunning());
+ }
+
+ @Test
+ public void test_getRunnerTaskState_withoutKubernetesPeonLifecycle_returnsPending()
+ {
+ Assert.assertEquals(RunnerTaskState.PENDING, workItem.getRunnerTaskState());
+ }
+
+ @Test
+ public void test_getRunnerTaskState_withKubernetesPeonLifecycle_returnsPending()
+ {
+ workItem.setKubernetesPeonLifecycle(new KubernetesPeonLifecycle(
+ task,
+ null,
+ null,
+ null
+ ));
+
+ Assert.assertEquals(RunnerTaskState.PENDING, workItem.getRunnerTaskState());
+ }
+
+ @Test
+ public void test_getRunnerTaskState_withKubernetesPeonLifecycle_inPendingState_returnsPending()
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(
+ task,
+ null,
+ null,
+ null
+ ) {
+ @Override
+ protected State getState()
+ {
+ return State.PENDING;
+ }
+ };
+
+ workItem.setKubernetesPeonLifecycle(peonLifecycle);
+
+ Assert.assertEquals(RunnerTaskState.PENDING, workItem.getRunnerTaskState());
+ }
+
+ @Test
+ public void test_getRunnerTaskState_withKubernetesPeonLifecycle_inRunningState_returnsRunning()
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(
+ task,
+ null,
+ null,
+ null
+ ) {
+ @Override
+ protected State getState()
+ {
+ return State.RUNNING;
+ }
+ };
+
+ workItem.setKubernetesPeonLifecycle(peonLifecycle);
+
+ Assert.assertEquals(RunnerTaskState.RUNNING, workItem.getRunnerTaskState());
+ }
+
+ @Test
+ public void test_getRunnerTaskState_withKubernetesPeonLifecycle_inStoppedState_returnsNone()
+ {
+ KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(
+ task,
+ null,
+ null,
+ null
+ ) {
+ @Override
+ protected State getState()
+ {
+ return State.STOPPED;
+ }
+ };
+
+ workItem.setKubernetesPeonLifecycle(peonLifecycle);
+
+ Assert.assertEquals(RunnerTaskState.NONE, workItem.getRunnerTaskState());
+ }
+
+ @Test
+ public void test_streamTaskLogs_withoutKubernetesPeonLifecycle()
+ {
+ Assert.assertFalse(workItem.streamTaskLogs().isPresent());
+ }
+
+ @Test
+ public void test_streamTaskLogs_withKubernetesPeonLifecycle()
+ {
+ workItem.setKubernetesPeonLifecycle(new KubernetesPeonLifecycle(
+ task,
+ null,
+ null,
+ null
+ ));
+ Assert.assertFalse(workItem.streamTaskLogs().isPresent());
+ }
+
+ @Test
+ public void test_getLocation_withoutKubernetesPeonLifecycle()
+ {
+ Assert.assertEquals(TaskLocation.unknown(), workItem.getLocation());
+ }
+
+ @Test
+ public void test_getLocation_withKubernetesPeonLifecycle()
+ {
+ workItem.setKubernetesPeonLifecycle(new KubernetesPeonLifecycle(
+ task,
+ null,
+ null,
+ null
+ ));
+
+ Assert.assertEquals(TaskLocation.unknown(), workItem.getLocation());
+ }
+
+ @Test
+ public void test_getTaskType()
+ {
+ Assert.assertEquals(task.getType(), workItem.getTaskType());
+ }
+
+ @Test
+ public void test_getDataSource()
+ {
+ Assert.assertEquals(task.getDataSource(), workItem.getDataSource());
+ }
+}
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/TestPeonLifecycleFactory.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/TestPeonLifecycleFactory.java
new file mode 100644
index 00000000000..333b0490bac
--- /dev/null
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/TestPeonLifecycleFactory.java
@@ -0,0 +1,38 @@
+/*
+ * 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 class TestPeonLifecycleFactory implements PeonLifecycleFactory
+{
+ private final KubernetesPeonLifecycle kubernetesPeonLifecycle;
+
+ public TestPeonLifecycleFactory(KubernetesPeonLifecycle kubernetesPeonLifecycle)
+ {
+ this.kubernetesPeonLifecycle = kubernetesPeonLifecycle;
+ }
+
+ @Override
+ public KubernetesPeonLifecycle build(Task task)
+ {
+ return kubernetesPeonLifecycle;
+ }
+}
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DruidKubernetesPeonClientTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DruidKubernetesPeonClientTest.java
deleted file mode 100644
index ebf7d8c0a44..00000000000
--- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DruidKubernetesPeonClientTest.java
+++ /dev/null
@@ -1,269 +0,0 @@
-/*
- * 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 com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import io.fabric8.kubernetes.api.model.ObjectMeta;
-import io.fabric8.kubernetes.api.model.PodListBuilder;
-import io.fabric8.kubernetes.api.model.PodTemplateSpec;
-import io.fabric8.kubernetes.api.model.batch.v1.Job;
-import io.fabric8.kubernetes.api.model.batch.v1.JobBuilder;
-import io.fabric8.kubernetes.api.model.batch.v1.JobStatus;
-import io.fabric8.kubernetes.api.model.batch.v1.JobStatusBuilder;
-import io.fabric8.kubernetes.client.KubernetesClient;
-import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient;
-import io.fabric8.kubernetes.client.server.mock.KubernetesMockServer;
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.Test;
-
-import java.io.InputStream;
-import java.net.HttpURLConnection;
-import java.sql.Timestamp;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-
-@EnableKubernetesMockClient(crud = true)
-public class DruidKubernetesPeonClientTest
-{
-
- private KubernetesClient client;
- private KubernetesMockServer server;
-
- @Test
- void testWaitingForAPodToGetReadyThatDoesntExist()
- {
- DruidKubernetesPeonClient client = new DruidKubernetesPeonClient(new TestKubernetesClient(this.client), "test",
- false
- );
- JobResponse jobResponse = client.waitForJobCompletion(new K8sTaskId("some-task"), 1, TimeUnit.SECONDS);
- Assertions.assertEquals(PeonPhase.FAILED, jobResponse.getPhase());
- Assertions.assertNull(jobResponse.getJob());
- }
-
- @Test
- void testWaitingForAPodToGetReadySuccess()
- {
- DruidKubernetesPeonClient peonClient = new DruidKubernetesPeonClient(new TestKubernetesClient(this.client), "test",
- false
- );
- Job job = new JobBuilder()
- .withNewMetadata()
- .withName("sometask")
- .endMetadata()
- .withStatus(new JobStatusBuilder().withActive(null).withSucceeded(1).build())
- .build();
- client.batch().v1().jobs().inNamespace("test").create(job);
- JobResponse jobResponse = peonClient.waitForJobCompletion(new K8sTaskId("sometask"), 1, TimeUnit.SECONDS);
- Assertions.assertEquals(PeonPhase.SUCCEEDED, jobResponse.getPhase());
- Assertions.assertEquals(job.getStatus().getSucceeded(), jobResponse.getJob().getStatus().getSucceeded());
- }
-
- @Test
- void testTheFlow()
- {
- DruidKubernetesPeonClient peonClient = new DruidKubernetesPeonClient(new TestKubernetesClient(this.client), "test",
- false
- );
- List currentJobs = peonClient.listAllPeonJobs();
- Assertions.assertEquals(0, currentJobs.size());
- Job job = new JobBuilder()
- .withNewMetadata()
- .withName("job_name")
- .addToLabels(DruidK8sConstants.LABEL_KEY, "true")
- .endMetadata()
- .withNewSpec()
- .withTemplate(new PodTemplateSpec(new ObjectMeta(), K8sTestUtils.getDummyPodSpec()))
- .endSpec().build();
- client.batch().v1().jobs().inNamespace("test").create(job);
- currentJobs = peonClient.listAllPeonJobs();
- Assertions.assertEquals(1, currentJobs.size());
- }
-
- @Test
- void testCleanup() throws KubernetesResourceNotFoundException
- {
- DruidKubernetesPeonClient peonClient = new DruidKubernetesPeonClient(new TestKubernetesClient(this.client), "test",
- false
- );
- Job active = mockJob(true, new Timestamp(System.currentTimeMillis()));
- long tenMinutesAgo = System.currentTimeMillis() - TimeUnit.MINUTES.toMillis(10);
- Job dontKillYet = mockJob(false, new Timestamp(tenMinutesAgo));
- long oneHourAgo = System.currentTimeMillis() - TimeUnit.MINUTES.toMillis(60);
- Job killThisOne = mockJob(false, new Timestamp(oneHourAgo));
-
- List jobs = Lists.newArrayList(active, dontKillYet, killThisOne);
- List toDelete = peonClient.getJobsToCleanup(jobs, 30, TimeUnit.MINUTES);
- Assertions.assertEquals(1, toDelete.size()); // should only cleanup one job
- Assertions.assertEquals(killThisOne, Iterables.getOnlyElement(toDelete)); // should only cleanup one job
- }
-
- @Test
- void testCleanupReturnValue() throws KubernetesResourceNotFoundException
- {
- DruidKubernetesPeonClient peonClient = new DruidKubernetesPeonClient(new TestKubernetesClient(this.client), "test",
- false
- );
- Assertions.assertFalse(peonClient.cleanUpJob(new K8sTaskId("sometask")));
-
- Job job = new JobBuilder()
- .withNewMetadata()
- .withName("sometask")
- .addToLabels(DruidK8sConstants.LABEL_KEY, "true")
- .endMetadata()
- .withNewSpec()
- .withTemplate(new PodTemplateSpec(new ObjectMeta(), K8sTestUtils.getDummyPodSpec()))
- .endSpec().build();
- client.batch().v1().jobs().inNamespace("test").create(job);
- Assertions.assertTrue(peonClient.cleanUpJob(new K8sTaskId("sometask")));
- }
-
- @Test
- void watchingALogThatDoesntExist()
- {
- DruidKubernetesPeonClient peonClient = new DruidKubernetesPeonClient(
- new TestKubernetesClient(this.client), "test",
- false
- );
- Optional stream = peonClient.getPeonLogs(new K8sTaskId("foo"));
- Assertions.assertFalse(stream.isPresent());
- }
-
- @Test
- void testGetLogReaderForJob()
- {
- server.expect().get()
- .withPath("/apis/batch/v1/namespaces/test/jobs/foo")
- .andReturn(HttpURLConnection.HTTP_OK, new JobBuilder()
- .withNewMetadata()
- .withName("foo")
- .withUid("uid")
- .endMetadata()
- .withNewSpec()
- .withNewTemplate()
- .withNewSpec()
- .addNewContainer()
- .withName("main")
- .endContainer()
- .endSpec()
- .endTemplate()
- .endSpec()
- .build()
- ).once();
-
- server.expect().get()
- .withPath("/api/v1/namespaces/test/pods?labelSelector=controller-uid%3Duid")
- .andReturn(HttpURLConnection.HTTP_OK, new PodListBuilder()
- .addNewItem()
- .withNewMetadata()
- .withName("foo")
- .addNewOwnerReference()
- .withUid("uid")
- .withController(true)
- .endOwnerReference()
- .endMetadata()
- .withNewSpec()
- .addNewContainer()
- .withName("main")
- .endContainer()
- .endSpec()
- .endItem()
- .build()
- ).once();
-
- server.expect().get()
- .withPath("/api/v1/namespaces/test/pods/foo/log?pretty=false&container=main")
- .andReturn(HttpURLConnection.HTTP_OK, "data")
- .once();
-
- DruidKubernetesPeonClient peonClient = new DruidKubernetesPeonClient(
- new TestKubernetesClient(client),
- "test",
- false
- );
-
- Optional logs = peonClient.getPeonLogs(new K8sTaskId("foo"));
- Assertions.assertTrue(logs.isPresent());
- }
-
- @Test
- void testGetLogReaderForJobWithoutPodReturnsEmptyOptional()
- {
- server.expect().get()
- .withPath("/apis/batch/v1/namespaces/test/jobs/foo")
- .andReturn(HttpURLConnection.HTTP_OK, new JobBuilder()
- .withNewMetadata()
- .withName("foo")
- .withUid("uid")
- .endMetadata()
- .withNewSpec()
- .withNewTemplate()
- .withNewSpec()
- .addNewContainer()
- .withName("main")
- .endContainer()
- .endSpec()
- .endTemplate()
- .endSpec()
- .build()
- ).once();
-
- DruidKubernetesPeonClient peonClient = new DruidKubernetesPeonClient(
- new TestKubernetesClient(client),
- "test",
- false
- );
-
- Optional logs = peonClient.getPeonLogs(new K8sTaskId("foo"));
- Assertions.assertFalse(logs.isPresent());
- }
-
- @Test
- void testGetLogReaderForJobWithoutJobReturnsEmptyOptional()
- {
- DruidKubernetesPeonClient peonClient = new DruidKubernetesPeonClient(
- new TestKubernetesClient(client),
- "test",
- false
- );
-
- Optional logs = peonClient.getPeonLogs(new K8sTaskId("foo"));
- Assertions.assertFalse(logs.isPresent());
- }
-
- private Job mockJob(boolean active, Timestamp timestamp)
- {
- Job job = mock(Job.class);
- JobStatus status = mock(JobStatus.class);
- if (active) {
- when(status.getActive()).thenReturn(1);
- } else {
- when(status.getActive()).thenReturn(null);
- when(status.getCompletionTime()).thenReturn(timestamp.toString());
- }
- when(job.getStatus()).thenReturn(status);
- return job;
- }
-}
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/JobResponseTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/JobResponseTest.java
index 5f6328ce548..2e2043578aa 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/JobResponseTest.java
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/JobResponseTest.java
@@ -19,68 +19,67 @@
package org.apache.druid.k8s.overlord.common;
-import com.google.common.base.Optional;
-import io.fabric8.kubernetes.api.model.ObjectMeta;
import io.fabric8.kubernetes.api.model.batch.v1.Job;
-import io.fabric8.kubernetes.api.model.batch.v1.JobStatus;
+import io.fabric8.kubernetes.api.model.batch.v1.JobBuilder;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
class JobResponseTest
{
-
@Test
void testCompletionTime()
{
- Job job = mock(Job.class);
- ObjectMeta metadata = mock(ObjectMeta.class);
- when(metadata.getName()).thenReturn("job");
- when(job.getMetadata()).thenReturn(metadata);
- JobStatus jobStatus = mock(JobStatus.class);
- when(jobStatus.getStartTime()).thenReturn("2022-09-19T23:31:50Z");
- when(jobStatus.getCompletionTime()).thenReturn("2022-09-19T23:32:48Z");
- when(job.getStatus()).thenReturn(jobStatus);
+ Job job = new JobBuilder()
+ .withNewMetadata()
+ .withName("job")
+ .endMetadata()
+ .withNewStatus()
+ .withStartTime("2022-09-19T23:31:50Z")
+ .withCompletionTime("2022-09-19T23:32:48Z")
+ .endStatus()
+ .build();
+
JobResponse response = new JobResponse(job, PeonPhase.SUCCEEDED);
- Optional duration = response.getJobDuration();
- Assertions.assertEquals(Long.valueOf(58000L), duration.get());
+
+ Assertions.assertEquals(58000L, response.getJobDuration());
}
@Test
void testNoDuration()
{
- Job job = mock(Job.class);
- ObjectMeta metadata = mock(ObjectMeta.class);
- when(metadata.getName()).thenReturn("job");
- when(job.getMetadata()).thenReturn(metadata);
- JobStatus jobStatus = mock(JobStatus.class);
- when(jobStatus.getStartTime()).thenReturn("2022-09-19T23:31:50Z");
- when(job.getStatus()).thenReturn(jobStatus);
+ Job job = new JobBuilder()
+ .withNewMetadata()
+ .withName("job")
+ .endMetadata()
+ .withNewStatus()
+ .withStartTime("2022-09-19T23:31:50Z")
+ .endStatus()
+ .build();
+
JobResponse response = new JobResponse(job, PeonPhase.SUCCEEDED);
- Optional duration = response.getJobDuration();
- Assertions.assertFalse(duration.isPresent());
+
+ Assertions.assertEquals(-1, response.getJobDuration());
}
@Test
void testMakingCodeCoverageHappy()
{
- Job job = mock(Job.class);
- ObjectMeta metadata = mock(ObjectMeta.class);
- when(metadata.getName()).thenReturn("job");
- when(job.getMetadata()).thenReturn(metadata);
- when(job.getStatus()).thenReturn(null);
+ Job job = new JobBuilder()
+ .withNewMetadata()
+ .withName("job")
+ .endMetadata()
+ .build();
+
JobResponse response = new JobResponse(job, PeonPhase.SUCCEEDED);
- Optional duration = response.getJobDuration();
- Assertions.assertFalse(duration.isPresent());
+
+ Assertions.assertEquals(-1, response.getJobDuration());
}
@Test
void testNullJob()
{
JobResponse response = new JobResponse(null, PeonPhase.SUCCEEDED);
- Optional duration = response.getJobDuration();
- Assertions.assertFalse(duration.isPresent());
+ long duration = response.getJobDuration();
+ Assertions.assertEquals(-1, duration);
}
}
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java
index 6bdd3678bc8..c0affe9573c 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java
@@ -20,16 +20,9 @@
package org.apache.druid.k8s.overlord.common;
import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import io.fabric8.kubernetes.api.model.Pod;
-import io.fabric8.kubernetes.api.model.PodBuilder;
import io.fabric8.kubernetes.api.model.PodSpec;
import io.fabric8.kubernetes.api.model.PodSpecBuilder;
-import io.fabric8.kubernetes.api.model.PodTemplateSpec;
-import io.fabric8.kubernetes.api.model.batch.v1.Job;
import io.fabric8.kubernetes.client.utils.Serialization;
-import org.apache.commons.text.CharacterPredicates;
-import org.apache.commons.text.RandomStringGenerator;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.LocalInputSource;
import org.apache.druid.data.input.impl.NoopInputFormat;
@@ -53,29 +46,6 @@ public class K8sTestUtils
private static final IndexSpec INDEX_SPEC = IndexSpec.DEFAULT;
-
- /*
- * The k8s mock server can't launch pods from jobs, so we will fake it out by taking a job
- * grabbing the podSpec and launching it ourselves for testing.
- */
- @SuppressWarnings("javadoc")
- public static Pod createPodFromJob(Job job)
- {
- RandomStringGenerator random = new RandomStringGenerator.Builder().withinRange('0', 'z')
- .filteredBy(CharacterPredicates.LETTERS).build();
- PodTemplateSpec podTemplate = job.getSpec().getTemplate();
- return new PodBuilder()
- .withNewMetadata()
- .withName(new K8sTaskId(job.getMetadata().getName()).getK8sTaskId() + "-" + random.generate(5))
- .withLabels(ImmutableMap.of("job-name", new K8sTaskId(job.getMetadata().getName()).getK8sTaskId(),
- DruidK8sConstants.LABEL_KEY, "true"
- )
- )
- .endMetadata()
- .withSpec(podTemplate.getSpec())
- .build();
- }
-
public static PodSpec getDummyPodSpec()
{
return new PodSpecBuilder()
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java
new file mode 100644
index 00000000000..208773a3aaa
--- /dev/null
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java
@@ -0,0 +1,497 @@
+/*
+ * 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.PodBuilder;
+import io.fabric8.kubernetes.api.model.PodListBuilder;
+import io.fabric8.kubernetes.api.model.batch.v1.Job;
+import io.fabric8.kubernetes.api.model.batch.v1.JobBuilder;
+import io.fabric8.kubernetes.client.KubernetesClient;
+import io.fabric8.kubernetes.client.KubernetesClientTimeoutException;
+import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient;
+import io.fabric8.kubernetes.client.server.mock.KubernetesMockServer;
+import org.apache.druid.java.util.common.StringUtils;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.InputStream;
+import java.net.HttpURLConnection;
+import java.sql.Timestamp;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@EnableKubernetesMockClient(crud = true)
+public class KubernetesPeonClientTest
+{
+ private static final String ID = "id";
+ private static final String JOB_NAME = ID;
+ private static final String POD_NAME = "name";
+ private static final String NAMESPACE = "namespace";
+
+ private KubernetesClient client;
+ private KubernetesMockServer server;
+ private KubernetesClientApi clientApi;
+ private KubernetesPeonClient instance;
+
+ @BeforeEach
+ public void setup()
+ {
+ clientApi = new TestKubernetesClient(this.client);
+ instance = new KubernetesPeonClient(clientApi, NAMESPACE, false);
+ }
+
+ @Test
+ void test_launchPeonJobAndWaitForStart()
+ {
+ Job job = new JobBuilder()
+ .withNewMetadata()
+ .withName(JOB_NAME)
+ .endMetadata()
+ .build();
+
+ Pod pod = new PodBuilder()
+ .withNewMetadata()
+ .withName(POD_NAME)
+ .addToLabels("job-name", JOB_NAME)
+ .endMetadata()
+ .withNewStatus()
+ .withPodIP("ip")
+ .endStatus()
+ .build();
+
+ client.pods().inNamespace(NAMESPACE).resource(pod).create();
+
+ Pod peonPod = instance.launchPeonJobAndWaitForStart(job, 1, TimeUnit.SECONDS);
+
+ Assertions.assertNotNull(peonPod);
+ }
+
+ @Test
+ void test_launchPeonJobAndWaitForStart_withDisappearingPod_throwsKubernetesClientTimeoutException()
+ {
+ Job job = new JobBuilder()
+ .withNewMetadata()
+ .withName(JOB_NAME)
+ .endMetadata()
+ .build();
+
+ server.expect().get()
+ .withPath("/api/v1/namespaces/namespace/pods?labelSelector=job-name%3Did")
+ .andReturn(HttpURLConnection.HTTP_OK, new PodListBuilder()
+ .addNewItem()
+ .withNewMetadata()
+ .withName(POD_NAME)
+ .addToLabels("job-name", JOB_NAME)
+ .endMetadata()
+ .endItem()
+ .build()
+ ).once();
+
+ Assertions.assertThrows(
+ KubernetesClientTimeoutException.class,
+ () -> instance.launchPeonJobAndWaitForStart(job, 1, TimeUnit.SECONDS)
+ );
+ }
+
+ @Test
+ void test_waitForPeonJobCompletion_withSuccessfulJob_returnsJobResponseWithJobAndSucceededPeonPhase()
+ {
+ Job job = new JobBuilder()
+ .withNewMetadata()
+ .withName(JOB_NAME)
+ .endMetadata()
+ .withNewStatus()
+ .withActive(null)
+ .withSucceeded(1)
+ .endStatus()
+ .build();
+
+ client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create();
+
+ JobResponse jobResponse = instance.waitForPeonJobCompletion(
+ new K8sTaskId(ID),
+ 1,
+ TimeUnit.SECONDS
+ );
+
+ Assertions.assertEquals(PeonPhase.SUCCEEDED, jobResponse.getPhase());
+ Assertions.assertNotNull(jobResponse.getJob());
+ }
+
+ @Test
+ void test_waitForPeonJobCompletion_withFailedJob_returnsJobResponseWithJobAndFailedPeonPhase()
+ {
+ Job job = new JobBuilder()
+ .withNewMetadata()
+ .withName(JOB_NAME)
+ .endMetadata()
+ .withNewStatus()
+ .withActive(null)
+ .withFailed(1)
+ .endStatus()
+ .build();
+
+ client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create();
+
+ JobResponse jobResponse = instance.waitForPeonJobCompletion(
+ new K8sTaskId(ID),
+ 1,
+ TimeUnit.SECONDS
+ );
+
+ Assertions.assertEquals(PeonPhase.FAILED, jobResponse.getPhase());
+ Assertions.assertNotNull(jobResponse.getJob());
+ }
+
+ @Test
+ void test_waitforPeonJobCompletion_withoutRunningJob_returnsJobResponseWithEmptyJobAndFailedPeonPhase()
+ {
+ JobResponse jobResponse = instance.waitForPeonJobCompletion(
+ new K8sTaskId(ID),
+ 1,
+ TimeUnit.SECONDS
+ );
+
+ Assertions.assertEquals(PeonPhase.FAILED, jobResponse.getPhase());
+ Assertions.assertNull(jobResponse.getJob());
+ }
+
+ @Test
+ void test_deletePeonJob_withJob_returnsTrue()
+ {
+ Job job = new JobBuilder()
+ .withNewMetadata()
+ .withName(JOB_NAME)
+ .endMetadata()
+ .build();
+
+ client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create();
+
+ Assertions.assertTrue(instance.deletePeonJob(new K8sTaskId(ID)));
+ }
+
+ @Test
+ void test_deletePeonJob_withoutJob_returnsFalse()
+ {
+ Assertions.assertFalse(instance.deletePeonJob(new K8sTaskId(ID)));
+ }
+
+ @Test
+ void test_deletePeonJob_withJob_withDebugJobsTrue_skipsDelete()
+ {
+ KubernetesPeonClient instance = new KubernetesPeonClient(
+ new TestKubernetesClient(this.client),
+ NAMESPACE,
+ true
+ );
+
+ Job job = new JobBuilder()
+ .withNewMetadata()
+ .withName(JOB_NAME)
+ .endMetadata()
+ .build();
+
+ client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create();
+
+ Assertions.assertTrue(instance.deletePeonJob(new K8sTaskId(ID)));
+
+ Assertions.assertNotNull(
+ client.batch().v1().jobs().inNamespace(NAMESPACE).withName(ID).get()
+ );
+ }
+
+ @Test
+ void test_deletePeonJob_withoutJob_withDebugJobsTrue_skipsDelete()
+ {
+ KubernetesPeonClient instance = new KubernetesPeonClient(
+ new TestKubernetesClient(this.client),
+ NAMESPACE,
+ true
+ );
+
+ Assertions.assertTrue(instance.deletePeonJob(new K8sTaskId(ID)));
+ }
+
+ @Test
+ void test_getPeonLogs_withJob_returnsInputStreamInOptional()
+ {
+ server.expect().get()
+ .withPath("/apis/batch/v1/namespaces/namespace/jobs/id")
+ .andReturn(HttpURLConnection.HTTP_OK, new JobBuilder()
+ .withNewMetadata()
+ .withName(JOB_NAME)
+ .withUid("uid")
+ .endMetadata()
+ .withNewSpec()
+ .withNewTemplate()
+ .withNewSpec()
+ .addNewContainer()
+ .withName("main")
+ .endContainer()
+ .endSpec()
+ .endTemplate()
+ .endSpec()
+ .build()
+ ).once();
+
+ server.expect().get()
+ .withPath("/api/v1/namespaces/namespace/pods?labelSelector=controller-uid%3Duid")
+ .andReturn(HttpURLConnection.HTTP_OK, new PodListBuilder()
+ .addNewItem()
+ .withNewMetadata()
+ .withName(POD_NAME)
+ .addNewOwnerReference()
+ .withUid("uid")
+ .withController(true)
+ .endOwnerReference()
+ .endMetadata()
+ .withNewSpec()
+ .addNewContainer()
+ .withName("main")
+ .endContainer()
+ .endSpec()
+ .endItem()
+ .build()
+ ).once();
+
+ server.expect().get()
+ .withPath("/api/v1/namespaces/namespace/pods/id/log?pretty=false&container=main")
+ .andReturn(HttpURLConnection.HTTP_OK, "data")
+ .once();
+
+ Optional maybeInputStream = instance.getPeonLogs(new K8sTaskId(ID));
+ Assertions.assertTrue(maybeInputStream.isPresent());
+ }
+
+ @Test
+ void test_getPeonLogs_withJobWithoutPod_returnsEmptyOptional()
+ {
+ Job job = new JobBuilder()
+ .withNewMetadata()
+ .withName(JOB_NAME)
+ .endMetadata()
+ .build();
+
+ client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create();
+
+ Optional maybeInputStream = instance.getPeonLogs(new K8sTaskId(ID));
+ Assertions.assertFalse(maybeInputStream.isPresent());
+ }
+
+ @Test
+ void test_getPeonLogs_withoutJob_returnsEmptyOptional()
+ {
+ Optional stream = instance.getPeonLogs(new K8sTaskId(ID));
+ Assertions.assertFalse(stream.isPresent());
+ }
+
+ @Test
+ void test_getPeonJobs_withJob_returnsPodList()
+ {
+ Job job = new JobBuilder()
+ .withNewMetadata()
+ .withName(JOB_NAME)
+ .addToLabels("druid.k8s.peons", "true")
+ .endMetadata()
+ .build();
+
+ client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create();
+
+ List jobs = instance.getPeonJobs();
+
+ Assertions.assertEquals(1, jobs.size());
+ }
+
+ @Test
+ void test_getPeonJobs_withoutJob_returnsEmptyList()
+ {
+ List jobs = instance.getPeonJobs();
+ Assertions.assertEquals(0, jobs.size());
+ }
+
+ @Test
+ void test_deleteCompletedPeonJobsOlderThan_withoutJob_returnsZero()
+ {
+ int deleted = instance.deleteCompletedPeonJobsOlderThan(1, TimeUnit.MINUTES);
+ Assertions.assertEquals(0, deleted);
+ }
+
+ @Test
+ void test_deleteCompletedPeonJobsOlderThan_withActiveJob_returnsZero()
+ {
+ Job job = new JobBuilder()
+ .withNewMetadata()
+ .withName(JOB_NAME)
+ .endMetadata()
+ .withNewStatus()
+ .withActive(1)
+ .endStatus()
+ .build();
+
+ client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create();
+
+ int deleted = instance.deleteCompletedPeonJobsOlderThan(1, TimeUnit.MINUTES);
+
+ Assertions.assertEquals(0, deleted);
+ }
+
+ @Test
+ void test_deleteCompletedPeonJobsOlderThan_withoutDeleteableJob_returnsZero()
+ {
+ Job job = new JobBuilder()
+ .withNewMetadata()
+ .withName(JOB_NAME)
+ .addToLabels("druid.k8s.peons", "true")
+ .endMetadata()
+ .withNewStatus()
+ .withCompletionTime(new Timestamp(System.currentTimeMillis()).toString())
+ .endStatus()
+ .build();
+
+ client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create();
+
+ int deleted = instance.deleteCompletedPeonJobsOlderThan(1, TimeUnit.MINUTES);
+
+ Assertions.assertEquals(0, deleted);
+ }
+
+ @Test
+ void test_deleteCompletedPeonJobsOlderThan_withDeletableJob_returnsOne()
+ {
+ Job job = new JobBuilder()
+ .withNewMetadata()
+ .withName(JOB_NAME)
+ .addToLabels("druid.k8s.peons", "true")
+ .endMetadata()
+ .withNewStatus()
+ .withCompletionTime(new Timestamp(System.currentTimeMillis() - TimeUnit.MINUTES.toMillis(5)).toString())
+ .endStatus()
+ .build();
+
+ client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create();
+
+ int deleted = instance.deleteCompletedPeonJobsOlderThan(1, TimeUnit.MINUTES);
+
+ Assertions.assertEquals(1, deleted);
+ }
+
+ @Test
+ void test_deleteCompletedPeonJobsOlderThan_withActiveAndDeletableAndNotDeletableJob_returnsOne()
+ {
+ Job activeJob = new JobBuilder()
+ .withNewMetadata()
+ .withName(StringUtils.format("%s-active", JOB_NAME))
+ .endMetadata()
+ .withNewStatus()
+ .withActive(1)
+ .endStatus()
+ .build();
+
+ Job deletableJob = new JobBuilder()
+ .withNewMetadata()
+ .withName(StringUtils.format("%s-deleteable", JOB_NAME))
+ .addToLabels("druid.k8s.peons", "true")
+ .endMetadata()
+ .withNewStatus()
+ .withCompletionTime(new Timestamp(System.currentTimeMillis() - TimeUnit.MINUTES.toMillis(5)).toString())
+ .endStatus()
+ .build();
+
+ Job undeletableJob = new JobBuilder()
+ .withNewMetadata()
+ .withName(StringUtils.format("%s-undeletable", JOB_NAME))
+ .addToLabels("druid.k8s.peons", "true")
+ .endMetadata()
+ .withNewStatus()
+ .withCompletionTime(new Timestamp(System.currentTimeMillis()).toString())
+ .endStatus()
+ .build();
+
+ client.batch().v1().jobs().inNamespace(NAMESPACE).resource(activeJob).create();
+ client.batch().v1().jobs().inNamespace(NAMESPACE).resource(deletableJob).create();
+ client.batch().v1().jobs().inNamespace(NAMESPACE).resource(undeletableJob).create();
+
+ int deleted = instance.deleteCompletedPeonJobsOlderThan(1, TimeUnit.MINUTES);
+
+ Assertions.assertEquals(1, deleted);
+ }
+
+ @Test
+ void test_getPeonPod_withPod_returnsPodInOptional()
+ {
+ Pod pod = new PodBuilder()
+ .withNewMetadata()
+ .withName(POD_NAME)
+ .addToLabels("job-name", JOB_NAME)
+ .endMetadata()
+ .build();
+
+ client.pods().inNamespace(NAMESPACE).resource(pod).create();
+
+ Optional maybePod = instance.getPeonPod(new K8sTaskId(ID));
+
+ Assertions.assertTrue(maybePod.isPresent());
+ }
+
+ @Test
+ void test_getPeonPod_withoutPod_returnsEmptyOptional()
+ {
+ Optional maybePod = instance.getPeonPod(new K8sTaskId(ID));
+ Assertions.assertFalse(maybePod.isPresent());
+ }
+
+ @Test
+ void test_getPeonPodWithRetries_withPod_returnsPod()
+ {
+ server.expect().get()
+ .withPath("/api/v1/namespaces/namespace/pods?labelSelector=job-name%3Did")
+ .andReturn(HttpURLConnection.HTTP_OK, new PodListBuilder().build())
+ .once();
+
+ server.expect().get()
+ .withPath("/api/v1/namespaces/namespace/pods?labelSelector=job-name%3Did")
+ .andReturn(HttpURLConnection.HTTP_OK, new PodListBuilder()
+ .addNewItem()
+ .withNewMetadata()
+ .withName(POD_NAME)
+ .addToLabels("job-name", JOB_NAME)
+ .endMetadata()
+ .endItem()
+ .build()
+ ).once();
+
+ Pod pod = instance.getPeonPodWithRetries(new K8sTaskId(ID));
+
+ Assertions.assertNotNull(pod);
+ }
+
+ @Test
+ void test_getPeonPodWithRetries_withoutPod_raisesKubernetesResourceNotFoundException()
+ {
+ Assertions.assertThrows(
+ KubernetesResourceNotFoundException.class,
+ () -> instance.getPeonPodWithRetries(clientApi.getClient(), new K8sTaskId(ID), 1, 1),
+ StringUtils.format("K8s pod with label: job-name=%s not found", ID)
+ );
+ }
+}
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DruidPeonClientIntegrationTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java
similarity index 84%
rename from extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DruidPeonClientIntegrationTest.java
rename to extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java
index 318d3f8810c..ebec69dc117 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DruidPeonClientIntegrationTest.java
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java
@@ -17,7 +17,7 @@
* under the License.
*/
-package org.apache.druid.k8s.overlord.common;
+package org.apache.druid.k8s.overlord.taskadapter;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper;
@@ -35,6 +35,14 @@ import org.apache.druid.indexing.common.task.IndexTask;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig;
import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig;
+import org.apache.druid.k8s.overlord.common.DruidKubernetesClient;
+import org.apache.druid.k8s.overlord.common.JobResponse;
+import org.apache.druid.k8s.overlord.common.K8sTaskId;
+import org.apache.druid.k8s.overlord.common.K8sTestUtils;
+import org.apache.druid.k8s.overlord.common.KubernetesClientApi;
+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.server.DruidNode;
import org.apache.druid.server.log.StartupLoggingConfig;
import org.junit.jupiter.api.BeforeEach;
@@ -66,7 +74,7 @@ public class DruidPeonClientIntegrationTest
private TaskConfig taskConfig;
private DruidNode druidNode;
private KubernetesClientApi k8sClient;
- private DruidKubernetesPeonClient peonClient;
+ private KubernetesPeonClient peonClient;
private ObjectMapper jsonMapper;
@BeforeEach
@@ -82,7 +90,7 @@ public class DruidPeonClientIntegrationTest
new NamedType(IndexTask.IndexTuningConfig.class, "index")
);
k8sClient = new DruidKubernetesClient();
- peonClient = new DruidKubernetesPeonClient(k8sClient, "default", false);
+ peonClient = new KubernetesPeonClient(k8sClient, "default", false);
druidNode = new DruidNode(
"test",
null,
@@ -103,8 +111,9 @@ public class DruidPeonClientIntegrationTest
PodSpec podSpec = K8sTestUtils.getDummyPodSpec();
Task task = K8sTestUtils.getTask();
- KubernetesTaskRunnerConfig config = new KubernetesTaskRunnerConfig();
- config.namespace = "default";
+ KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder()
+ .withNamespace("default")
+ .build();
K8sTaskAdapter adapter = new SingleContainerTaskAdapter(
k8sClient,
config,
@@ -121,10 +130,10 @@ public class DruidPeonClientIntegrationTest
Job job = adapter.createJobFromPodSpec(podSpec, task, context);
// launch the job and wait to start...
- peonClient.launchJobAndWaitForStart(job, 1, TimeUnit.MINUTES);
+ peonClient.launchPeonJobAndWaitForStart(job, 1, TimeUnit.MINUTES);
// there should be one job that is a k8s peon job that exists
- List jobs = peonClient.listAllPeonJobs();
+ List jobs = peonClient.getPeonJobs();
assertEquals(1, jobs.size());
K8sTaskId taskId = new K8sTaskId(task.getId());
@@ -150,7 +159,7 @@ public class DruidPeonClientIntegrationTest
// now copy the task.json file from the pod and make sure its the same as our task.json we expected
Path downloadPath = Paths.get(tempDir.toAbsolutePath().toString(), "task.json");
- Pod mainJobPod = peonClient.getMainJobPod(taskId);
+ Pod mainJobPod = peonClient.getPeonPodWithRetries(taskId);
k8sClient.executeRequest(client -> {
client.pods()
.inNamespace("default")
@@ -165,16 +174,16 @@ public class DruidPeonClientIntegrationTest
assertEquals(task, taskFromPod);
- JobResponse jobStatusResult = peonClient.waitForJobCompletion(taskId, 2, TimeUnit.MINUTES);
+ JobResponse jobStatusResult = peonClient.waitForPeonJobCompletion(taskId, 2, TimeUnit.MINUTES);
thread.join();
assertEquals(PeonPhase.SUCCEEDED, jobStatusResult.getPhase());
// as long as there were no exceptions we are good!
assertEquals(expectedLogs, actualLogs);
// cleanup my job
- assertTrue(peonClient.cleanUpJob(taskId));
+ assertTrue(peonClient.deletePeonJob(taskId));
// we cleaned up the job, none should exist
- List existingJobs = peonClient.listAllPeonJobs();
+ List existingJobs = peonClient.getPeonJobs();
assertEquals(0, existingJobs.size());
}
}
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java
similarity index 91%
rename from extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTaskAdapterTest.java
rename to extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java
index a9420431578..519e7177cbe 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTaskAdapterTest.java
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java
@@ -17,12 +17,13 @@
* under the License.
*/
-package org.apache.druid.k8s.overlord.common;
+package org.apache.druid.k8s.overlord.taskadapter;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.google.api.client.util.Joiner;
+import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
@@ -50,6 +51,12 @@ import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig;
import org.apache.druid.java.util.common.HumanReadableBytes;
import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig;
+import org.apache.druid.k8s.overlord.common.DruidK8sConstants;
+import org.apache.druid.k8s.overlord.common.K8sTestUtils;
+import org.apache.druid.k8s.overlord.common.KubernetesExecutor;
+import org.apache.druid.k8s.overlord.common.KubernetesResourceNotFoundException;
+import org.apache.druid.k8s.overlord.common.PeonCommandContext;
+import org.apache.druid.k8s.overlord.common.TestKubernetesClient;
import org.apache.druid.server.DruidNode;
import org.apache.druid.server.log.StartupLoggingConfig;
import org.junit.jupiter.api.Assertions;
@@ -121,10 +128,11 @@ class K8sTaskAdapterTest
}
};
- KubernetesTaskRunnerConfig config = new KubernetesTaskRunnerConfig();
- config.namespace = "test";
- config.annotations.put("annotation_key", "annotation_value");
- config.labels.put("label_key", "label_value");
+ KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder()
+ .withNamespace("test")
+ .withAnnotations(ImmutableMap.of("annotation_key", "annotation_value"))
+ .withLabels(ImmutableMap.of("label_key", "label_value"))
+ .build();
K8sTaskAdapter adapter = new SingleContainerTaskAdapter(
testClient,
config,
@@ -149,8 +157,9 @@ class K8sTaskAdapterTest
{
// given a task create a k8s job
TestKubernetesClient testClient = new TestKubernetesClient(client);
- KubernetesTaskRunnerConfig config = new KubernetesTaskRunnerConfig();
- config.namespace = "test";
+ KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder()
+ .withNamespace("test")
+ .build();
K8sTaskAdapter adapter = new SingleContainerTaskAdapter(
testClient,
config,
@@ -264,8 +273,9 @@ class K8sTaskAdapterTest
new ArrayList<>(),
new File("/tmp/")
);
- KubernetesTaskRunnerConfig config = new KubernetesTaskRunnerConfig();
- config.namespace = "test";
+ KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder()
+ .withNamespace("test")
+ .build();
K8sTaskAdapter adapter = new SingleContainerTaskAdapter(
testClient,
config,
@@ -285,7 +295,10 @@ class K8sTaskAdapterTest
);
// we have an override, but nothing in the overlord
- config.peonMonitors = jsonMapper.readValue("[\"org.apache.druid.java.util.metrics.JvmMonitor\"]", List.class);
+ config = KubernetesTaskRunnerConfig.builder()
+ .withNamespace("test")
+ .withPeonMonitors(ImmutableList.of("org.apache.druid.java.util.metrics.JvmMonitor"))
+ .build();
adapter = new SingleContainerTaskAdapter(
testClient,
config,
@@ -300,10 +313,9 @@ class K8sTaskAdapterTest
.filter(x -> x.getName().equals("druid_monitoring_monitors"))
.findFirst()
.get();
- assertEquals(jsonMapper.writeValueAsString(config.peonMonitors), env.getValue());
+ assertEquals(jsonMapper.writeValueAsString(config.getPeonMonitors()), env.getValue());
// we override what is in the overlord
- config.peonMonitors = jsonMapper.readValue("[\"org.apache.druid.java.util.metrics.JvmMonitor\"]", List.class);
adapter = new SingleContainerTaskAdapter(
testClient,
config,
@@ -324,7 +336,7 @@ class K8sTaskAdapterTest
.filter(x -> x.getName().equals("druid_monitoring_monitors"))
.findFirst()
.get();
- assertEquals(jsonMapper.writeValueAsString(config.peonMonitors), env.getValue());
+ assertEquals(jsonMapper.writeValueAsString(config.getPeonMonitors()), env.getValue());
}
@Test
@@ -332,8 +344,9 @@ class K8sTaskAdapterTest
{
TestKubernetesClient testClient = new TestKubernetesClient(client);
Pod pod = K8sTestUtils.fileToResource("ephemeralPodSpec.yaml", Pod.class);
- KubernetesTaskRunnerConfig config = new KubernetesTaskRunnerConfig();
- config.namespace = "test";
+ KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder()
+ .withNamespace("test")
+ .build();
SingleContainerTaskAdapter adapter =
new SingleContainerTaskAdapter(testClient,
config, taskConfig,
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/MultiContainerTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java
similarity index 90%
rename from extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/MultiContainerTaskAdapterTest.java
rename to extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java
index 28f037126c7..aa93856ded8 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/MultiContainerTaskAdapterTest.java
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java
@@ -17,11 +17,12 @@
* under the License.
*/
-package org.apache.druid.k8s.overlord.common;
+package org.apache.druid.k8s.overlord.taskadapter;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.jsontype.NamedType;
+import com.google.common.collect.ImmutableList;
import io.fabric8.kubernetes.api.model.Pod;
import io.fabric8.kubernetes.api.model.PodSpec;
import io.fabric8.kubernetes.api.model.batch.v1.Job;
@@ -35,6 +36,9 @@ import org.apache.druid.indexing.common.task.IndexTask;
import org.apache.druid.indexing.common.task.NoopTask;
import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig;
import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig;
+import org.apache.druid.k8s.overlord.common.K8sTestUtils;
+import org.apache.druid.k8s.overlord.common.PeonCommandContext;
+import org.apache.druid.k8s.overlord.common.TestKubernetesClient;
import org.apache.druid.server.DruidNode;
import org.apache.druid.server.log.StartupLoggingConfig;
import org.junit.jupiter.api.Assertions;
@@ -45,7 +49,6 @@ import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
-import java.util.List;
@EnableKubernetesMockClient(crud = true)
class MultiContainerTaskAdapterTest
@@ -86,8 +89,9 @@ class MultiContainerTaskAdapterTest
{
TestKubernetesClient testClient = new TestKubernetesClient(client);
Pod pod = K8sTestUtils.fileToResource("multiContainerPodSpec.yaml", Pod.class);
- KubernetesTaskRunnerConfig config = new KubernetesTaskRunnerConfig();
- config.namespace = "test";
+ KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder()
+ .withNamespace("test")
+ .build();
MultiContainerTaskAdapter adapter = new MultiContainerTaskAdapter(
testClient,
config,
@@ -132,9 +136,10 @@ class MultiContainerTaskAdapterTest
{
TestKubernetesClient testClient = new TestKubernetesClient(client);
Pod pod = K8sTestUtils.fileToResource("multiContainerPodSpecOrder.yaml", Pod.class);
- KubernetesTaskRunnerConfig config = new KubernetesTaskRunnerConfig();
- config.namespace = "test";
- config.primaryContainerName = "primary";
+ KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder()
+ .withNamespace("test")
+ .withPrimaryContainerName("primary")
+ .build();
MultiContainerTaskAdapter adapter = new MultiContainerTaskAdapter(
testClient,
config,
@@ -181,10 +186,11 @@ class MultiContainerTaskAdapterTest
{
TestKubernetesClient testClient = new TestKubernetesClient(client);
Pod pod = K8sTestUtils.fileToResource("podSpec.yaml", Pod.class);
- KubernetesTaskRunnerConfig config = new KubernetesTaskRunnerConfig();
- config.namespace = "test";
- config.primaryContainerName = "primary";
- config.peonMonitors = jsonMapper.readValue("[\"org.apache.druid.java.util.metrics.JvmMonitor\"]", List.class);
+ KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder()
+ .withNamespace("test")
+ .withPrimaryContainerName("primary")
+ .withPeonMonitors(ImmutableList.of("org.apache.druid.java.util.metrics.JvmMonitor"))
+ .build();
MultiContainerTaskAdapter adapter = new MultiContainerTaskAdapter(testClient,
config,
taskConfig,
@@ -193,7 +199,7 @@ class MultiContainerTaskAdapterTest
jsonMapper);
NoopTask task = NoopTask.create("id", 1);
PodSpec spec = pod.getSpec();
- K8sTaskAdapter.massageSpec(spec, config.primaryContainerName);
+ K8sTaskAdapter.massageSpec(spec, config.getPrimaryContainerName());
Job actual = adapter.createJobFromPodSpec(
spec,
task,
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/PodTemplateTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java
similarity index 97%
rename from extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/PodTemplateTaskAdapterTest.java
rename to extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java
index 1e35b90a53d..2a9381aa3d3 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/PodTemplateTaskAdapterTest.java
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java
@@ -17,7 +17,7 @@
* under the License.
*/
-package org.apache.druid.k8s.overlord.common;
+package org.apache.druid.k8s.overlord.taskadapter;
import com.fasterxml.jackson.databind.ObjectMapper;
import io.fabric8.kubernetes.api.model.PodTemplate;
@@ -32,6 +32,9 @@ import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.IOE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig;
+import org.apache.druid.k8s.overlord.common.Base64Compression;
+import org.apache.druid.k8s.overlord.common.DruidK8sConstants;
+import org.apache.druid.k8s.overlord.common.K8sTestUtils;
import org.apache.druid.server.DruidNode;
import org.junit.Assert;
import org.junit.jupiter.api.Assertions;
@@ -58,7 +61,7 @@ public class PodTemplateTaskAdapterTest
@BeforeEach
public void setup()
{
- taskRunnerConfig = new KubernetesTaskRunnerConfig();
+ taskRunnerConfig = KubernetesTaskRunnerConfig.builder().build();
taskConfig = new TaskConfigBuilder().setBaseDir("/tmp").build();
node = new DruidNode(
"test",
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/SingleContainerTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java
similarity index 92%
rename from extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/SingleContainerTaskAdapterTest.java
rename to extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java
index 35d3d0d8288..3e51c5a7acb 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/SingleContainerTaskAdapterTest.java
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java
@@ -17,7 +17,7 @@
* under the License.
*/
-package org.apache.druid.k8s.overlord.common;
+package org.apache.druid.k8s.overlord.taskadapter;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper;
@@ -34,6 +34,9 @@ import org.apache.druid.indexing.common.task.IndexTask;
import org.apache.druid.indexing.common.task.NoopTask;
import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig;
import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig;
+import org.apache.druid.k8s.overlord.common.K8sTestUtils;
+import org.apache.druid.k8s.overlord.common.PeonCommandContext;
+import org.apache.druid.k8s.overlord.common.TestKubernetesClient;
import org.apache.druid.server.DruidNode;
import org.apache.druid.server.log.StartupLoggingConfig;
import org.junit.jupiter.api.Assertions;
@@ -84,8 +87,9 @@ class SingleContainerTaskAdapterTest
{
TestKubernetesClient testClient = new TestKubernetesClient(client);
Pod pod = K8sTestUtils.fileToResource("multiContainerPodSpec.yaml", Pod.class);
- KubernetesTaskRunnerConfig config = new KubernetesTaskRunnerConfig();
- config.namespace = "test";
+ KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder()
+ .withNamespace("test")
+ .build();
SingleContainerTaskAdapter adapter = new SingleContainerTaskAdapter(
testClient,
config,
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/kubernetesTaskRunnerConfig.json b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/kubernetesTaskRunnerConfig.json
new file mode 100644
index 00000000000..637a6386114
--- /dev/null
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/kubernetesTaskRunnerConfig.json
@@ -0,0 +1,5 @@
+{
+ "namespace": "namespace",
+ "primaryContainerName": "name",
+ "disableClientProxy": true
+}
\ No newline at end of file