mirror of https://github.com/apache/druid.git
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:
parent
9eebeead44
commit
58dcbf9399
|
@ -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>
|
||||
|
|
|
@ -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)
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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(
|
|
@ -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();
|
|
@ -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())
|
|
@ -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;
|
||||
|
|
@ -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;
|
|
@ -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
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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,
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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)
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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,
|
|
@ -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,
|
|
@ -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",
|
|
@ -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,
|
|
@ -0,0 +1,5 @@
|
|||
{
|
||||
"namespace": "namespace",
|
||||
"primaryContainerName": "name",
|
||||
"disableClientProxy": true
|
||||
}
|
Loading…
Reference in New Issue