Browse Source

[Test-3288][server] Fix github ci unit test oom (#3305)

* [Improvement] Test github action oom

* Optimize the test case performance

* Remove the unused import
pull/3/MERGE
Yichao Yang 4 years ago committed by GitHub
parent
commit
0577dc9780
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 4
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java
  2. 14
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java
  3. 23
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerService.java
  4. 21
      dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterExecThreadTest.java
  5. 18
      dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/consumer/TaskPriorityQueueConsumerTest.java
  6. 2
      dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/ExecutorDispatcherTest.java
  7. 1
      dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/RoundRobinHostManagerTest.java
  8. 1
      dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/registry/MasterRegistryTest.java
  9. 71
      dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecThreadTest.java
  10. 9
      dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/registry/ZookeeperNodeManagerTest.java
  11. 7
      dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/processor/TaskCallbackServiceTest.java
  12. 1
      dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/registry/WorkerRegistryTest.java

4
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java

@ -123,8 +123,8 @@ public class MasterServer {
// self tolerant // self tolerant
this.zkMasterClient.start(); this.zkMasterClient.start();
// // scheduler start
masterSchedulerService.start(); this.masterSchedulerService.start();
// start QuartzExecutors // start QuartzExecutors
// what system should do if exception // what system should do if exception

14
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java

@ -35,7 +35,6 @@ import org.apache.dolphinscheduler.dao.utils.DagHelper;
import org.apache.dolphinscheduler.remote.NettyRemotingClient; import org.apache.dolphinscheduler.remote.NettyRemotingClient;
import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.utils.AlertManager; import org.apache.dolphinscheduler.server.utils.AlertManager;
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.dolphinscheduler.service.process.ProcessService;
import org.apache.dolphinscheduler.service.quartz.cron.CronUtils; import org.apache.dolphinscheduler.service.quartz.cron.CronUtils;
import org.slf4j.Logger; import org.slf4j.Logger;
@ -123,12 +122,12 @@ public class MasterExecThread implements Runnable {
/** /**
* alert manager * alert manager
*/ */
private AlertManager alertManager = new AlertManager(); private AlertManager alertManager;
/** /**
* the object of DAG * the object of DAG
*/ */
private DAG<String,TaskNode,TaskNodeRelation> dag; private DAG<String, TaskNode, TaskNodeRelation> dag;
/** /**
* process service * process service
@ -151,15 +150,20 @@ public class MasterExecThread implements Runnable {
* @param processService processService * @param processService processService
* @param nettyRemotingClient nettyRemotingClient * @param nettyRemotingClient nettyRemotingClient
*/ */
public MasterExecThread(ProcessInstance processInstance, ProcessService processService, NettyRemotingClient nettyRemotingClient){ public MasterExecThread(ProcessInstance processInstance
, ProcessService processService
, NettyRemotingClient nettyRemotingClient
, AlertManager alertManager
, MasterConfig masterConfig) {
this.processService = processService; this.processService = processService;
this.processInstance = processInstance; this.processInstance = processInstance;
this.masterConfig = SpringApplicationContext.getBean(MasterConfig.class); this.masterConfig = masterConfig;
int masterTaskExecNum = masterConfig.getMasterExecTaskNum(); int masterTaskExecNum = masterConfig.getMasterExecTaskNum();
this.taskExecService = ThreadUtils.newDaemonFixedThreadExecutor("Master-Task-Exec-Thread", this.taskExecService = ThreadUtils.newDaemonFixedThreadExecutor("Master-Task-Exec-Thread",
masterTaskExecNum); masterTaskExecNum);
this.nettyRemotingClient = nettyRemotingClient; this.nettyRemotingClient = nettyRemotingClient;
this.alertManager = alertManager;
} }

23
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerService.java

@ -16,6 +16,11 @@
*/ */
package org.apache.dolphinscheduler.server.master.runner; package org.apache.dolphinscheduler.server.master.runner;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import javax.annotation.PostConstruct;
import org.apache.curator.framework.imps.CuratorFrameworkState; import org.apache.curator.framework.imps.CuratorFrameworkState;
import org.apache.curator.framework.recipes.locks.InterProcessMutex; import org.apache.curator.framework.recipes.locks.InterProcessMutex;
import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.Constants;
@ -28,6 +33,7 @@ import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.remote.NettyRemotingClient; import org.apache.dolphinscheduler.remote.NettyRemotingClient;
import org.apache.dolphinscheduler.remote.config.NettyClientConfig; import org.apache.dolphinscheduler.remote.config.NettyClientConfig;
import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.utils.AlertManager;
import org.apache.dolphinscheduler.server.zk.ZKMasterClient; import org.apache.dolphinscheduler.server.zk.ZKMasterClient;
import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.dolphinscheduler.service.process.ProcessService;
import org.slf4j.Logger; import org.slf4j.Logger;
@ -35,10 +41,6 @@ import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired; import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service; import org.springframework.stereotype.Service;
import javax.annotation.PostConstruct;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
/** /**
* master scheduler thread * master scheduler thread
*/ */
@ -68,6 +70,11 @@ public class MasterSchedulerService extends Thread {
@Autowired @Autowired
private MasterConfig masterConfig; private MasterConfig masterConfig;
/**
* alert manager
*/
private AlertManager alertManager = new AlertManager();
/** /**
* netty remoting client * netty remoting client
*/ */
@ -139,7 +146,13 @@ public class MasterSchedulerService extends Thread {
this.masterConfig.getMasterExecThreads() - activeCount, command); this.masterConfig.getMasterExecThreads() - activeCount, command);
if (processInstance != null) { if (processInstance != null) {
logger.info("start master exec thread , split DAG ..."); logger.info("start master exec thread , split DAG ...");
masterExecService.execute(new MasterExecThread(processInstance, processService, nettyRemotingClient)); masterExecService.execute(
new MasterExecThread(
processInstance
, processService
, nettyRemotingClient
, alertManager
, masterConfig));
} }
}catch (Exception e){ }catch (Exception e){
logger.error("scan command error ", e); logger.error("scan command error ", e);

21
dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterExecThreadTest.java

@ -72,8 +72,6 @@ public class MasterExecThreadTest {
applicationContext = mock(ApplicationContext.class); applicationContext = mock(ApplicationContext.class);
config = new MasterConfig(); config = new MasterConfig();
config.setMasterExecTaskNum(1); config.setMasterExecTaskNum(1);
SpringApplicationContext springApplicationContext = new SpringApplicationContext();
springApplicationContext.setApplicationContext(applicationContext);
Mockito.when(applicationContext.getBean(MasterConfig.class)).thenReturn(config); Mockito.when(applicationContext.getBean(MasterConfig.class)).thenReturn(config);
processInstance = mock(ProcessInstance.class); processInstance = mock(ProcessInstance.class);
@ -84,14 +82,17 @@ public class MasterExecThreadTest {
Mockito.when(processInstance.getScheduleTime()).thenReturn(DateUtils.stringToDate("2020-01-01 00:00:00")); Mockito.when(processInstance.getScheduleTime()).thenReturn(DateUtils.stringToDate("2020-01-01 00:00:00"));
Map<String, String> cmdParam = new HashMap<>(); Map<String, String> cmdParam = new HashMap<>();
cmdParam.put(CMDPARAM_COMPLEMENT_DATA_START_DATE, "2020-01-01 00:00:00"); cmdParam.put(CMDPARAM_COMPLEMENT_DATA_START_DATE, "2020-01-01 00:00:00");
cmdParam.put(CMDPARAM_COMPLEMENT_DATA_END_DATE, "2020-01-31 23:00:00"); cmdParam.put(CMDPARAM_COMPLEMENT_DATA_END_DATE, "2020-01-20 23:00:00");
Mockito.when(processInstance.getCommandParam()).thenReturn(JSONUtils.toJsonString(cmdParam)); Mockito.when(processInstance.getCommandParam()).thenReturn(JSONUtils.toJsonString(cmdParam));
ProcessDefinition processDefinition = new ProcessDefinition(); ProcessDefinition processDefinition = new ProcessDefinition();
processDefinition.setGlobalParamMap(Collections.EMPTY_MAP); processDefinition.setGlobalParamMap(Collections.EMPTY_MAP);
processDefinition.setGlobalParamList(Collections.EMPTY_LIST); processDefinition.setGlobalParamList(Collections.EMPTY_LIST);
Mockito.when(processInstance.getProcessDefinition()).thenReturn(processDefinition); Mockito.when(processInstance.getProcessDefinition()).thenReturn(processDefinition);
masterExecThread = PowerMockito.spy(new MasterExecThread(processInstance, processService,null)); masterExecThread = PowerMockito.spy(new MasterExecThread(
processInstance
, processService
,null, null, config));
// prepareProcess init dag // prepareProcess init dag
Field dag = MasterExecThread.class.getDeclaredField("dag"); Field dag = MasterExecThread.class.getDeclaredField("dag");
dag.setAccessible(true); dag.setAccessible(true);
@ -114,11 +115,11 @@ public class MasterExecThreadTest {
Method method = MasterExecThread.class.getDeclaredMethod("executeComplementProcess"); Method method = MasterExecThread.class.getDeclaredMethod("executeComplementProcess");
method.setAccessible(true); method.setAccessible(true);
method.invoke(masterExecThread); method.invoke(masterExecThread);
// one create save, and 1-30 for next save, and last day 31 no save // one create save, and 1-30 for next save, and last day 20 no save
verify(processService, times(31)).saveProcessInstance(processInstance); verify(processService, times(20)).saveProcessInstance(processInstance);
}catch (Exception e){ }catch (Exception e){
e.printStackTrace(); e.printStackTrace();
Assert.assertTrue(false); Assert.fail();
} }
} }
@ -133,10 +134,10 @@ public class MasterExecThreadTest {
Method method = MasterExecThread.class.getDeclaredMethod("executeComplementProcess"); Method method = MasterExecThread.class.getDeclaredMethod("executeComplementProcess");
method.setAccessible(true); method.setAccessible(true);
method.invoke(masterExecThread); method.invoke(masterExecThread);
// one create save, and 15(1 to 31 step 2) for next save, and last day 31 no save // one create save, and 9(1 to 20 step 2) for next save, and last day 31 no save
verify(processService, times(15)).saveProcessInstance(processInstance); verify(processService, times(9)).saveProcessInstance(processInstance);
}catch (Exception e){ }catch (Exception e){
Assert.assertTrue(false); Assert.fail();
} }
} }

