wuxiaofei
4 years ago
64 changed files with 1459 additions and 326 deletions
@ -0,0 +1,143 @@
|
||||
/* |
||||
* 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.dolphinscheduler.server.worker.runner; |
||||
|
||||
import org.apache.dolphinscheduler.common.enums.Event; |
||||
import org.apache.dolphinscheduler.common.enums.ExecutionStatus; |
||||
import org.apache.dolphinscheduler.common.thread.Stopper; |
||||
import org.apache.dolphinscheduler.common.thread.ThreadUtils; |
||||
import org.apache.dolphinscheduler.remote.command.TaskExecuteResponseCommand; |
||||
import org.apache.dolphinscheduler.server.entity.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.server.worker.cache.ResponceCache; |
||||
import org.apache.dolphinscheduler.server.worker.cache.TaskExecutionContextCacheManager; |
||||
import org.apache.dolphinscheduler.server.worker.cache.impl.TaskExecutionContextCacheManagerImpl; |
||||
import org.apache.dolphinscheduler.server.worker.config.WorkerConfig; |
||||
import org.apache.dolphinscheduler.server.worker.processor.TaskCallbackService; |
||||
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; |
||||
|
||||
import java.util.concurrent.DelayQueue; |
||||
import java.util.concurrent.ExecutorService; |
||||
|
||||
import org.slf4j.Logger; |
||||
import org.slf4j.LoggerFactory; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
/** |
||||
* Manage tasks |
||||
*/ |
||||
@Component |
||||
public class WorkerManagerThread implements Runnable { |
||||
|
||||
private final Logger logger = LoggerFactory.getLogger(WorkerManagerThread.class); |
||||
|
||||
/** |
||||
* task queue |
||||
*/ |
||||
private final DelayQueue<TaskExecuteThread> workerExecuteQueue = new DelayQueue<>(); |
||||
|
||||
/** |
||||
* worker config |
||||
*/ |
||||
private final WorkerConfig workerConfig; |
||||
|
||||
/** |
||||
* thread executor service |
||||
*/ |
||||
private final ExecutorService workerExecService; |
||||
|
||||
/** |
||||
* taskExecutionContextCacheManager |
||||
*/ |
||||
private TaskExecutionContextCacheManager taskExecutionContextCacheManager; |
||||
|
||||
/** |
||||
* task callback service |
||||
*/ |
||||
private final TaskCallbackService taskCallbackService; |
||||
|
||||
public WorkerManagerThread() { |
||||
this.workerConfig = SpringApplicationContext.getBean(WorkerConfig.class); |
||||
this.taskExecutionContextCacheManager = SpringApplicationContext.getBean(TaskExecutionContextCacheManagerImpl.class); |
||||
this.workerExecService = ThreadUtils.newDaemonFixedThreadExecutor("Worker-Execute-Thread", this.workerConfig.getWorkerExecThreads()); |
||||
this.taskCallbackService = SpringApplicationContext.getBean(TaskCallbackService.class); |
||||
} |
||||
|
||||
/** |
||||
* get queue size |
||||
* |
||||
* @return queue size |
||||
*/ |
||||
public int getQueueSize() { |
||||
return workerExecuteQueue.size(); |
||||
} |
||||
|
||||
/** |
||||
* Kill tasks that have not been executed, like delay task |
||||
* then send Response to Master, update the execution status of task instance |
||||
*/ |
||||
public void killTaskBeforeExecuteByInstanceId(Integer taskInstanceId) { |
||||
workerExecuteQueue.stream() |
||||
.filter(taskExecuteThread -> taskExecuteThread.getTaskExecutionContext().getTaskInstanceId() == taskInstanceId) |
||||
.forEach(workerExecuteQueue::remove); |
||||
sendTaskKillResponse(taskInstanceId); |
||||
} |
||||
|
||||
/** |
||||
* kill task before execute , like delay task |
||||
*/ |
||||
private void sendTaskKillResponse(Integer taskInstanceId) { |
||||
TaskExecutionContext taskExecutionContext = taskExecutionContextCacheManager.getByTaskInstanceId(taskInstanceId); |
||||
if (taskExecutionContext == null) { |
||||
return; |
||||
} |
||||
TaskExecuteResponseCommand responseCommand = new TaskExecuteResponseCommand(taskExecutionContext.getTaskInstanceId()); |
||||
responseCommand.setStatus(ExecutionStatus.KILL.getCode()); |
||||
ResponceCache.get().cache(taskExecutionContext.getTaskInstanceId(), responseCommand.convert2Command(), Event.RESULT); |
||||
taskCallbackService.sendResult(taskExecutionContext.getTaskInstanceId(), responseCommand.convert2Command()); |
||||
} |
||||
|
||||
/** |
||||
* submit task |
||||
* |
||||
* @param taskExecuteThread taskExecuteThread |
||||
* @return submit result |
||||
*/ |
||||
public boolean offer(TaskExecuteThread taskExecuteThread) { |
||||
return workerExecuteQueue.offer(taskExecuteThread); |
||||
} |
||||
|
||||
public void start() { |
||||
Thread thread = new Thread(this, this.getClass().getName()); |
||||
thread.start(); |
||||
} |
||||
|
||||
@Override |
||||
public void run() { |
||||
Thread.currentThread().setName("Worker-Execute-Manager-Thread"); |
||||
TaskExecuteThread taskExecuteThread; |
||||
while (Stopper.isRunning()) { |
||||
try { |
||||
taskExecuteThread = workerExecuteQueue.take(); |
||||
workerExecService.submit(taskExecuteThread); |
||||
} catch (Exception e) { |
||||
logger.error("An unexpected interrupt is happened, " |
||||
+ "the exception will be ignored and this thread will continue to run", e); |
||||
} |
||||
} |
||||
} |
||||
} |
@ -0,0 +1,195 @@
|
||||
/* |
||||
* 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.dolphinscheduler.server.worker.processor; |
||||
|
||||
import org.apache.dolphinscheduler.common.enums.ExecutionStatus; |
||||
import org.apache.dolphinscheduler.common.thread.ThreadUtils; |
||||
import org.apache.dolphinscheduler.common.utils.FileUtils; |
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils; |
||||
import org.apache.dolphinscheduler.common.utils.LoggerUtils; |
||||
import org.apache.dolphinscheduler.remote.command.Command; |
||||
import org.apache.dolphinscheduler.remote.command.CommandType; |
||||
import org.apache.dolphinscheduler.remote.command.TaskExecuteAckCommand; |
||||
import org.apache.dolphinscheduler.remote.command.TaskExecuteRequestCommand; |
||||
import org.apache.dolphinscheduler.remote.utils.ChannelUtils; |
||||
import org.apache.dolphinscheduler.remote.utils.JsonSerializer; |
||||
import org.apache.dolphinscheduler.server.entity.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.server.worker.cache.impl.TaskExecutionContextCacheManagerImpl; |
||||
import org.apache.dolphinscheduler.server.worker.config.WorkerConfig; |
||||
import org.apache.dolphinscheduler.server.worker.runner.TaskExecuteThread; |
||||
import org.apache.dolphinscheduler.server.worker.runner.WorkerManagerThread; |
||||
import org.apache.dolphinscheduler.service.alert.AlertClientService; |
||||
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; |
||||
|
||||
import java.util.Date; |
||||
import java.util.concurrent.ExecutorService; |
||||
|
||||
import org.junit.Assert; |
||||
import org.junit.Before; |
||||
import org.junit.Test; |
||||
import org.junit.runner.RunWith; |
||||
import org.mockito.Mockito; |
||||
import org.powermock.api.mockito.PowerMockito; |
||||
import org.powermock.core.classloader.annotations.PrepareForTest; |
||||
import org.powermock.modules.junit4.PowerMockRunner; |
||||
import org.slf4j.Logger; |
||||
import org.slf4j.LoggerFactory; |
||||
|
||||
/** |
||||
* test task execute processor |
||||
*/ |
||||
@RunWith(PowerMockRunner.class) |
||||
@PrepareForTest({SpringApplicationContext.class, TaskCallbackService.class, WorkerConfig.class, FileUtils.class, |
||||
JsonSerializer.class, JSONUtils.class, ThreadUtils.class, ExecutorService.class, ChannelUtils.class}) |
||||
public class TaskExecuteProcessorTest { |
||||
|
||||
private TaskExecutionContext taskExecutionContext; |
||||
|
||||
private TaskCallbackService taskCallbackService; |
||||
|
||||
private ExecutorService workerExecService; |
||||
|
||||
private WorkerConfig workerConfig; |
||||
|
||||
private Command command; |
||||
|
||||
private Command ackCommand; |
||||
|
||||
private TaskExecuteRequestCommand taskRequestCommand; |
||||
|
||||
private TaskExecutionContextCacheManagerImpl taskExecutionContextCacheManager; |
||||
|
||||
private AlertClientService alertClientService; |
||||
|
||||
private WorkerManagerThread workerManager; |
||||
|
||||
@Before |
||||
public void before() throws Exception { |
||||
// init task execution context
|
||||
taskExecutionContext = getTaskExecutionContext(); |
||||
workerConfig = new WorkerConfig(); |
||||
workerConfig.setWorkerExecThreads(1); |
||||
workerConfig.setListenPort(1234); |
||||
command = new Command(); |
||||
command.setType(CommandType.TASK_EXECUTE_REQUEST); |
||||
ackCommand = new TaskExecuteAckCommand().convert2Command(); |
||||
taskRequestCommand = new TaskExecuteRequestCommand(); |
||||
alertClientService = PowerMockito.mock(AlertClientService.class); |
||||
workerExecService = PowerMockito.mock(ExecutorService.class); |
||||
PowerMockito.when(workerExecService.submit(Mockito.any(TaskExecuteThread.class))) |
||||
.thenReturn(null); |
||||
|
||||
PowerMockito.mockStatic(ChannelUtils.class); |
||||
PowerMockito.when(ChannelUtils.toAddress(null)).thenReturn(null); |
||||
|
||||
taskExecutionContextCacheManager = PowerMockito.mock(TaskExecutionContextCacheManagerImpl.class); |
||||
taskCallbackService = PowerMockito.mock(TaskCallbackService.class); |
||||
PowerMockito.doNothing().when(taskCallbackService).sendAck(taskExecutionContext.getTaskInstanceId(), ackCommand); |
||||
|
||||
PowerMockito.mockStatic(SpringApplicationContext.class); |
||||
PowerMockito.when(SpringApplicationContext.getBean(TaskCallbackService.class)) |
||||
.thenReturn(taskCallbackService); |
||||
PowerMockito.when(SpringApplicationContext.getBean(WorkerConfig.class)) |
||||
.thenReturn(workerConfig); |
||||
PowerMockito.when(SpringApplicationContext.getBean(TaskExecutionContextCacheManagerImpl.class)) |
||||
.thenReturn(null); |
||||
PowerMockito.when(SpringApplicationContext.getBean(TaskExecutionContextCacheManagerImpl.class)) |
||||
.thenReturn(taskExecutionContextCacheManager); |
||||
|
||||
Logger taskLogger = LoggerFactory.getLogger(LoggerUtils.buildTaskId(LoggerUtils.TASK_LOGGER_INFO_PREFIX, |
||||
taskExecutionContext.getProcessDefineId(), |
||||
taskExecutionContext.getProcessInstanceId(), |
||||
taskExecutionContext.getTaskInstanceId())); |
||||
|
||||
workerManager = PowerMockito.mock(WorkerManagerThread.class); |
||||
PowerMockito.when(workerManager.offer(new TaskExecuteThread(taskExecutionContext, taskCallbackService, taskLogger, alertClientService))).thenReturn(Boolean.TRUE); |
||||
|
||||
PowerMockito.when(SpringApplicationContext.getBean(WorkerManagerThread.class)) |
||||
.thenReturn(workerManager); |
||||
|
||||
PowerMockito.mockStatic(ThreadUtils.class); |
||||
PowerMockito.when(ThreadUtils.newDaemonFixedThreadExecutor("Worker-Execute-Thread", workerConfig.getWorkerExecThreads())) |
||||
.thenReturn(workerExecService); |
||||
|
||||
PowerMockito.mockStatic(JsonSerializer.class); |
||||
PowerMockito.when(JsonSerializer.deserialize(command.getBody(), TaskExecuteRequestCommand.class)) |
||||
.thenReturn(taskRequestCommand); |
||||
|
||||
PowerMockito.mockStatic(JSONUtils.class); |
||||
PowerMockito.when(JSONUtils.parseObject(command.getBody(), TaskExecuteRequestCommand.class)) |
||||
.thenReturn(taskRequestCommand); |
||||
PowerMockito.when(JSONUtils.parseObject(taskRequestCommand.getTaskExecutionContext(), TaskExecutionContext.class)) |
||||
.thenReturn(taskExecutionContext); |
||||
|
||||
PowerMockito.mockStatic(FileUtils.class); |
||||
PowerMockito.when(FileUtils.getProcessExecDir(taskExecutionContext.getProjectId(), |
||||
taskExecutionContext.getProcessDefineId(), |
||||
taskExecutionContext.getProcessInstanceId(), |
||||
taskExecutionContext.getTaskInstanceId())) |
||||
.thenReturn(taskExecutionContext.getExecutePath()); |
||||
PowerMockito.doNothing().when(FileUtils.class, "createWorkDirIfAbsent", taskExecutionContext.getExecutePath()); |
||||
|
||||
SimpleTaskExecuteThread simpleTaskExecuteThread = new SimpleTaskExecuteThread(null, null, null, alertClientService); |
||||
PowerMockito.whenNew(TaskExecuteThread.class).withAnyArguments() |
||||
.thenReturn(simpleTaskExecuteThread); |
||||
} |
||||
|
||||
@Test |
||||
public void testNormalExecution() { |
||||
TaskExecuteProcessor processor = new TaskExecuteProcessor(); |
||||
processor.process(null, command); |
||||
|
||||
Assert.assertEquals(ExecutionStatus.RUNNING_EXECUTION, taskExecutionContext.getCurrentExecutionStatus()); |
||||
} |
||||
|
||||
@Test |
||||
public void testDelayExecution() { |
||||
taskExecutionContext.setDelayTime(1); |
||||
TaskExecuteProcessor processor = new TaskExecuteProcessor(); |
||||
processor.process(null, command); |
||||
|
||||
Assert.assertEquals(ExecutionStatus.DELAY_EXECUTION, taskExecutionContext.getCurrentExecutionStatus()); |
||||
} |
||||
|
||||
public TaskExecutionContext getTaskExecutionContext() { |
||||
TaskExecutionContext taskExecutionContext = new TaskExecutionContext(); |
||||
taskExecutionContext.setProcessId(12345); |
||||
taskExecutionContext.setProcessDefineId(1); |
||||
taskExecutionContext.setProcessInstanceId(1); |
||||
taskExecutionContext.setTaskInstanceId(1); |
||||
taskExecutionContext.setTaskType("sql"); |
||||
taskExecutionContext.setFirstSubmitTime(new Date()); |
||||
taskExecutionContext.setDelayTime(0); |
||||
taskExecutionContext.setLogPath("/tmp/test.log"); |
||||
taskExecutionContext.setHost("localhost"); |
||||
taskExecutionContext.setExecutePath("/tmp/dolphinscheduler/exec/process/1/2/3/4"); |
||||
return taskExecutionContext; |
||||
} |
||||
|
||||
private static class SimpleTaskExecuteThread extends TaskExecuteThread { |
||||
|
||||
public SimpleTaskExecuteThread(TaskExecutionContext taskExecutionContext, TaskCallbackService taskCallbackService, Logger taskLogger, AlertClientService alertClientService) { |
||||
super(taskExecutionContext, taskCallbackService, taskLogger, alertClientService); |
||||
} |
||||
|
||||
@Override |
||||
public void run() { |
||||
//
|
||||
} |
||||
} |
||||
} |
@ -0,0 +1,187 @@
|
||||
/* |
||||
* 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.dolphinscheduler.server.worker.runner; |
||||
|
||||
import org.apache.dolphinscheduler.common.enums.ExecutionStatus; |
||||
import org.apache.dolphinscheduler.common.model.TaskNode; |
||||
import org.apache.dolphinscheduler.common.task.AbstractParameters; |
||||
import org.apache.dolphinscheduler.common.thread.Stopper; |
||||
import org.apache.dolphinscheduler.common.utils.CommonUtils; |
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils; |
||||
import org.apache.dolphinscheduler.common.utils.LoggerUtils; |
||||
import org.apache.dolphinscheduler.common.utils.OSUtils; |
||||
import org.apache.dolphinscheduler.remote.command.Command; |
||||
import org.apache.dolphinscheduler.remote.command.TaskExecuteAckCommand; |
||||
import org.apache.dolphinscheduler.remote.command.TaskExecuteResponseCommand; |
||||
import org.apache.dolphinscheduler.server.entity.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.server.worker.cache.impl.TaskExecutionContextCacheManagerImpl; |
||||
import org.apache.dolphinscheduler.server.worker.config.WorkerConfig; |
||||
import org.apache.dolphinscheduler.server.worker.processor.TaskCallbackService; |
||||
import org.apache.dolphinscheduler.server.worker.task.AbstractTask; |
||||
import org.apache.dolphinscheduler.server.worker.task.TaskManager; |
||||
import org.apache.dolphinscheduler.service.alert.AlertClientService; |
||||
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; |
||||
|
||||
import java.util.Collections; |
||||
import java.util.Date; |
||||
import java.util.List; |
||||
|
||||
import org.junit.Assert; |
||||
import org.junit.Before; |
||||
import org.junit.Test; |
||||
import org.junit.runner.RunWith; |
||||
import org.powermock.api.mockito.PowerMockito; |
||||
import org.powermock.core.classloader.annotations.PrepareForTest; |
||||
import org.powermock.modules.junit4.PowerMockRunner; |
||||
import org.slf4j.Logger; |
||||
import org.slf4j.LoggerFactory; |
||||
|
||||
/** |
||||
* test worker manager thread. |
||||
*/ |
||||
@RunWith(PowerMockRunner.class) |
||||
@PrepareForTest({ |
||||
Stopper.class, |
||||
TaskManager.class, |
||||
JSONUtils.class, |
||||
CommonUtils.class, |
||||
SpringApplicationContext.class, |
||||
OSUtils.class}) |
||||
public class WorkerManagerThreadTest { |
||||
|
||||
private TaskCallbackService taskCallbackService; |
||||
|
||||
private WorkerManagerThread workerManager; |
||||
|
||||
private TaskExecutionContext taskExecutionContext; |
||||
|
||||
private AlertClientService alertClientService; |
||||
|
||||
private Logger taskLogger; |
||||
|
||||
@Before |
||||
public void before() { |
||||
// init task execution context, logger
|
||||
taskExecutionContext = new TaskExecutionContext(); |
||||
taskExecutionContext.setProcessId(12345); |
||||
taskExecutionContext.setProcessDefineId(1); |
||||
taskExecutionContext.setProcessInstanceId(1); |
||||
taskExecutionContext.setTaskInstanceId(1); |
||||
taskExecutionContext.setTenantCode("test"); |
||||
taskExecutionContext.setTaskType(""); |
||||
taskExecutionContext.setFirstSubmitTime(new Date()); |
||||
taskExecutionContext.setDelayTime(0); |
||||
taskExecutionContext.setLogPath("/tmp/test.log"); |
||||
taskExecutionContext.setHost("localhost"); |
||||
taskExecutionContext.setExecutePath("/tmp/dolphinscheduler/exec/process/1/2/3/4"); |
||||
|
||||
Command ackCommand = new TaskExecuteAckCommand().convert2Command(); |
||||
Command responseCommand = new TaskExecuteResponseCommand(taskExecutionContext.getTaskInstanceId()).convert2Command(); |
||||
|
||||
taskLogger = LoggerFactory.getLogger(LoggerUtils.buildTaskId( |
||||
LoggerUtils.TASK_LOGGER_INFO_PREFIX, |
||||
taskExecutionContext.getProcessDefineId(), |
||||
taskExecutionContext.getProcessInstanceId(), |
||||
taskExecutionContext.getTaskInstanceId() |
||||
)); |
||||
|
||||
TaskExecutionContextCacheManagerImpl taskExecutionContextCacheManager = new TaskExecutionContextCacheManagerImpl(); |
||||
taskExecutionContextCacheManager.cacheTaskExecutionContext(taskExecutionContext); |
||||
|
||||
alertClientService = PowerMockito.mock(AlertClientService.class); |
||||
WorkerConfig workerConfig = PowerMockito.mock(WorkerConfig.class); |
||||
taskCallbackService = PowerMockito.mock(TaskCallbackService.class); |
||||
PowerMockito.doNothing().when(taskCallbackService).sendAck(taskExecutionContext.getTaskInstanceId(), ackCommand); |
||||
PowerMockito.doNothing().when(taskCallbackService).sendResult(taskExecutionContext.getTaskInstanceId(), responseCommand); |
||||
PowerMockito.mockStatic(SpringApplicationContext.class); |
||||
PowerMockito.when(SpringApplicationContext.getBean(TaskExecutionContextCacheManagerImpl.class)) |
||||
.thenReturn(taskExecutionContextCacheManager); |
||||
PowerMockito.when(SpringApplicationContext.getBean(WorkerConfig.class)) |
||||
.thenReturn(workerConfig); |
||||
PowerMockito.when(SpringApplicationContext.getBean(TaskCallbackService.class)) |
||||
.thenReturn(taskCallbackService); |
||||
PowerMockito.when(workerConfig.getWorkerExecThreads()).thenReturn(5); |
||||
workerManager = new WorkerManagerThread(); |
||||
|
||||
PowerMockito.mockStatic(TaskManager.class); |
||||
PowerMockito.when(TaskManager.newTask(taskExecutionContext, taskLogger, alertClientService)) |
||||
.thenReturn(new SimpleTask(taskExecutionContext, taskLogger)); |
||||
PowerMockito.mockStatic(JSONUtils.class); |
||||
PowerMockito.when(JSONUtils.parseObject(taskExecutionContext.getTaskJson(), TaskNode.class)) |
||||
.thenReturn(new TaskNode()); |
||||
PowerMockito.mockStatic(CommonUtils.class); |
||||
PowerMockito.when(CommonUtils.getSystemEnvPath()).thenReturn("/user_home/.bash_profile"); |
||||
List<String> osUserList = Collections.singletonList("test"); |
||||
PowerMockito.mockStatic(OSUtils.class); |
||||
PowerMockito.when(OSUtils.getUserList()).thenReturn(osUserList); |
||||
PowerMockito.mockStatic(Stopper.class); |
||||
PowerMockito.when(Stopper.isRunning()).thenReturn(true, false); |
||||
} |
||||
|
||||
@Test |
||||
public void testSendTaskKillResponse() { |
||||
TaskExecuteThread taskExecuteThread = new TaskExecuteThread(taskExecutionContext, taskCallbackService, taskLogger, alertClientService); |
||||
workerManager.offer(taskExecuteThread); |
||||
Assert.assertEquals(1, workerManager.getQueueSize()); |
||||
workerManager.killTaskBeforeExecuteByInstanceId(1); |
||||
Assert.assertEquals(0, workerManager.getQueueSize()); |
||||
} |
||||
|
||||
@Test |
||||
public void testRun() { |
||||
TaskExecuteThread taskExecuteThread = new TaskExecuteThread(taskExecutionContext, taskCallbackService, taskLogger, alertClientService); |
||||
workerManager.offer(taskExecuteThread); |
||||
Assert.assertEquals(1, workerManager.getQueueSize()); |
||||
workerManager.run(); |
||||
Assert.assertEquals(0, workerManager.getQueueSize()); |
||||
} |
||||
|
||||
private static class SimpleTask extends AbstractTask { |
||||
|
||||
protected SimpleTask(TaskExecutionContext taskExecutionContext, Logger logger) { |
||||
super(taskExecutionContext, logger); |
||||
// pid
|
||||
this.processId = taskExecutionContext.getProcessId(); |
||||
} |
||||
|
||||
@Override |
||||
public AbstractParameters getParameters() { |
||||
return null; |
||||
} |
||||
|
||||
@Override |
||||
public void init() { |
||||
|
||||
} |
||||
|
||||
@Override |
||||
public void handle() { |
||||
|
||||
} |
||||
|
||||
@Override |
||||
public void after() { |
||||
|
||||
} |
||||
|
||||
@Override |
||||
public ExecutionStatus getExitStatus() { |
||||
return ExecutionStatus.SUCCESS; |
||||
} |
||||
} |
||||
} |
@ -0,0 +1,53 @@
|
||||
/* |
||||
* 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.dolphinscheduler.server.worker.task; |
||||
|
||||
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; |
||||
|
||||
import org.junit.Before; |
||||
import org.junit.Test; |
||||
import org.junit.runner.RunWith; |
||||
import org.powermock.core.classloader.annotations.PrepareForTest; |
||||
import org.powermock.modules.junit4.PowerMockRunner; |
||||
import org.slf4j.Logger; |
||||
import org.slf4j.LoggerFactory; |
||||
|
||||
@RunWith(PowerMockRunner.class) |
||||
@PrepareForTest({SpringApplicationContext.class}) |
||||
public class AbstractCommandExecutorTest { |
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(AbstractCommandExecutorTest.class); |
||||
|
||||
private ShellCommandExecutor shellCommandExecutor; |
||||
|
||||
@Before |
||||
public void before() throws Exception { |
||||
System.setProperty("log4j2.disable.jmx", Boolean.TRUE.toString()); |
||||
shellCommandExecutor = new ShellCommandExecutor(null); |
||||
} |
||||
|
||||
@Test |
||||
public void testSetTaskResultString() { |
||||
shellCommandExecutor.setTaskResultString("shellReturn"); |
||||
} |
||||
|
||||
@Test |
||||
public void testGetTaskResultString() { |
||||
logger.info(shellCommandExecutor.getTaskResultString()); |
||||
} |
||||
} |
@ -0,0 +1,127 @@
|
||||
/* |
||||
* 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.dolphinscheduler.server.worker.task; |
||||
|
||||
import static org.mockito.ArgumentMatchers.anyString; |
||||
|
||||
import org.apache.dolphinscheduler.server.entity.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.server.worker.task.CommandExecuteResult; |
||||
import org.apache.dolphinscheduler.server.worker.task.ShellCommandExecutor; |
||||
import org.apache.dolphinscheduler.server.worker.task.shell.ShellTask; |
||||
import org.apache.dolphinscheduler.server.worker.task.shell.ShellTaskTest; |
||||
|
||||
import java.nio.file.Files; |
||||
import java.nio.file.Paths; |
||||
import java.util.Date; |
||||
import java.util.HashMap; |
||||
import java.util.Map; |
||||
|
||||
import org.junit.Before; |
||||
import org.junit.Test; |
||||
import org.junit.runner.RunWith; |
||||
import org.powermock.api.mockito.PowerMockito; |
||||
import org.powermock.core.classloader.annotations.PrepareForTest; |
||||
import org.powermock.modules.junit4.PowerMockRunner; |
||||
import org.slf4j.Logger; |
||||
import org.slf4j.LoggerFactory; |
||||
|
||||
/** |
||||
* shell task return test. |
||||
*/ |
||||
@RunWith(PowerMockRunner.class) |
||||
@PrepareForTest({ShellTask.class}) |
||||
public class ShellTaskReturnTest { |
||||
private static final Logger logger = LoggerFactory.getLogger(ShellTaskTest.class); |
||||
|
||||
private ShellTask shellTask; |
||||
private ShellCommandExecutor shellCommandExecutor; |
||||
private TaskExecutionContext taskExecutionContext; |
||||
private CommandExecuteResult commandExecuteResult; |
||||
|
||||
@Before |
||||
public void before() throws Exception { |
||||
System.setProperty("log4j2.disable.jmx", Boolean.TRUE.toString()); |
||||
shellCommandExecutor = PowerMockito.mock(ShellCommandExecutor.class); |
||||
PowerMockito.whenNew(ShellCommandExecutor.class).withAnyArguments().thenReturn(shellCommandExecutor); |
||||
taskExecutionContext = new TaskExecutionContext(); |
||||
taskExecutionContext.setTaskInstanceId(1); |
||||
taskExecutionContext.setTaskName("kris test"); |
||||
taskExecutionContext.setTaskType("SHELL"); |
||||
taskExecutionContext.setHost("127.0.0.1:1234"); |
||||
taskExecutionContext.setExecutePath("/tmp"); |
||||
taskExecutionContext.setLogPath("/log"); |
||||
taskExecutionContext.setTaskJson( |
||||
"{\"conditionResult\":\"{\\\"successNode\\\":[\\\"\\\"]," |
||||
+ "\\\"failedNode\\\":[\\\"\\\"]}\",\"conditionsTask\":false," |
||||
+ "\"depList\":[],\"dependence\":\"{}\",\"forbidden\":false,\"id\":\"" |
||||
+ "tasks-16849\",\"maxRetryTimes\":0,\"name\":\"shell test 001\"," |
||||
+ "\"params\":\"{\\\"rawScript\\\":\\\"#!/bin/sh\\\\necho $[yyyy-MM-dd HH:mm:ss +3]\\\\necho \\\\\\\" ?? " |
||||
+ "${time1} \\\\\\\"\\\\necho \\\\\\\" ????? ${time2}\\\\\\\"\\\\n\\\"," |
||||
+ "\\\"localParams\\\":[{\\\"prop\\\":\\\"time1\\\",\\\"direct\\\":\\\"IN\\\",\\\"type\\\":" |
||||
+ "\\\"VARCHAR\\\",\\\"value\\\":\\\"$[yyyy-MM-dd HH:mm:ss]\\\"}," |
||||
+ "{\\\"prop\\\":\\\"time2\\\",\\\"direct\\\":\\\"IN\\\",\\\"type\\\":\\\"VARCHAR\\\",\\\"value\\\":\\\"${time_gb}\\\"}" |
||||
+ "],\\\"resourceList\\\":[]}\",\"preTasks\":\"[]\",\"retryInterval\":1," |
||||
+ "\"runFlag\":\"NORMAL\",\"taskInstancePriority\":\"MEDIUM\",\"taskTimeoutParameter\":" |
||||
+ "{\"enable\":false,\"interval\":0},\"timeout\":\"{\\\"enable\\\":false,\\\"strategy\\\":\\\"\\\"}\"," |
||||
+ "\"type\":\"SHELL\",\"workerGroup\":\"default\"}"); |
||||
taskExecutionContext.setProcessInstanceId(1); |
||||
taskExecutionContext.setGlobalParams("[{\"direct\":\"IN\",\"prop\":\"time_gb\",\"type\":\"VARCHAR\",\"value\":\"2020-12-16 17:18:33\"}]"); |
||||
taskExecutionContext.setExecutorId(1); |
||||
taskExecutionContext.setCmdTypeIfComplement(5); |
||||
taskExecutionContext.setTenantCode("roo"); |
||||
taskExecutionContext.setScheduleTime(new Date()); |
||||
taskExecutionContext.setQueue("default"); |
||||
taskExecutionContext.setTaskParams( |
||||
"{\"rawScript\":\"#!/bin/sh\\necho $[yyyy-MM-dd HH:mm:ss +3]\\necho \\\" ?? ${time1} \\\"\\necho \\\" ????? ${time2}\\\"\\n\",\"localParams\":" |
||||
+ |
||||
"[{\"prop\":\"time1\",\"direct\":\"OUT\",\"type\":\"VARCHAR\",\"value\":\"$[yyyy-MM-dd HH:mm:ss]\"},{\"prop\":\"time2\",\"direct\":\"IN\",\"type\":\"VARCHAR" |
||||
+ "\",\"value\":\"${time_gb}\"}],\"resourceList\":[]}"); |
||||
Map<String, String> definedParams = new HashMap<>(); |
||||
definedParams.put("time_gb", "2020-12-16 00:00:00"); |
||||
taskExecutionContext.setDefinedParams(definedParams); |
||||
PowerMockito.mockStatic(Files.class); |
||||
PowerMockito.when(Files.exists(Paths.get(anyString()))).thenReturn(true); |
||||
commandExecuteResult = new CommandExecuteResult(); |
||||
commandExecuteResult.setAppIds("appId"); |
||||
commandExecuteResult.setExitStatusCode(0); |
||||
commandExecuteResult.setProcessId(1); |
||||
} |
||||
|
||||
@Test |
||||
public void testShellReturnString() { |
||||
shellTask = new ShellTask(taskExecutionContext, logger); |
||||
shellTask.init(); |
||||
try { |
||||
PowerMockito.when(shellCommandExecutor.run(anyString())).thenReturn(commandExecuteResult); |
||||
} catch (Exception e) { |
||||
e.printStackTrace(); |
||||
} |
||||
shellTask.setResult("shell return string"); |
||||
logger.info("shell return string:{}", shellTask.getResultString()); |
||||
} |
||||
|
||||
@Test |
||||
public void testSetTaskResultString() { |
||||
shellCommandExecutor.setTaskResultString("shellReturn"); |
||||
} |
||||
|
||||
@Test |
||||
public void testGetTaskResultString() { |
||||
logger.info(shellCommandExecutor.getTaskResultString()); |
||||
} |
||||
} |
Binary file not shown.
Loading…
Reference in new issue