Move some lifecycle management from doTask -> shutdown for the mm-less task runner (#14895)

* save work

* Add syncronized

* Don't shutdown in run

* Adding unit tests

* Cleanup lifecycle

* Fix tests

* remove newline
This commit is contained in:
George Shiqi Wu 2023-08-25 12:50:38 -04:00 committed by GitHub
parent ad32f84586
commit 95b0de61d1
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 52 additions and 68 deletions

View File

@ -137,7 +137,6 @@ public class KubernetesPeonLifecycle
} }
catch (Exception e) { catch (Exception e) {
log.info("Failed to run task: %s", taskId.getOriginalTaskId()); log.info("Failed to run task: %s", taskId.getOriginalTaskId());
shutdown();
throw e; throw e;
} }
finally { finally {
@ -168,10 +167,9 @@ public class KubernetesPeonLifecycle
finally { finally {
try { try {
saveLogs(); saveLogs();
shutdown();
} }
catch (Exception e) { catch (Exception e) {
log.warn(e, "Task [%s] cleanup failed", taskId); log.warn(e, "Log processing failed for task [%s]", taskId);
} }
stopTask(); stopTask();
@ -188,7 +186,7 @@ public class KubernetesPeonLifecycle
*/ */
protected void shutdown() protected void shutdown()
{ {
if (State.PENDING.equals(state.get()) || State.RUNNING.equals(state.get())) { if (State.PENDING.equals(state.get()) || State.RUNNING.equals(state.get()) || State.STOPPED.equals(state.get())) {
kubernetesClient.deletePeonJob(taskId); kubernetesClient.deletePeonJob(taskId);
} }
} }
@ -223,7 +221,7 @@ public class KubernetesPeonLifecycle
*/ */
protected TaskLocation getTaskLocation() protected TaskLocation getTaskLocation()
{ {
if (!State.RUNNING.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()); log.debug("Can't get task location for non-running job. [%s]", taskId.getOriginalTaskId());
return TaskLocation.unknown(); return TaskLocation.unknown();
} }
@ -251,7 +249,6 @@ public class KubernetesPeonLifecycle
Boolean.parseBoolean(pod.getMetadata().getAnnotations().getOrDefault(DruidK8sConstants.TLS_ENABLED, "false")), Boolean.parseBoolean(pod.getMetadata().getAnnotations().getOrDefault(DruidK8sConstants.TLS_ENABLED, "false")),
pod.getMetadata() != null ? pod.getMetadata().getName() : "" pod.getMetadata() != null ? pod.getMetadata().getName() : ""
); );
log.info("K8s task %s is running at location %s", taskId.getOriginalTaskId(), taskLocation);
} }
return taskLocation; return taskLocation;

View File

@ -182,10 +182,6 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner
KubernetesWorkItem workItem = tasks.get(task.getId()); KubernetesWorkItem workItem = tasks.get(task.getId());
if (workItem == null) { if (workItem == null) {
throw new ISE("Task [%s] disappeared", task.getId());
}
if (workItem.isShutdownRequested()) {
throw new ISE("Task [%s] has been shut down", task.getId()); throw new ISE("Task [%s] has been shut down", task.getId());
} }
@ -213,11 +209,6 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner
log.error(e, "Task [%s] execution caught an exception", task.getId()); log.error(e, "Task [%s] execution caught an exception", task.getId());
throw new RuntimeException(e); throw new RuntimeException(e);
} }
finally {
synchronized (tasks) {
tasks.remove(task.getId());
}
}
} }
@VisibleForTesting @VisibleForTesting
@ -271,6 +262,10 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner
return; return;
} }
synchronized (tasks) {
tasks.remove(taskid);
}
workItem.shutdown(); workItem.shutdown();
} }
@ -440,6 +435,17 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner
.collect(Collectors.toList()); .collect(Collectors.toList());
} }
@Override
public TaskLocation getTaskLocation(String taskId)
{
final KubernetesWorkItem workItem = tasks.get(taskId);
if (workItem == null) {
return TaskLocation.unknown();
} else {
return workItem.getLocation();
}
}
@Nullable @Nullable
@Override @Override
public RunnerTaskState getRunnerTaskState(String taskId) public RunnerTaskState getRunnerTaskState(String taskId)

