queue tasks in kubernetes task runner if capacity is fully utilized (#14156)

* queue tasks if all slots in use

* Declare hamcrest-core dependency

* Use AtomicBoolean for shutdown requested

* Use AtomicReference for peon lifecycle state

* fix uninitialized read error

* fix indentations

* Make tasks protected

* fix KubernetesTaskRunnerConfig deserialization

* ensure k8s task runner max capacity is Integer.MAX_VALUE

* set job duration as task status duration

* Address pr comments

---------

Co-authored-by: George Shiqi Wu <george.wu@imply.io>
This commit is contained in:
Nicholas Lippis 2023-05-12 11:41:44 -04:00 committed by GitHub
parent 9eebeead44
commit 58dcbf9399
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
35 changed files with 3415 additions and 1855 deletions

View File

@ -157,6 +157,18 @@
<version>5.8.2</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.hamcrest</groupId>
<artifactId>hamcrest-core</artifactId>
<version>2.2</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.hamcrest</groupId>
<artifactId>hamcrest</artifactId>
<version>2.2</version>
<scope>test</scope>
</dependency>
<!-- others -->
<dependency>

View File

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

View File

@ -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> 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<InputStream> 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<Pod> 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<InputStream> 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<InputStream> 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());
}
}
}

View File

@ -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
);
}
}

View File

