k8s-based-ingestion: Wait for task lifecycles to enter RUNNING state before returning from KubernetesTaskRunner.start (#17446)

* Add a wait on start() for task lifecycle to go into running

* handle exceptions

* Fix logging messages

* Don't pass in the settable future as a arg

* add some unit tests
This commit is contained in:
George Shiqi Wu 2024-11-08 08:13:35 -08:00 committed by GitHub
parent d8162163c8
commit 5764183d4e
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
8 changed files with 311 additions and 132 deletions

View File

@ -610,6 +610,7 @@ Druid selects the pod template `podSpecWithHighMemRequests.yaml`.
|`druid.indexer.runner.maxTaskDuration`| `Duration` | Max time a task is allowed to run for before getting killed |`PT4H`|No| |`druid.indexer.runner.maxTaskDuration`| `Duration` | Max time a task is allowed to run for before getting killed |`PT4H`|No|
|`druid.indexer.runner.taskCleanupDelay`| `Duration` | How long do jobs stay around before getting reaped from K8s |`P2D`|No| |`druid.indexer.runner.taskCleanupDelay`| `Duration` | How long do jobs stay around before getting reaped from K8s |`P2D`|No|
|`druid.indexer.runner.taskCleanupInterval`| `Duration` | How often to check for jobs to be reaped |`PT10M`|No| |`druid.indexer.runner.taskCleanupInterval`| `Duration` | How often to check for jobs to be reaped |`PT10M`|No|
|`druid.indexer.runner.taskJoinTimeout`| `Duration` | Timeout for gathering metadata about existing tasks on startup |`PT1M`|No|
|`druid.indexer.runner.K8sjobLaunchTimeout`| `Duration` | How long to wait to launch a K8s task before marking it as failed, on a resource constrained cluster it may take some time. |`PT1H`|No| |`druid.indexer.runner.K8sjobLaunchTimeout`| `Duration` | How long to wait to launch a K8s task before marking it as failed, on a resource constrained cluster it may take some time. |`PT1H`|No|
|`druid.indexer.runner.javaOptsArray`| `JsonArray` | java opts for the task. |`-Xmx1g`|No| |`druid.indexer.runner.javaOptsArray`| `JsonArray` | java opts for the task. |`-Xmx1g`|No|
|`druid.indexer.runner.labels`| `JsonObject` | Additional labels you want to add to peon pod |`{}`|No| |`druid.indexer.runner.labels`| `JsonObject` | Additional labels you want to add to peon pod |`{}`|No|

View File

@ -22,6 +22,8 @@ package org.apache.druid.k8s.overlord;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Optional; import com.google.common.base.Optional;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.SettableFuture;
import io.fabric8.kubernetes.api.model.Pod; import io.fabric8.kubernetes.api.model.Pod;
import io.fabric8.kubernetes.api.model.PodStatus; import io.fabric8.kubernetes.api.model.PodStatus;
import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.api.model.batch.v1.Job;
@ -90,6 +92,8 @@ public class KubernetesPeonLifecycle
private final KubernetesPeonClient kubernetesClient; private final KubernetesPeonClient kubernetesClient;
private final ObjectMapper mapper; private final ObjectMapper mapper;
private final TaskStateListener stateListener; private final TaskStateListener stateListener;
private final SettableFuture<Boolean> taskStartedSuccessfullyFuture;
@MonotonicNonNull @MonotonicNonNull
private LogWatch logWatch; private LogWatch logWatch;
@ -109,6 +113,7 @@ public class KubernetesPeonLifecycle
this.taskLogs = taskLogs; this.taskLogs = taskLogs;
this.mapper = mapper; this.mapper = mapper;
this.stateListener = stateListener; this.stateListener = stateListener;
this.taskStartedSuccessfullyFuture = SettableFuture.create();
} }
/** /**
@ -137,11 +142,13 @@ public class KubernetesPeonLifecycle
launchTimeout, launchTimeout,
TimeUnit.MILLISECONDS TimeUnit.MILLISECONDS
); );
return join(timeout); return join(timeout);
} }
catch (Exception e) { catch (Exception e) {
log.info("Failed to run task: %s", taskId.getOriginalTaskId()); log.info("Failed to run task: %s", taskId.getOriginalTaskId());
if (!taskStartedSuccessfullyFuture.isDone()) {
taskStartedSuccessfullyFuture.set(false);
}
throw e; throw e;
} }
finally { finally {
@ -179,7 +186,7 @@ public class KubernetesPeonLifecycle
{ {
try { try {
updateState(new State[]{State.NOT_STARTED, State.PENDING}, State.RUNNING); updateState(new State[]{State.NOT_STARTED, State.PENDING}, State.RUNNING);
taskStartedSuccessfullyFuture.set(true);
JobResponse jobResponse = kubernetesClient.waitForPeonJobCompletion( JobResponse jobResponse = kubernetesClient.waitForPeonJobCompletion(
taskId, taskId,
timeout, timeout,
@ -188,6 +195,12 @@ public class KubernetesPeonLifecycle
return getTaskStatus(jobResponse.getJobDuration()); return getTaskStatus(jobResponse.getJobDuration());
} }
catch (Exception e) {
if (!taskStartedSuccessfullyFuture.isDone()) {
taskStartedSuccessfullyFuture.set(false);
}
throw e;
}
finally { finally {
try { try {
saveLogs(); saveLogs();
@ -195,7 +208,6 @@ public class KubernetesPeonLifecycle
catch (Exception e) { catch (Exception e) {
log.warn(e, "Log processing failed for task [%s]", taskId); log.warn(e, "Log processing failed for task [%s]", taskId);
} }
stopTask(); stopTask();
} }
} }
@ -246,7 +258,10 @@ public class KubernetesPeonLifecycle
protected TaskLocation getTaskLocation() protected TaskLocation getTaskLocation()
{ {
if (State.PENDING.equals(state.get()) || State.NOT_STARTED.equals(state.get())) { if (State.PENDING.equals(state.get()) || State.NOT_STARTED.equals(state.get())) {
log.debug("Can't get task location for non-running job. [%s]", taskId.getOriginalTaskId()); /* This should not actually ever happen because KubernetesTaskRunner.start() should not return until all running tasks
have already gone into State.RUNNING, so getTaskLocation should not be called.
*/
log.warn("Can't get task location for non-running job. [%s]", taskId.getOriginalTaskId());
return TaskLocation.unknown(); return TaskLocation.unknown();
} }
@ -257,6 +272,10 @@ public class KubernetesPeonLifecycle
if (taskLocation == null) { if (taskLocation == null) {
Optional<Pod> maybePod = kubernetesClient.getPeonPod(taskId.getK8sJobName()); Optional<Pod> maybePod = kubernetesClient.getPeonPod(taskId.getK8sJobName());
if (!maybePod.isPresent()) { if (!maybePod.isPresent()) {
/* Arguably we should throw a exception here but leaving it as a warn log to prevent unexpected errors.
If there is strange behavior during overlord restarts the operator should look for this warn log.
*/
log.warn("Could not get task location from k8s for task [%s].", taskId);
return TaskLocation.unknown(); return TaskLocation.unknown();
} }
@ -264,6 +283,7 @@ public class KubernetesPeonLifecycle
PodStatus podStatus = pod.getStatus(); PodStatus podStatus = pod.getStatus();
if (podStatus == null || podStatus.getPodIP() == null) { if (podStatus == null || podStatus.getPodIP() == null) {
log.warn("Could not get task location from k8s for task [%s].", taskId);
return TaskLocation.unknown(); return TaskLocation.unknown();
} }
taskLocation = TaskLocation.create( taskLocation = TaskLocation.create(
@ -376,4 +396,17 @@ public class KubernetesPeonLifecycle
); );
stateListener.stateChanged(state.get(), taskId.getOriginalTaskId()); stateListener.stateChanged(state.get(), taskId.getOriginalTaskId());
} }
/**
* Retrieves the current {@link ListenableFuture} representing whether the task started successfully
*
* <p>This future can be used to track whether the task started successfully, with a boolean result
* indicating success (true) or failure (false) when the task starts.
*
* @return a {@link ListenableFuture} representing whether the task started successfully.
*/
protected ListenableFuture<Boolean> getTaskStartedSuccessfullyFuture()
{
return taskStartedSuccessfullyFuture;
}
} }