View File

@ -30,13 +30,10 @@ import org.apache.druid.indexing.overlord.TaskRunnerWorkItem;
import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.ISE;
import java.io.InputStream; import java.io.InputStream;
import java.util.concurrent.atomic.AtomicBoolean;
public class KubernetesWorkItem extends TaskRunnerWorkItem public class KubernetesWorkItem extends TaskRunnerWorkItem
{ {
private final Task task; private final Task task;
private final AtomicBoolean shutdownRequested = new AtomicBoolean(false);
private KubernetesPeonLifecycle kubernetesPeonLifecycle = null; private KubernetesPeonLifecycle kubernetesPeonLifecycle = null;
public KubernetesWorkItem(Task task, ListenableFuture<TaskStatus> statusFuture) public KubernetesWorkItem(Task task, ListenableFuture<TaskStatus> statusFuture)
@ -53,7 +50,6 @@ public class KubernetesWorkItem extends TaskRunnerWorkItem
protected synchronized void shutdown() protected synchronized void shutdown()
{ {
this.shutdownRequested.set(true);
if (this.kubernetesPeonLifecycle != null) { if (this.kubernetesPeonLifecycle != null) {
this.kubernetesPeonLifecycle.startWatchingLogs(); this.kubernetesPeonLifecycle.startWatchingLogs();
@ -61,11 +57,6 @@ public class KubernetesWorkItem extends TaskRunnerWorkItem
} }
} }
protected boolean isShutdownRequested()
{
return shutdownRequested.get();
}
protected boolean isPending() protected boolean isPending()
{ {
return RunnerTaskState.PENDING.equals(getRunnerTaskState()); return RunnerTaskState.PENDING.equals(getRunnerTaskState());

View File

@ -198,9 +198,6 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
EasyMock.anyLong(), EasyMock.anyLong(),
EasyMock.eq(TimeUnit.MILLISECONDS) EasyMock.eq(TimeUnit.MILLISECONDS)
)).andReturn(null); )).andReturn(null);
EasyMock.expect(kubernetesClient.deletePeonJob(
new K8sTaskId(ID)
)).andReturn(true);
Assert.assertEquals(KubernetesPeonLifecycle.State.NOT_STARTED, peonLifecycle.getState()); Assert.assertEquals(KubernetesPeonLifecycle.State.NOT_STARTED, peonLifecycle.getState());
stateListener.stateChanged(KubernetesPeonLifecycle.State.PENDING, ID); stateListener.stateChanged(KubernetesPeonLifecycle.State.PENDING, ID);
EasyMock.expectLastCall().once(); EasyMock.expectLastCall().once();
@ -245,7 +242,6 @@ 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);
replayAll(); replayAll();
@ -298,7 +294,6 @@ 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());
@ -353,7 +348,6 @@ 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());
@ -408,7 +402,6 @@ 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());
@ -459,7 +452,6 @@ 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());
@ -512,7 +504,6 @@ 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());
@ -554,8 +545,6 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
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();
@ -908,8 +897,11 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport
stateListener stateListener
); );
setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.STOPPED); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.STOPPED);
EasyMock.expect(kubernetesClient.getPeonPod(k8sTaskId.getK8sJobName())).andReturn(Optional.absent()).once();
replayAll();
Assert.assertEquals(TaskLocation.unknown(), peonLifecycle.getTaskLocation()); Assert.assertEquals(TaskLocation.unknown(), peonLifecycle.getTaskLocation());
verifyAll();
} }
private void setPeonLifecycleState(KubernetesPeonLifecycle peonLifecycle, KubernetesPeonLifecycle.State state) private void setPeonLifecycleState(KubernetesPeonLifecycle peonLifecycle, KubernetesPeonLifecycle.State state)

View File

