Browse Source

Add rpc benchmark test (#14797)

3.2.1-prepare
Wenjun Ruan 1 year ago committed by GitHub
parent
commit
d223d654cc
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 2
      .github/CODEOWNERS
  2. 2
      docs/docs/en/about/glossary.md
  3. 2
      docs/docs/zh/about/glossary.md
  4. 2
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/aspect/CacheEvictAspect.java
  5. 2
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/executor/workflow/instance/pause/pause/PauseExecuteFunction.java
  6. 4
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/executor/workflow/instance/stop/StopExecuteFunction.java
  7. 8
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ExecutorServiceImpl.java
  8. 8
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/LoggerServiceImpl.java
  9. 2
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/MetricsCleanUpServiceImpl.java
  10. 8
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/TaskInstanceServiceImpl.java
  11. 36
      dolphinscheduler-extract/dolphinscheduler-extract-base/src/main/java/org/apache/dolphinscheduler/extract/base/client/JdkDynamicRpcClientProxyFactory.java
  12. 14
      dolphinscheduler-extract/dolphinscheduler-extract-base/src/main/java/org/apache/dolphinscheduler/extract/base/client/SingletonJdkDynamicRpcClientProxyFactory.java
  13. 2
      dolphinscheduler-extract/dolphinscheduler-extract-base/src/main/java/org/apache/dolphinscheduler/extract/base/server/JdkDynamicServerHandler.java
  14. 94
      dolphinscheduler-extract/dolphinscheduler-extract-base/src/test/java/org/apache/dolphinscheduler/extract/base/client/SingletonJdkDynamicRpcClientProxyFactoryTest.java
  15. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskDelayEventHandler.java
  16. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskResultEventHandler.java
  17. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRunningEventHandler.java
  18. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskUpdatePidEventHandler.java
  19. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StreamTaskExecuteRunnable.java
  20. 10
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnable.java
  21. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteThreadPool.java
  22. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcher.java
  23. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcher.java
  24. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecuteRunningEventSender.java
  25. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionFinishEventSender.java
  26. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/LogicTaskExecuteRunnableKillOperator.java
  27. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/LogicTaskExecuteRunnablePauseOperator.java
  28. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/LogicTaskExecuteRunnableTimeoutOperator.java
  29. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskExecuteRunnableKillOperator.java
  30. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskExecuteRunnablePauseOperator.java
  31. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskExecuteRunnableTimeoutOperator.java
  32. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicLogicTask.java
  33. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTask.java
  34. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/service/WorkerFailoverService.java
  35. 5
      dolphinscheduler-microbench/pom.xml
  36. 29
      dolphinscheduler-microbench/src/main/java/org/apache/dolphinscheduler/microbench/rpc/IService.java
  37. 27
      dolphinscheduler-microbench/src/main/java/org/apache/dolphinscheduler/microbench/rpc/IServiceImpl.java
  38. 76
      dolphinscheduler-microbench/src/main/java/org/apache/dolphinscheduler/microbench/rpc/RpcBenchMarkTest.java
  39. 8
      dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessServiceImpl.java
  40. 2
      dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskInstanceExecutionFinishEventSender.java
  41. 2
      dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskInstanceExecutionInfoUpdateEventSender.java
  42. 2
      dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskInstanceExecutionRunningEventSender.java
  43. 2
      dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecuteRunnable.java

2
.github/CODEOWNERS

@ -33,7 +33,7 @@
/dolphinscheduler-master/ @caishunfeng @SbloodyS @ruanwenjun /dolphinscheduler-master/ @caishunfeng @SbloodyS @ruanwenjun
/dolphinscheduler-worker/ @caishunfeng @SbloodyS @ruanwenjun /dolphinscheduler-worker/ @caishunfeng @SbloodyS @ruanwenjun
/dolphinscheduler-service/ @caishunfeng /dolphinscheduler-service/ @caishunfeng
/dolphinscheduler-remote/ @caishunfeng /dolphinscheduler-extract/ @caishunfeng @ruanwenjun
/dolphinscheduler-spi/ @caishunfeng /dolphinscheduler-spi/ @caishunfeng
/dolphinscheduler-task-plugin/ @caishunfeng @SbloodyS @zhuangchong /dolphinscheduler-task-plugin/ @caishunfeng @SbloodyS @zhuangchong
/dolphinscheduler-tools/ @caishunfeng @SbloodyS @zhongjiajie @EricGao888 /dolphinscheduler-tools/ @caishunfeng @SbloodyS @zhongjiajie @EricGao888

2
docs/docs/en/about/glossary.md

@ -61,7 +61,7 @@ process fails and ends
- dolphinscheduler-dao provides operations such as database access. - dolphinscheduler-dao provides operations such as database access.
- dolphinscheduler-remote client and server based on netty - dolphinscheduler-extract dolphinscheduler extract module, providing master/worker/alert sdk.
- dolphinscheduler-service service module, including Quartz, Zookeeper, log client access service, easy to call server - dolphinscheduler-service service module, including Quartz, Zookeeper, log client access service, easy to call server
module and api module module and api module

2
docs/docs/zh/about/glossary.md

@ -45,7 +45,7 @@
- dolphinscheduler-dao 提供数据库访问等操作。 - dolphinscheduler-dao 提供数据库访问等操作。
- dolphinscheduler-remote 基于 netty 的客户端、服务端 - dolphinscheduler-extract extract模块,包含master/worker/alert的sdk
- dolphinscheduler-service service模块,包含Quartz、Zookeeper、日志客户端访问服务,便于server模块和api模块调用 - dolphinscheduler-service service模块,包含Quartz、Zookeeper、日志客户端访问服务,便于server模块和api模块调用

2
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/aspect/CacheEvictAspect.java

@ -147,7 +147,7 @@ public class CacheEvictAspect {
return; return;
} }
for (Server server : serverList) { for (Server server : serverList) {
IMasterCacheService masterCacheService = SingletonJdkDynamicRpcClientProxyFactory.getInstance() IMasterCacheService masterCacheService = SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(server.getHost() + ":" + server.getPort(), IMasterCacheService.class); .getProxyClient(server.getHost() + ":" + server.getPort(), IMasterCacheService.class);
masterCacheService.cacheExpire(new CacheExpireRequest(cacheType, cacheKey)); masterCacheService.cacheExpire(new CacheExpireRequest(cacheType, cacheKey));
} }

2
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/executor/workflow/instance/pause/pause/PauseExecuteFunction.java