18
dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/consumer/TaskPriorityQueueConsumerTest.java

@ -17,11 +17,18 @@
package org.apache.dolphinscheduler.server.master.consumer; package org.apache.dolphinscheduler.server.master.consumer;
import java.util.Date;
import org.apache.dolphinscheduler.common.enums.CommandType; import org.apache.dolphinscheduler.common.enums.CommandType;
import org.apache.dolphinscheduler.common.enums.DbType; import org.apache.dolphinscheduler.common.enums.DbType;
import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.common.enums.Priority; import org.apache.dolphinscheduler.common.enums.Priority;
import org.apache.dolphinscheduler.dao.entity.*; import org.apache.dolphinscheduler.common.thread.Stopper;
import org.apache.dolphinscheduler.dao.entity.DataSource;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.dao.entity.Tenant;
import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.dispatch.ExecutorDispatcher; import org.apache.dolphinscheduler.server.master.dispatch.ExecutorDispatcher;
import org.apache.dolphinscheduler.server.master.dispatch.executor.NettyExecutorManager; import org.apache.dolphinscheduler.server.master.dispatch.executor.NettyExecutorManager;
@ -34,6 +41,7 @@ import org.apache.dolphinscheduler.service.process.ProcessService;
import org.apache.dolphinscheduler.service.queue.TaskPriorityQueue; import org.apache.dolphinscheduler.service.queue.TaskPriorityQueue;
import org.apache.dolphinscheduler.service.zk.ZookeeperCachedOperator; import org.apache.dolphinscheduler.service.zk.ZookeeperCachedOperator;
import org.apache.dolphinscheduler.service.zk.ZookeeperConfig; import org.apache.dolphinscheduler.service.zk.ZookeeperConfig;
import org.junit.After;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import org.junit.runner.RunWith; import org.junit.runner.RunWith;
@ -42,8 +50,6 @@ import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.test.context.ContextConfiguration; import org.springframework.test.context.ContextConfiguration;
import org.springframework.test.context.junit4.SpringJUnit4ClassRunner; import org.springframework.test.context.junit4.SpringJUnit4ClassRunner;
import java.util.Date;
@RunWith(SpringJUnit4ClassRunner.class) @RunWith(SpringJUnit4ClassRunner.class)
@ContextConfiguration(classes={DependencyConfig.class, SpringApplicationContext.class, SpringZKServer.class, @ContextConfiguration(classes={DependencyConfig.class, SpringApplicationContext.class, SpringZKServer.class,
@ -250,5 +256,11 @@ public class TaskPriorityQueueConsumerTest {
taskPriorityQueueConsumer.taskInstanceIsFinalState(1); taskPriorityQueueConsumer.taskInstanceIsFinalState(1);
} }
@After
public void close() {
Stopper.stop();
}
} }

2
dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/ExecutorDispatcherTest.java

@ -78,5 +78,7 @@ public class ExecutorDispatcherTest {
ExecutionContext executionContext = ExecutionContextTestUtils.getExecutionContext(port); ExecutionContext executionContext = ExecutionContextTestUtils.getExecutionContext(port);
executorDispatcher.dispatch(executionContext); executorDispatcher.dispatch(executionContext);
workerRegistry.unRegistry();
} }
} }