@ -152,8 +152,6 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
Assert.assertEquals(taskStatus, future.get()); Assert.assertEquals(taskStatus, future.get());
verifyAll(); verifyAll();
Assert.assertFalse(runner.tasks.containsKey(task.getId()));
} }
@Test @Test
@ -191,8 +189,6 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
Assert.assertTrue(e.getCause() instanceof RuntimeException); Assert.assertTrue(e.getCause() instanceof RuntimeException);
verifyAll(); verifyAll();
Assert.assertFalse(runner.tasks.containsKey(task.getId()));
} }
@Test @Test
@ -208,8 +204,6 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
Assert.assertEquals(taskStatus, future.get()); Assert.assertEquals(taskStatus, future.get());
verifyAll(); verifyAll();
Assert.assertFalse(runner.tasks.containsKey(task.getId()));
} }
@Test @Test
@ -236,28 +230,11 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
Assert.assertTrue(e.getCause() instanceof RuntimeException); Assert.assertTrue(e.getCause() instanceof RuntimeException);
verifyAll(); verifyAll();
Assert.assertFalse(runner.tasks.containsKey(task.getId()));
}
@Test
public void test_doTask_withoutWorkItem_throwsRuntimeException()
{
Assert.assertThrows(
"Task [id] disappeared",
RuntimeException.class,
() -> runner.doTask(task, true)
);
} }
@Test @Test
public void test_doTask_whenShutdownRequested_throwsRuntimeException() public void test_doTask_whenShutdownRequested_throwsRuntimeException()
{ {
KubernetesWorkItem workItem = new KubernetesWorkItem(task, null);
workItem.shutdown();
runner.tasks.put(task.getId(), workItem);
Assert.assertThrows( Assert.assertThrows(
"Task [id] has been shut down", "Task [id] has been shut down",
RuntimeException.class, RuntimeException.class,
@ -266,13 +243,7 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
} }
@Test @Test
public void test_shutdown_withoutExistingTask() public void test_shutdown_withExistingTask_removesTaskFromMap()
{
runner.shutdown(task.getId(), "");
}
@Test
public void test_shutdown_withExistingTask()
{ {
KubernetesWorkItem workItem = new KubernetesWorkItem(task, null) { KubernetesWorkItem workItem = new KubernetesWorkItem(task, null) {
@Override @Override
@ -282,7 +253,13 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
}; };
runner.tasks.put(task.getId(), workItem); runner.tasks.put(task.getId(), workItem);
runner.shutdown(task.getId(), "");
Assert.assertTrue(runner.tasks.isEmpty());
}
@Test
public void test_shutdown_withoutExistingTask()
{
runner.shutdown(task.getId(), ""); runner.shutdown(task.getId(), "");
} }
@ -629,6 +606,30 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
verifyAll(); verifyAll();
} }
@Test
public void test_getTaskLocation_withExistingTask()
{
KubernetesWorkItem workItem = new KubernetesWorkItem(task, null) {
@Override
public TaskLocation getLocation()
{
return TaskLocation.create("host", 0, 1, false);
}
};
runner.tasks.put(task.getId(), workItem);
TaskLocation taskLocation = runner.getTaskLocation(task.getId());
Assert.assertEquals(TaskLocation.create("host", 0, 1, false), taskLocation);
}
@Test
public void test_getTaskLocation_noTaskFound()
{
TaskLocation taskLocation = runner.getTaskLocation(task.getId());
Assert.assertEquals(TaskLocation.unknown(), taskLocation);
}
@Test @Test
public void test_getTotalCapacity() public void test_getTotalCapacity()
{ {
@ -644,6 +645,5 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
Assert.assertEquals(1, runner.getUsedCapacity()); Assert.assertEquals(1, runner.getUsedCapacity());
runner.tasks.remove(task.getId()); runner.tasks.remove(task.getId());
Assert.assertEquals(0, runner.getUsedCapacity()); Assert.assertEquals(0, runner.getUsedCapacity());
} }
} }

View File

@ -75,7 +75,6 @@ public class KubernetesWorkItemTest extends EasyMockSupport
public void test_shutdown_withoutKubernetesPeonLifecycle() public void test_shutdown_withoutKubernetesPeonLifecycle()
{ {
workItem.shutdown(); workItem.shutdown();
Assert.assertTrue(workItem.isShutdownRequested());
} }
@Test @Test
@ -91,7 +90,6 @@ public class KubernetesWorkItemTest extends EasyMockSupport
workItem.shutdown(); workItem.shutdown();
verifyAll(); verifyAll();
Assert.assertTrue(workItem.isShutdownRequested());
} }
@Test @Test