@ -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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
// to cleanup old jobs that might not have been deleted.
private final ScheduledExecutorService cleanupExecutor;
protected final ConcurrentHashMap<String, K8sWorkItem> tasks = new ConcurrentHashMap<>();
protected final ConcurrentHashMap<String, KubernetesWorkItem> tasks = new ConcurrentHashMap<>();
protected final TaskAdapter adapter;
protected final KubernetesPeonClient client;
private final ObjectMapper mapper;
private final KubernetesTaskRunnerConfig k8sConfig;
private final TaskQueueConfig taskQueueConfig;
private final TaskLogs taskLogs;
private final KubernetesPeonClient client;
private final KubernetesTaskRunnerConfig config;
private final ListeningExecutorService exec;
private final HttpClient httpClient;
private final PeonLifecycleFactory peonLifecycleFactory;
public KubernetesTaskRunner(
ObjectMapper mapper,
TaskAdapter adapter,
KubernetesTaskRunnerConfig k8sConfig,
TaskQueueConfig taskQueueConfig,
TaskLogs taskLogs,
KubernetesTaskRunnerConfig config,
KubernetesPeonClient client,
HttpClient httpClient
HttpClient httpClient,
PeonLifecycleFactory peonLifecycleFactory
)
{
this.mapper = mapper;
this.adapter = adapter;
this.k8sConfig = k8sConfig;
this.taskQueueConfig = taskQueueConfig;
this.taskLogs = taskLogs;
this.config = config;
this.client = client;
this.httpClient = httpClient;
this.peonLifecycleFactory = peonLifecycleFactory;
this.cleanupExecutor = Executors.newScheduledThreadPool(1);
this.exec = MoreExecutors.listeningDecorator(
Execs.multiThreaded(taskQueueConfig.getMaxSize(), "k8s-task-runner-%d")
);
Preconditions.checkArgument(
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<InputStream> 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<TaskStatus> 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<Job> 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<InputStream> 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<TaskStatus> 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<InputStream> 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<InputStream> 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<Pair<Task, ListenableFuture<TaskStatus>>> restore()
{
return ImmutableList.of();
List<Pair<Task, ListenableFuture<TaskStatus>>> 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<String, Long> getTotalTaskSlotCount()
{
return ImmutableMap.of("taskQueue", (long) taskQueueConfig.getMaxSize());
return ImmutableMap.of("taskQueue", (long) config.getCapacity());
}
@Override
public Collection<? extends TaskRunnerWorkItem> getKnownTasks()
{
List<TaskRunnerWorkItem> result = new ArrayList<>();
for (Job existingTask : client.listAllPeonJobs()) {
try {
Task task = adapter.toTask(existingTask);
ListenableFuture<TaskStatus> 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<TaskRunnerWorkItem> getRunningTasks()
{
List<TaskRunnerWorkItem> result = new ArrayList<>();
for (Job existingTask : client.listAllPeonJobs().stream().filter(JobStatus::isActive).collect(Collectors.toSet())) {
try {
Task task = adapter.toTask(existingTask);
ListenableFuture<TaskStatus> 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<TaskRunnerWorkItem> 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<TaskStatus> statusFuture)
{
super(task.getId(), statusFuture);
this.task = task;
this.client = client;
}
public K8sWorkItem(
KubernetesPeonClient client,
Task task,
ListenableFuture<TaskStatus> 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();
}
}
}

View File

@ -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<String> peonMonitors = new ArrayList<>();
private List<String> peonMonitors = ImmutableList.of();
@JsonProperty
@NotNull
public List<String> javaOptsArray = new ArrayList<>();
private List<String> javaOptsArray = ImmutableList.of();
@JsonProperty
@NotNull
public String classpath = System.getProperty("java.class.path");
private Map<String, String> labels = ImmutableMap.of();
@JsonProperty
@NotNull
public Map<String, String> labels = new HashMap<>();
private Map<String, String> annotations = ImmutableMap.of();
@JsonProperty
@Min(1)
@Max(Integer.MAX_VALUE)
@NotNull
public Map<String, String> annotations = new HashMap<>();
private Integer capacity = Integer.MAX_VALUE;
@JsonProperty
@NotNull
List<String> 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<String> peonMonitors,
List<String> javaOptsArray,
Map<String, String> labels,
Map<String, String> 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<String> getPeonMonitors()
{
return peonMonitors;
}
public List<String> getJavaOptsArray()
{
return javaOptsArray;
}
public Map<String, String> getLabels()
{
return labels;
}
public Map<String, String> 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<String> peonMonitors;
private List<String> javaOptsArray;
private Map<String, String> labels;
private Map<String, String> 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<String> peonMonitors)
{
this.peonMonitors = peonMonitors;
return this;
}
public Builder withJavaOptsArray(List<String> javaOptsArray)
{
this.javaOptsArray = javaOptsArray;
return this;
}
public Builder withLabels(Map<String, String> labels)
{
this.labels = labels;
return this;
}
public Builder withAnnotations(Map<String, String> 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
);
}
}
}

View File

@ -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<Kubernetes
private final HttpClient httpClient;
private final KubernetesTaskRunnerConfig kubernetesTaskRunnerConfig;
private final StartupLoggingConfig startupLoggingConfig;
private final TaskQueueConfig taskQueueConfig;
private final TaskLogs taskLogs;
private final DruidNode druidNode;
private final TaskConfig taskConfig;
@ -67,19 +64,16 @@ public class KubernetesTaskRunnerFactory implements TaskRunnerFactory<Kubernetes
@EscalatedGlobal final HttpClient httpClient,
KubernetesTaskRunnerConfig kubernetesTaskRunnerConfig,
StartupLoggingConfig startupLoggingConfig,
@JacksonInject TaskQueueConfig taskQueueConfig,
TaskLogs taskLogs,
@Self DruidNode druidNode,
TaskConfig taskConfig,
Properties properties
)
{
this.smileMapper = smileMapper;
this.httpClient = httpClient;
this.kubernetesTaskRunnerConfig = kubernetesTaskRunnerConfig;
this.startupLoggingConfig = startupLoggingConfig;
this.taskQueueConfig = taskQueueConfig;
this.taskLogs = taskLogs;
this.druidNode = druidNode;
this.taskConfig = taskConfig;
@ -90,7 +84,7 @@ public class KubernetesTaskRunnerFactory implements TaskRunnerFactory<Kubernetes
public KubernetesTaskRunner build()
{
DruidKubernetesClient client;
if (kubernetesTaskRunnerConfig.disableClientProxy) {
if (kubernetesTaskRunnerConfig.isDisableClientProxy()) {
Config config = new ConfigBuilder().build();
config.setHttpsProxy(null);
config.setHttpProxy(null);
@ -99,14 +93,18 @@ public class KubernetesTaskRunnerFactory implements TaskRunnerFactory<Kubernetes
client = new DruidKubernetesClient();
}
KubernetesPeonClient peonClient = new KubernetesPeonClient(
client,
kubernetesTaskRunnerConfig.getNamespace(),
kubernetesTaskRunnerConfig.isDebugJobs()
);
runner = new KubernetesTaskRunner(
smileMapper,
buildTaskAdapter(client),
kubernetesTaskRunnerConfig,
taskQueueConfig,
taskLogs,
new DruidKubernetesPeonClient(client, kubernetesTaskRunnerConfig.namespace, kubernetesTaskRunnerConfig.debugJobs),
httpClient
peonClient,
httpClient,
new KubernetesPeonLifecycleFactory(peonClient, taskLogs, smileMapper)
);
return runner;
}
@ -126,7 +124,7 @@ public class KubernetesTaskRunnerFactory implements TaskRunnerFactory<Kubernetes
TYPE_NAME
));
if (adapter != null && !MultiContainerTaskAdapter.TYPE.equals(adapter) && kubernetesTaskRunnerConfig.sidecarSupport) {
if (adapter != null && !MultiContainerTaskAdapter.TYPE.equals(adapter) && kubernetesTaskRunnerConfig.isSidecarSupport()) {
throw new IAE(
"Invalid pod adapter [%s], only pod adapter [%s] can be specified when sidecarSupport is enabled",
adapter,
@ -134,7 +132,7 @@ public class KubernetesTaskRunnerFactory implements TaskRunnerFactory<Kubernetes
);
}
if (MultiContainerTaskAdapter.TYPE.equals(adapter) || kubernetesTaskRunnerConfig.sidecarSupport) {
if (MultiContainerTaskAdapter.TYPE.equals(adapter) || kubernetesTaskRunnerConfig.isSidecarSupport()) {
return new MultiContainerTaskAdapter(
client,
kubernetesTaskRunnerConfig,

View File

@ -0,0 +1,125 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.k8s.overlord;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.util.concurrent.ListenableFuture;
import org.apache.druid.indexer.RunnerTaskState;
import org.apache.druid.indexer.TaskLocation;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.TaskRunnerWorkItem;
import org.apache.druid.java.util.common.ISE;
import java.io.InputStream;
import java.util.concurrent.atomic.AtomicBoolean;
public class KubernetesWorkItem extends TaskRunnerWorkItem
{
private final Task task;
private final AtomicBoolean shutdownRequested = new AtomicBoolean(false);
private KubernetesPeonLifecycle kubernetesPeonLifecycle = null;
public KubernetesWorkItem(Task task, ListenableFuture<TaskStatus> 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<InputStream> 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();
}
}

View File

@ -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);
}

View File

@ -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<Throwable> IS_TRANSIENT = e -> e instanceof KubernetesResourceNotFoundException;
}

View File

@ -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<Job> 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<InputStream> 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<Job> 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<Job> 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<Job> getJobsToCleanup(List<Job> candidates, long howFarBack, TimeUnit timeUnit)
{
List<Job> 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");
}
}
}