View File

@ -29,6 +29,7 @@ import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.MoreExecutors;
import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.api.model.batch.v1.Job;
import org.apache.druid.common.guava.FutureUtils;
import org.apache.druid.indexer.RunnerTaskState; import org.apache.druid.indexer.RunnerTaskState;
import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskLocation;
import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatus;
@ -55,12 +56,14 @@ import org.apache.druid.k8s.overlord.common.KubernetesPeonClient;
import org.apache.druid.k8s.overlord.taskadapter.TaskAdapter; import org.apache.druid.k8s.overlord.taskadapter.TaskAdapter;
import org.apache.druid.tasklogs.TaskLogStreamer; import org.apache.druid.tasklogs.TaskLogStreamer;
import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpMethod;
import org.joda.time.DateTime;
import org.joda.time.Duration; import org.joda.time.Duration;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.net.URL; import java.net.URL;
import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
@ -146,16 +149,28 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner
public ListenableFuture<TaskStatus> run(Task task) public ListenableFuture<TaskStatus> run(Task task)
{ {
synchronized (tasks) { synchronized (tasks) {
return tasks.computeIfAbsent(task.getId(), k -> new KubernetesWorkItem(task, exec.submit(() -> runTask(task)))) return tasks.computeIfAbsent(task.getId(), k -> new KubernetesWorkItem(
.getResult(); task,
exec.submit(() -> runTask(task)),
peonLifecycleFactory.build(
task,
this::emitTaskStateMetrics
)
)).getResult();
} }
} }
protected ListenableFuture<TaskStatus> joinAsync(Task task) protected KubernetesWorkItem joinAsync(Task task)
{ {
synchronized (tasks) { synchronized (tasks) {
return tasks.computeIfAbsent(task.getId(), k -> new KubernetesWorkItem(task, exec.submit(() -> joinTask(task)))) return tasks.computeIfAbsent(task.getId(), k -> new KubernetesWorkItem(
.getResult(); task,
exec.submit(() -> joinTask(task)),
peonLifecycleFactory.build(
task,
this::emitTaskStateMetrics
)
));
} }
} }
@ -173,10 +188,7 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner
protected TaskStatus doTask(Task task, boolean run) protected TaskStatus doTask(Task task, boolean run)
{ {
try { try {
KubernetesPeonLifecycle peonLifecycle = peonLifecycleFactory.build( KubernetesPeonLifecycle peonLifecycle;
task,
this::emitTaskStateMetrics
);
synchronized (tasks) { synchronized (tasks) {
KubernetesWorkItem workItem = tasks.get(task.getId()); KubernetesWorkItem workItem = tasks.get(task.getId());
@ -185,7 +197,7 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner
throw new ISE("Task [%s] has been shut down", task.getId()); throw new ISE("Task [%s] has been shut down", task.getId());
} }
workItem.setKubernetesPeonLifecycle(peonLifecycle); peonLifecycle = workItem.getPeonLifeycle();
} }
TaskStatus taskStatus; TaskStatus taskStatus;
@ -321,16 +333,53 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner
public void start() public void start()
{ {
log.info("Starting K8sTaskRunner..."); log.info("Starting K8sTaskRunner...");
// Load tasks from previously running jobs and wait for their statuses to be updated asynchronously. // Load tasks from previously running jobs and wait for their statuses to start running.
for (Job job : client.getPeonJobs()) { final List<ListenableFuture<Boolean>> taskStatusActiveList = new ArrayList<>();
final List<Job> peonJobs = client.getPeonJobs();
log.info("Locating [%,d] active tasks.", peonJobs.size());
for (Job job : peonJobs) {
try { try {
joinAsync(adapter.toTask(job)); KubernetesWorkItem kubernetesWorkItem = joinAsync(adapter.toTask(job));
taskStatusActiveList.add(kubernetesWorkItem.getPeonLifeycle().getTaskStartedSuccessfullyFuture());
} }
catch (IOException e) { catch (IOException e) {
log.error(e, "Error deserializing task from job [%s]", job.getMetadata().getName()); log.error(e, "Error deserializing task from job [%s]", job.getMetadata().getName());
} }
} }
log.info("Loaded %,d tasks from previous run", tasks.size());
try {
final DateTime nowUtc = DateTimes.nowUtc();
final long timeoutMs = nowUtc.plus(config.getTaskJoinTimeout()).getMillis() - nowUtc.getMillis();
if (timeoutMs > 0) {
FutureUtils.coalesce(taskStatusActiveList).get(timeoutMs, TimeUnit.MILLISECONDS);
}
log.info("Located [%,d] active tasks.", taskStatusActiveList.size());
}
catch (Exception e) {
final long numInitialized =
tasks.values()
.stream()
.filter(item -> {
if (item.getPeonLifeycle().getTaskStartedSuccessfullyFuture().isDone()) {
try {
return item.getPeonLifeycle().getTaskStartedSuccessfullyFuture().get();
}
catch (InterruptedException | ExecutionException ex) {
return false;
}
} else {
return false;
}
}).count();
log.warn(
e,
"Located [%,d] out of [%,d] active tasks (timeout = %s). Locating others asynchronously.",
numInitialized,
taskStatusActiveList.size(),
config.getTaskJoinTimeout()
);
}
cleanupExecutor.scheduleAtFixedRate( cleanupExecutor.scheduleAtFixedRate(
() -> () ->
@ -342,7 +391,7 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner
config.getTaskCleanupInterval().toStandardDuration().getMillis(), config.getTaskCleanupInterval().toStandardDuration().getMillis(),
TimeUnit.MILLISECONDS TimeUnit.MILLISECONDS
); );
log.debug("Started cleanup executor for jobs older than %s...", config.getTaskCleanupDelay()); log.info("Started cleanup executor for jobs older than %s...", config.getTaskCleanupDelay());
} }
@Override @Override

View File

@ -85,6 +85,12 @@ public class KubernetesTaskRunnerConfig
// interval for k8s job cleanup to run // interval for k8s job cleanup to run
private Period taskCleanupInterval = new Period("PT10m"); private Period taskCleanupInterval = new Period("PT10m");
@JsonProperty
@NotNull
// how long to wait to join peon k8s jobs on startup
private Period taskJoinTimeout = new Period("PT1M");
@JsonProperty @JsonProperty
@NotNull @NotNull
// how long to wait for the peon k8s job to launch // how long to wait for the peon k8s job to launch
@ -140,7 +146,8 @@ public class KubernetesTaskRunnerConfig
int cpuCoreInMicro, int cpuCoreInMicro,
Map<String, String> labels, Map<String, String> labels,
Map<String, String> annotations, Map<String, String> annotations,
Integer capacity Integer capacity,
Period taskJoinTimeout
) )
{ {
this.namespace = namespace; this.namespace = namespace;
@ -181,6 +188,10 @@ public class KubernetesTaskRunnerConfig
k8sjobLaunchTimeout, k8sjobLaunchTimeout,
this.k8sjobLaunchTimeout this.k8sjobLaunchTimeout
); );
this.taskJoinTimeout = ObjectUtils.defaultIfNull(
taskJoinTimeout,
this.taskJoinTimeout
);
this.peonMonitors = ObjectUtils.defaultIfNull( this.peonMonitors = ObjectUtils.defaultIfNull(
peonMonitors, peonMonitors,
this.peonMonitors this.peonMonitors
@ -247,6 +258,11 @@ public class KubernetesTaskRunnerConfig
{ {
return maxTaskDuration; return maxTaskDuration;
} }
public Period getTaskJoinTimeout()
{
return taskJoinTimeout;
}
public Period getTaskCleanupDelay() public Period getTaskCleanupDelay()
{ {
@ -317,6 +333,7 @@ public class KubernetesTaskRunnerConfig
private Map<String, String> labels; private Map<String, String> labels;
private Map<String, String> annotations; private Map<String, String> annotations;
private Integer capacity; private Integer capacity;
private Period taskJoinTimeout;
public Builder() public Builder()
{ {
@ -425,6 +442,12 @@ public class KubernetesTaskRunnerConfig
return this; return this;
} }
public Builder withTaskJoinTimeout(Period taskJoinTimeout)
{
this.taskJoinTimeout = taskJoinTimeout;
return this;
}
public KubernetesTaskRunnerConfig build() public KubernetesTaskRunnerConfig build()
{ {
return new KubernetesTaskRunnerConfig( return new KubernetesTaskRunnerConfig(
@ -444,7 +467,8 @@ public class KubernetesTaskRunnerConfig
this.cpuCoreInMicro, this.cpuCoreInMicro,
this.labels, this.labels,
this.annotations, this.annotations,
this.capacity this.capacity,
this.taskJoinTimeout
); );
} }
} }

View File

@ -20,7 +20,6 @@
package org.apache.druid.k8s.overlord; package org.apache.druid.k8s.overlord;
import com.google.common.base.Optional; import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListenableFuture;
import org.apache.druid.indexer.RunnerTaskState; import org.apache.druid.indexer.RunnerTaskState;
import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskLocation;
@ -34,27 +33,19 @@ import java.io.InputStream;
public class KubernetesWorkItem extends TaskRunnerWorkItem public class KubernetesWorkItem extends TaskRunnerWorkItem
{ {
private final Task task; private final Task task;
private KubernetesPeonLifecycle kubernetesPeonLifecycle = null; private final KubernetesPeonLifecycle kubernetesPeonLifecycle;
public KubernetesWorkItem(Task task, ListenableFuture<TaskStatus> statusFuture) public KubernetesWorkItem(Task task, ListenableFuture<TaskStatus> statusFuture, KubernetesPeonLifecycle kubernetesPeonLifecycle)
{ {
super(task.getId(), statusFuture); super(task.getId(), statusFuture);
this.task = task; this.task = task;
}
protected synchronized void setKubernetesPeonLifecycle(KubernetesPeonLifecycle kubernetesPeonLifecycle)
{
Preconditions.checkState(this.kubernetesPeonLifecycle == null);
this.kubernetesPeonLifecycle = kubernetesPeonLifecycle; this.kubernetesPeonLifecycle = kubernetesPeonLifecycle;
} }
protected synchronized void shutdown() protected synchronized void shutdown()
{ {
this.kubernetesPeonLifecycle.startWatchingLogs();
if (this.kubernetesPeonLifecycle != null) { this.kubernetesPeonLifecycle.shutdown();
this.kubernetesPeonLifecycle.startWatchingLogs();
this.kubernetesPeonLifecycle.shutdown();
}
} }
protected boolean isPending() protected boolean isPending()
@ -88,18 +79,12 @@ public class KubernetesWorkItem extends TaskRunnerWorkItem
protected Optional<InputStream> streamTaskLogs() protected Optional<InputStream> streamTaskLogs()
{ {
if (kubernetesPeonLifecycle == null) {
return Optional.absent();
}
return kubernetesPeonLifecycle.streamLogs(); return kubernetesPeonLifecycle.streamLogs();
} }
@Override @Override
public TaskLocation getLocation() public TaskLocation getLocation()
{ {
if (kubernetesPeonLifecycle == null) {
return TaskLocation.unknown();
}
return kubernetesPeonLifecycle.getTaskLocation(); return kubernetesPeonLifecycle.getTaskLocation();
} }
@ -119,4 +104,9 @@ public class KubernetesWorkItem extends TaskRunnerWorkItem
{ {
return task; return task;
} }
protected KubernetesPeonLifecycle getPeonLifeycle()
{
return this.kubernetesPeonLifecycle;
}
} }

View File

@ -52,6 +52,7 @@ import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.lang.reflect.Field; import java.lang.reflect.Field;
import java.nio.charset.StandardCharsets; import java.nio.charset.StandardCharsets;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReference;
@ -264,6 +265,53 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
Assert.assertEquals(KubernetesPeonLifecycle.State.STOPPED, peonLifecycle.getState()); Assert.assertEquals(KubernetesPeonLifecycle.State.STOPPED, peonLifecycle.getState());
} }
@Test
public void test_run_whenExceptionRaised_setsStartStatusFutureToFalse() throws ExecutionException, InterruptedException
{
KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(
task,
kubernetesClient,
taskLogs,
mapper,
stateListener
)
{
@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.eq(task),
EasyMock.anyLong(),
EasyMock.eq(TimeUnit.MILLISECONDS)
)).andReturn(null);
Assert.assertEquals(KubernetesPeonLifecycle.State.NOT_STARTED, peonLifecycle.getState());
stateListener.stateChanged(KubernetesPeonLifecycle.State.PENDING, ID);
EasyMock.expectLastCall().once();
stateListener.stateChanged(KubernetesPeonLifecycle.State.STOPPED, ID);
EasyMock.expectLastCall().once();
replayAll();
Assert.assertThrows(
Exception.class,
() -> peonLifecycle.run(job, 0L, 0L, false)
);
verifyAll();
Assert.assertEquals(KubernetesPeonLifecycle.State.STOPPED, peonLifecycle.getState());
Assert.assertTrue(peonLifecycle.getTaskStartedSuccessfullyFuture().isDone());
Assert.assertFalse(peonLifecycle.getTaskStartedSuccessfullyFuture().get());
}
@Test @Test
public void test_join_withoutJob_returnsFailedTaskStatus() throws IOException public void test_join_withoutJob_returnsFailedTaskStatus() throws IOException
{ {
@ -313,6 +361,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
stateListener stateListener
); );
Assert.assertFalse(peonLifecycle.getTaskStartedSuccessfullyFuture().isDone());
Job job = new JobBuilder() Job job = new JobBuilder()
.withNewMetadata() .withNewMetadata()
.withName(ID) .withName(ID)
@ -347,7 +396,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
TaskStatus taskStatus = peonLifecycle.join(0L); TaskStatus taskStatus = peonLifecycle.join(0L);
verifyAll(); verifyAll();
Assert.assertTrue(peonLifecycle.getTaskStartedSuccessfullyFuture().isDone());
Assert.assertEquals(SUCCESS.withDuration(58000), taskStatus); Assert.assertEquals(SUCCESS.withDuration(58000), taskStatus);
Assert.assertEquals(KubernetesPeonLifecycle.State.STOPPED, peonLifecycle.getState()); Assert.assertEquals(KubernetesPeonLifecycle.State.STOPPED, peonLifecycle.getState());
} }