1
dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/RoundRobinHostManagerTest.java

@ -74,5 +74,6 @@ public class RoundRobinHostManagerTest {
Host host = roundRobinHostManager.select(context); Host host = roundRobinHostManager.select(context);
Assert.assertTrue(StringUtils.isNotEmpty(host.getAddress())); Assert.assertTrue(StringUtils.isNotEmpty(host.getAddress()));
Assert.assertTrue(host.getAddress().equalsIgnoreCase(NetUtils.getHost() + ":" + workerConfig.getListenPort())); Assert.assertTrue(host.getAddress().equalsIgnoreCase(NetUtils.getHost() + ":" + workerConfig.getListenPort()));
workerRegistry.unRegistry();
} }
} }

1
dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/registry/MasterRegistryTest.java

@ -58,6 +58,7 @@ public class MasterRegistryTest {
String masterNodePath = masterPath + "/" + (Constants.LOCAL_ADDRESS + ":" + masterConfig.getListenPort()); String masterNodePath = masterPath + "/" + (Constants.LOCAL_ADDRESS + ":" + masterConfig.getListenPort());
String heartbeat = zookeeperRegistryCenter.getZookeeperCachedOperator().get(masterNodePath); String heartbeat = zookeeperRegistryCenter.getZookeeperCachedOperator().get(masterNodePath);
Assert.assertEquals(HEARTBEAT_FOR_ZOOKEEPER_INFO_LENGTH, heartbeat.split(",").length); Assert.assertEquals(HEARTBEAT_FOR_ZOOKEEPER_INFO_LENGTH, heartbeat.split(",").length);
masterRegistry.unRegistry();
} }
@Test @Test

