Browse Source

[Feature-13763][K8S Task] collect real-time log (#14379)

* [Improvement-13763][K8S Task] collect real-time log

* fix codesmell

* get pod watcher until pod is ready

* fix codesmell

* specify container name & loop waiting pod creation

* sleep when pod is not initialized

---------

Co-authored-by: Jay Chung <zhongjiajie955@gmail.com>
3.2.1-prepare
Aaron Wang 1 year ago committed by GitHub
parent
commit
04800a48d1
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 4
      dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractCommandExecutor.java
  2. 57
      dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/am/KubernetesApplicationManager.java
  3. 7
      dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/am/KubernetesApplicationManagerContext.java
  4. 2
      dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/k8s/AbstractK8sTask.java
  5. 12
      dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/k8s/AbstractK8sTaskExecutor.java
  6. 116
      dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/k8s/impl/K8sTaskExecutor.java
  7. 22
      dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/utils/ProcessUtils.java
  8. 9
      dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/test/java/org/apache/dolphinscheduler/plugin/task/api/k8s/K8sTaskExecutorTest.java

4
dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractCommandExecutor.java

@ -201,7 +201,7 @@ public abstract class AbstractCommandExecutor {
// Wait the task log process finished. // Wait the task log process finished.
taskOutputFuture.get(); taskOutputFuture.get();
} catch (ExecutionException e) { } catch (ExecutionException e) {
logger.info("Handle task log error", e); logger.error("Handle task log error", e);
} }
} }
@ -272,7 +272,7 @@ public abstract class AbstractCommandExecutor {
ThreadUtils.sleep(SLEEP_TIME_MILLIS * 5L); ThreadUtils.sleep(SLEEP_TIME_MILLIS * 5L);
try ( try (
LogWatch watcher = ProcessUtils.getPodLogWatcher(taskRequest.getK8sTaskExecutionContext(), LogWatch watcher = ProcessUtils.getPodLogWatcher(taskRequest.getK8sTaskExecutionContext(),
taskRequest.getTaskAppId())) { taskRequest.getTaskAppId(), "")) {
if (watcher == null) { if (watcher == null) {
throw new RuntimeException("The driver pod does not exist."); throw new RuntimeException("The driver pod does not exist.");
} else { } else {

57
dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/am/KubernetesApplicationManager.java

@ -17,9 +17,11 @@
package org.apache.dolphinscheduler.plugin.task.api.am; package org.apache.dolphinscheduler.plugin.task.api.am;
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.SLEEP_TIME_MILLIS;
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.UNIQUE_LABEL_NAME; import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.UNIQUE_LABEL_NAME;
import org.apache.dolphinscheduler.common.enums.ResourceManagerType; import org.apache.dolphinscheduler.common.enums.ResourceManagerType;
import org.apache.dolphinscheduler.common.thread.ThreadUtils;
import org.apache.dolphinscheduler.plugin.task.api.K8sTaskExecutionContext; import org.apache.dolphinscheduler.plugin.task.api.K8sTaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.TaskException; import org.apache.dolphinscheduler.plugin.task.api.TaskException;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
@ -54,6 +56,8 @@ public class KubernetesApplicationManager implements ApplicationManager {
private static final String FAILED = "Failed"; private static final String FAILED = "Failed";
private static final String UNKNOWN = "Unknown"; private static final String UNKNOWN = "Unknown";
private static final int MAX_RETRY_TIMES = 10;
/** /**
* cache k8s client for same task * cache k8s client for same task
*/ */
@ -67,7 +71,7 @@ public class KubernetesApplicationManager implements ApplicationManager {
boolean isKill; boolean isKill;
String labelValue = kubernetesApplicationManagerContext.getLabelValue(); String labelValue = kubernetesApplicationManagerContext.getLabelValue();
FilterWatchListDeletable<Pod, PodList, PodResource> watchList = FilterWatchListDeletable<Pod, PodList, PodResource> watchList =
getDriverPod(kubernetesApplicationManagerContext); getListenPod(kubernetesApplicationManagerContext);
try { try {
if (getApplicationStatus(kubernetesApplicationManagerContext, watchList).isFailure()) { if (getApplicationStatus(kubernetesApplicationManagerContext, watchList).isFailure()) {
log.error("Driver pod is in FAILED or UNKNOWN status."); log.error("Driver pod is in FAILED or UNKNOWN status.");
@ -97,16 +101,24 @@ public class KubernetesApplicationManager implements ApplicationManager {
* @param kubernetesApplicationManagerContext * @param kubernetesApplicationManagerContext
* @return * @return
*/ */
private FilterWatchListDeletable<Pod, PodList, PodResource> getDriverPod(KubernetesApplicationManagerContext kubernetesApplicationManagerContext) { private FilterWatchListDeletable<Pod, PodList, PodResource> getListenPod(KubernetesApplicationManagerContext kubernetesApplicationManagerContext) {
KubernetesClient client = getClient(kubernetesApplicationManagerContext); KubernetesClient client = getClient(kubernetesApplicationManagerContext);
String labelValue = kubernetesApplicationManagerContext.getLabelValue(); String labelValue = kubernetesApplicationManagerContext.getLabelValue();
FilterWatchListDeletable<Pod, PodList, PodResource> watchList = client.pods() List<Pod> podList = null;
.inNamespace(kubernetesApplicationManagerContext.getK8sTaskExecutionContext().getNamespace()) FilterWatchListDeletable<Pod, PodList, PodResource> watchList = null;
.withLabel(UNIQUE_LABEL_NAME, labelValue); int retryTimes = 0;
List<Pod> podList = watchList.list().getItems(); while (CollectionUtils.isEmpty(podList) && retryTimes < MAX_RETRY_TIMES) {
if (podList.size() != 1) { watchList = client.pods()
log.warn("Expected driver pod 1, but get {}.", podList.size()); .inNamespace(kubernetesApplicationManagerContext.getK8sTaskExecutionContext().getNamespace())
.withLabel(UNIQUE_LABEL_NAME, labelValue);
podList = watchList.list().getItems();
if (!CollectionUtils.isEmpty(podList)) {
break;
}
ThreadUtils.sleep(SLEEP_TIME_MILLIS);
retryTimes += 1;
} }
return watchList; return watchList;
} }
@ -153,7 +165,7 @@ public class KubernetesApplicationManager implements ApplicationManager {
String phase; String phase;
try { try {
if (Objects.isNull(watchList)) { if (Objects.isNull(watchList)) {
watchList = getDriverPod(kubernetesApplicationManagerContext); watchList = getListenPod(kubernetesApplicationManagerContext);
} }
List<Pod> driverPod = watchList.list().getItems(); List<Pod> driverPod = watchList.list().getItems();
if (!driverPod.isEmpty()) { if (!driverPod.isEmpty()) {
@ -180,16 +192,27 @@ public class KubernetesApplicationManager implements ApplicationManager {
*/ */
public LogWatch getPodLogWatcher(KubernetesApplicationManagerContext kubernetesApplicationManagerContext) { public LogWatch getPodLogWatcher(KubernetesApplicationManagerContext kubernetesApplicationManagerContext) {
KubernetesClient client = getClient(kubernetesApplicationManagerContext); KubernetesClient client = getClient(kubernetesApplicationManagerContext);
FilterWatchListDeletable<Pod, PodList, PodResource> watchList = boolean podIsReady = false;
getDriverPod(kubernetesApplicationManagerContext); Pod pod = null;
List<Pod> driverPod = watchList.list().getItems(); while (!podIsReady) {
if (CollectionUtils.isEmpty(driverPod)) { FilterWatchListDeletable<Pod, PodList, PodResource> watchList =
return null; getListenPod(kubernetesApplicationManagerContext);
List<Pod> podList = watchList == null ? null : watchList.list().getItems();
if (CollectionUtils.isEmpty(podList)) {
return null;
}
pod = podList.get(0);
String phase = pod.getStatus().getPhase();
if (phase.equals(PENDING) || phase.equals(UNKNOWN)) {
ThreadUtils.sleep(SLEEP_TIME_MILLIS);
} else {
podIsReady = true;
}
} }
Pod driver = driverPod.get(0);
return client.pods().inNamespace(driver.getMetadata().getNamespace()) return client.pods().inNamespace(pod.getMetadata().getNamespace())
.withName(driver.getMetadata().getName()) .withName(pod.getMetadata().getName())
.inContainer(kubernetesApplicationManagerContext.getContainerName())
.watchLog(); .watchLog();
} }

7
dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/am/KubernetesApplicationManagerContext.java

@ -32,8 +32,13 @@ public class KubernetesApplicationManagerContext implements ApplicationManagerCo
private final K8sTaskExecutionContext k8sTaskExecutionContext; private final K8sTaskExecutionContext k8sTaskExecutionContext;
/** /**
* driver pod label value * pod label value
*/ */
private final String labelValue; private final String labelValue;
/**
* container name (optional)
*/
private final String containerName;
} }

2
dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/k8s/AbstractK8sTask.java

@ -48,7 +48,7 @@ public abstract class AbstractK8sTask extends AbstractRemoteTask {
setExitStatusCode(response.getExitStatusCode()); setExitStatusCode(response.getExitStatusCode());
setAppIds(response.getAppIds()); setAppIds(response.getAppIds());
} catch (Exception e) { } catch (Exception e) {
log.error("k8s task submit failed with error", e); log.error("k8s task submit failed with error");
exitStatusCode = -1; exitStatusCode = -1;
throw new TaskException("Execute k8s task error", e); throw new TaskException("Execute k8s task error", e);
} }

12
dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/k8s/AbstractK8sTaskExecutor.java

@ -17,8 +17,6 @@
package org.apache.dolphinscheduler.plugin.task.api.k8s; package org.apache.dolphinscheduler.plugin.task.api.k8s;
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.EXIT_CODE_FAILURE;
import org.apache.dolphinscheduler.plugin.task.api.TaskException; import org.apache.dolphinscheduler.plugin.task.api.TaskException;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.model.TaskResponse; import org.apache.dolphinscheduler.plugin.task.api.model.TaskResponse;
@ -32,14 +30,12 @@ public abstract class AbstractK8sTaskExecutor {
protected Logger log; protected Logger log;
protected TaskExecutionContext taskRequest; protected TaskExecutionContext taskRequest;
protected K8sUtils k8sUtils; protected K8sUtils k8sUtils;
protected StringBuilder logStringBuffer;
protected Yaml yaml; protected Yaml yaml;
protected AbstractK8sTaskExecutor(Logger log, TaskExecutionContext taskRequest) { protected AbstractK8sTaskExecutor(Logger log, TaskExecutionContext taskRequest) {
this.log = log; this.log = log;
this.taskRequest = taskRequest; this.taskRequest = taskRequest;
this.k8sUtils = new K8sUtils(); this.k8sUtils = new K8sUtils();
this.logStringBuffer = new StringBuilder();
this.yaml = new Yaml(); this.yaml = new Yaml();
} }
@ -53,14 +49,6 @@ public abstract class AbstractK8sTaskExecutor {
} }
} }
public void flushLog(TaskResponse taskResponse) {
if (logStringBuffer.length() != 0 && taskResponse.getExitStatusCode() == EXIT_CODE_FAILURE) {
log.error(logStringBuffer.toString());
} else if (logStringBuffer.length() != 0) {
log.info(logStringBuffer.toString());
}
}
public abstract void submitJob2k8s(String k8sParameterStr); public abstract void submitJob2k8s(String k8sParameterStr);
public abstract void stopJobOnK8s(String k8sParameterStr); public abstract void stopJobOnK8s(String k8sParameterStr);

116
dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/k8s/impl/K8sTaskExecutor.java

@ -30,7 +30,9 @@ import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.MI;
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.NAME_LABEL; import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.NAME_LABEL;
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.RESTART_POLICY; import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.RESTART_POLICY;
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_INSTANCE_ID; import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_INSTANCE_ID;
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.UNIQUE_LABEL_NAME;
import org.apache.dolphinscheduler.common.thread.ThreadUtils;
import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.plugin.task.api.K8sTaskExecutionContext; import org.apache.dolphinscheduler.plugin.task.api.K8sTaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.TaskConstants; import org.apache.dolphinscheduler.plugin.task.api.TaskConstants;
@ -41,16 +43,23 @@ import org.apache.dolphinscheduler.plugin.task.api.enums.TaskTimeoutStrategy;
import org.apache.dolphinscheduler.plugin.task.api.k8s.AbstractK8sTaskExecutor; import org.apache.dolphinscheduler.plugin.task.api.k8s.AbstractK8sTaskExecutor;
import org.apache.dolphinscheduler.plugin.task.api.k8s.K8sTaskMainParameters; import org.apache.dolphinscheduler.plugin.task.api.k8s.K8sTaskMainParameters;
import org.apache.dolphinscheduler.plugin.task.api.model.TaskResponse; import org.apache.dolphinscheduler.plugin.task.api.model.TaskResponse;
import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
import org.apache.dolphinscheduler.plugin.task.api.utils.MapUtils; import org.apache.dolphinscheduler.plugin.task.api.utils.MapUtils;
import org.apache.dolphinscheduler.plugin.task.api.utils.ProcessUtils;
import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.StringUtils;
import java.io.BufferedReader;
import java.io.InputStreamReader;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Locale; import java.util.Locale;
import java.util.Map; import java.util.Map;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import org.slf4j.Logger; import org.slf4j.Logger;
@ -67,6 +76,7 @@ import io.fabric8.kubernetes.api.model.batch.v1.JobStatus;
import io.fabric8.kubernetes.client.Watch; import io.fabric8.kubernetes.client.Watch;
import io.fabric8.kubernetes.client.Watcher; import io.fabric8.kubernetes.client.Watcher;
import io.fabric8.kubernetes.client.WatcherException; import io.fabric8.kubernetes.client.WatcherException;
import io.fabric8.kubernetes.client.dsl.LogWatch;
/** /**
* K8sTaskExecutor used to submit k8s task to K8S * K8sTaskExecutor used to submit k8s task to K8S
@ -74,6 +84,9 @@ import io.fabric8.kubernetes.client.WatcherException;
public class K8sTaskExecutor extends AbstractK8sTaskExecutor { public class K8sTaskExecutor extends AbstractK8sTaskExecutor {
private Job job; private Job job;
protected boolean podLogOutputIsFinished = false;
protected Future<?> podLogOutputFuture;
public K8sTaskExecutor(Logger logger, TaskExecutionContext taskRequest) { public K8sTaskExecutor(Logger logger, TaskExecutionContext taskRequest) {
super(logger, taskRequest); super(logger, taskRequest);
} }
@ -100,6 +113,8 @@ public class K8sTaskExecutor extends AbstractK8sTaskExecutor {
Map<String, String> labelMap = k8STaskMainParameters.getLabelMap(); Map<String, String> labelMap = k8STaskMainParameters.getLabelMap();
labelMap.put(LAYER_LABEL, LAYER_LABEL_VALUE); labelMap.put(LAYER_LABEL, LAYER_LABEL_VALUE);
labelMap.put(NAME_LABEL, k8sJobName); labelMap.put(NAME_LABEL, k8sJobName);
Map<String, String> podLabelMap = new HashMap<>();
podLabelMap.put(UNIQUE_LABEL_NAME, taskRequest.getTaskAppId());
EnvVar taskInstanceIdVar = new EnvVar(TASK_INSTANCE_ID, taskInstanceId, null); EnvVar taskInstanceIdVar = new EnvVar(TASK_INSTANCE_ID, taskInstanceId, null);
List<EnvVar> envVars = new ArrayList<>(); List<EnvVar> envVars = new ArrayList<>();
envVars.add(taskInstanceIdVar); envVars.add(taskInstanceIdVar);
@ -150,6 +165,9 @@ public class K8sTaskExecutor extends AbstractK8sTaskExecutor {
.withNewSpec() .withNewSpec()
.withTtlSecondsAfterFinished(JOB_TTL_SECONDS) .withTtlSecondsAfterFinished(JOB_TTL_SECONDS)
.withNewTemplate() .withNewTemplate()
.withNewMetadata()
.withLabels(podLabelMap)
.endMetadata()
.withNewSpec() .withNewSpec()
.addNewContainer() .addNewContainer()
.withName(k8sJobName) .withName(k8sJobName)
@ -170,36 +188,36 @@ public class K8sTaskExecutor extends AbstractK8sTaskExecutor {
return jobBuilder.build(); return jobBuilder.build();
} }
public void registerBatchJobWatcher(Job job, String taskInstanceId, TaskResponse taskResponse, public void registerBatchJobWatcher(Job job, String taskInstanceId, TaskResponse taskResponse) {
K8sTaskMainParameters k8STaskMainParameters) {
CountDownLatch countDownLatch = new CountDownLatch(1); CountDownLatch countDownLatch = new CountDownLatch(1);
Watcher<Job> watcher = new Watcher<Job>() { Watcher<Job> watcher = new Watcher<Job>() {
@Override @Override
public void eventReceived(Action action, Job job) { public void eventReceived(Action action, Job job) {
log.info("event received : job:{} action:{}", job.getMetadata().getName(), action); try (
if (action != Action.ADDED) { final LogUtils.MDCAutoClosableContext mdcAutoClosableContext =
int jobStatus = getK8sJobStatus(job); LogUtils.setTaskInstanceLogFullPathMDC(taskRequest.getLogPath())) {
log.info("job {} status {}", job.getMetadata().getName(), jobStatus); log.info("event received : job:{} action:{}", job.getMetadata().getName(), action);
if (jobStatus == TaskConstants.RUNNING_CODE) { if (action != Action.ADDED) {
return; int jobStatus = getK8sJobStatus(job);
log.info("job {} status {}", job.getMetadata().getName(), jobStatus);
if (jobStatus == TaskConstants.RUNNING_CODE) {
return;
}
setTaskStatus(jobStatus, taskInstanceId, taskResponse);
countDownLatch.countDown();
} }
setTaskStatus(jobStatus, taskInstanceId, taskResponse, k8STaskMainParameters);
countDownLatch.countDown();
} }
} }
@Override @Override
public void onClose(WatcherException e) { public void onClose(WatcherException e) {
logStringBuffer.append(String.format("[K8sJobExecutor-%s] fail in k8s: %s", job.getMetadata().getName(), log.error("[K8sJobExecutor-{}] fail in k8s: {}", job.getMetadata().getName(), e.getMessage());
e.getMessage()));
taskResponse.setExitStatusCode(EXIT_CODE_FAILURE); taskResponse.setExitStatusCode(EXIT_CODE_FAILURE);
countDownLatch.countDown(); countDownLatch.countDown();
} }
}; };
Watch watch = null; try (Watch watch = k8sUtils.createBatchJobWatcher(job.getMetadata().getName(), watcher)) {
try {
watch = k8sUtils.createBatchJobWatcher(job.getMetadata().getName(), watcher);
boolean timeoutFlag = taskRequest.getTaskTimeoutStrategy() == TaskTimeoutStrategy.FAILED boolean timeoutFlag = taskRequest.getTaskTimeoutStrategy() == TaskTimeoutStrategy.FAILED
|| taskRequest.getTaskTimeoutStrategy() == TaskTimeoutStrategy.WARNFAILED; || taskRequest.getTaskTimeoutStrategy() == TaskTimeoutStrategy.WARNFAILED;
if (timeoutFlag) { if (timeoutFlag) {
@ -208,7 +226,6 @@ public class K8sTaskExecutor extends AbstractK8sTaskExecutor {
} else { } else {
countDownLatch.await(); countDownLatch.await();
} }
flushLog(taskResponse);
} catch (InterruptedException e) { } catch (InterruptedException e) {
log.error("job failed in k8s: {}", e.getMessage(), e); log.error("job failed in k8s: {}", e.getMessage(), e);
Thread.currentThread().interrupt(); Thread.currentThread().interrupt();
@ -216,19 +233,42 @@ public class K8sTaskExecutor extends AbstractK8sTaskExecutor {
} catch (Exception e) { } catch (Exception e) {
log.error("job failed in k8s: {}", e.getMessage(), e); log.error("job failed in k8s: {}", e.getMessage(), e);
taskResponse.setExitStatusCode(EXIT_CODE_FAILURE); taskResponse.setExitStatusCode(EXIT_CODE_FAILURE);
} finally {
if (watch != null) {
watch.close();
}
} }
} }
private void parsePodLogOutput() {
ExecutorService collectPodLogExecutorService = ThreadUtils
.newSingleDaemonScheduledExecutorService("CollectPodLogOutput-thread-" + taskRequest.getTaskName());
String taskInstanceId = String.valueOf(taskRequest.getTaskInstanceId());
String taskName = taskRequest.getTaskName().toLowerCase(Locale.ROOT);
String containerName = String.format("%s-%s", taskName, taskInstanceId);
podLogOutputFuture = collectPodLogExecutorService.submit(() -> {
try (
final LogUtils.MDCAutoClosableContext mdcAutoClosableContext =
LogUtils.setTaskInstanceLogFullPathMDC(taskRequest.getLogPath());
LogWatch watcher = ProcessUtils.getPodLogWatcher(taskRequest.getK8sTaskExecutionContext(),
taskRequest.getTaskAppId(), containerName)) {
String line;
try (BufferedReader reader = new BufferedReader(new InputStreamReader(watcher.getOutput()))) {
while ((line = reader.readLine()) != null) {
log.info("[K8S-pod-log] {}", line);
}
}
} catch (Exception e) {
throw new RuntimeException(e);
} finally {
podLogOutputIsFinished = true;
}
});
collectPodLogExecutorService.shutdown();
}
@Override @Override
public TaskResponse run(String k8sParameterStr) throws Exception { public TaskResponse run(String k8sParameterStr) throws Exception {
TaskResponse result = new TaskResponse(); TaskResponse result = new TaskResponse();
int taskInstanceId = taskRequest.getTaskInstanceId(); int taskInstanceId = taskRequest.getTaskInstanceId();
K8sTaskMainParameters k8STaskMainParameters =
JSONUtils.parseObject(k8sParameterStr, K8sTaskMainParameters.class);
try { try {
if (null == TaskExecutionContextCacheManager.getByTaskInstanceId(taskInstanceId)) { if (null == TaskExecutionContextCacheManager.getByTaskInstanceId(taskInstanceId)) {
result.setExitStatusCode(EXIT_CODE_KILL); result.setExitStatusCode(EXIT_CODE_KILL);
@ -242,9 +282,20 @@ public class K8sTaskExecutor extends AbstractK8sTaskExecutor {
String configYaml = k8sTaskExecutionContext.getConfigYaml(); String configYaml = k8sTaskExecutionContext.getConfigYaml();
k8sUtils.buildClient(configYaml); k8sUtils.buildClient(configYaml);
submitJob2k8s(k8sParameterStr); submitJob2k8s(k8sParameterStr);
registerBatchJobWatcher(job, Integer.toString(taskInstanceId), result, k8STaskMainParameters); parsePodLogOutput();
registerBatchJobWatcher(job, Integer.toString(taskInstanceId), result);
if (podLogOutputFuture != null) {
try {
// Wait kubernetes pod log collection finished
podLogOutputFuture.get();
} catch (ExecutionException e) {
log.error("Handle pod log error", e);
}
}
} catch (Exception e) { } catch (Exception e) {
cancelApplication(k8sParameterStr); cancelApplication(k8sParameterStr);
Thread.currentThread().interrupt();
result.setExitStatusCode(EXIT_CODE_FAILURE); result.setExitStatusCode(EXIT_CODE_FAILURE);
throw e; throw e;
} }
@ -270,9 +321,9 @@ public class K8sTaskExecutor extends AbstractK8sTaskExecutor {
stopJobOnK8s(k8sParameterStr); stopJobOnK8s(k8sParameterStr);
String namespaceName = k8STaskMainParameters.getNamespaceName(); String namespaceName = k8STaskMainParameters.getNamespaceName();
k8sUtils.createJob(namespaceName, job); k8sUtils.createJob(namespaceName, job);
log.info("[K8sJobExecutor-{}-{}] submitted job successfully", taskName, taskInstanceId); log.info("[K8sJobExecutor-{}-{}] submitted job successfully", taskName, taskInstanceId);
} catch (Exception e) { } catch (Exception e) {
log.error("[K8sJobExecutor-{}-{}] fail to submit job", taskName, taskInstanceId); log.error("[K8sJobExecutor-{}-{}] fail to submit job", taskName, taskInstanceId);
throw new TaskException("K8sJobExecutor fail to submit job", e); throw new TaskException("K8sJobExecutor fail to submit job", e);
} }
} }
@ -288,7 +339,7 @@ public class K8sTaskExecutor extends AbstractK8sTaskExecutor {
k8sUtils.deleteJob(jobName, namespaceName); k8sUtils.deleteJob(jobName, namespaceName);
} }
} catch (Exception e) { } catch (Exception e) {
log.error("[K8sJobExecutor-{}] fail to stop job", jobName); log.error("[K8sJobExecutor-{}] fail to stop job", jobName);
throw new TaskException("K8sJobExecutor fail to stop job", e); throw new TaskException("K8sJobExecutor fail to stop job", e);
} }
} }
@ -304,21 +355,16 @@ public class K8sTaskExecutor extends AbstractK8sTaskExecutor {
} }
} }
public void setTaskStatus(int jobStatus, String taskInstanceId, TaskResponse taskResponse, public void setTaskStatus(int jobStatus, String taskInstanceId, TaskResponse taskResponse) {
K8sTaskMainParameters k8STaskMainParameters) {
if (jobStatus == EXIT_CODE_SUCCESS || jobStatus == EXIT_CODE_FAILURE) { if (jobStatus == EXIT_CODE_SUCCESS || jobStatus == EXIT_CODE_FAILURE) {
if (null == TaskExecutionContextCacheManager.getByTaskInstanceId(Integer.valueOf(taskInstanceId))) { if (null == TaskExecutionContextCacheManager.getByTaskInstanceId(Integer.valueOf(taskInstanceId))) {
logStringBuffer.append(String.format("[K8sJobExecutor-%s] killed", job.getMetadata().getName())); log.info("[K8sJobExecutor-{}] killed", job.getMetadata().getName());
taskResponse.setExitStatusCode(EXIT_CODE_KILL); taskResponse.setExitStatusCode(EXIT_CODE_KILL);
} else if (jobStatus == EXIT_CODE_SUCCESS) { } else if (jobStatus == EXIT_CODE_SUCCESS) {
logStringBuffer log.info("[K8sJobExecutor-{}] succeed in k8s", job.getMetadata().getName());
.append(String.format("[K8sJobExecutor-%s] succeed in k8s", job.getMetadata().getName()));
taskResponse.setExitStatusCode(EXIT_CODE_SUCCESS); taskResponse.setExitStatusCode(EXIT_CODE_SUCCESS);
} else { } else {
String errorMessage = log.error("[K8sJobExecutor-{}] fail in k8s", job.getMetadata().getName());
k8sUtils.getPodLog(job.getMetadata().getName(), k8STaskMainParameters.getNamespaceName());
logStringBuffer.append(String.format("[K8sJobExecutor-%s] fail in k8s: %s", job.getMetadata().getName(),
errorMessage));
taskResponse.setExitStatusCode(EXIT_CODE_FAILURE); taskResponse.setExitStatusCode(EXIT_CODE_FAILURE);
} }
} }

22
dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/utils/ProcessUtils.java

@ -142,12 +142,14 @@ public final class ProcessUtils {
*/ */
public static void cancelApplication(TaskExecutionContext taskExecutionContext) { public static void cancelApplication(TaskExecutionContext taskExecutionContext) {
try { try {
if (Objects.nonNull(taskExecutionContext.getK8sTaskExecutionContext()) && if (Objects.nonNull(taskExecutionContext.getK8sTaskExecutionContext())) {
!TASK_TYPE_SET_K8S.contains(taskExecutionContext.getTaskType())) { if (!TASK_TYPE_SET_K8S.contains(taskExecutionContext.getTaskType())) {
applicationManagerMap.get(ResourceManagerType.KUBERNETES) // Set empty container name for Spark on K8S task
.killApplication(new KubernetesApplicationManagerContext( applicationManagerMap.get(ResourceManagerType.KUBERNETES)
taskExecutionContext.getK8sTaskExecutionContext(), .killApplication(new KubernetesApplicationManagerContext(
taskExecutionContext.getTaskAppId())); taskExecutionContext.getK8sTaskExecutionContext(),
taskExecutionContext.getTaskAppId(), ""));
}
} else { } else {
String host = taskExecutionContext.getHost(); String host = taskExecutionContext.getHost();
String executePath = taskExecutionContext.getExecutePath(); String executePath = taskExecutionContext.getExecutePath();
@ -197,7 +199,7 @@ public final class ProcessUtils {
KubernetesApplicationManager applicationManager = KubernetesApplicationManager applicationManager =
(KubernetesApplicationManager) applicationManagerMap.get(ResourceManagerType.KUBERNETES); (KubernetesApplicationManager) applicationManagerMap.get(ResourceManagerType.KUBERNETES);
return applicationManager return applicationManager
.getApplicationStatus(new KubernetesApplicationManagerContext(k8sTaskExecutionContext, taskAppId)); .getApplicationStatus(new KubernetesApplicationManagerContext(k8sTaskExecutionContext, taskAppId, ""));
} }
/** /**
@ -207,12 +209,14 @@ public final class ProcessUtils {
* @param taskAppId * @param taskAppId
* @return * @return
*/ */
public static LogWatch getPodLogWatcher(K8sTaskExecutionContext k8sTaskExecutionContext, String taskAppId) { public static LogWatch getPodLogWatcher(K8sTaskExecutionContext k8sTaskExecutionContext, String taskAppId,
String containerName) {
KubernetesApplicationManager applicationManager = KubernetesApplicationManager applicationManager =
(KubernetesApplicationManager) applicationManagerMap.get(ResourceManagerType.KUBERNETES); (KubernetesApplicationManager) applicationManagerMap.get(ResourceManagerType.KUBERNETES);
return applicationManager return applicationManager
.getPodLogWatcher(new KubernetesApplicationManagerContext(k8sTaskExecutionContext, taskAppId)); .getPodLogWatcher(
new KubernetesApplicationManagerContext(k8sTaskExecutionContext, taskAppId, containerName));
} }
} }

9
dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/test/java/org/apache/dolphinscheduler/plugin/task/api/k8s/K8sTaskExecutorTest.java

@ -34,6 +34,8 @@ import java.util.Map;
import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import io.fabric8.kubernetes.api.model.NodeSelectorRequirement; import io.fabric8.kubernetes.api.model.NodeSelectorRequirement;
import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.api.model.batch.v1.Job;
@ -41,6 +43,8 @@ import io.fabric8.kubernetes.api.model.batch.v1.JobStatus;
public class K8sTaskExecutorTest { public class K8sTaskExecutorTest {
private static final Logger logger = LoggerFactory.getLogger(K8sTaskExecutorTest.class);
private K8sTaskExecutor k8sTaskExecutor = null; private K8sTaskExecutor k8sTaskExecutor = null;
private K8sTaskMainParameters k8sTaskMainParameters = null; private K8sTaskMainParameters k8sTaskMainParameters = null;
private final String image = "ds-dev"; private final String image = "ds-dev";
@ -66,7 +70,7 @@ public class K8sTaskExecutorTest {
requirement.setKey("node-label"); requirement.setKey("node-label");
requirement.setOperator("In"); requirement.setOperator("In");
requirement.setValues(Arrays.asList("1234", "123456")); requirement.setValues(Arrays.asList("1234", "123456"));
k8sTaskExecutor = new K8sTaskExecutor(null, taskRequest); k8sTaskExecutor = new K8sTaskExecutor(logger, taskRequest);
k8sTaskMainParameters = new K8sTaskMainParameters(); k8sTaskMainParameters = new K8sTaskMainParameters();
k8sTaskMainParameters.setImage(image); k8sTaskMainParameters.setImage(image);
k8sTaskMainParameters.setImagePullPolicy(imagePullPolicy); k8sTaskMainParameters.setImagePullPolicy(imagePullPolicy);
@ -90,9 +94,8 @@ public class K8sTaskExecutorTest {
public void testSetTaskStatusNormal() { public void testSetTaskStatusNormal() {
int jobStatus = 0; int jobStatus = 0;
TaskResponse taskResponse = new TaskResponse(); TaskResponse taskResponse = new TaskResponse();
K8sTaskMainParameters k8STaskMainParameters = new K8sTaskMainParameters();
k8sTaskExecutor.setJob(job); k8sTaskExecutor.setJob(job);
k8sTaskExecutor.setTaskStatus(jobStatus, String.valueOf(taskInstanceId), taskResponse, k8STaskMainParameters); k8sTaskExecutor.setTaskStatus(jobStatus, String.valueOf(taskInstanceId), taskResponse);
Assertions.assertEquals(0, Integer.compare(EXIT_CODE_KILL, taskResponse.getExitStatusCode())); Assertions.assertEquals(0, Integer.compare(EXIT_CODE_KILL, taskResponse.getExitStatusCode()));
} }
@Test @Test

Loading…
Cancel
Save