View File

@ -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<Long> getJobDuration()
public long getJobDuration()
{
Optional<Long> 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;
}
}

View File

@ -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<Job> 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<InputStream> 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<Job> listAllPeonJobs();
public Optional<InputStream> 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<Job> 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<Job> 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<Job> getJobsToCleanup(List<Job> candidates, long howFarBack, TimeUnit timeUnit)
{
List<Job> 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<Pod> getPeonPod(K8sTaskId taskId)
{
return clientApi.executeRequest(client -> getPeonPod(client, taskId));
}
private Optional<Pod> getPeonPod(KubernetesClient client, K8sTaskId taskId)
{
List<Pod> 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<Pod> 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");
}
}
}

View File

@ -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<String, String> addJobSpecificAnnotations(PeonCommandContext context, K8sTaskId k8sTaskId)
{
Map<String, String> annotations = taskRunnerConfig.annotations;
annotations.put(DruidK8sConstants.TASK_ID, k8sTaskId.getOriginalTaskId());
annotations.put(DruidK8sConstants.TLS_ENABLED, String.valueOf(context.isEnableTls()));
return annotations;
return ImmutableMap.<String, String>builder()
.putAll(taskRunnerConfig.getAnnotations())
.put(DruidK8sConstants.TASK_ID, k8sTaskId.getOriginalTaskId())
.put(DruidK8sConstants.TLS_ENABLED, String.valueOf(context.isEnableTls()))
.build();
}
protected Map<String, String> addJobSpecificLabels()
{
Map<String, String> labels = taskRunnerConfig.labels;
labels.put(DruidK8sConstants.LABEL_KEY, "true");
return labels;
return ImmutableMap.<String, String>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<String> javaOpts(Task task)
{
final List<String> javaOpts = new ArrayList<>();
Iterables.addAll(javaOpts, taskRunnerConfig.javaOptsArray);
Iterables.addAll(javaOpts, taskRunnerConfig.getJavaOptsArray());
// Override task specific javaOpts
Object taskJavaOpts = task.getContextValue(

View File

@ -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();

View File

@ -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<String, String> getJobLabels(KubernetesTaskRunnerConfig config, Task task)
{
return ImmutableMap.<String, String>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<String, String> getJobAnnotations(KubernetesTaskRunnerConfig config, Task task)
{
return ImmutableMap.<String, String>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())

View File

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

View File

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

View File

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

View File

@ -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));
}
}