71
dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecThreadTest.java

@ -17,66 +17,75 @@
package org.apache.dolphinscheduler.server.master.runner; package org.apache.dolphinscheduler.server.master.runner;
import java.util.HashSet;
import java.util.Set;
import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.server.master.consumer.TaskPriorityQueueConsumer;
import org.apache.dolphinscheduler.server.master.dispatch.ExecutorDispatcher;
import org.apache.dolphinscheduler.server.master.dispatch.executor.NettyExecutorManager;
import org.apache.dolphinscheduler.server.registry.DependencyConfig;
import org.apache.dolphinscheduler.server.registry.ZookeeperNodeManager;
import org.apache.dolphinscheduler.server.registry.ZookeeperRegistryCenter; import org.apache.dolphinscheduler.server.registry.ZookeeperRegistryCenter;
import org.apache.dolphinscheduler.server.zk.SpringZKServer;
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.dolphinscheduler.service.process.ProcessService;
import org.apache.dolphinscheduler.service.zk.CuratorZookeeperClient; import org.junit.Assert;
import org.apache.dolphinscheduler.service.zk.ZookeeperCachedOperator; import org.junit.Before;
import org.apache.dolphinscheduler.service.zk.ZookeeperConfig;
import org.junit.Test; import org.junit.Test;
import org.junit.runner.RunWith; import org.junit.runner.RunWith;
import org.mockito.Mockito; import org.mockito.Mockito;
import org.mockito.junit.MockitoJUnitRunner;
import org.powermock.api.mockito.PowerMockito;
import org.powermock.core.classloader.annotations.PrepareForTest;
import org.springframework.context.ApplicationContext; import org.springframework.context.ApplicationContext;
import org.springframework.test.context.ContextConfiguration;
import org.springframework.test.context.junit4.SpringJUnit4ClassRunner;
import java.util.HashSet; import com.google.common.collect.Sets;
import java.util.Set;
@RunWith(SpringJUnit4ClassRunner.class) @RunWith(MockitoJUnitRunner.Silent.class)
@ContextConfiguration(classes={DependencyConfig.class, SpringApplicationContext.class, SpringZKServer.class, @PrepareForTest(MasterTaskExecThread.class)
NettyExecutorManager.class, ExecutorDispatcher.class, ZookeeperRegistryCenter.class, TaskPriorityQueueConsumer.class,
ZookeeperNodeManager.class, ZookeeperCachedOperator.class, ZookeeperConfig.class, CuratorZookeeperClient.class})
public class MasterTaskExecThreadTest { public class MasterTaskExecThreadTest {
private MasterTaskExecThread masterTaskExecThread;
private SpringApplicationContext springApplicationContext;
private ZookeeperRegistryCenter zookeeperRegistryCenter;
@Before
public void setUp() {
ApplicationContext applicationContext = PowerMockito.mock(ApplicationContext.class);
this.springApplicationContext = new SpringApplicationContext();
springApplicationContext.setApplicationContext(applicationContext);
this.zookeeperRegistryCenter = PowerMockito.mock(ZookeeperRegistryCenter.class);
PowerMockito.when(SpringApplicationContext.getBean(ZookeeperRegistryCenter.class))
.thenReturn(this.zookeeperRegistryCenter);
this.masterTaskExecThread = new MasterTaskExecThread(null);
}
@Test @Test
public void testExistsValidWorkerGroup1(){ public void testExistsValidWorkerGroup1(){
ZookeeperRegistryCenter zookeeperRegistryCenter = Mockito.mock(ZookeeperRegistryCenter.class);
Mockito.when(zookeeperRegistryCenter.getWorkerGroupDirectly()).thenReturn(null); Mockito.when(zookeeperRegistryCenter.getWorkerGroupDirectly()).thenReturn(Sets.newHashSet());
MasterTaskExecThread masterTaskExecThread = new MasterTaskExecThread(null); boolean b = masterTaskExecThread.existsValidWorkerGroup("default");
masterTaskExecThread.existsValidWorkerGroup("default"); Assert.assertFalse(b);
} }
@Test @Test
public void testExistsValidWorkerGroup2(){ public void testExistsValidWorkerGroup2(){
ZookeeperRegistryCenter zookeeperRegistryCenter = Mockito.mock(ZookeeperRegistryCenter.class);
Set<String> workerGorups = new HashSet<>(); Set<String> workerGorups = new HashSet<>();
workerGorups.add("test1"); workerGorups.add("test1");
workerGorups.add("test2"); workerGorups.add("test2");
Mockito.when(zookeeperRegistryCenter.getWorkerGroupDirectly()).thenReturn(workerGorups); Mockito.when(zookeeperRegistryCenter.getWorkerGroupDirectly()).thenReturn(workerGorups);
MasterTaskExecThread masterTaskExecThread = new MasterTaskExecThread(null); boolean b = masterTaskExecThread.existsValidWorkerGroup("default");
masterTaskExecThread.existsValidWorkerGroup("default"); Assert.assertFalse(b);
} }
@Test @Test
public void testExistsValidWorkerGroup3(){ public void testExistsValidWorkerGroup3(){
ZookeeperRegistryCenter zookeeperRegistryCenter = Mockito.mock(ZookeeperRegistryCenter.class);
Set<String> workerGorups = new HashSet<>(); Set<String> workerGorups = new HashSet<>();
workerGorups.add("test1"); workerGorups.add("test1");
Mockito.when(zookeeperRegistryCenter.getWorkerGroupDirectly()).thenReturn(workerGorups); Mockito.when(zookeeperRegistryCenter.getWorkerGroupDirectly()).thenReturn(workerGorups);
Mockito.when(zookeeperRegistryCenter.getWorkerGroupNodesDirectly("test1")).thenReturn(workerGorups); Mockito.when(zookeeperRegistryCenter.getWorkerGroupNodesDirectly("test1")).thenReturn(workerGorups);
MasterTaskExecThread masterTaskExecThread = new MasterTaskExecThread(null); boolean b = masterTaskExecThread.existsValidWorkerGroup("test1");
masterTaskExecThread.existsValidWorkerGroup("test1"); Assert.assertTrue(b);
} }
@Test @Test
@ -84,17 +93,15 @@ public class MasterTaskExecThreadTest {
ProcessService processService = Mockito.mock(ProcessService.class); ProcessService processService = Mockito.mock(ProcessService.class);
ApplicationContext applicationContext = Mockito.mock(ApplicationContext.class); Mockito.when(this.springApplicationContext.getBean(ProcessService.class))
SpringApplicationContext springApplicationContext = new SpringApplicationContext(); .thenReturn(processService);
springApplicationContext.setApplicationContext(applicationContext);
Mockito.when(applicationContext.getBean(ProcessService.class)).thenReturn(processService);
TaskInstance taskInstance = getTaskInstance(); TaskInstance taskInstance = getTaskInstance();
Mockito.when(processService.findTaskInstanceById(252612)) Mockito.when(processService.findTaskInstanceById(252612))
.thenReturn(taskInstance); .thenReturn(taskInstance);
Mockito.when(processService.updateTaskInstance(taskInstance)) Mockito.when(processService.updateTaskInstance(taskInstance))
.thenReturn(true); .thenReturn(true);
MasterTaskExecThread masterTaskExecThread = new MasterTaskExecThread(taskInstance); MasterTaskExecThread masterTaskExecThread = new MasterTaskExecThread(taskInstance);
masterTaskExecThread.pauseTask(); masterTaskExecThread.pauseTask();

9
dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/registry/ZookeeperNodeManagerTest.java

@ -17,6 +17,9 @@
package org.apache.dolphinscheduler.server.registry; package org.apache.dolphinscheduler.server.registry;
import java.util.Map;
import java.util.Set;
import org.apache.dolphinscheduler.common.utils.CollectionUtils; import org.apache.dolphinscheduler.common.utils.CollectionUtils;
import org.apache.dolphinscheduler.common.utils.NetUtils; import org.apache.dolphinscheduler.common.utils.NetUtils;
import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.config.MasterConfig;
@ -33,9 +36,6 @@ import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.test.context.ContextConfiguration; import org.springframework.test.context.ContextConfiguration;
import org.springframework.test.context.junit4.SpringJUnit4ClassRunner; import org.springframework.test.context.junit4.SpringJUnit4ClassRunner;
import java.util.Map;
import java.util.Set;
/** /**
* zookeeper node manager test * zookeeper node manager test
*/ */
@ -75,6 +75,7 @@ public class ZookeeperNodeManagerTest {
Assert.assertTrue(CollectionUtils.isNotEmpty(masterNodes)); Assert.assertTrue(CollectionUtils.isNotEmpty(masterNodes));
Assert.assertEquals(1, masterNodes.size()); Assert.assertEquals(1, masterNodes.size());
Assert.assertEquals(NetUtils.getHost() + ":" + masterConfig.getListenPort(), masterNodes.iterator().next()); Assert.assertEquals(NetUtils.getHost() + ":" + masterConfig.getListenPort(), masterNodes.iterator().next());
workerRegistry.unRegistry();
} }
@Test @Test
@ -88,6 +89,7 @@ public class ZookeeperNodeManagerTest {
Map<String, Set<String>> workerGroupNodes = zookeeperNodeManager.getWorkerGroupNodes(); Map<String, Set<String>> workerGroupNodes = zookeeperNodeManager.getWorkerGroupNodes();
Assert.assertEquals(1, workerGroupNodes.size()); Assert.assertEquals(1, workerGroupNodes.size());
Assert.assertEquals("default".trim(), workerGroupNodes.keySet().iterator().next()); Assert.assertEquals("default".trim(), workerGroupNodes.keySet().iterator().next());
workerRegistry.unRegistry();
} }
@Test @Test
@ -103,5 +105,6 @@ public class ZookeeperNodeManagerTest {
Assert.assertTrue(CollectionUtils.isNotEmpty(workerNodes)); Assert.assertTrue(CollectionUtils.isNotEmpty(workerNodes));
Assert.assertEquals(1, workerNodes.size()); Assert.assertEquals(1, workerNodes.size());
Assert.assertEquals(NetUtils.getHost() + ":" + workerConfig.getListenPort(), workerNodes.iterator().next()); Assert.assertEquals(NetUtils.getHost() + ":" + workerConfig.getListenPort(), workerNodes.iterator().next());
workerRegistry.unRegistry();
} }
} }