@ -58,7 +58,7 @@ public class PauseExecuteFunction implements ExecuteFunction<PauseExecuteRequest
try { try {
// todo: direct call the workflow instance pause method // todo: direct call the workflow instance pause method
ITaskInstanceExecutionEventListener iTaskInstanceExecutionEventListener = ITaskInstanceExecutionEventListener iTaskInstanceExecutionEventListener =
SingletonJdkDynamicRpcClientProxyFactory.getInstance() SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(workflowInstance.getHost(), ITaskInstanceExecutionEventListener.class); .getProxyClient(workflowInstance.getHost(), ITaskInstanceExecutionEventListener.class);
iTaskInstanceExecutionEventListener.onWorkflowInstanceInstanceStateChange( iTaskInstanceExecutionEventListener.onWorkflowInstanceInstanceStateChange(
new WorkflowInstanceStateChangeEvent( new WorkflowInstanceStateChangeEvent(

4
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/executor/workflow/instance/stop/StopExecuteFunction.java

@ -59,8 +59,8 @@ public class StopExecuteFunction implements ExecuteFunction<StopRequest, StopRes
try { try {
// todo: direct call the workflow instance stop method // todo: direct call the workflow instance stop method
ITaskInstanceExecutionEventListener iTaskInstanceExecutionEventListener = ITaskInstanceExecutionEventListener iTaskInstanceExecutionEventListener =
SingletonJdkDynamicRpcClientProxyFactory.getInstance() SingletonJdkDynamicRpcClientProxyFactory.getProxyClient(workflowInstance.getHost(),
.getProxyClient(workflowInstance.getHost(), ITaskInstanceExecutionEventListener.class); ITaskInstanceExecutionEventListener.class);
iTaskInstanceExecutionEventListener.onWorkflowInstanceInstanceStateChange( iTaskInstanceExecutionEventListener.onWorkflowInstanceInstanceStateChange(
new WorkflowInstanceStateChangeEvent( new WorkflowInstanceStateChangeEvent(
workflowInstance.getId(), 0, workflowInstance.getState(), workflowInstance.getId(), 0)); workflowInstance.getId(), 0, workflowInstance.getState(), workflowInstance.getId(), 0));

8
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ExecutorServiceImpl.java

@ -656,7 +656,7 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
WorkflowInstanceStateChangeEvent workflowStateEventChangeRequest = new WorkflowInstanceStateChangeEvent( WorkflowInstanceStateChangeEvent workflowStateEventChangeRequest = new WorkflowInstanceStateChangeEvent(
processInstance.getId(), 0, processInstance.getState(), processInstance.getId(), 0); processInstance.getId(), 0, processInstance.getState(), processInstance.getId(), 0);
ITaskInstanceExecutionEventListener iTaskInstanceExecutionEventListener = ITaskInstanceExecutionEventListener iTaskInstanceExecutionEventListener =
SingletonJdkDynamicRpcClientProxyFactory.getInstance() SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(processInstance.getHost(), ITaskInstanceExecutionEventListener.class); .getProxyClient(processInstance.getHost(), ITaskInstanceExecutionEventListener.class);
iTaskInstanceExecutionEventListener.onWorkflowInstanceInstanceStateChange(workflowStateEventChangeRequest); iTaskInstanceExecutionEventListener.onWorkflowInstanceInstanceStateChange(workflowStateEventChangeRequest);
putMsg(result, Status.SUCCESS); putMsg(result, Status.SUCCESS);
@ -684,7 +684,7 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
taskGroupQueue.setForceStart(Flag.YES.getCode()); taskGroupQueue.setForceStart(Flag.YES.getCode());
processService.updateTaskGroupQueue(taskGroupQueue); processService.updateTaskGroupQueue(taskGroupQueue);
log.info("Sending force start command to master: {}.", processInstance.getHost()); log.info("Sending force start command to master: {}.", processInstance.getHost());
ILogicTaskInstanceOperator iLogicTaskInstanceOperator = SingletonJdkDynamicRpcClientProxyFactory.getInstance() ILogicTaskInstanceOperator iLogicTaskInstanceOperator = SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(processInstance.getHost(), ILogicTaskInstanceOperator.class); .getProxyClient(processInstance.getHost(), ILogicTaskInstanceOperator.class);
iLogicTaskInstanceOperator.forceStartTaskInstance( iLogicTaskInstanceOperator.forceStartTaskInstance(
new TaskInstanceForceStartRequest(processInstance.getId(), taskGroupQueue.getTaskId())); new TaskInstanceForceStartRequest(processInstance.getId(), taskGroupQueue.getTaskId()));
@ -1152,7 +1152,7 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
log.error("Process instance does not exist, processInstanceId:{}.", processInstanceId); log.error("Process instance does not exist, processInstanceId:{}.", processInstanceId);
return null; return null;
} }
IWorkflowInstanceService iWorkflowInstanceService = SingletonJdkDynamicRpcClientProxyFactory.getInstance() IWorkflowInstanceService iWorkflowInstanceService = SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(processInstance.getHost(), IWorkflowInstanceService.class); .getProxyClient(processInstance.getHost(), IWorkflowInstanceService.class);
return iWorkflowInstanceService.getWorkflowExecutingData(processInstanceId); return iWorkflowInstanceService.getWorkflowExecutingData(processInstanceId);
} }
@ -1189,7 +1189,7 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
taskExecuteStartMessage.setStartParams(startParams); taskExecuteStartMessage.setStartParams(startParams);
taskExecuteStartMessage.setDryRun(dryRun); taskExecuteStartMessage.setDryRun(dryRun);
IStreamingTaskOperator streamingTaskOperator = SingletonJdkDynamicRpcClientProxyFactory.getInstance() IStreamingTaskOperator streamingTaskOperator = SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(server.getHost() + ":" + server.getPort(), IStreamingTaskOperator.class); .getProxyClient(server.getHost() + ":" + server.getPort(), IStreamingTaskOperator.class);
StreamingTaskTriggerResponse streamingTaskTriggerResponse = StreamingTaskTriggerResponse streamingTaskTriggerResponse =
streamingTaskOperator.triggerStreamingTask(taskExecuteStartMessage); streamingTaskOperator.triggerStreamingTask(taskExecuteStartMessage);

8
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/LoggerServiceImpl.java

@ -225,7 +225,7 @@ public class LoggerServiceImpl extends BaseServiceImpl implements LoggerService
String logContent = null; String logContent = null;
if (TaskUtils.isLogicTask(taskInstance.getTaskType())) { if (TaskUtils.isLogicTask(taskInstance.getTaskType())) {
IMasterLogService masterLogService = SingletonJdkDynamicRpcClientProxyFactory.getInstance() IMasterLogService masterLogService = SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(taskInstance.getHost(), IMasterLogService.class); .getProxyClient(taskInstance.getHost(), IMasterLogService.class);
try { try {
LogicTaskInstanceLogPageQueryRequest logicTaskInstanceLogPageQueryRequest = LogicTaskInstanceLogPageQueryRequest logicTaskInstanceLogPageQueryRequest =
@ -237,7 +237,7 @@ public class LoggerServiceImpl extends BaseServiceImpl implements LoggerService
log.error("Query LogicTaskInstance log error", ex); log.error("Query LogicTaskInstance log error", ex);
} }
} else { } else {
IWorkerLogService iWorkerLogService = SingletonJdkDynamicRpcClientProxyFactory.getInstance() IWorkerLogService iWorkerLogService = SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(host, IWorkerLogService.class); .getProxyClient(host, IWorkerLogService.class);
try { try {
TaskInstanceLogPageQueryRequest taskInstanceLogPageQueryRequest = TaskInstanceLogPageQueryRequest taskInstanceLogPageQueryRequest =
@ -282,7 +282,7 @@ public class LoggerServiceImpl extends BaseServiceImpl implements LoggerService
byte[] logBytes = new byte[0]; byte[] logBytes = new byte[0];
if (TaskUtils.isLogicTask(taskInstance.getTaskType())) { if (TaskUtils.isLogicTask(taskInstance.getTaskType())) {
IMasterLogService masterLogService = SingletonJdkDynamicRpcClientProxyFactory.getInstance() IMasterLogService masterLogService = SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(taskInstance.getHost(), IMasterLogService.class); .getProxyClient(taskInstance.getHost(), IMasterLogService.class);
try { try {
LogicTaskInstanceLogFileDownloadRequest logicTaskInstanceLogFileDownloadRequest = LogicTaskInstanceLogFileDownloadRequest logicTaskInstanceLogFileDownloadRequest =
@ -294,7 +294,7 @@ public class LoggerServiceImpl extends BaseServiceImpl implements LoggerService
log.error("Query LogicTaskInstance log error", ex); log.error("Query LogicTaskInstance log error", ex);
} }
} else { } else {
IWorkerLogService iWorkerLogService = SingletonJdkDynamicRpcClientProxyFactory.getInstance() IWorkerLogService iWorkerLogService = SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(host, IWorkerLogService.class); .getProxyClient(host, IWorkerLogService.class);
try { try {
TaskInstanceLogFileDownloadRequest taskInstanceLogFileDownloadRequest = TaskInstanceLogFileDownloadRequest taskInstanceLogFileDownloadRequest =

2
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/MetricsCleanUpServiceImpl.java

@ -50,7 +50,7 @@ public class MetricsCleanUpServiceImpl implements MetricsCleanUpService {
private void cleanUpWorkflowMetrics(Server server, Long workflowDefinitionCode) { private void cleanUpWorkflowMetrics(Server server, Long workflowDefinitionCode) {
try { try {
IWorkflowInstanceService iWorkflowInstanceService = IWorkflowInstanceService iWorkflowInstanceService =
SingletonJdkDynamicRpcClientProxyFactory.getInstance().getProxyClient( SingletonJdkDynamicRpcClientProxyFactory.getProxyClient(
String.format("%s:%s", server.getHost(), server.getPort()), IWorkflowInstanceService.class); String.format("%s:%s", server.getHost(), server.getPort()), IWorkflowInstanceService.class);
iWorkflowInstanceService.clearWorkflowMetrics(workflowDefinitionCode); iWorkflowInstanceService.clearWorkflowMetrics(workflowDefinitionCode);
} catch (Exception e) { } catch (Exception e) {

8
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/TaskInstanceServiceImpl.java

@ -290,7 +290,7 @@ public class TaskInstanceServiceImpl extends BaseServiceImpl implements TaskInst
return result; return result;
} }
IStreamingTaskInstanceOperator streamingTaskInstanceOperator = IStreamingTaskInstanceOperator streamingTaskInstanceOperator =
SingletonJdkDynamicRpcClientProxyFactory.getInstance() SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(taskInstance.getHost(), IStreamingTaskInstanceOperator.class); .getProxyClient(taskInstance.getHost(), IStreamingTaskInstanceOperator.class);
TaskInstanceTriggerSavepointResponse taskInstanceTriggerSavepointResponse = TaskInstanceTriggerSavepointResponse taskInstanceTriggerSavepointResponse =
streamingTaskInstanceOperator.triggerSavepoint(new TaskInstanceTriggerSavepointRequest(taskInstanceId)); streamingTaskInstanceOperator.triggerSavepoint(new TaskInstanceTriggerSavepointRequest(taskInstanceId));
@ -322,7 +322,7 @@ public class TaskInstanceServiceImpl extends BaseServiceImpl implements TaskInst
} }
// todo: we only support streaming task for now // todo: we only support streaming task for now
ITaskInstanceOperator iTaskInstanceOperator = SingletonJdkDynamicRpcClientProxyFactory.getInstance() ITaskInstanceOperator iTaskInstanceOperator = SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(taskInstance.getHost(), ITaskInstanceOperator.class); .getProxyClient(taskInstance.getHost(), ITaskInstanceOperator.class);
TaskInstanceKillResponse taskInstanceKillResponse = TaskInstanceKillResponse taskInstanceKillResponse =
iTaskInstanceOperator.killTask(new TaskInstanceKillRequest(taskInstanceId)); iTaskInstanceOperator.killTask(new TaskInstanceKillRequest(taskInstanceId));
@ -381,11 +381,11 @@ public class TaskInstanceServiceImpl extends BaseServiceImpl implements TaskInst
// delete log // delete log
if (StringUtils.isNotEmpty(taskInstance.getLogPath())) { if (StringUtils.isNotEmpty(taskInstance.getLogPath())) {
if (TaskUtils.isLogicTask(taskInstance.getTaskType())) { if (TaskUtils.isLogicTask(taskInstance.getTaskType())) {
IMasterLogService masterLogService = SingletonJdkDynamicRpcClientProxyFactory.getInstance() IMasterLogService masterLogService = SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(taskInstance.getHost(), IMasterLogService.class); .getProxyClient(taskInstance.getHost(), IMasterLogService.class);
masterLogService.removeLogicTaskInstanceLog(taskInstance.getLogPath()); masterLogService.removeLogicTaskInstanceLog(taskInstance.getLogPath());
} else { } else {
IWorkerLogService workerLogService = SingletonJdkDynamicRpcClientProxyFactory.getInstance() IWorkerLogService workerLogService = SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(taskInstance.getHost(), IWorkerLogService.class); .getProxyClient(taskInstance.getHost(), IWorkerLogService.class);
workerLogService.removeTaskInstanceLog(taskInstance.getLogPath()); workerLogService.removeTaskInstanceLog(taskInstance.getLogPath());
} }

36
dolphinscheduler-extract/dolphinscheduler-extract-base/src/main/java/org/apache/dolphinscheduler/extract/base/client/JdkDynamicRpcClientProxyFactory.java

@ -21,9 +21,16 @@ import org.apache.dolphinscheduler.extract.base.NettyRemotingClient;
import org.apache.dolphinscheduler.extract.base.utils.Host; import org.apache.dolphinscheduler.extract.base.utils.Host;
import java.lang.reflect.Proxy; import java.lang.reflect.Proxy;
import java.time.Duration;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import lombok.SneakyThrows;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
/** /**
* This class is used to create a proxy client which will transform local method invocation to remove invocation. * This class is used to create a proxy client which will transform local method invocation to remove invocation.
*/ */
@ -31,21 +38,34 @@ public class JdkDynamicRpcClientProxyFactory implements IRpcClientProxyFactory {
private final NettyRemotingClient nettyRemotingClient; private final NettyRemotingClient nettyRemotingClient;
// todo: use guava cache to avoid memory leak private static final LoadingCache<String, Map<String, Object>> proxyClientCache = CacheBuilder.newBuilder()
private final Map<String, Map<String, Object>> proxyClientCache = new ConcurrentHashMap<>(); // expire here to remove dead host
.expireAfterAccess(Duration.ofHours(1))
.build(new CacheLoader<String, Map<String, Object>>() {
@Override
public Map<String, Object> load(String key) {
return new ConcurrentHashMap<>();
}
});
public JdkDynamicRpcClientProxyFactory(NettyRemotingClient nettyRemotingClient) { public JdkDynamicRpcClientProxyFactory(NettyRemotingClient nettyRemotingClient) {
this.nettyRemotingClient = nettyRemotingClient; this.nettyRemotingClient = nettyRemotingClient;
} }
@SneakyThrows
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@Override @Override
public <T> T getProxyClient(String serverHost, Class<T> clientInterface) { public <T> T getProxyClient(String serverHost, Class<T> clientInterface) {
return (T) proxyClientCache return (T) proxyClientCache.get(serverHost)
.computeIfAbsent(serverHost, key -> new ConcurrentHashMap<>()) .computeIfAbsent(clientInterface.getName(), key -> newProxyClient(serverHost, clientInterface));
.computeIfAbsent(clientInterface.getName(), }
key -> Proxy.newProxyInstance(
clientInterface.getClassLoader(), new Class[]{clientInterface}, @SuppressWarnings("unchecked")
new ClientInvocationHandler(Host.of(serverHost), nettyRemotingClient))); private <T> T newProxyClient(String serverHost, Class<T> clientInterface) {
return (T) Proxy.newProxyInstance(
clientInterface.getClassLoader(),
new Class[]{clientInterface},
new ClientInvocationHandler(Host.of(serverHost), nettyRemotingClient));
} }
} }

14
dolphinscheduler-extract/dolphinscheduler-extract-base/src/main/java/org/apache/dolphinscheduler/extract/base/client/SingletonJdkDynamicRpcClientProxyFactory.java

@ -20,17 +20,13 @@ package org.apache.dolphinscheduler.extract.base.client;
import org.apache.dolphinscheduler.extract.base.NettyRemotingClientFactory; import org.apache.dolphinscheduler.extract.base.NettyRemotingClientFactory;
import org.apache.dolphinscheduler.extract.base.config.NettyClientConfig; import org.apache.dolphinscheduler.extract.base.config.NettyClientConfig;
public class SingletonJdkDynamicRpcClientProxyFactory extends JdkDynamicRpcClientProxyFactory { public class SingletonJdkDynamicRpcClientProxyFactory {
private static final SingletonJdkDynamicRpcClientProxyFactory INSTANCE = private static final JdkDynamicRpcClientProxyFactory INSTANCE = new JdkDynamicRpcClientProxyFactory(
new SingletonJdkDynamicRpcClientProxyFactory(); NettyRemotingClientFactory.buildNettyRemotingClient(new NettyClientConfig()));
private SingletonJdkDynamicRpcClientProxyFactory() { public static <T> T getProxyClient(String serverAddress, Class<T> clazz) {
super(NettyRemotingClientFactory.buildNettyRemotingClient(new NettyClientConfig())); return INSTANCE.getProxyClient(serverAddress, clazz);
}
public static SingletonJdkDynamicRpcClientProxyFactory getInstance() {
return INSTANCE;
} }
} }

2
dolphinscheduler-extract/dolphinscheduler-extract-base/src/main/java/org/apache/dolphinscheduler/extract/base/server/JdkDynamicServerHandler.java

@ -96,7 +96,7 @@ public class JdkDynamicServerHandler extends ChannelInboundHandlerAdapter {
StandardRpcRequest standardRpcRequest = StandardRpcRequest standardRpcRequest =
JsonSerializer.deserialize(transporter.getBody(), StandardRpcRequest.class); JsonSerializer.deserialize(transporter.getBody(), StandardRpcRequest.class);
Object[] args; Object[] args;
if (standardRpcRequest.getArgs().length == 0) { if (standardRpcRequest.getArgs() == null || standardRpcRequest.getArgs().length == 0) {
args = null; args = null;
} else { } else {
args = new Object[standardRpcRequest.getArgs().length]; args = new Object[standardRpcRequest.getArgs().length];

94
dolphinscheduler-extract/dolphinscheduler-extract-base/src/test/java/org/apache/dolphinscheduler/extract/base/client/SingletonJdkDynamicRpcClientProxyFactoryTest.java

@ -0,0 +1,94 @@
/*
* 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.extract.base.client;
import org.apache.dolphinscheduler.extract.base.NettyRemotingServer;
import org.apache.dolphinscheduler.extract.base.RpcMethod;
import org.apache.dolphinscheduler.extract.base.RpcService;
import org.apache.dolphinscheduler.extract.base.config.NettyServerConfig;
import org.apache.dolphinscheduler.extract.base.server.SpringServerMethodInvokerDiscovery;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
public class SingletonJdkDynamicRpcClientProxyFactoryTest {
private NettyRemotingServer nettyRemotingServer;
@BeforeEach
public void setUp() {
nettyRemotingServer = new NettyRemotingServer(new NettyServerConfig(12345));
nettyRemotingServer.start();
new SpringServerMethodInvokerDiscovery(nettyRemotingServer)
.postProcessAfterInitialization(new IServiceImpl(), "iServiceImpl");
}
@Test
public void getProxyClient() {
IService proxyClient =
SingletonJdkDynamicRpcClientProxyFactory.getProxyClient("localhost:12345", IService.class);
Assertions.assertNotNull(proxyClient);
}
@Test
public void testPing() {
IService proxyClient =
SingletonJdkDynamicRpcClientProxyFactory.getProxyClient("localhost:12345", IService.class);
String ping = proxyClient.ping("ping");
Assertions.assertEquals("pong", ping);
}
@Test
public void testVoid() {
IService proxyClient =
SingletonJdkDynamicRpcClientProxyFactory.getProxyClient("localhost:12345", IService.class);
Assertions.assertDoesNotThrow(proxyClient::voidMethod);
}
@AfterEach
public void tearDown() {
nettyRemotingServer.close();
}
@RpcService
public interface IService {
@RpcMethod
String ping(String ping);
@RpcMethod
void voidMethod();
}
public static class IServiceImpl implements IService {
@Override
public String ping(String ping) {
return "pong";
}
@Override
public void voidMethod() {
System.out.println("void method");
}
}
}

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskDelayEventHandler.java

@ -114,7 +114,7 @@ public class TaskDelayEventHandler implements TaskEventHandler {
private void sendAckToWorker(TaskEvent taskEvent) { private void sendAckToWorker(TaskEvent taskEvent) {
// If event handle success, send ack to worker to otherwise the worker will retry this event // If event handle success, send ack to worker to otherwise the worker will retry this event
ITaskInstanceExecutionEventAckListener instanceExecutionEventAckListener = ITaskInstanceExecutionEventAckListener instanceExecutionEventAckListener =
SingletonJdkDynamicRpcClientProxyFactory.getInstance() SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(taskEvent.getWorkerAddress(), ITaskInstanceExecutionEventAckListener.class); .getProxyClient(taskEvent.getWorkerAddress(), ITaskInstanceExecutionEventAckListener.class);
instanceExecutionEventAckListener.handleTaskInstanceExecutionRunningEventAck( instanceExecutionEventAckListener.handleTaskInstanceExecutionRunningEventAck(
TaskInstanceExecutionRunningEventAck.success(taskEvent.getTaskInstanceId())); TaskInstanceExecutionRunningEventAck.success(taskEvent.getTaskInstanceId()));

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskResultEventHandler.java

@ -116,7 +116,7 @@ public class TaskResultEventHandler implements TaskEventHandler {
public void sendAckToWorker(TaskEvent taskEvent) { public void sendAckToWorker(TaskEvent taskEvent) {
ITaskInstanceExecutionEventAckListener instanceExecutionEventAckListener = ITaskInstanceExecutionEventAckListener instanceExecutionEventAckListener =
SingletonJdkDynamicRpcClientProxyFactory.getInstance() SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(taskEvent.getWorkerAddress(), ITaskInstanceExecutionEventAckListener.class); .getProxyClient(taskEvent.getWorkerAddress(), ITaskInstanceExecutionEventAckListener.class);
instanceExecutionEventAckListener.handleTaskInstanceExecutionFinishEventAck( instanceExecutionEventAckListener.handleTaskInstanceExecutionFinishEventAck(
TaskInstanceExecutionFinishEventAck.success(taskEvent.getTaskInstanceId())); TaskInstanceExecutionFinishEventAck.success(taskEvent.getTaskInstanceId()));

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRunningEventHandler.java

@ -110,7 +110,7 @@ public class TaskRunningEventHandler implements TaskEventHandler {
private void sendAckToWorker(TaskEvent taskEvent) { private void sendAckToWorker(TaskEvent taskEvent) {
// If event handle success, send ack to worker to otherwise the worker will retry this event // If event handle success, send ack to worker to otherwise the worker will retry this event
ITaskInstanceExecutionEventAckListener instanceExecutionEventAckListener = ITaskInstanceExecutionEventAckListener instanceExecutionEventAckListener =
SingletonJdkDynamicRpcClientProxyFactory.getInstance() SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(taskEvent.getWorkerAddress(), ITaskInstanceExecutionEventAckListener.class); .getProxyClient(taskEvent.getWorkerAddress(), ITaskInstanceExecutionEventAckListener.class);
instanceExecutionEventAckListener.handleTaskInstanceExecutionRunningEventAck( instanceExecutionEventAckListener.handleTaskInstanceExecutionRunningEventAck(
TaskInstanceExecutionRunningEventAck.success(taskEvent.getTaskInstanceId())); TaskInstanceExecutionRunningEventAck.success(taskEvent.getTaskInstanceId()));

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskUpdatePidEventHandler.java

@ -94,7 +94,7 @@ public class TaskUpdatePidEventHandler implements TaskEventHandler {
private void sendAckToWorker(TaskEvent taskEvent) { private void sendAckToWorker(TaskEvent taskEvent) {
// If event handle success, send ack to worker to otherwise the worker will retry this event // If event handle success, send ack to worker to otherwise the worker will retry this event
ITaskInstanceExecutionEventAckListener instanceExecutionEventAckListener = ITaskInstanceExecutionEventAckListener instanceExecutionEventAckListener =
SingletonJdkDynamicRpcClientProxyFactory.getInstance() SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(taskEvent.getWorkerAddress(), ITaskInstanceExecutionEventAckListener.class); .getProxyClient(taskEvent.getWorkerAddress(), ITaskInstanceExecutionEventAckListener.class);
instanceExecutionEventAckListener.handleTaskInstanceExecutionInfoEventAck( instanceExecutionEventAckListener.handleTaskInstanceExecutionInfoEventAck(
TaskInstanceExecutionInfoEventAck.success(taskEvent.getTaskInstanceId())); TaskInstanceExecutionInfoEventAck.success(taskEvent.getTaskInstanceId()));

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StreamTaskExecuteRunnable.java

@ -473,7 +473,7 @@ public class StreamTaskExecuteRunnable implements Runnable {
private void sendAckToWorker(TaskEvent taskEvent) { private void sendAckToWorker(TaskEvent taskEvent) {
// If event handle success, send ack to worker to otherwise the worker will retry this event // If event handle success, send ack to worker to otherwise the worker will retry this event
ITaskInstanceExecutionEventAckListener instanceExecutionEventAckListener = ITaskInstanceExecutionEventAckListener instanceExecutionEventAckListener =
SingletonJdkDynamicRpcClientProxyFactory.getInstance() SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(taskEvent.getWorkerAddress(), ITaskInstanceExecutionEventAckListener.class); .getProxyClient(taskEvent.getWorkerAddress(), ITaskInstanceExecutionEventAckListener.class);
if (taskEvent.getEvent() == TaskEventType.RUNNING) { if (taskEvent.getEvent() == TaskEventType.RUNNING) {
log.error("taskEvent.getChannel() is null, taskEvent:{}", taskEvent); log.error("taskEvent.getChannel() is null, taskEvent:{}", taskEvent);

10
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnable.java

@ -60,6 +60,7 @@ import org.apache.dolphinscheduler.dao.utils.TaskCacheUtils;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory;
import org.apache.dolphinscheduler.extract.master.ILogicTaskInstanceOperator; import org.apache.dolphinscheduler.extract.master.ILogicTaskInstanceOperator;
import org.apache.dolphinscheduler.extract.master.transportor.TaskInstanceWakeupRequest; import org.apache.dolphinscheduler.extract.master.transportor.TaskInstanceWakeupRequest;
import org.apache.dolphinscheduler.extract.worker.ITaskInstanceOperator;
import org.apache.dolphinscheduler.extract.worker.transportor.UpdateWorkflowHostRequest; import org.apache.dolphinscheduler.extract.worker.transportor.UpdateWorkflowHostRequest;
import org.apache.dolphinscheduler.extract.worker.transportor.UpdateWorkflowHostResponse; import org.apache.dolphinscheduler.extract.worker.transportor.UpdateWorkflowHostResponse;
import org.apache.dolphinscheduler.plugin.task.api.enums.DependResult; import org.apache.dolphinscheduler.plugin.task.api.enums.DependResult;
@ -466,7 +467,7 @@ public class WorkflowExecuteRunnable implements IWorkflowExecuteRunnable {
} else { } else {
ProcessInstance processInstance = ProcessInstance processInstance =
processService.findProcessInstanceById(nextTaskInstance.getProcessInstanceId()); processService.findProcessInstanceById(nextTaskInstance.getProcessInstanceId());
ILogicTaskInstanceOperator taskInstanceOperator = SingletonJdkDynamicRpcClientProxyFactory.getInstance() ILogicTaskInstanceOperator taskInstanceOperator = SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(processInstance.getHost(), ILogicTaskInstanceOperator.class); .getProxyClient(processInstance.getHost(), ILogicTaskInstanceOperator.class);
taskInstanceOperator.wakeupTaskInstance( taskInstanceOperator.wakeupTaskInstance(
new TaskInstanceWakeupRequest(processInstance.getId(), nextTaskInstance.getId())); new TaskInstanceWakeupRequest(processInstance.getId(), nextTaskInstance.getId()));
@ -1385,10 +1386,9 @@ public class WorkflowExecuteRunnable implements IWorkflowExecuteRunnable {
return false; return false;
} }
try { try {
org.apache.dolphinscheduler.extract.worker.ITaskInstanceOperator iTaskInstanceOperator = ITaskInstanceOperator iTaskInstanceOperator =
SingletonJdkDynamicRpcClientProxyFactory.getInstance() SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(taskInstance.getHost(), .getProxyClient(taskInstance.getHost(), ITaskInstanceOperator.class);
org.apache.dolphinscheduler.extract.worker.ITaskInstanceOperator.class);
UpdateWorkflowHostResponse updateWorkflowHostResponse = iTaskInstanceOperator.updateWorkflowInstanceHost( UpdateWorkflowHostResponse updateWorkflowHostResponse = iTaskInstanceOperator.updateWorkflowInstanceHost(
new UpdateWorkflowHostRequest(taskInstance.getId(), masterConfig.getMasterAddress())); new UpdateWorkflowHostRequest(taskInstance.getId(), masterConfig.getMasterAddress()));
if (!updateWorkflowHostResponse.isSuccess()) { if (!updateWorkflowHostResponse.isSuccess()) {

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteThreadPool.java

@ -226,7 +226,7 @@ public class WorkflowExecuteThreadPool extends ThreadPoolTaskExecutor {
return; return;
} }
ITaskInstanceExecutionEventListener iTaskInstanceExecutionEventListener = ITaskInstanceExecutionEventListener iTaskInstanceExecutionEventListener =
SingletonJdkDynamicRpcClientProxyFactory.getInstance() SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(processInstanceHost, ITaskInstanceExecutionEventListener.class); .getProxyClient(processInstanceHost, ITaskInstanceExecutionEventListener.class);
WorkflowInstanceStateChangeEvent workflowInstanceStateChangeEvent = new WorkflowInstanceStateChangeEvent( WorkflowInstanceStateChangeEvent workflowInstanceStateChangeEvent = new WorkflowInstanceStateChangeEvent(

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcher.java

@ -51,7 +51,7 @@ public class MasterTaskDispatcher extends BaseTaskDispatcher {
protected void doDispatch(TaskExecuteRunnable taskExecuteRunnable) throws TaskDispatchException { protected void doDispatch(TaskExecuteRunnable taskExecuteRunnable) throws TaskDispatchException {
TaskExecutionContext taskExecutionContext = taskExecuteRunnable.getTaskExecutionContext(); TaskExecutionContext taskExecutionContext = taskExecuteRunnable.getTaskExecutionContext();
try { try {
ILogicTaskInstanceOperator taskInstanceOperator = SingletonJdkDynamicRpcClientProxyFactory.getInstance() ILogicTaskInstanceOperator taskInstanceOperator = SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(taskExecutionContext.getHost(), ILogicTaskInstanceOperator.class); .getProxyClient(taskExecutionContext.getHost(), ILogicTaskInstanceOperator.class);
LogicTaskDispatchResponse logicTaskDispatchResponse = taskInstanceOperator LogicTaskDispatchResponse logicTaskDispatchResponse = taskInstanceOperator
.dispatchLogicTask(new LogicTaskDispatchRequest(taskExecuteRunnable.getTaskExecutionContext())); .dispatchLogicTask(new LogicTaskDispatchRequest(taskExecuteRunnable.getTaskExecutionContext()));

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcher.java

@ -56,7 +56,7 @@ public class WorkerTaskDispatcher extends BaseTaskDispatcher {
protected void doDispatch(TaskExecuteRunnable taskExecuteRunnable) throws TaskDispatchException { protected void doDispatch(TaskExecuteRunnable taskExecuteRunnable) throws TaskDispatchException {
TaskExecutionContext taskExecutionContext = taskExecuteRunnable.getTaskExecutionContext(); TaskExecutionContext taskExecutionContext = taskExecuteRunnable.getTaskExecutionContext();
try { try {
ITaskInstanceOperator taskInstanceOperator = SingletonJdkDynamicRpcClientProxyFactory.getInstance() ITaskInstanceOperator taskInstanceOperator = SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(taskExecutionContext.getHost(), ITaskInstanceOperator.class); .getProxyClient(taskExecutionContext.getHost(), ITaskInstanceOperator.class);
TaskInstanceDispatchResponse taskInstanceDispatchResponse = taskInstanceOperator TaskInstanceDispatchResponse taskInstanceDispatchResponse = taskInstanceOperator
.dispatchTask(new TaskInstanceDispatchRequest(taskExecuteRunnable.getTaskExecutionContext())); .dispatchTask(new TaskInstanceDispatchRequest(taskExecuteRunnable.getTaskExecutionContext()));

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecuteRunningEventSender.java

@ -34,7 +34,7 @@ public class LogicTaskInstanceExecuteRunningEventSender
@Override @Override
public void sendMessage(TaskInstanceExecutionRunningEvent taskInstanceExecutionRunningEvent) { public void sendMessage(TaskInstanceExecutionRunningEvent taskInstanceExecutionRunningEvent) {
ITaskInstanceExecutionEventListener iTaskInstanceExecutionEventListener = ITaskInstanceExecutionEventListener iTaskInstanceExecutionEventListener =
SingletonJdkDynamicRpcClientProxyFactory.getInstance() SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(taskInstanceExecutionRunningEvent.getHost(), .getProxyClient(taskInstanceExecutionRunningEvent.getHost(),
ITaskInstanceExecutionEventListener.class); ITaskInstanceExecutionEventListener.class);
iTaskInstanceExecutionEventListener.onTaskInstanceExecutionRunning(taskInstanceExecutionRunningEvent); iTaskInstanceExecutionEventListener.onTaskInstanceExecutionRunning(taskInstanceExecutionRunningEvent);

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionFinishEventSender.java

@ -32,7 +32,7 @@ public class LogicTaskInstanceExecutionFinishEventSender
@Override @Override
public void sendMessage(TaskInstanceExecutionFinishEvent message) { public void sendMessage(TaskInstanceExecutionFinishEvent message) {
ITaskInstanceExecutionEventListener iTaskInstanceExecutionEventListener = ITaskInstanceExecutionEventListener iTaskInstanceExecutionEventListener =
SingletonJdkDynamicRpcClientProxyFactory.getInstance() SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(message.getHost(), ITaskInstanceExecutionEventListener.class); .getProxyClient(message.getHost(), ITaskInstanceExecutionEventListener.class);
iTaskInstanceExecutionEventListener.onTaskInstanceExecutionFinish(message); iTaskInstanceExecutionEventListener.onTaskInstanceExecutionFinish(message);
} }

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/LogicTaskExecuteRunnableKillOperator.java

@ -45,7 +45,7 @@ public class LogicTaskExecuteRunnableKillOperator extends BaseTaskExecuteRunnabl
taskInstance.getName()); taskInstance.getName());
return; return;
} }
final ILogicTaskInstanceOperator taskInstanceOperator = SingletonJdkDynamicRpcClientProxyFactory.getInstance() final ILogicTaskInstanceOperator taskInstanceOperator = SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(taskInstance.getHost(), ILogicTaskInstanceOperator.class); .getProxyClient(taskInstance.getHost(), ILogicTaskInstanceOperator.class);
final LogicTaskKillRequest logicTaskKillRequest = new LogicTaskKillRequest(taskInstance.getId()); final LogicTaskKillRequest logicTaskKillRequest = new LogicTaskKillRequest(taskInstance.getId());
final LogicTaskKillResponse logicTaskKillResponse = taskInstanceOperator.killLogicTask(logicTaskKillRequest); final LogicTaskKillResponse logicTaskKillResponse = taskInstanceOperator.killLogicTask(logicTaskKillRequest);

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/LogicTaskExecuteRunnablePauseOperator.java

@ -40,7 +40,7 @@ public class LogicTaskExecuteRunnablePauseOperator extends BaseTaskExecuteRunnab
taskInstance.getName()); taskInstance.getName());
return; return;
} }
final ILogicTaskInstanceOperator taskInstanceOperator = SingletonJdkDynamicRpcClientProxyFactory.getInstance() final ILogicTaskInstanceOperator taskInstanceOperator = SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(taskInstance.getHost(), ILogicTaskInstanceOperator.class); .getProxyClient(taskInstance.getHost(), ILogicTaskInstanceOperator.class);
final LogicTaskPauseRequest logicTaskPauseRequest = new LogicTaskPauseRequest(taskInstance.getId()); final LogicTaskPauseRequest logicTaskPauseRequest = new LogicTaskPauseRequest(taskInstance.getId());
final LogicTaskPauseResponse logicTaskPauseResponse = final LogicTaskPauseResponse logicTaskPauseResponse =

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/LogicTaskExecuteRunnableTimeoutOperator.java

@ -47,7 +47,7 @@ public class LogicTaskExecuteRunnableTimeoutOperator extends BaseTaskExecuteRunn
} }
final ILogicTaskInstanceOperator iLogicTaskInstanceOperator = final ILogicTaskInstanceOperator iLogicTaskInstanceOperator =
SingletonJdkDynamicRpcClientProxyFactory.getInstance() SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(taskInstance.getHost(), ILogicTaskInstanceOperator.class); .getProxyClient(taskInstance.getHost(), ILogicTaskInstanceOperator.class);
final LogicTaskKillRequest taskInstanceKillRequest = new LogicTaskKillRequest(taskInstance.getId()); final LogicTaskKillRequest taskInstanceKillRequest = new LogicTaskKillRequest(taskInstance.getId());

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskExecuteRunnableKillOperator.java

@ -43,7 +43,7 @@ public class TaskExecuteRunnableKillOperator extends BaseTaskExecuteRunnableKill
log.info("TaskInstance {} host is empty, no need to killRemoteTask", taskInstance.getName()); log.info("TaskInstance {} host is empty, no need to killRemoteTask", taskInstance.getName());
return; return;
} }
ITaskInstanceOperator taskInstanceOperator = SingletonJdkDynamicRpcClientProxyFactory.getInstance() ITaskInstanceOperator taskInstanceOperator = SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(taskInstance.getHost(), ITaskInstanceOperator.class); .getProxyClient(taskInstance.getHost(), ITaskInstanceOperator.class);
TaskInstanceKillRequest taskInstanceKillRequest = new TaskInstanceKillRequest(taskInstance.getId()); TaskInstanceKillRequest taskInstanceKillRequest = new TaskInstanceKillRequest(taskInstance.getId());
TaskInstanceKillResponse taskInstanceKillResponse = taskInstanceOperator.killTask(taskInstanceKillRequest); TaskInstanceKillResponse taskInstanceKillResponse = taskInstanceOperator.killTask(taskInstanceKillRequest);

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskExecuteRunnablePauseOperator.java

@ -48,7 +48,7 @@ public class TaskExecuteRunnablePauseOperator implements TaskExecuteRunnableOper
log.info("The TaskInstance: {} host is null, no need to pauseRemoteTaskInstance", taskInstance.getName()); log.info("The TaskInstance: {} host is null, no need to pauseRemoteTaskInstance", taskInstance.getName());
return; return;
} }
final ITaskInstanceOperator taskInstanceOperator = SingletonJdkDynamicRpcClientProxyFactory.getInstance() final ITaskInstanceOperator taskInstanceOperator = SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(taskInstance.getHost(), ITaskInstanceOperator.class); .getProxyClient(taskInstance.getHost(), ITaskInstanceOperator.class);
final TaskInstancePauseRequest taskInstancePauseRequest = new TaskInstancePauseRequest(taskInstance.getId()); final TaskInstancePauseRequest taskInstancePauseRequest = new TaskInstancePauseRequest(taskInstance.getId());
final TaskInstancePauseResponse taskInstancePauseResponse = final TaskInstancePauseResponse taskInstancePauseResponse =

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskExecuteRunnableTimeoutOperator.java

@ -45,7 +45,7 @@ public class TaskExecuteRunnableTimeoutOperator extends BaseTaskExecuteRunnableT
return; return;
} }
final ITaskInstanceOperator iTaskInstanceOperator = SingletonJdkDynamicRpcClientProxyFactory.getInstance() final ITaskInstanceOperator iTaskInstanceOperator = SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(taskInstance.getHost(), ITaskInstanceOperator.class); .getProxyClient(taskInstance.getHost(), ITaskInstanceOperator.class);
final TaskInstanceKillRequest taskInstanceKillRequest = new TaskInstanceKillRequest(taskInstance.getId()); final TaskInstanceKillRequest taskInstanceKillRequest = new TaskInstanceKillRequest(taskInstance.getId());

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicLogicTask.java

@ -286,7 +286,7 @@ public class DynamicLogicTask extends BaseAsyncLogicTask<DynamicParameters> {
private void sendToSubProcess(TaskExecutionContext taskExecutionContext, ProcessInstance subProcessInstance) { private void sendToSubProcess(TaskExecutionContext taskExecutionContext, ProcessInstance subProcessInstance) {
final ITaskInstanceExecutionEventListener iTaskInstanceExecutionEventListener = final ITaskInstanceExecutionEventListener iTaskInstanceExecutionEventListener =
SingletonJdkDynamicRpcClientProxyFactory.getInstance() SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(subProcessInstance.getHost(), ITaskInstanceExecutionEventListener.class); .getProxyClient(subProcessInstance.getHost(), ITaskInstanceExecutionEventListener.class);
final WorkflowInstanceStateChangeEvent workflowInstanceStateChangeEvent = new WorkflowInstanceStateChangeEvent( final WorkflowInstanceStateChangeEvent workflowInstanceStateChangeEvent = new WorkflowInstanceStateChangeEvent(
taskExecutionContext.getProcessInstanceId(), taskExecutionContext.getProcessInstanceId(),

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTask.java

@ -136,7 +136,7 @@ public class SubWorkflowLogicTask extends BaseAsyncLogicTask<SubProcessParameter
private void sendToSubProcess(TaskExecutionContext taskExecutionContext, private void sendToSubProcess(TaskExecutionContext taskExecutionContext,
ProcessInstance subProcessInstance) { ProcessInstance subProcessInstance) {
final ITaskInstanceExecutionEventListener iTaskInstanceExecutionEventListener = final ITaskInstanceExecutionEventListener iTaskInstanceExecutionEventListener =
SingletonJdkDynamicRpcClientProxyFactory.getInstance() SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(subProcessInstance.getHost(), ITaskInstanceExecutionEventListener.class); .getProxyClient(subProcessInstance.getHost(), ITaskInstanceExecutionEventListener.class);
final WorkflowInstanceStateChangeEvent workflowInstanceStateChangeEvent = new WorkflowInstanceStateChangeEvent( final WorkflowInstanceStateChangeEvent workflowInstanceStateChangeEvent = new WorkflowInstanceStateChangeEvent(
taskExecutionContext.getProcessInstanceId(), taskExecutionContext.getProcessInstanceId(),

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/service/WorkerFailoverService.java

@ -274,7 +274,7 @@ public class WorkerFailoverService {
.create(); .create();
// only kill yarn/k8s job if exists , the local thread has exited // only kill yarn/k8s job if exists , the local thread has exited
log.info("TaskInstance failover begin kill the task related yarn or k8s job"); log.info("TaskInstance failover begin kill the task related yarn or k8s job");
IWorkerLogService iWorkerLogService = SingletonJdkDynamicRpcClientProxyFactory.getInstance() IWorkerLogService iWorkerLogService = SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(taskInstance.getHost(), IWorkerLogService.class); .getProxyClient(taskInstance.getHost(), IWorkerLogService.class);
GetAppIdResponse getAppIdResponse = GetAppIdResponse getAppIdResponse =
iWorkerLogService.getAppId(new GetAppIdRequest(taskInstance.getId(), taskInstance.getLogPath())); iWorkerLogService.getAppId(new GetAppIdRequest(taskInstance.getId(), taskInstance.getLogPath()));

5
dolphinscheduler-microbench/pom.xml

@ -48,6 +48,11 @@
<dependencies> <dependencies>
<dependency>
<groupId>org.apache.dolphinscheduler</groupId>
<artifactId>dolphinscheduler-extract-base</artifactId>
</dependency>
<dependency> <dependency>
<groupId>org.openjdk.jmh</groupId> <groupId>org.openjdk.jmh</groupId>
<artifactId>jmh-core</artifactId> <artifactId>jmh-core</artifactId>

29
dolphinscheduler-microbench/src/main/java/org/apache/dolphinscheduler/microbench/rpc/IService.java

@ -0,0 +1,29 @@
/*
* 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.microbench.rpc;
import org.apache.dolphinscheduler.extract.base.RpcMethod;
import org.apache.dolphinscheduler.extract.base.RpcService;
@RpcService
public interface IService {
@RpcMethod
String ping(String pingRequest);
}

27
dolphinscheduler-microbench/src/main/java/org/apache/dolphinscheduler/microbench/rpc/IServiceImpl.java

@ -0,0 +1,27 @@
/*
* 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.microbench.rpc;
public class IServiceImpl implements IService {
@Override
public String ping(String pingRequest) {
return "I get " + pingRequest + ", I am Pong!";
}
}

76
dolphinscheduler-microbench/src/main/java/org/apache/dolphinscheduler/microbench/rpc/RpcBenchMarkTest.java

@ -0,0 +1,76 @@
/*
* 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.microbench.rpc;
import org.apache.dolphinscheduler.extract.base.NettyRemotingServer;
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory;
import org.apache.dolphinscheduler.extract.base.config.NettyServerConfig;
import org.apache.dolphinscheduler.extract.base.server.SpringServerMethodInvokerDiscovery;
import org.apache.dolphinscheduler.microbench.base.AbstractBaseBenchmark;
import java.util.concurrent.TimeUnit;
import lombok.extern.slf4j.Slf4j;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Measurement;
import org.openjdk.jmh.annotations.Mode;
import org.openjdk.jmh.annotations.OutputTimeUnit;
import org.openjdk.jmh.annotations.Scope;
import org.openjdk.jmh.annotations.Setup;
import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.annotations.TearDown;
import org.openjdk.jmh.annotations.Warmup;
import org.openjdk.jmh.infra.Blackhole;
@Slf4j
@Warmup(iterations = 5, time = 1)
@Measurement(iterations = 10, time = 1)
@State(Scope.Benchmark)
@BenchmarkMode({Mode.Throughput, Mode.AverageTime, Mode.SampleTime})
public class RpcBenchMarkTest extends AbstractBaseBenchmark {
private NettyRemotingServer nettyRemotingServer;
private IService iService;
@Setup
public void before() {
nettyRemotingServer = new NettyRemotingServer(new NettyServerConfig(12345));
nettyRemotingServer.start();
SpringServerMethodInvokerDiscovery springServerMethodInvokerDiscovery =
new SpringServerMethodInvokerDiscovery(nettyRemotingServer);
springServerMethodInvokerDiscovery.postProcessAfterInitialization(new IServiceImpl(), "iServiceImpl");
iService =
SingletonJdkDynamicRpcClientProxyFactory.getProxyClient("localhost:12345", IService.class);
}
@Benchmark
@BenchmarkMode({Mode.Throughput, Mode.AverageTime, Mode.SampleTime})
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void sendTest(Blackhole bh) {
String pong = iService.ping("ping");
bh.consume(pong);
}
@TearDown
public void after() {
nettyRemotingServer.close();
}
}

8
dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessServiceImpl.java

@ -389,8 +389,8 @@ public class ProcessServiceImpl implements ProcessService {
if (update) { if (update) {
try { try {
final ITaskInstanceExecutionEventListener iTaskInstanceExecutionEventListener = final ITaskInstanceExecutionEventListener iTaskInstanceExecutionEventListener =
SingletonJdkDynamicRpcClientProxyFactory.getInstance() SingletonJdkDynamicRpcClientProxyFactory.getProxyClient(info.getHost(),
.getProxyClient(info.getHost(), ITaskInstanceExecutionEventListener.class); ITaskInstanceExecutionEventListener.class);
final WorkflowInstanceStateChangeEvent workflowInstanceStateChangeEvent = final WorkflowInstanceStateChangeEvent workflowInstanceStateChangeEvent =
new WorkflowInstanceStateChangeEvent(info.getId(), 0, info.getState(), info.getId(), 0); new WorkflowInstanceStateChangeEvent(info.getId(), 0, info.getState(), info.getId(), 0);
iTaskInstanceExecutionEventListener iTaskInstanceExecutionEventListener
@ -516,11 +516,11 @@ public class ProcessServiceImpl implements ProcessService {
continue; continue;
} }
if (TaskUtils.isLogicTask(taskInstance.getTaskType())) { if (TaskUtils.isLogicTask(taskInstance.getTaskType())) {
IMasterLogService masterLogService = SingletonJdkDynamicRpcClientProxyFactory.getInstance() IMasterLogService masterLogService = SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(taskInstance.getHost(), IMasterLogService.class); .getProxyClient(taskInstance.getHost(), IMasterLogService.class);
masterLogService.removeLogicTaskInstanceLog(taskLogPath); masterLogService.removeLogicTaskInstanceLog(taskLogPath);
} else { } else {
IWorkerLogService iWorkerLogService = SingletonJdkDynamicRpcClientProxyFactory.getInstance() IWorkerLogService iWorkerLogService = SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(taskInstance.getHost(), IWorkerLogService.class); .getProxyClient(taskInstance.getHost(), IWorkerLogService.class);
iWorkerLogService.removeTaskInstanceLog(taskLogPath); iWorkerLogService.removeTaskInstanceLog(taskLogPath);
} }

2
dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskInstanceExecutionFinishEventSender.java

@ -33,7 +33,7 @@ public class TaskInstanceExecutionFinishEventSender
@Override @Override
public void sendEvent(TaskInstanceExecutionFinishEvent taskInstanceExecutionFinishEvent) { public void sendEvent(TaskInstanceExecutionFinishEvent taskInstanceExecutionFinishEvent) {
ITaskInstanceExecutionEventListener iTaskInstanceExecutionEventListener = ITaskInstanceExecutionEventListener iTaskInstanceExecutionEventListener =
SingletonJdkDynamicRpcClientProxyFactory.getInstance() SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(taskInstanceExecutionFinishEvent.getHost(), .getProxyClient(taskInstanceExecutionFinishEvent.getHost(),
ITaskInstanceExecutionEventListener.class); ITaskInstanceExecutionEventListener.class);
iTaskInstanceExecutionEventListener.onTaskInstanceExecutionFinish(taskInstanceExecutionFinishEvent); iTaskInstanceExecutionEventListener.onTaskInstanceExecutionFinish(taskInstanceExecutionFinishEvent);

2
dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskInstanceExecutionInfoUpdateEventSender.java

@ -35,7 +35,7 @@ public class TaskInstanceExecutionInfoUpdateEventSender
@Override @Override
public void sendEvent(TaskInstanceExecutionInfoEvent taskInstanceExecutionInfoEvent) { public void sendEvent(TaskInstanceExecutionInfoEvent taskInstanceExecutionInfoEvent) {
ITaskInstanceExecutionEventListener iTaskInstanceExecutionEventListener = ITaskInstanceExecutionEventListener iTaskInstanceExecutionEventListener =
SingletonJdkDynamicRpcClientProxyFactory.getInstance() SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(taskInstanceExecutionInfoEvent.getHost(), .getProxyClient(taskInstanceExecutionInfoEvent.getHost(),
ITaskInstanceExecutionEventListener.class); ITaskInstanceExecutionEventListener.class);
iTaskInstanceExecutionEventListener.onTaskInstanceExecutionInfoUpdate(taskInstanceExecutionInfoEvent); iTaskInstanceExecutionEventListener.onTaskInstanceExecutionInfoUpdate(taskInstanceExecutionInfoEvent);

2
dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskInstanceExecutionRunningEventSender.java

@ -35,7 +35,7 @@ public class TaskInstanceExecutionRunningEventSender
@Override @Override
public void sendEvent(TaskInstanceExecutionRunningEvent taskInstanceExecutionRunningEvent) { public void sendEvent(TaskInstanceExecutionRunningEvent taskInstanceExecutionRunningEvent) {
ITaskInstanceExecutionEventListener iTaskInstanceExecutionEventListener = ITaskInstanceExecutionEventListener iTaskInstanceExecutionEventListener =
SingletonJdkDynamicRpcClientProxyFactory.getInstance() SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(taskInstanceExecutionRunningEvent.getHost(), .getProxyClient(taskInstanceExecutionRunningEvent.getHost(),
ITaskInstanceExecutionEventListener.class); ITaskInstanceExecutionEventListener.class);
iTaskInstanceExecutionEventListener.onTaskInstanceExecutionRunning(taskInstanceExecutionRunningEvent); iTaskInstanceExecutionEventListener.onTaskInstanceExecutionRunning(taskInstanceExecutionRunningEvent);

2
dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecuteRunnable.java

@ -257,7 +257,7 @@ public abstract class WorkerTaskExecuteRunnable implements Runnable {
task.getExitStatus() == TaskExecutionStatus.SUCCESS ? WarningType.SUCCESS.getCode() task.getExitStatus() == TaskExecutionStatus.SUCCESS ? WarningType.SUCCESS.getCode()
: WarningType.FAILURE.getCode()); : WarningType.FAILURE.getCode());
try { try {
IAlertOperator alertOperator = SingletonJdkDynamicRpcClientProxyFactory.getInstance() IAlertOperator alertOperator = SingletonJdkDynamicRpcClientProxyFactory
.getProxyClient(alertServerAddress.getAddress(), IAlertOperator.class); .getProxyClient(alertServerAddress.getAddress(), IAlertOperator.class);
AlertSendResponse alertSendResponse = alertOperator.sendAlert(alertSendRequest); AlertSendResponse alertSendResponse = alertOperator.sendAlert(alertSendRequest);
log.info("Send alert to: {} successfully, response: {}", alertServerAddress, alertSendResponse); log.info("Send alert to: {} successfully, response: {}", alertServerAddress, alertSendResponse);

Loading…
Cancel
Save