Revert "Separate task lifecycle from kubernetes/location lifecycle (#15133)" (#15346)

This reverts commit dc0b163e192545c802b7fe2b3271e035cc1e70ff.
This commit is contained in:
George Shiqi Wu 2023-11-08 13:12:30 -05:00 committed by GitHub
parent b7d7f84bce
commit 130bfbfc6d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
34 changed files with 325 additions and 326 deletions

View File

@ -285,4 +285,10 @@ public class KubernetesAndWorkerTaskRunner implements TaskLogStreamer, WorkerTas
{ {
kubernetesTaskRunner.updateStatus(task, status); kubernetesTaskRunner.updateStatus(task, status);
} }
@Override
public void updateLocation(Task task, TaskLocation location)
{
kubernetesTaskRunner.updateLocation(task, location);
}
} }

View File

@ -31,9 +31,6 @@ import org.apache.commons.io.IOUtils;
import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskLocation;
import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.TaskRunnerListener;
import org.apache.druid.indexing.overlord.TaskRunnerUtils;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.k8s.overlord.common.DruidK8sConstants; import org.apache.druid.k8s.overlord.common.DruidK8sConstants;
@ -50,8 +47,6 @@ import java.nio.charset.StandardCharsets;
import java.nio.file.Files; import java.nio.file.Files;
import java.nio.file.Path; import java.nio.file.Path;
import java.util.Arrays; import java.util.Arrays;
import java.util.List;
import java.util.concurrent.Executor;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReference;
@ -94,8 +89,6 @@ 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 List<Pair<TaskRunnerListener, Executor>> listeners;
@MonotonicNonNull @MonotonicNonNull
private LogWatch logWatch; private LogWatch logWatch;
@ -106,8 +99,7 @@ public class KubernetesPeonLifecycle
KubernetesPeonClient kubernetesClient, KubernetesPeonClient kubernetesClient,
TaskLogs taskLogs, TaskLogs taskLogs,
ObjectMapper mapper, ObjectMapper mapper,
TaskStateListener stateListener, TaskStateListener stateListener
List<Pair<TaskRunnerListener, Executor>> listeners
) )
{ {
this.taskId = new K8sTaskId(task); this.taskId = new K8sTaskId(task);
@ -116,7 +108,6 @@ public class KubernetesPeonLifecycle
this.taskLogs = taskLogs; this.taskLogs = taskLogs;
this.mapper = mapper; this.mapper = mapper;
this.stateListener = stateListener; this.stateListener = stateListener;
this.listeners = listeners;
} }
/** /**
@ -187,11 +178,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);
TaskRunnerUtils.notifyLocationChanged(
listeners,
task.getId(),
getTaskLocation()
);
JobResponse jobResponse = kubernetesClient.waitForPeonJobCompletion( JobResponse jobResponse = kubernetesClient.waitForPeonJobCompletion(
taskId, taskId,
timeout, timeout,
@ -203,16 +190,14 @@ public class KubernetesPeonLifecycle
finally { finally {
try { try {
saveLogs(); saveLogs();
shutdown();
} }
catch (Exception e) { catch (Exception e) {
log.warn(e, "Cleanup failed for task [%s]", taskId); log.warn(e, "Log processing failed for task [%s]", taskId);
} }
finally {
stopTask(); stopTask();
} }
} }
}
/** /**
* Shutdown Kubernetes job and associated pods * Shutdown Kubernetes job and associated pods

View File

@ -21,14 +21,9 @@ package org.apache.druid.k8s.overlord;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.TaskRunnerListener;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; import org.apache.druid.k8s.overlord.common.KubernetesPeonClient;
import org.apache.druid.tasklogs.TaskLogs; import org.apache.druid.tasklogs.TaskLogs;
import java.util.List;
import java.util.concurrent.Executor;
public class KubernetesPeonLifecycleFactory implements PeonLifecycleFactory public class KubernetesPeonLifecycleFactory implements PeonLifecycleFactory
{ {
private final KubernetesPeonClient client; private final KubernetesPeonClient client;
@ -47,15 +42,14 @@ public class KubernetesPeonLifecycleFactory implements PeonLifecycleFactory
} }
@Override @Override
public KubernetesPeonLifecycle build(Task task, KubernetesPeonLifecycle.TaskStateListener stateListener, List<Pair<TaskRunnerListener, Executor>> listeners) public KubernetesPeonLifecycle build(Task task, KubernetesPeonLifecycle.TaskStateListener stateListener)
{ {
return new KubernetesPeonLifecycle( return new KubernetesPeonLifecycle(
task, task,
client, client,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
listeners
); );
} }
} }

View File

@ -146,20 +146,16 @@ 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 -> { return tasks.computeIfAbsent(task.getId(), k -> new KubernetesWorkItem(task, exec.submit(() -> runTask(task))))
ListenableFuture<TaskStatus> unused = exec.submit(() -> runTask(task)); .getResult();
return new KubernetesWorkItem(task);
}).getResult();
} }
} }
protected ListenableFuture<TaskStatus> joinAsync(Task task) protected ListenableFuture<TaskStatus> joinAsync(Task task)
{ {
synchronized (tasks) { synchronized (tasks) {
return tasks.computeIfAbsent(task.getId(), k -> { return tasks.computeIfAbsent(task.getId(), k -> new KubernetesWorkItem(task, exec.submit(() -> joinTask(task))))
ListenableFuture<TaskStatus> unused = exec.submit(() -> joinTask(task)); .getResult();
return new KubernetesWorkItem(task);
}).getResult();
} }
} }
@ -176,12 +172,10 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner
@VisibleForTesting @VisibleForTesting
protected TaskStatus doTask(Task task, boolean run) protected TaskStatus doTask(Task task, boolean run)
{ {
TaskStatus taskStatus = TaskStatus.failure(task.getId(), "Task execution never started");
try { try {
KubernetesPeonLifecycle peonLifecycle = peonLifecycleFactory.build( KubernetesPeonLifecycle peonLifecycle = peonLifecycleFactory.build(
task, task,
this::emitTaskStateMetrics, this::emitTaskStateMetrics
listeners
); );
synchronized (tasks) { synchronized (tasks) {
@ -194,6 +188,7 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner
workItem.setKubernetesPeonLifecycle(peonLifecycle); workItem.setKubernetesPeonLifecycle(peonLifecycle);
} }
TaskStatus taskStatus;
if (run) { if (run) {
taskStatus = peonLifecycle.run( taskStatus = peonLifecycle.run(
adapter.fromTask(task), adapter.fromTask(task),
@ -206,17 +201,15 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner
config.getTaskTimeout().toStandardDuration().getMillis() config.getTaskTimeout().toStandardDuration().getMillis()
); );
} }
updateStatus(task, taskStatus);
return taskStatus; return taskStatus;
} }
catch (Exception e) { catch (Exception e) {
log.error(e, "Task [%s] execution caught an exception", task.getId()); log.error(e, "Task [%s] execution caught an exception", task.getId());
taskStatus = TaskStatus.failure(task.getId(), "Could not start task execution");
throw new RuntimeException(e); throw new RuntimeException(e);
} }
finally {
updateStatus(task, taskStatus);
TaskRunnerUtils.notifyLocationChanged(listeners, task.getId(), TaskLocation.unknown());
}
} }
@VisibleForTesting @VisibleForTesting
@ -249,13 +242,13 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner
@Override @Override
public void updateStatus(Task task, TaskStatus status) public void updateStatus(Task task, TaskStatus status)
{ {
KubernetesWorkItem workItem = tasks.get(task.getId()); TaskRunnerUtils.notifyStatusChanged(listeners, task.getId(), status);
if (workItem != null && !workItem.getResult().isDone() && status.isComplete()) {
workItem.setResult(status);
} }
// Notify listeners even if the result is set to handle the shutdown case. @Override
TaskRunnerUtils.notifyStatusChanged(listeners, task.getId(), status); public void updateLocation(Task task, TaskLocation location)
{
TaskRunnerUtils.notifyLocationChanged(listeners, task.getId(), location);
} }
@Override @Override
@ -424,16 +417,6 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner
final Pair<TaskRunnerListener, Executor> listenerPair = Pair.of(listener, executor); final Pair<TaskRunnerListener, Executor> listenerPair = Pair.of(listener, executor);
log.debug("Registered listener [%s]", listener.getListenerId()); log.debug("Registered listener [%s]", listener.getListenerId());
listeners.add(listenerPair); listeners.add(listenerPair);
for (Map.Entry<String, KubernetesWorkItem> entry : tasks.entrySet()) {
if (entry.getValue().isRunning()) {
TaskRunnerUtils.notifyLocationChanged(
ImmutableList.of(listenerPair),
entry.getKey(),
entry.getValue().getLocation()
);
}
}
} }
@Override @Override

View File

@ -78,7 +78,7 @@ public class KubernetesTaskRunnerConfig
@JsonProperty @JsonProperty
@NotNull @NotNull
// how long to wait for the jobs to be cleaned up. // how long to wait for the jobs to be cleaned up.
private Period taskCleanupDelay = new Period("PT1H"); private Period taskCleanupDelay = new Period("P2D");
@JsonProperty @JsonProperty
@NotNull @NotNull

View File

@ -19,10 +19,9 @@
package org.apache.druid.k8s.overlord; package org.apache.druid.k8s.overlord;
import com.google.common.annotations.VisibleForTesting;
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.SettableFuture; 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;
import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatus;
@ -37,18 +36,9 @@ public class KubernetesWorkItem extends TaskRunnerWorkItem
private final Task task; private final Task task;
private KubernetesPeonLifecycle kubernetesPeonLifecycle = null; private KubernetesPeonLifecycle kubernetesPeonLifecycle = null;
private final SettableFuture<TaskStatus> result; public KubernetesWorkItem(Task task, ListenableFuture<TaskStatus> statusFuture)
public KubernetesWorkItem(Task task)
{ {
this(task, SettableFuture.create()); super(task.getId(), statusFuture);
}
@VisibleForTesting
public KubernetesWorkItem(Task task, SettableFuture<TaskStatus> result)
{
super(task.getId(), result);
this.result = result;
this.task = task; this.task = task;
} }
@ -61,7 +51,7 @@ public class KubernetesWorkItem extends TaskRunnerWorkItem
protected synchronized void shutdown() protected synchronized void shutdown()
{ {
if (this.kubernetesPeonLifecycle != null && !result.isDone()) { if (this.kubernetesPeonLifecycle != null) {
this.kubernetesPeonLifecycle.startWatchingLogs(); this.kubernetesPeonLifecycle.startWatchingLogs();
this.kubernetesPeonLifecycle.shutdown(); this.kubernetesPeonLifecycle.shutdown();
} }
@ -129,9 +119,4 @@ public class KubernetesWorkItem extends TaskRunnerWorkItem
{ {
return task; return task;
} }
public void setResult(TaskStatus status)
{
result.set(status);
}
} }

View File

@ -20,13 +20,8 @@
package org.apache.druid.k8s.overlord; package org.apache.druid.k8s.overlord;
import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.TaskRunnerListener;
import org.apache.druid.java.util.common.Pair;
import java.util.List;
import java.util.concurrent.Executor;
public interface PeonLifecycleFactory public interface PeonLifecycleFactory
{ {
KubernetesPeonLifecycle build(Task task, KubernetesPeonLifecycle.TaskStateListener stateListener, List<Pair<TaskRunnerListener, Executor>> listeners); KubernetesPeonLifecycle build(Task task, KubernetesPeonLifecycle.TaskStateListener stateListener);
} }

View File

@ -32,10 +32,12 @@ public class JobResponse
private static final EmittingLogger LOGGER = new EmittingLogger(JobResponse.class); private static final EmittingLogger LOGGER = new EmittingLogger(JobResponse.class);
private final Job job; private final Job job;
private final PeonPhase phase;
public JobResponse(@Nullable Job job) public JobResponse(@Nullable Job job, PeonPhase phase)
{ {
this.job = job; this.job = job;
this.phase = phase;
} }
public Job getJob() public Job getJob()
@ -43,6 +45,11 @@ public class JobResponse
return job; return job;
} }
public PeonPhase getPhase()
{
return phase;
}
public long getJobDuration() public long getJobDuration()
{ {
long duration = -1L; long duration = -1L;

View File

@ -101,13 +101,13 @@ public class KubernetesPeonClient
); );
if (job == null) { if (job == null) {
log.info("K8s job for the task [%s] was not found. It can happen if the task was canceled", taskId); log.info("K8s job for the task [%s] was not found. It can happen if the task was canceled", taskId);
return new JobResponse(null); return new JobResponse(null, PeonPhase.FAILED);
} }
if (job.getStatus().getSucceeded() != null) { if (job.getStatus().getSucceeded() != null) {
return new JobResponse(job); return new JobResponse(job, PeonPhase.SUCCEEDED);
} }
log.warn("Task %s failed with status %s", taskId, job.getStatus()); log.warn("Task %s failed with status %s", taskId, job.getStatus());
return new JobResponse(job); return new JobResponse(job, PeonPhase.FAILED);
}); });
} }

View File

@ -0,0 +1,62 @@
/*
* 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 io.fabric8.kubernetes.api.model.Pod;
import java.util.Arrays;
import java.util.Map;
import java.util.function.Function;
import java.util.stream.Collectors;
public enum PeonPhase
{
PENDING("Pending"),
SUCCEEDED("Succeeded"),
FAILED("Failed"),
UNKNOWN("Unknown"),
RUNNING("Running");
private static final Map<String, PeonPhase> PHASE_MAP = Arrays.stream(PeonPhase.values())
.collect(Collectors.toMap(
PeonPhase::getPhase,
Function.identity()
));
private final String phase;
PeonPhase(String phase)
{
this.phase = phase;
}
public String getPhase()
{
return phase;
}
public static PeonPhase getPhaseFor(Pod pod)
{
if (pod == null) {
return UNKNOWN;
}
return PHASE_MAP.get(pod.getStatus().getPhase());
}
}

View File

@ -362,4 +362,13 @@ public class KubernetesAndWorkerTaskRunnerTest extends EasyMockSupport
runner.updateStatus(task, TaskStatus.running(ID)); runner.updateStatus(task, TaskStatus.running(ID));
verifyAll(); verifyAll();
} }
@Test
public void test_updateLocation()
{
kubernetesTaskRunner.updateLocation(task, TaskLocation.unknown());
replayAll();
runner.updateLocation(task, TaskLocation.unknown());
verifyAll();
}
} }

View File

@ -21,7 +21,6 @@ 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.collect.ImmutableList;
import io.fabric8.kubernetes.api.model.Pod; import io.fabric8.kubernetes.api.model.Pod;
import io.fabric8.kubernetes.api.model.PodBuilder; import io.fabric8.kubernetes.api.model.PodBuilder;
import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.api.model.batch.v1.Job;
@ -32,12 +31,11 @@ import org.apache.druid.indexer.TaskLocation;
import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.TestUtils;
import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.TaskRunnerListener;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.k8s.overlord.common.JobResponse; import org.apache.druid.k8s.overlord.common.JobResponse;
import org.apache.druid.k8s.overlord.common.K8sTaskId; import org.apache.druid.k8s.overlord.common.K8sTaskId;
import org.apache.druid.k8s.overlord.common.K8sTestUtils; import org.apache.druid.k8s.overlord.common.K8sTestUtils;
import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; import org.apache.druid.k8s.overlord.common.KubernetesPeonClient;
import org.apache.druid.k8s.overlord.common.PeonPhase;
import org.apache.druid.tasklogs.TaskLogs; import org.apache.druid.tasklogs.TaskLogs;
import org.easymock.EasyMock; import org.easymock.EasyMock;
import org.easymock.EasyMockRunner; import org.easymock.EasyMockRunner;
@ -52,8 +50,6 @@ import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.lang.reflect.Field; import java.lang.reflect.Field;
import java.nio.charset.StandardCharsets; import java.nio.charset.StandardCharsets;
import java.util.List;
import java.util.concurrent.Executor;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReference;
@ -69,8 +65,6 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
@Mock LogWatch logWatch; @Mock LogWatch logWatch;
@Mock KubernetesPeonLifecycle.TaskStateListener stateListener; @Mock KubernetesPeonLifecycle.TaskStateListener stateListener;
List<Pair<TaskRunnerListener, Executor>> listeners = ImmutableList.of();
private ObjectMapper mapper; private ObjectMapper mapper;
private Task task; private Task task;
private K8sTaskId k8sTaskId; private K8sTaskId k8sTaskId;
@ -92,8 +86,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
kubernetesClient, kubernetesClient,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
listeners
) )
{ {
@Override @Override
@ -138,8 +131,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
kubernetesClient, kubernetesClient,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
listeners
) )
{ {
@Override @Override
@ -183,8 +175,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
kubernetesClient, kubernetesClient,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
listeners
) )
{ {
@Override @Override
@ -233,8 +224,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
kubernetesClient, kubernetesClient,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
listeners
) )
{ {
@Override @Override
@ -278,19 +268,15 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
kubernetesClient, kubernetesClient,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
listeners
); );
EasyMock.expect(kubernetesClient.waitForPeonJobCompletion( EasyMock.expect(kubernetesClient.waitForPeonJobCompletion(
EasyMock.eq(k8sTaskId), EasyMock.eq(k8sTaskId),
EasyMock.anyLong(), EasyMock.anyLong(),
EasyMock.eq(TimeUnit.MILLISECONDS) EasyMock.eq(TimeUnit.MILLISECONDS)
)).andReturn(new JobResponse(null)); )).andReturn(new JobResponse(null, PeonPhase.FAILED));
EasyMock.expect(kubernetesClient.getPeonLogWatcher(k8sTaskId)).andReturn(Optional.of(logWatch)); EasyMock.expect(kubernetesClient.getPeonLogWatcher(k8sTaskId)).andReturn(Optional.of(logWatch));
EasyMock.expect(kubernetesClient.getPeonPod(k8sTaskId.getK8sJobName())).andReturn(Optional.absent());
EasyMock.expect(kubernetesClient.deletePeonJob(k8sTaskId)).andReturn(true);
EasyMock.expect(taskLogs.streamTaskStatus(ID)).andReturn(Optional.absent()); EasyMock.expect(taskLogs.streamTaskStatus(ID)).andReturn(Optional.absent());
taskLogs.pushTaskLog(EasyMock.eq(ID), EasyMock.anyObject(File.class)); taskLogs.pushTaskLog(EasyMock.eq(ID), EasyMock.anyObject(File.class));
EasyMock.expectLastCall(); EasyMock.expectLastCall();
@ -316,15 +302,12 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
@Test @Test
public void test_join() throws IOException public void test_join() throws IOException
{ {
Executor executor = EasyMock.mock(Executor.class);
TaskRunnerListener taskRunnerListener = EasyMock.mock(TaskRunnerListener.class);
KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle( KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle(
task, task,
kubernetesClient, kubernetesClient,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
ImmutableList.of(Pair.of(taskRunnerListener, executor))
); );
Job job = new JobBuilder() Job job = new JobBuilder()
@ -342,12 +325,8 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
EasyMock.eq(k8sTaskId), EasyMock.eq(k8sTaskId),
EasyMock.anyLong(), EasyMock.anyLong(),
EasyMock.eq(TimeUnit.MILLISECONDS) EasyMock.eq(TimeUnit.MILLISECONDS)
)).andReturn(new JobResponse(job)); )).andReturn(new JobResponse(job, PeonPhase.SUCCEEDED));
EasyMock.expect(kubernetesClient.getPeonLogWatcher(k8sTaskId)).andReturn(Optional.of(logWatch)); EasyMock.expect(kubernetesClient.getPeonLogWatcher(k8sTaskId)).andReturn(Optional.of(logWatch));
EasyMock.expect(kubernetesClient.getPeonPod(k8sTaskId.getK8sJobName())).andReturn(
Optional.of(new PodBuilder().withNewMetadata().withName("job-pod").endMetadata().withNewStatus().withPodIP("ip").endStatus().build())
);
EasyMock.expect(kubernetesClient.deletePeonJob(k8sTaskId)).andReturn(true);
EasyMock.expect(taskLogs.streamTaskStatus(ID)).andReturn(Optional.of( EasyMock.expect(taskLogs.streamTaskStatus(ID)).andReturn(Optional.of(
IOUtils.toInputStream(mapper.writeValueAsString(SUCCESS), StandardCharsets.UTF_8) IOUtils.toInputStream(mapper.writeValueAsString(SUCCESS), StandardCharsets.UTF_8)
)); ));
@ -359,10 +338,6 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
EasyMock.expectLastCall().once(); EasyMock.expectLastCall().once();
logWatch.close(); logWatch.close();
EasyMock.expectLastCall(); EasyMock.expectLastCall();
executor.execute(EasyMock.anyObject());
EasyMock.expectLastCall();
taskRunnerListener.locationChanged(EasyMock.anyObject(), EasyMock.anyObject());
EasyMock.expectLastCall();
Assert.assertEquals(KubernetesPeonLifecycle.State.NOT_STARTED, peonLifecycle.getState()); Assert.assertEquals(KubernetesPeonLifecycle.State.NOT_STARTED, peonLifecycle.getState());
@ -384,8 +359,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
kubernetesClient, kubernetesClient,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
listeners
); );
Job job = new JobBuilder() Job job = new JobBuilder()
@ -401,15 +375,8 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
EasyMock.eq(k8sTaskId), EasyMock.eq(k8sTaskId),
EasyMock.anyLong(), EasyMock.anyLong(),
EasyMock.eq(TimeUnit.MILLISECONDS) EasyMock.eq(TimeUnit.MILLISECONDS)
)).andReturn(new JobResponse(job)); )).andReturn(new JobResponse(job, PeonPhase.SUCCEEDED));
EasyMock.expect(kubernetesClient.getPeonLogWatcher(k8sTaskId)).andReturn(Optional.of(logWatch)); EasyMock.expect(kubernetesClient.getPeonLogWatcher(k8sTaskId)).andReturn(Optional.of(logWatch));
// Only update the location the first time, second call doesn't reach this point in the logic
EasyMock.expect(kubernetesClient.getPeonPod(k8sTaskId.getK8sJobName())).andReturn(
Optional.of(new PodBuilder().withNewMetadata().withName("job-pod").endMetadata().withNewStatus().withPodIP("ip").endStatus().build())
);
// Always try to delete the job
EasyMock.expect(kubernetesClient.deletePeonJob(k8sTaskId)).andReturn(true).times(2);
EasyMock.expect(taskLogs.streamTaskStatus(ID)).andReturn( EasyMock.expect(taskLogs.streamTaskStatus(ID)).andReturn(
Optional.of(IOUtils.toInputStream(mapper.writeValueAsString(SUCCESS), StandardCharsets.UTF_8)) Optional.of(IOUtils.toInputStream(mapper.writeValueAsString(SUCCESS), StandardCharsets.UTF_8))
); );
@ -452,8 +419,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
kubernetesClient, kubernetesClient,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
listeners
); );
Job job = new JobBuilder() Job job = new JobBuilder()
@ -469,12 +435,8 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
EasyMock.eq(k8sTaskId), EasyMock.eq(k8sTaskId),
EasyMock.anyLong(), EasyMock.anyLong(),
EasyMock.eq(TimeUnit.MILLISECONDS) EasyMock.eq(TimeUnit.MILLISECONDS)
)).andReturn(new JobResponse(job)); )).andReturn(new JobResponse(job, PeonPhase.SUCCEEDED));
EasyMock.expect(kubernetesClient.getPeonLogWatcher(k8sTaskId)).andReturn(Optional.of(logWatch)); EasyMock.expect(kubernetesClient.getPeonLogWatcher(k8sTaskId)).andReturn(Optional.of(logWatch));
EasyMock.expect(kubernetesClient.getPeonPod(k8sTaskId.getK8sJobName())).andReturn(
Optional.of(new PodBuilder().withNewMetadata().withName("job-pod").endMetadata().withNewStatus().withPodIP("ip").endStatus().build())
);
EasyMock.expect(kubernetesClient.deletePeonJob(k8sTaskId)).andReturn(true);
EasyMock.expect(taskLogs.streamTaskStatus(ID)).andReturn(Optional.absent()); EasyMock.expect(taskLogs.streamTaskStatus(ID)).andReturn(Optional.absent());
taskLogs.pushTaskLog(EasyMock.eq(ID), EasyMock.anyObject(File.class)); taskLogs.pushTaskLog(EasyMock.eq(ID), EasyMock.anyObject(File.class));
EasyMock.expectLastCall(); EasyMock.expectLastCall();
@ -507,8 +469,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
kubernetesClient, kubernetesClient,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
listeners
); );
Job job = new JobBuilder() Job job = new JobBuilder()
@ -524,12 +485,8 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
EasyMock.eq(k8sTaskId), EasyMock.eq(k8sTaskId),
EasyMock.anyLong(), EasyMock.anyLong(),
EasyMock.eq(TimeUnit.MILLISECONDS) EasyMock.eq(TimeUnit.MILLISECONDS)
)).andReturn(new JobResponse(job)); )).andReturn(new JobResponse(job, PeonPhase.SUCCEEDED));
EasyMock.expect(kubernetesClient.getPeonLogWatcher(k8sTaskId)).andReturn(Optional.of(logWatch)); EasyMock.expect(kubernetesClient.getPeonLogWatcher(k8sTaskId)).andReturn(Optional.of(logWatch));
EasyMock.expect(kubernetesClient.getPeonPod(k8sTaskId.getK8sJobName())).andReturn(
Optional.of(new PodBuilder().withNewMetadata().withName("job-pod").endMetadata().withNewStatus().withPodIP("ip").endStatus().build())
);
EasyMock.expect(kubernetesClient.deletePeonJob(k8sTaskId)).andReturn(true);
EasyMock.expect(taskLogs.streamTaskStatus(ID)).andThrow(new IOException()); EasyMock.expect(taskLogs.streamTaskStatus(ID)).andThrow(new IOException());
taskLogs.pushTaskLog(EasyMock.eq(ID), EasyMock.anyObject(File.class)); taskLogs.pushTaskLog(EasyMock.eq(ID), EasyMock.anyObject(File.class));
EasyMock.expectLastCall(); EasyMock.expectLastCall();
@ -562,8 +519,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
kubernetesClient, kubernetesClient,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
listeners
); );
Job job = new JobBuilder() Job job = new JobBuilder()
@ -579,11 +535,8 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
EasyMock.eq(k8sTaskId), EasyMock.eq(k8sTaskId),
EasyMock.anyLong(), EasyMock.anyLong(),
EasyMock.eq(TimeUnit.MILLISECONDS) EasyMock.eq(TimeUnit.MILLISECONDS)
)).andReturn(new JobResponse(job)); )).andReturn(new JobResponse(job, PeonPhase.SUCCEEDED));
EasyMock.expect(kubernetesClient.getPeonLogWatcher(k8sTaskId)).andReturn(Optional.of(logWatch)); EasyMock.expect(kubernetesClient.getPeonLogWatcher(k8sTaskId)).andReturn(Optional.of(logWatch));
EasyMock.expect(kubernetesClient.getPeonPod(k8sTaskId.getK8sJobName())).andReturn(
Optional.of(new PodBuilder().withNewMetadata().withName("job-pod").endMetadata().withNewStatus().withPodIP("ip").endStatus().build())
);
EasyMock.expect(taskLogs.streamTaskStatus(ID)).andReturn( EasyMock.expect(taskLogs.streamTaskStatus(ID)).andReturn(
Optional.of(IOUtils.toInputStream(mapper.writeValueAsString(SUCCESS), StandardCharsets.UTF_8)) Optional.of(IOUtils.toInputStream(mapper.writeValueAsString(SUCCESS), StandardCharsets.UTF_8))
); );
@ -596,9 +549,6 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
logWatch.close(); logWatch.close();
EasyMock.expectLastCall(); EasyMock.expectLastCall();
// We should still try to cleanup the Job after
EasyMock.expect(kubernetesClient.deletePeonJob(k8sTaskId)).andReturn(true);
Assert.assertEquals(KubernetesPeonLifecycle.State.NOT_STARTED, peonLifecycle.getState()); Assert.assertEquals(KubernetesPeonLifecycle.State.NOT_STARTED, peonLifecycle.getState());
replayAll(); replayAll();
@ -619,8 +569,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
kubernetesClient, kubernetesClient,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
listeners
); );
EasyMock.expect(kubernetesClient.waitForPeonJobCompletion( EasyMock.expect(kubernetesClient.waitForPeonJobCompletion(
@ -629,9 +578,6 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
EasyMock.eq(TimeUnit.MILLISECONDS) EasyMock.eq(TimeUnit.MILLISECONDS)
)).andThrow(new RuntimeException()); )).andThrow(new RuntimeException());
EasyMock.expect(kubernetesClient.getPeonPod(k8sTaskId.getK8sJobName())).andReturn(
Optional.of(new PodBuilder().withNewMetadata().withName("job-pod").endMetadata().withNewStatus().withPodIP("ip").endStatus().build())
);
// We should still try to push logs // We should still try to push logs
EasyMock.expect(kubernetesClient.getPeonLogWatcher(k8sTaskId)).andReturn(Optional.of(logWatch)); EasyMock.expect(kubernetesClient.getPeonLogWatcher(k8sTaskId)).andReturn(Optional.of(logWatch));
taskLogs.pushTaskLog(EasyMock.eq(ID), EasyMock.anyObject(File.class)); taskLogs.pushTaskLog(EasyMock.eq(ID), EasyMock.anyObject(File.class));
@ -642,7 +588,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
EasyMock.expectLastCall().once(); EasyMock.expectLastCall().once();
logWatch.close(); logWatch.close();
EasyMock.expectLastCall(); EasyMock.expectLastCall();
EasyMock.expect(kubernetesClient.deletePeonJob(k8sTaskId)).andReturn(true);
Assert.assertEquals(KubernetesPeonLifecycle.State.NOT_STARTED, peonLifecycle.getState()); Assert.assertEquals(KubernetesPeonLifecycle.State.NOT_STARTED, peonLifecycle.getState());
replayAll(); replayAll();
@ -662,8 +608,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
kubernetesClient, kubernetesClient,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
listeners
); );
peonLifecycle.shutdown(); peonLifecycle.shutdown();
} }
@ -676,8 +621,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
kubernetesClient, kubernetesClient,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
listeners
); );
setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.PENDING); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.PENDING);
@ -698,8 +642,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
kubernetesClient, kubernetesClient,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
listeners
); );
setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING);
@ -720,8 +663,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
kubernetesClient, kubernetesClient,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
listeners
); );
setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.STOPPED); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.STOPPED);
@ -736,8 +678,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
kubernetesClient, kubernetesClient,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
listeners
); );
setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.NOT_STARTED); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.NOT_STARTED);
@ -752,8 +693,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
kubernetesClient, kubernetesClient,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
listeners
); );
setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.PENDING); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.PENDING);
@ -768,8 +708,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
kubernetesClient, kubernetesClient,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
listeners
); );
setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING);
@ -792,8 +731,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
kubernetesClient, kubernetesClient,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
listeners
); );
setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.STOPPED); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.STOPPED);
@ -809,8 +747,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
kubernetesClient, kubernetesClient,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
listeners
); );
setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.NOT_STARTED); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.NOT_STARTED);
@ -826,8 +763,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
kubernetesClient, kubernetesClient,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
listeners
); );
setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.PENDING); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.PENDING);
@ -843,8 +779,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
kubernetesClient, kubernetesClient,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
listeners
); );
setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING);
@ -866,8 +801,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
kubernetesClient, kubernetesClient,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
listeners
); );
setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING);
@ -895,8 +829,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
kubernetesClient, kubernetesClient,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
listeners
); );
setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING);
@ -932,8 +865,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
kubernetesClient, kubernetesClient,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
listeners
); );
setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING);
@ -969,8 +901,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
kubernetesClient, kubernetesClient,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
listeners
); );
setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING);
@ -1007,8 +938,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
kubernetesClient, kubernetesClient,
taskLogs, taskLogs,
mapper, mapper,
stateListener, stateListener
listeners
); );
setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.STOPPED); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.STOPPED);
EasyMock.expect(kubernetesClient.getPeonPod(k8sTaskId.getK8sJobName())).andReturn(Optional.absent()).once(); EasyMock.expect(kubernetesClient.getPeonPod(k8sTaskId.getK8sJobName())).andReturn(Optional.absent()).once();

View File

@ -47,7 +47,7 @@ public class KubernetesTaskRunnerConfigTest
Assert.assertNull(config.getGraceTerminationPeriodSeconds()); Assert.assertNull(config.getGraceTerminationPeriodSeconds());
Assert.assertTrue(config.isDisableClientProxy()); Assert.assertTrue(config.isDisableClientProxy());
Assert.assertEquals(new Period("PT4H"), config.getTaskTimeout()); Assert.assertEquals(new Period("PT4H"), config.getTaskTimeout());
Assert.assertEquals(new Period("PT1H"), config.getTaskCleanupDelay()); Assert.assertEquals(new Period("P2D"), config.getTaskCleanupDelay());
Assert.assertEquals(new Period("PT10m"), config.getTaskCleanupInterval()); Assert.assertEquals(new Period("PT10m"), config.getTaskCleanupInterval());
Assert.assertEquals(new Period("PT1H"), config.getTaskLaunchTimeout()); Assert.assertEquals(new Period("PT1H"), config.getTaskLaunchTimeout());
Assert.assertEquals(ImmutableList.of(), config.getPeonMonitors()); Assert.assertEquals(ImmutableList.of(), config.getPeonMonitors());
@ -72,7 +72,7 @@ public class KubernetesTaskRunnerConfigTest
Assert.assertNull(config.getGraceTerminationPeriodSeconds()); Assert.assertNull(config.getGraceTerminationPeriodSeconds());
Assert.assertTrue(config.isDisableClientProxy()); Assert.assertTrue(config.isDisableClientProxy());
Assert.assertEquals(new Period("PT4H"), config.getTaskTimeout()); Assert.assertEquals(new Period("PT4H"), config.getTaskTimeout());
Assert.assertEquals(new Period("PT1H"), config.getTaskCleanupDelay()); Assert.assertEquals(new Period("P2D"), config.getTaskCleanupDelay());
Assert.assertEquals(new Period("PT10m"), config.getTaskCleanupInterval()); Assert.assertEquals(new Period("PT10m"), config.getTaskCleanupInterval());
Assert.assertEquals(new Period("PT1H"), config.getTaskLaunchTimeout()); Assert.assertEquals(new Period("PT1H"), config.getTaskLaunchTimeout());
Assert.assertEquals(ImmutableList.of(), config.getPeonMonitors()); Assert.assertEquals(ImmutableList.of(), config.getPeonMonitors());

View File

@ -28,10 +28,8 @@ import io.fabric8.kubernetes.api.model.batch.v1.JobBuilder;
import org.apache.commons.io.IOUtils; import org.apache.commons.io.IOUtils;
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.TaskState;
import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.TaskRunnerListener;
import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; import org.apache.druid.indexing.overlord.TaskRunnerWorkItem;
import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.java.util.emitter.service.ServiceEventBuilder; import org.apache.druid.java.util.emitter.service.ServiceEventBuilder;
@ -78,9 +76,6 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
@Mock private KubernetesPeonLifecycle kubernetesPeonLifecycle; @Mock private KubernetesPeonLifecycle kubernetesPeonLifecycle;
@Mock private ServiceEmitter emitter; @Mock private ServiceEmitter emitter;
@Mock private Executor executor;
@Mock private TaskRunnerListener taskRunnerListener;
private KubernetesTaskRunnerConfig config; private KubernetesTaskRunnerConfig config;
private KubernetesTaskRunner runner; private KubernetesTaskRunner runner;
private Task task; private Task task;
@ -121,7 +116,11 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
{ {
return tasks.computeIfAbsent( return tasks.computeIfAbsent(
task.getId(), task.getId(),
k -> new KubernetesWorkItem(task)).getResult(); k -> new KubernetesWorkItem(
task,
Futures.immediateFuture(TaskStatus.success(task.getId()))
)
).getResult();
} }
}; };
@ -250,7 +249,7 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
} }
@Test @Test
public void test_run_whenExceptionThrown_throwsRuntimeException() throws Exception public void test_run_whenExceptionThrown_throwsRuntimeException() throws IOException
{ {
Job job = new JobBuilder() Job job = new JobBuilder()
.withNewMetadata() .withNewMetadata()
@ -270,91 +269,13 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
replayAll(); replayAll();
ListenableFuture<TaskStatus> future = runner.run(task); ListenableFuture<TaskStatus> future = runner.run(task);
TaskStatus taskStatus = future.get();
Assert.assertEquals(TaskState.FAILED, taskStatus.getStatusCode()); Exception e = Assert.assertThrows(ExecutionException.class, future::get);
Assert.assertEquals("Could not start task execution", taskStatus.getErrorMsg()); Assert.assertTrue(e.getCause() instanceof RuntimeException);
verifyAll(); verifyAll();
} }
@Test
public void test_run_updateStatus() throws ExecutionException, InterruptedException
{
KubernetesTaskRunner runner = new KubernetesTaskRunner(
taskAdapter,
config,
peonClient,
httpClient,
new TestPeonLifecycleFactory(kubernetesPeonLifecycle),
emitter
);
KubernetesWorkItem workItem = new KubernetesWorkItem(task);
runner.tasks.put(task.getId(), workItem);
TaskStatus completeTaskStatus = TaskStatus.success(task.getId());
replayAll();
runner.updateStatus(task, completeTaskStatus);
verifyAll();
assertTrue(workItem.getResult().isDone());
assertEquals(completeTaskStatus, workItem.getResult().get());
}
@Test
public void test_run_updateStatus_running()
{
KubernetesTaskRunner runner = new KubernetesTaskRunner(
taskAdapter,
config,
peonClient,
httpClient,
new TestPeonLifecycleFactory(kubernetesPeonLifecycle),
emitter
);
KubernetesWorkItem workItem = new KubernetesWorkItem(task);
runner.tasks.put(task.getId(), workItem);
TaskStatus runningTaskStatus = TaskStatus.running(task.getId());
replayAll();
runner.updateStatus(task, runningTaskStatus);
verifyAll();
assertFalse(workItem.getResult().isDone());
}
@Test
public void test_registerListener_runningTask()
{
KubernetesTaskRunner runner = new KubernetesTaskRunner(
taskAdapter,
config,
peonClient,
httpClient,
new TestPeonLifecycleFactory(kubernetesPeonLifecycle),
emitter
);
KubernetesPeonLifecycle runningKubernetesPeonLifecycle = EasyMock.mock(KubernetesPeonLifecycle.class);
EasyMock.expect(runningKubernetesPeonLifecycle.getState()).andReturn(KubernetesPeonLifecycle.State.RUNNING);
EasyMock.expect(runningKubernetesPeonLifecycle.getTaskLocation()).andReturn(TaskLocation.unknown());
KubernetesWorkItem workItem = new KubernetesWorkItem(task);
workItem.setKubernetesPeonLifecycle(runningKubernetesPeonLifecycle);
runner.tasks.put(task.getId(), workItem);
Executor executor = EasyMock.mock(Executor.class);
TaskRunnerListener taskRunnerListener = EasyMock.mock(TaskRunnerListener.class);
executor.execute(EasyMock.anyObject());
EasyMock.expectLastCall();
taskRunnerListener.locationChanged(EasyMock.anyObject(), EasyMock.anyObject());
EasyMock.expectLastCall();
replayAll();
EasyMock.replay(runningKubernetesPeonLifecycle);
runner.registerListener(taskRunnerListener, executor);
verifyAll();
EasyMock.verify(runningKubernetesPeonLifecycle);
}
@Test @Test
public void test_join_withoutExistingTask() throws ExecutionException, InterruptedException public void test_join_withoutExistingTask() throws ExecutionException, InterruptedException
{ {
@ -382,15 +303,16 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
} }
@Test @Test
public void test_join_whenExceptionThrown_throwsRuntimeException() throws ExecutionException, InterruptedException public void test_join_whenExceptionThrown_throwsRuntimeException()
{ {
EasyMock.expect(kubernetesPeonLifecycle.join(EasyMock.anyLong())).andThrow(new IllegalStateException()); EasyMock.expect(kubernetesPeonLifecycle.join(EasyMock.anyLong())).andThrow(new IllegalStateException());
replayAll(); replayAll();
ListenableFuture<TaskStatus> future = runner.joinAsync(task); ListenableFuture<TaskStatus> future = runner.joinAsync(task);
TaskStatus taskStatus = future.get();
Assert.assertEquals(TaskState.FAILED, taskStatus.getStatusCode()); Exception e = Assert.assertThrows(ExecutionException.class, future::get);
Assert.assertTrue(e.getCause() instanceof RuntimeException);
verifyAll(); verifyAll();
} }

View File

@ -56,7 +56,6 @@ public class KubernetesWorkItemTest extends EasyMockSupport
null, null,
null, null,
null, null,
null,
null null
)); ));
@ -67,7 +66,6 @@ public class KubernetesWorkItemTest extends EasyMockSupport
null, null,
null, null,
null, null,
null,
null null
)) ))
); );
@ -82,8 +80,6 @@ public class KubernetesWorkItemTest extends EasyMockSupport
@Test @Test
public void test_shutdown_withKubernetesPeonLifecycle() public void test_shutdown_withKubernetesPeonLifecycle()
{ {
KubernetesWorkItem workItem = new KubernetesWorkItem(task);
kubernetesPeonLifecycle.shutdown(); kubernetesPeonLifecycle.shutdown();
EasyMock.expectLastCall(); EasyMock.expectLastCall();
kubernetesPeonLifecycle.startWatchingLogs(); kubernetesPeonLifecycle.startWatchingLogs();
@ -91,6 +87,7 @@ public class KubernetesWorkItemTest extends EasyMockSupport
replayAll(); replayAll();
workItem.setKubernetesPeonLifecycle(kubernetesPeonLifecycle); workItem.setKubernetesPeonLifecycle(kubernetesPeonLifecycle);
workItem.shutdown(); workItem.shutdown();
verifyAll(); verifyAll();
} }
@ -161,7 +158,6 @@ public class KubernetesWorkItemTest extends EasyMockSupport
null, null,
null, null,
null, null,
null,
null null
)); ));
@ -176,7 +172,6 @@ public class KubernetesWorkItemTest extends EasyMockSupport
null, null,
null, null,
null, null,
null,
null null
) { ) {
@Override @Override
@ -199,7 +194,6 @@ public class KubernetesWorkItemTest extends EasyMockSupport
null, null,
null, null,
null, null,
null,
null null
) { ) {
@Override @Override
@ -222,7 +216,6 @@ public class KubernetesWorkItemTest extends EasyMockSupport
null, null,
null, null,
null, null,
null,
null null
) { ) {
@Override @Override
@ -251,7 +244,6 @@ public class KubernetesWorkItemTest extends EasyMockSupport
null, null,
null, null,
null, null,
null,
null null
)); ));
Assert.assertFalse(workItem.streamTaskLogs().isPresent()); Assert.assertFalse(workItem.streamTaskLogs().isPresent());
@ -271,7 +263,6 @@ public class KubernetesWorkItemTest extends EasyMockSupport
null, null,
null, null,
null, null,
null,
null null
)); ));

View File

@ -20,11 +20,6 @@
package org.apache.druid.k8s.overlord; package org.apache.druid.k8s.overlord;
import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.TaskRunnerListener;
import org.apache.druid.java.util.common.Pair;
import java.util.List;
import java.util.concurrent.Executor;
public class TestPeonLifecycleFactory implements PeonLifecycleFactory public class TestPeonLifecycleFactory implements PeonLifecycleFactory
{ {
@ -36,7 +31,7 @@ public class TestPeonLifecycleFactory implements PeonLifecycleFactory
} }
@Override @Override
public KubernetesPeonLifecycle build(Task task, KubernetesPeonLifecycle.TaskStateListener stateListener, List<Pair<TaskRunnerListener, Executor>> listeners) public KubernetesPeonLifecycle build(Task task, KubernetesPeonLifecycle.TaskStateListener stateListener)
{ {
return kubernetesPeonLifecycle; return kubernetesPeonLifecycle;
} }

View File

@ -39,7 +39,7 @@ class JobResponseTest
.endStatus() .endStatus()
.build(); .build();
JobResponse response = new JobResponse(job); JobResponse response = new JobResponse(job, PeonPhase.SUCCEEDED);
Assertions.assertEquals(58000L, response.getJobDuration()); Assertions.assertEquals(58000L, response.getJobDuration());
} }
@ -56,7 +56,7 @@ class JobResponseTest
.endStatus() .endStatus()
.build(); .build();
JobResponse response = new JobResponse(job); JobResponse response = new JobResponse(job, PeonPhase.SUCCEEDED);
Assertions.assertEquals(-1, response.getJobDuration()); Assertions.assertEquals(-1, response.getJobDuration());
} }
@ -70,7 +70,7 @@ class JobResponseTest
.endMetadata() .endMetadata()
.build(); .build();
JobResponse response = new JobResponse(job); JobResponse response = new JobResponse(job, PeonPhase.SUCCEEDED);
Assertions.assertEquals(-1, response.getJobDuration()); Assertions.assertEquals(-1, response.getJobDuration());
} }
@ -78,7 +78,7 @@ class JobResponseTest
@Test @Test
void testNullJob() void testNullJob()
{ {
JobResponse response = new JobResponse(null); JobResponse response = new JobResponse(null, PeonPhase.SUCCEEDED);
long duration = response.getJobDuration(); long duration = response.getJobDuration();
Assertions.assertEquals(-1, duration); Assertions.assertEquals(-1, duration);
} }

View File

@ -153,6 +153,7 @@ public class KubernetesPeonClientTest
TimeUnit.SECONDS TimeUnit.SECONDS
); );
Assertions.assertEquals(PeonPhase.SUCCEEDED, jobResponse.getPhase());
Assertions.assertNotNull(jobResponse.getJob()); Assertions.assertNotNull(jobResponse.getJob());
} }
@ -177,6 +178,7 @@ public class KubernetesPeonClientTest
TimeUnit.SECONDS TimeUnit.SECONDS
); );
Assertions.assertEquals(PeonPhase.FAILED, jobResponse.getPhase());
Assertions.assertNotNull(jobResponse.getJob()); Assertions.assertNotNull(jobResponse.getJob());
} }
@ -189,6 +191,7 @@ public class KubernetesPeonClientTest
TimeUnit.SECONDS TimeUnit.SECONDS
); );
Assertions.assertEquals(PeonPhase.FAILED, jobResponse.getPhase());
Assertions.assertNull(jobResponse.getJob()); Assertions.assertNull(jobResponse.getJob());
} }

View File

@ -0,0 +1,43 @@
/*
* 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 io.fabric8.kubernetes.api.model.Pod;
import io.fabric8.kubernetes.api.model.PodStatus;
import org.junit.jupiter.api.Test;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public class PeonPhaseTest
{
@Test
void testGetPhaseForToMakeCoverageHappy()
{
Pod pod = mock(Pod.class);
PodStatus status = mock(PodStatus.class);
when(status.getPhase()).thenReturn("Succeeded");
when(pod.getStatus()).thenReturn(status);
assertEquals(PeonPhase.UNKNOWN, PeonPhase.getPhaseFor(null));
assertEquals(PeonPhase.SUCCEEDED, PeonPhase.getPhaseFor(pod));
}
}

View File

@ -38,11 +38,13 @@ import org.apache.druid.java.util.emitter.core.Event;
import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig;
import org.apache.druid.k8s.overlord.common.DruidKubernetesClient; 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.K8sTaskId;
import org.apache.druid.k8s.overlord.common.K8sTestUtils; import org.apache.druid.k8s.overlord.common.K8sTestUtils;
import org.apache.druid.k8s.overlord.common.KubernetesClientApi; import org.apache.druid.k8s.overlord.common.KubernetesClientApi;
import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; import org.apache.druid.k8s.overlord.common.KubernetesPeonClient;
import org.apache.druid.k8s.overlord.common.PeonCommandContext; 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.DruidNode;
import org.apache.druid.server.log.StartupLoggingConfig; import org.apache.druid.server.log.StartupLoggingConfig;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
@ -182,8 +184,9 @@ public class DruidPeonClientIntegrationTest
assertEquals(task, taskFromPod); assertEquals(task, taskFromPod);
peonClient.waitForPeonJobCompletion(taskId, 2, TimeUnit.MINUTES); JobResponse jobStatusResult = peonClient.waitForPeonJobCompletion(taskId, 2, TimeUnit.MINUTES);
thread.join(); thread.join();
assertEquals(PeonPhase.SUCCEEDED, jobStatusResult.getPhase());
// as long as there were no exceptions we are good! // as long as there were no exceptions we are good!
assertEquals(expectedLogs, actualLogs); assertEquals(expectedLogs, actualLogs);
// cleanup my job // cleanup my job

View File

@ -62,4 +62,4 @@ spec:
ephemeral-storage: 1Gi ephemeral-storage: 1Gi
hostname: "id-3e70afe5cd823dfc7dd308eea616426b" hostname: "id-3e70afe5cd823dfc7dd308eea616426b"
restartPolicy: "Never" restartPolicy: "Never"
ttlSecondsAfterFinished: 3600 ttlSecondsAfterFinished: 172800

View File

@ -105,4 +105,4 @@ spec:
name: "graveyard" name: "graveyard"
- emptyDir: {} - emptyDir: {}
name: "kubexit" name: "kubexit"
ttlSecondsAfterFinished: 3600 ttlSecondsAfterFinished: 172800

View File

@ -105,4 +105,4 @@ spec:
name: "graveyard" name: "graveyard"
- emptyDir: {} - emptyDir: {}
name: "kubexit" name: "kubexit"
ttlSecondsAfterFinished: 3600 ttlSecondsAfterFinished: 172800

View File

@ -16,7 +16,7 @@ metadata:
spec: spec:
activeDeadlineSeconds: 14400 activeDeadlineSeconds: 14400
backoffLimit: 0 backoffLimit: 0
ttlSecondsAfterFinished: 3600 ttlSecondsAfterFinished: 172800
template: template:
metadata: metadata:
labels: labels:

View File

@ -16,7 +16,7 @@ metadata:
spec: spec:
activeDeadlineSeconds: 14400 activeDeadlineSeconds: 14400
backoffLimit: 0 backoffLimit: 0
ttlSecondsAfterFinished: 3600 ttlSecondsAfterFinished: 172800
template: template:
metadata: metadata:
labels: labels:

View File

@ -16,7 +16,7 @@ metadata:
spec: spec:
activeDeadlineSeconds: 14400 activeDeadlineSeconds: 14400
backoffLimit: 0 backoffLimit: 0
ttlSecondsAfterFinished: 3600 ttlSecondsAfterFinished: 172800
template: template:
metadata: metadata:
labels: labels:

View File

@ -16,7 +16,7 @@ metadata:
spec: spec:
activeDeadlineSeconds: 14400 activeDeadlineSeconds: 14400
backoffLimit: 0 backoffLimit: 0
ttlSecondsAfterFinished: 3600 ttlSecondsAfterFinished: 172800
template: template:
metadata: metadata:
labels: labels:

View File

@ -104,4 +104,4 @@ spec:
name: "graveyard" name: "graveyard"
- emptyDir: {} - emptyDir: {}
name: "kubexit" name: "kubexit"
ttlSecondsAfterFinished: 3600 ttlSecondsAfterFinished: 172800

View File

@ -57,4 +57,4 @@ spec:
cpu: "1000m" cpu: "1000m"
memory: "2400000000" memory: "2400000000"
restartPolicy: "Never" restartPolicy: "Never"
ttlSecondsAfterFinished: 3600 ttlSecondsAfterFinished: 172800

View File

@ -23,15 +23,11 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.base.Optional;
import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskLocation;
import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.TaskRunner;
/* This class was added for mm-less ingestion in order to let the peon manage its own location lifecycle by submitting
actions to the overlord. https://github.com/apache/druid/pull/15133 moved this location logic to the overlord itself
so this Action is no longer needed. For backwards compatibility with old peons, this class was left in but can be deprecated
for a later druid release.
*/
@Deprecated
public class UpdateLocationAction implements TaskAction<Void> public class UpdateLocationAction implements TaskAction<Void>
{ {
@JsonIgnore @JsonIgnore
@ -62,6 +58,10 @@ public class UpdateLocationAction implements TaskAction<Void>
@Override @Override
public Void perform(Task task, TaskActionToolbox toolbox) public Void perform(Task task, TaskActionToolbox toolbox)
{ {
Optional<TaskRunner> taskRunner = toolbox.getTaskRunner();
if (taskRunner.isPresent()) {
taskRunner.get().updateLocation(task, taskLocation);
}
return null; return null;
} }

View File

@ -25,11 +25,13 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Objects; import com.google.common.base.Objects;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.druid.common.utils.IdUtils; import org.apache.druid.common.utils.IdUtils;
import org.apache.druid.indexer.TaskLocation;
import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskLock;
import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.actions.LockListAction; import org.apache.druid.indexing.common.actions.LockListAction;
import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.indexing.common.actions.UpdateLocationAction;
import org.apache.druid.indexing.common.actions.UpdateStatusAction; import org.apache.druid.indexing.common.actions.UpdateStatusAction;
import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.FileUtils;
import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.IAE;
@ -40,12 +42,14 @@ import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.apache.druid.query.Query; import org.apache.druid.query.Query;
import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunner;
import org.apache.druid.segment.indexing.BatchIOConfig; import org.apache.druid.segment.indexing.BatchIOConfig;
import org.apache.druid.server.DruidNode;
import org.joda.time.Interval; import org.joda.time.Interval;
import javax.annotation.Nonnull; import javax.annotation.Nonnull;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.net.InetAddress;
import java.nio.file.Files; import java.nio.file.Files;
import java.nio.file.Paths; import java.nio.file.Paths;
import java.util.HashMap; import java.util.HashMap;
@ -151,6 +155,11 @@ public abstract class AbstractTask implements Task
FileUtils.mkdirp(attemptDir); FileUtils.mkdirp(attemptDir);
reportsFile = new File(attemptDir, "report.json"); reportsFile = new File(attemptDir, "report.json");
statusFile = new File(attemptDir, "status.json"); statusFile = new File(attemptDir, "status.json");
InetAddress hostName = InetAddress.getLocalHost();
DruidNode node = toolbox.getTaskExecutorNode();
toolbox.getTaskActionClient().submit(new UpdateLocationAction(TaskLocation.create(
hostName.getHostAddress(), node.getPlaintextPort(), node.getTlsPort(), node.isEnablePlaintextPort()
)));
} }
log.debug("Task setup complete"); log.debug("Task setup complete");
return null; return null;
@ -203,6 +212,7 @@ public abstract class AbstractTask implements Task
// report back to the overlord // report back to the overlord
UpdateStatusAction status = new UpdateStatusAction("", taskStatusToReport); UpdateStatusAction status = new UpdateStatusAction("", taskStatusToReport);
toolbox.getTaskActionClient().submit(status); toolbox.getTaskActionClient().submit(status);
toolbox.getTaskActionClient().submit(new UpdateLocationAction(TaskLocation.unknown()));
if (reportsFile != null && reportsFile.exists()) { if (reportsFile != null && reportsFile.exists()) {
toolbox.getTaskLogPusher().pushTaskReports(id, reportsFile); toolbox.getTaskLogPusher().pushTaskReports(id, reportsFile);

View File

@ -141,6 +141,11 @@ public interface TaskRunner
// do nothing // do nothing
} }
default void updateLocation(Task task, TaskLocation location)
{
// do nothing
}
/** /**
* The maximum number of tasks this TaskRunner can run concurrently. * The maximum number of tasks this TaskRunner can run concurrently.
* Can return -1 if this method is not implemented or capacity can't be found. * Can return -1 if this method is not implemented or capacity can't be found.

View File

@ -0,0 +1,71 @@
/*
* 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.indexing.common.actions;
import com.google.common.base.Optional;
import org.apache.druid.indexer.TaskLocation;
import org.apache.druid.indexing.common.task.NoopTask;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.TaskRunner;
import org.junit.Test;
import java.net.InetAddress;
import java.net.UnknownHostException;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
public class UpdateLocationActionTest
{
@Test
public void testFlow() throws UnknownHostException
{
// get my task location
InetAddress hostName = InetAddress.getLocalHost();
TaskLocation myLocation = TaskLocation.create(hostName.getHostAddress(), 1, 2);
UpdateLocationAction action = new UpdateLocationAction(myLocation);
Task task = NoopTask.create();
TaskActionToolbox toolbox = mock(TaskActionToolbox.class);
TaskRunner runner = mock(TaskRunner.class);
when(toolbox.getTaskRunner()).thenReturn(Optional.of(runner));
action.perform(task, toolbox);
verify(runner, times(1)).updateLocation(eq(task), eq(myLocation));
}
@Test
public void testWithNoTaskRunner() throws UnknownHostException
{
// get my task location
InetAddress hostName = InetAddress.getLocalHost();
TaskLocation myLocation = TaskLocation.create(hostName.getHostAddress(), 1, 2);
UpdateLocationAction action = new UpdateLocationAction(myLocation);
Task task = NoopTask.create();
TaskActionToolbox toolbox = mock(TaskActionToolbox.class);
TaskRunner runner = mock(TaskRunner.class);
when(toolbox.getTaskRunner()).thenReturn(Optional.absent());
action.perform(task, toolbox);
verify(runner, never()).updateStatus(any(), any());
}
}

View File

@ -109,7 +109,7 @@ public class AbstractTaskTest
task.run(toolbox); task.run(toolbox);
// call it 3 times, once to update location in setup, then one for status and location in cleanup // call it 3 times, once to update location in setup, then one for status and location in cleanup
Mockito.verify(taskActionClient, times(1)).submit(any()); Mockito.verify(taskActionClient, times(3)).submit(any());
verify(pusher, times(1)).pushTaskReports(eq("myID"), any()); verify(pusher, times(1)).pushTaskReports(eq("myID"), any());
verify(pusher, times(1)).pushTaskStatus(eq("myID"), any()); verify(pusher, times(1)).pushTaskStatus(eq("myID"), any());
} }