7
dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/processor/TaskCallbackServiceTest.java

@ -16,7 +16,8 @@
*/ */
package org.apache.dolphinscheduler.server.worker.processor; package org.apache.dolphinscheduler.server.worker.processor;
import io.netty.channel.Channel; import java.util.Date;
import org.apache.dolphinscheduler.common.thread.Stopper; import org.apache.dolphinscheduler.common.thread.Stopper;
import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.remote.NettyRemotingClient; import org.apache.dolphinscheduler.remote.NettyRemotingClient;
@ -50,8 +51,7 @@ import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.test.context.ContextConfiguration; import org.springframework.test.context.ContextConfiguration;
import org.springframework.test.context.junit4.SpringJUnit4ClassRunner; import org.springframework.test.context.junit4.SpringJUnit4ClassRunner;
import java.io.IOException; import io.netty.channel.Channel;
import java.util.Date;
/** /**
* test task call back service * test task call back service
@ -189,6 +189,7 @@ public class TaskCallbackServiceTest {
nettyRemotingServer.close(); nettyRemotingServer.close();
nettyRemotingClient.close(); nettyRemotingClient.close();
masterRegistry.unRegistry();
} }
@Test @Test

1
dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/registry/WorkerRegistryTest.java

@ -143,6 +143,7 @@ public class WorkerRegistryTest {
Assert.assertEquals(0, testWorkerGroupPathZkChildren.size()); Assert.assertEquals(0, testWorkerGroupPathZkChildren.size());
Assert.assertEquals(0, defaultWorkerGroupPathZkChildren.size()); Assert.assertEquals(0, defaultWorkerGroupPathZkChildren.size());
workerRegistry.unRegistry();
} }
@Test @Test

Loading…
Cancel
Save