View File

@ -23,6 +23,7 @@ import com.google.common.base.Optional;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.SettableFuture;
import io.fabric8.kubernetes.api.model.batch.v1.Job; 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.JobBuilder;
import org.apache.commons.io.IOUtils; import org.apache.commons.io.IOUtils;
@ -103,6 +104,8 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
@Test @Test
public void test_start_withExistingJobs() throws IOException public void test_start_withExistingJobs() throws IOException
{ {
SettableFuture<Boolean> settableFuture = SettableFuture.create();
settableFuture.set(true);
KubernetesTaskRunner runner = new KubernetesTaskRunner( KubernetesTaskRunner runner = new KubernetesTaskRunner(
taskAdapter, taskAdapter,
config, config,
@ -113,15 +116,16 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
) )
{ {
@Override @Override
protected ListenableFuture<TaskStatus> joinAsync(Task task) protected KubernetesWorkItem joinAsync(Task task)
{ {
return tasks.computeIfAbsent( return tasks.computeIfAbsent(
task.getId(), task.getId(),
k -> new KubernetesWorkItem( k -> new KubernetesWorkItem(
task, task,
Futures.immediateFuture(TaskStatus.success(task.getId())) Futures.immediateFuture(TaskStatus.success(task.getId())),
kubernetesPeonLifecycle
) )
).getResult(); );
} }
}; };
@ -133,6 +137,67 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
EasyMock.expect(peonClient.getPeonJobs()).andReturn(ImmutableList.of(job)); EasyMock.expect(peonClient.getPeonJobs()).andReturn(ImmutableList.of(job));
EasyMock.expect(taskAdapter.toTask(job)).andReturn(task); EasyMock.expect(taskAdapter.toTask(job)).andReturn(task);
EasyMock.expect(kubernetesPeonLifecycle.getTaskStartedSuccessfullyFuture()).andReturn(
settableFuture
);
replayAll();
runner.start();
verifyAll();
Assert.assertNotNull(runner.tasks);
Assert.assertEquals(1, runner.tasks.size());
}
@Test
public void test_start_withExistingJobs_oneJobFails() throws IOException
{
SettableFuture<Boolean> settableFuture = SettableFuture.create();
settableFuture.set(true);
KubernetesTaskRunner runner = new KubernetesTaskRunner(
taskAdapter,
config,
peonClient,
httpClient,
new TestPeonLifecycleFactory(kubernetesPeonLifecycle),
emitter
)
{
@Override
protected KubernetesWorkItem joinAsync(Task task)
{
return tasks.computeIfAbsent(
task.getId(),
k -> new KubernetesWorkItem(
task,
Futures.immediateFuture(TaskStatus.success(task.getId())),
kubernetesPeonLifecycle
)
);
}
};
Job job = new JobBuilder()
.withNewMetadata()
.withName(ID)
.endMetadata()
.build();
Job job2 = new JobBuilder()
.withNewMetadata()
.withName("id2")
.endMetadata()
.build();
EasyMock.expect(peonClient.getPeonJobs()).andReturn(ImmutableList.of(job, job2));
EasyMock.expect(taskAdapter.toTask(job)).andReturn(task);
EasyMock.expect(taskAdapter.toTask(job2)).andThrow(new IOException("deserialization exception"));
EasyMock.expect(kubernetesPeonLifecycle.getTaskStartedSuccessfullyFuture()).andReturn(
settableFuture
);
replayAll(); replayAll();
@ -157,10 +222,9 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
) )
{ {
@Override @Override
protected ListenableFuture<TaskStatus> joinAsync(Task task) protected KubernetesWorkItem joinAsync(Task task)
{ {
return tasks.computeIfAbsent(task.getId(), k -> new KubernetesWorkItem(task, null)) return tasks.computeIfAbsent(task.getId(), k -> new KubernetesWorkItem(task, null, kubernetesPeonLifecycle));
.getResult();
} }
}; };
@ -193,7 +257,7 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
@Test @Test
public void test_streamTaskLog_withExistingTask() throws IOException public void test_streamTaskLog_withExistingTask() throws IOException
{ {
KubernetesWorkItem workItem = new KubernetesWorkItem(task, null) KubernetesWorkItem workItem = new KubernetesWorkItem(task, null, kubernetesPeonLifecycle)
{ {
@Override @Override
protected Optional<InputStream> streamTaskLogs() protected Optional<InputStream> streamTaskLogs()
@ -241,7 +305,7 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
@Test @Test
public void test_run_withExistingTask_returnsExistingWorkItem() public void test_run_withExistingTask_returnsExistingWorkItem()
{ {
KubernetesWorkItem workItem = new KubernetesWorkItem(task, null); KubernetesWorkItem workItem = new KubernetesWorkItem(task, null, kubernetesPeonLifecycle);
runner.tasks.put(task.getId(), workItem); runner.tasks.put(task.getId(), workItem);
ListenableFuture<TaskStatus> future = runner.run(task); ListenableFuture<TaskStatus> future = runner.run(task);
@ -286,8 +350,8 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
replayAll(); replayAll();
ListenableFuture<TaskStatus> future = runner.joinAsync(task); KubernetesWorkItem workItem = runner.joinAsync(task);
Assert.assertEquals(taskStatus, future.get()); Assert.assertEquals(taskStatus, workItem.getResult().get());
verifyAll(); verifyAll();
} }
@ -295,7 +359,7 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
@Test @Test
public void test_join_withExistingTask_returnsExistingWorkItem() public void test_join_withExistingTask_returnsExistingWorkItem()
{ {
KubernetesWorkItem workItem = new KubernetesWorkItem(task, null); KubernetesWorkItem workItem = new KubernetesWorkItem(task, null, kubernetesPeonLifecycle);
runner.tasks.put(task.getId(), workItem); runner.tasks.put(task.getId(), workItem);
ListenableFuture<TaskStatus> future = runner.run(task); ListenableFuture<TaskStatus> future = runner.run(task);
@ -310,9 +374,9 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
replayAll(); replayAll();
ListenableFuture<TaskStatus> future = runner.joinAsync(task); KubernetesWorkItem workItem = runner.joinAsync(task);
Exception e = Assert.assertThrows(ExecutionException.class, future::get); Exception e = Assert.assertThrows(ExecutionException.class, () -> workItem.getResult().get());
Assert.assertTrue(e.getCause() instanceof RuntimeException); Assert.assertTrue(e.getCause() instanceof RuntimeException);
verifyAll(); verifyAll();
@ -331,7 +395,7 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
@Test @Test
public void test_shutdown_withExistingTask_removesTaskFromMap() public void test_shutdown_withExistingTask_removesTaskFromMap()
{ {
KubernetesWorkItem workItem = new KubernetesWorkItem(task, statusFuture) { KubernetesWorkItem workItem = new KubernetesWorkItem(task, statusFuture, kubernetesPeonLifecycle) {
@Override @Override
protected synchronized void shutdown() protected synchronized void shutdown()
{ {
@ -348,7 +412,7 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
@Test @Test
public void test_shutdown_withExistingTask_futureIncomplete_removesTaskFromMap() public void test_shutdown_withExistingTask_futureIncomplete_removesTaskFromMap()
{ {
KubernetesWorkItem workItem = new KubernetesWorkItem(task, statusFuture) { KubernetesWorkItem workItem = new KubernetesWorkItem(task, statusFuture, kubernetesPeonLifecycle) {
@Override @Override
protected synchronized void shutdown() protected synchronized void shutdown()
{ {
@ -385,7 +449,7 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
@Test @Test
public void test_getKnownTasks() public void test_getKnownTasks()
{ {
KubernetesWorkItem workItem = new KubernetesWorkItem(task, null); KubernetesWorkItem workItem = new KubernetesWorkItem(task, null, kubernetesPeonLifecycle);
runner.tasks.put(task.getId(), workItem); runner.tasks.put(task.getId(), workItem);
@ -399,7 +463,7 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
public void test_getRunningTasks() public void test_getRunningTasks()
{ {
Task pendingTask = K8sTestUtils.createTask("pending-id", 0); Task pendingTask = K8sTestUtils.createTask("pending-id", 0);
KubernetesWorkItem pendingWorkItem = new KubernetesWorkItem(pendingTask, null) { KubernetesWorkItem pendingWorkItem = new KubernetesWorkItem(pendingTask, null, kubernetesPeonLifecycle) {
@Override @Override
protected RunnerTaskState getRunnerTaskState() protected RunnerTaskState getRunnerTaskState()
{ {
@ -409,7 +473,7 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
runner.tasks.put(pendingTask.getId(), pendingWorkItem); runner.tasks.put(pendingTask.getId(), pendingWorkItem);
Task runningTask = K8sTestUtils.createTask("running-id", 0); Task runningTask = K8sTestUtils.createTask("running-id", 0);
KubernetesWorkItem runningWorkItem = new KubernetesWorkItem(runningTask, null) { KubernetesWorkItem runningWorkItem = new KubernetesWorkItem(runningTask, null, kubernetesPeonLifecycle) {
@Override @Override
protected RunnerTaskState getRunnerTaskState() protected RunnerTaskState getRunnerTaskState()
{ {
@ -428,7 +492,7 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
public void test_getPendingTasks() public void test_getPendingTasks()
{ {
Task pendingTask = K8sTestUtils.createTask("pending-id", 0); Task pendingTask = K8sTestUtils.createTask("pending-id", 0);
KubernetesWorkItem pendingWorkItem = new KubernetesWorkItem(pendingTask, null) { KubernetesWorkItem pendingWorkItem = new KubernetesWorkItem(pendingTask, null, kubernetesPeonLifecycle) {
@Override @Override
protected RunnerTaskState getRunnerTaskState() protected RunnerTaskState getRunnerTaskState()
{ {
@ -438,7 +502,7 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
runner.tasks.put(pendingTask.getId(), pendingWorkItem); runner.tasks.put(pendingTask.getId(), pendingWorkItem);
Task runningTask = K8sTestUtils.createTask("running-id", 0); Task runningTask = K8sTestUtils.createTask("running-id", 0);
KubernetesWorkItem runningWorkItem = new KubernetesWorkItem(runningTask, null) { KubernetesWorkItem runningWorkItem = new KubernetesWorkItem(runningTask, null, kubernetesPeonLifecycle) {
@Override @Override
protected RunnerTaskState getRunnerTaskState() protected RunnerTaskState getRunnerTaskState()
{ {
@ -462,7 +526,7 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
@Test @Test
public void test_getRunnerTaskState_withExistingTask() public void test_getRunnerTaskState_withExistingTask()
{ {
KubernetesWorkItem workItem = new KubernetesWorkItem(task, null) { KubernetesWorkItem workItem = new KubernetesWorkItem(task, null, kubernetesPeonLifecycle) {
@Override @Override
protected RunnerTaskState getRunnerTaskState() protected RunnerTaskState getRunnerTaskState()
{ {
@ -477,7 +541,7 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
@Test @Test
public void test_streamTaskReports_withExistingTask() throws Exception public void test_streamTaskReports_withExistingTask() throws Exception
{ {
KubernetesWorkItem workItem = new KubernetesWorkItem(task, null) { KubernetesWorkItem workItem = new KubernetesWorkItem(task, null, kubernetesPeonLifecycle) {
@Override @Override
public TaskLocation getLocation() public TaskLocation getLocation()
{ {
@ -512,7 +576,7 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
@Test @Test
public void test_streamTaskReports_withUnknownTaskLocation_returnsEmptyOptional() throws Exception public void test_streamTaskReports_withUnknownTaskLocation_returnsEmptyOptional() throws Exception
{ {
KubernetesWorkItem workItem = new KubernetesWorkItem(task, null) { KubernetesWorkItem workItem = new KubernetesWorkItem(task, null, kubernetesPeonLifecycle) {
@Override @Override
public TaskLocation getLocation() public TaskLocation getLocation()
{ {
@ -529,7 +593,7 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
@Test @Test
public void test_streamTaskReports_whenInterruptedExceptionThrown_throwsRuntimeException() public void test_streamTaskReports_whenInterruptedExceptionThrown_throwsRuntimeException()
{ {
KubernetesWorkItem workItem = new KubernetesWorkItem(task, null) { KubernetesWorkItem workItem = new KubernetesWorkItem(task, null, kubernetesPeonLifecycle) {
@Override @Override
public TaskLocation getLocation() public TaskLocation getLocation()
{ {
@ -593,7 +657,7 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
@Test @Test
public void test_streamTaskReports_whenExecutionExceptionThrown_throwsRuntimeException() public void test_streamTaskReports_whenExecutionExceptionThrown_throwsRuntimeException()
{ {
KubernetesWorkItem workItem = new KubernetesWorkItem(task, null) { KubernetesWorkItem workItem = new KubernetesWorkItem(task, null, kubernetesPeonLifecycle) {
@Override @Override
public TaskLocation getLocation() public TaskLocation getLocation()
{ {
@ -618,7 +682,7 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
@Test @Test
public void test_metricsReported_whenTaskStateChange() public void test_metricsReported_whenTaskStateChange()
{ {
KubernetesWorkItem workItem = new KubernetesWorkItem(task, null) { KubernetesWorkItem workItem = new KubernetesWorkItem(task, null, kubernetesPeonLifecycle) {
@Override @Override
public TaskLocation getLocation() public TaskLocation getLocation()
{ {
@ -640,7 +704,7 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
@Test @Test
public void test_getTaskLocation_withExistingTask() public void test_getTaskLocation_withExistingTask()
{ {
KubernetesWorkItem workItem = new KubernetesWorkItem(task, null) { KubernetesWorkItem workItem = new KubernetesWorkItem(task, null, kubernetesPeonLifecycle) {
@Override @Override
public TaskLocation getLocation() public TaskLocation getLocation()
{ {
@ -657,7 +721,7 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
@Test @Test
public void test_getTaskLocation_throws() public void test_getTaskLocation_throws()
{ {
KubernetesWorkItem workItem = new KubernetesWorkItem(task, null) KubernetesWorkItem workItem = new KubernetesWorkItem(task, null, kubernetesPeonLifecycle)
{ {
@Override @Override
public TaskLocation getLocation() public TaskLocation getLocation()
@ -689,7 +753,7 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
public void test_getUsedCapacity() public void test_getUsedCapacity()
{ {
Assert.assertEquals(0, runner.getUsedCapacity()); Assert.assertEquals(0, runner.getUsedCapacity());
KubernetesWorkItem workItem = new KubernetesWorkItem(task, null); KubernetesWorkItem workItem = new KubernetesWorkItem(task, null, kubernetesPeonLifecycle);
runner.tasks.put(task.getId(), workItem); runner.tasks.put(task.getId(), workItem);
Assert.assertEquals(1, runner.getUsedCapacity()); Assert.assertEquals(1, runner.getUsedCapacity());
runner.tasks.remove(task.getId()); runner.tasks.remove(task.getId());

View File

@ -45,36 +45,6 @@ public class KubernetesWorkItemTest extends EasyMockSupport
public void setup() public void setup()
{ {
task = NoopTask.create(); task = NoopTask.create();
workItem = new KubernetesWorkItem(task, null);
}
@Test
public void test_setKubernetesPeonLifecycleTwice_throwsIllegalStateException()
{
workItem.setKubernetesPeonLifecycle(new KubernetesPeonLifecycle(
task,
null,
null,
null,
null
));
Assert.assertThrows(
IllegalStateException.class,
() -> workItem.setKubernetesPeonLifecycle(new KubernetesPeonLifecycle(
task,
null,
null,
null,
null
))
);
}
@Test
public void test_shutdown_withoutKubernetesPeonLifecycle()
{
workItem.shutdown();
} }
@Test @Test
@ -86,7 +56,11 @@ public class KubernetesWorkItemTest extends EasyMockSupport
EasyMock.expectLastCall(); EasyMock.expectLastCall();
replayAll(); replayAll();
workItem.setKubernetesPeonLifecycle(kubernetesPeonLifecycle); workItem = new KubernetesWorkItem(
task,
null,
kubernetesPeonLifecycle
);
workItem.shutdown(); workItem.shutdown();
verifyAll(); verifyAll();
@ -95,7 +69,7 @@ public class KubernetesWorkItemTest extends EasyMockSupport
@Test @Test
public void test_isPending_withTaskStateWaiting_returnsFalse() public void test_isPending_withTaskStateWaiting_returnsFalse()
{ {
workItem = new KubernetesWorkItem(task, null) { workItem = new KubernetesWorkItem(task, null, kubernetesPeonLifecycle) {
@Override @Override
protected RunnerTaskState getRunnerTaskState() protected RunnerTaskState getRunnerTaskState()
{ {
@ -108,7 +82,7 @@ public class KubernetesWorkItemTest extends EasyMockSupport
@Test @Test
public void test_isPending_withTaskStatePending_returnsTrue() public void test_isPending_withTaskStatePending_returnsTrue()
{ {
workItem = new KubernetesWorkItem(task, null) { workItem = new KubernetesWorkItem(task, null, kubernetesPeonLifecycle) {
@Override @Override
protected RunnerTaskState getRunnerTaskState() protected RunnerTaskState getRunnerTaskState()
{ {
@ -121,7 +95,7 @@ public class KubernetesWorkItemTest extends EasyMockSupport
@Test @Test
public void test_isRunning_withTaskStateWaiting_returnsFalse() public void test_isRunning_withTaskStateWaiting_returnsFalse()
{ {
workItem = new KubernetesWorkItem(task, null) { workItem = new KubernetesWorkItem(task, null, kubernetesPeonLifecycle) {
@Override @Override
protected RunnerTaskState getRunnerTaskState() protected RunnerTaskState getRunnerTaskState()
{ {
@ -134,7 +108,7 @@ public class KubernetesWorkItemTest extends EasyMockSupport
@Test @Test
public void test_isRunning_withTaskStatePending_returnsTrue() public void test_isRunning_withTaskStatePending_returnsTrue()
{ {
workItem = new KubernetesWorkItem(task, null) { workItem = new KubernetesWorkItem(task, null, kubernetesPeonLifecycle) {
@Override @Override
protected RunnerTaskState getRunnerTaskState() protected RunnerTaskState getRunnerTaskState()
{ {
@ -144,22 +118,17 @@ public class KubernetesWorkItemTest extends EasyMockSupport
Assert.assertTrue(workItem.isRunning()); Assert.assertTrue(workItem.isRunning());
} }
@Test
public void test_getRunnerTaskState_withoutKubernetesPeonLifecycle_returnsPending()
{
Assert.assertEquals(RunnerTaskState.PENDING, workItem.getRunnerTaskState());
}
@Test @Test
public void test_getRunnerTaskState_withKubernetesPeonLifecycle_returnsPending() public void test_getRunnerTaskState_withKubernetesPeonLifecycle_returnsPending()
{ {
workItem.setKubernetesPeonLifecycle(new KubernetesPeonLifecycle( KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(
task, task,
null, null,
null, null,
null, null,
null null
)); );
workItem = new KubernetesWorkItem(task, null, peonLifecycle);
Assert.assertEquals(RunnerTaskState.PENDING, workItem.getRunnerTaskState()); Assert.assertEquals(RunnerTaskState.PENDING, workItem.getRunnerTaskState());
} }
@ -181,7 +150,7 @@ public class KubernetesWorkItemTest extends EasyMockSupport
} }
}; };
workItem.setKubernetesPeonLifecycle(peonLifecycle); workItem = new KubernetesWorkItem(task, null, peonLifecycle);
Assert.assertEquals(RunnerTaskState.PENDING, workItem.getRunnerTaskState()); Assert.assertEquals(RunnerTaskState.PENDING, workItem.getRunnerTaskState());
} }
@ -203,7 +172,7 @@ public class KubernetesWorkItemTest extends EasyMockSupport
} }
}; };
workItem.setKubernetesPeonLifecycle(peonLifecycle); workItem = new KubernetesWorkItem(task, null, peonLifecycle);
Assert.assertEquals(RunnerTaskState.RUNNING, workItem.getRunnerTaskState()); Assert.assertEquals(RunnerTaskState.RUNNING, workItem.getRunnerTaskState());
} }
@ -225,46 +194,36 @@ public class KubernetesWorkItemTest extends EasyMockSupport
} }
}; };
workItem.setKubernetesPeonLifecycle(peonLifecycle); workItem = new KubernetesWorkItem(task, null, peonLifecycle);
Assert.assertEquals(RunnerTaskState.NONE, workItem.getRunnerTaskState()); Assert.assertEquals(RunnerTaskState.NONE, workItem.getRunnerTaskState());
} }
@Test
public void test_streamTaskLogs_withoutKubernetesPeonLifecycle()
{
Assert.assertFalse(workItem.streamTaskLogs().isPresent());
}
@Test @Test
public void test_streamTaskLogs_withKubernetesPeonLifecycle() public void test_streamTaskLogs_withKubernetesPeonLifecycle()
{ {
workItem.setKubernetesPeonLifecycle(new KubernetesPeonLifecycle( KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(
task, task,
null, null,
null, null,
null, null,
null null
)); );
workItem = new KubernetesWorkItem(task, null, peonLifecycle);
Assert.assertFalse(workItem.streamTaskLogs().isPresent()); Assert.assertFalse(workItem.streamTaskLogs().isPresent());
} }
@Test
public void test_getLocation_withoutKubernetesPeonLifecycle()
{
Assert.assertEquals(TaskLocation.unknown(), workItem.getLocation());
}
@Test @Test
public void test_getLocation_withKubernetesPeonLifecycle() public void test_getLocation_withKubernetesPeonLifecycle()
{ {
workItem.setKubernetesPeonLifecycle(new KubernetesPeonLifecycle( KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(
task, task,
null, null,
null, null,
null, null,
null null
)); );
workItem = new KubernetesWorkItem(task, null, peonLifecycle);
Assert.assertEquals(TaskLocation.unknown(), workItem.getLocation()); Assert.assertEquals(TaskLocation.unknown(), workItem.getLocation());
} }
@ -272,18 +231,28 @@ public class KubernetesWorkItemTest extends EasyMockSupport
@Test @Test
public void test_getTaskType() public void test_getTaskType()
{ {
workItem = new KubernetesWorkItem(task, null, kubernetesPeonLifecycle);
Assert.assertEquals(task.getType(), workItem.getTaskType()); Assert.assertEquals(task.getType(), workItem.getTaskType());
} }
@Test @Test
public void test_getDataSource() public void test_getDataSource()
{ {
workItem = new KubernetesWorkItem(task, null, kubernetesPeonLifecycle);
Assert.assertEquals(task.getDataSource(), workItem.getDataSource()); Assert.assertEquals(task.getDataSource(), workItem.getDataSource());
} }
@Test @Test
public void test_getTask() public void test_getTask()
{ {
workItem = new KubernetesWorkItem(task, null, kubernetesPeonLifecycle);
Assert.assertEquals(task, workItem.getTask()); Assert.assertEquals(task, workItem.getTask());
} }
@Test
public void test_peonLifeycle()
{
workItem = new KubernetesWorkItem(task, null, kubernetesPeonLifecycle);
Assert.assertEquals(kubernetesPeonLifecycle, workItem.getPeonLifeycle());
}
} }