View File

@ -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());
}
}

View File

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

View File

@ -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());
}
}

View File

@ -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;
}
}

View File

@ -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<Job> 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<Job> jobs = Lists.newArrayList(active, dontKillYet, killThisOne);
List<Job> 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<InputStream> 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<InputStream> 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<InputStream> logs = peonClient.getPeonLogs(new K8sTaskId("foo"));
Assertions.assertFalse(logs.isPresent());
}
@Test
void testGetLogReaderForJobWithoutJobReturnsEmptyOptional()
{
DruidKubernetesPeonClient peonClient = new DruidKubernetesPeonClient(
new TestKubernetesClient(client),
"test",
false
);
Optional<InputStream> 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;
}
}

View File

@ -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<Long> 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<Long> 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<Long> duration = response.getJobDuration();
Assertions.assertFalse(duration.isPresent());
Assertions.assertEquals(-1, response.getJobDuration());
}
@Test
void testNullJob()
{
JobResponse response = new JobResponse(null, PeonPhase.SUCCEEDED);
Optional<Long> duration = response.getJobDuration();
Assertions.assertFalse(duration.isPresent());
long duration = response.getJobDuration();
Assertions.assertEquals(-1, duration);
}
}

View File

@ -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()

View File

@ -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<InputStream> 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<InputStream> maybeInputStream = instance.getPeonLogs(new K8sTaskId(ID));
Assertions.assertFalse(maybeInputStream.isPresent());
}
@Test
void test_getPeonLogs_withoutJob_returnsEmptyOptional()
{
Optional<InputStream> 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<Job> jobs = instance.getPeonJobs();
Assertions.assertEquals(1, jobs.size());
}
@Test
void test_getPeonJobs_withoutJob_returnsEmptyList()
{
List<Job> 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<Pod> maybePod = instance.getPeonPod(new K8sTaskId(ID));
Assertions.assertTrue(maybePod.isPresent());
}
@Test
void test_getPeonPod_withoutPod_returnsEmptyOptional()
{
Optional<Pod> 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)
);
}
}

View File

@ -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<Job> jobs = peonClient.listAllPeonJobs();
List<Job> 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<Job> existingJobs = peonClient.listAllPeonJobs();
List<Job> existingJobs = peonClient.getPeonJobs();
assertEquals(0, existingJobs.size());
}
}

View File

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

View File

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

View File

@ -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",

View File

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

View File

@ -0,0 +1,5 @@
{
"namespace": "namespace",
"primaryContainerName": "name",
"disableClientProxy": true
}