Wenjun Ruan
2 years ago
committed by
GitHub
57 changed files with 1290 additions and 1372 deletions
@ -0,0 +1,57 @@
|
||||
/* |
||||
* 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.remote.command; |
||||
|
||||
import java.io.Serializable; |
||||
|
||||
import lombok.Data; |
||||
import lombok.NoArgsConstructor; |
||||
|
||||
/** |
||||
* This is the base class for rpc message. |
||||
* <p> |
||||
* Since we use async mode, the client send a message and will wait the target server |
||||
* send ack for the message, the client will retry during a while if he doesn't receive an ack. |
||||
* <p> |
||||
* When there is a network error, the server cannot send ack to the client by the origin channel, |
||||
* since the client has closed the channel, so the server need to know the command source. |
||||
*/ |
||||
@Data |
||||
@NoArgsConstructor |
||||
public abstract class BaseCommand implements Serializable { |
||||
|
||||
private static final long serialVersionUID = -1L; |
||||
|
||||
/** |
||||
* If the message receiver want to send ack to the sender, need to use this address. |
||||
*/ |
||||
protected String messageSenderAddress; |
||||
|
||||
/** |
||||
* The message receiver address. |
||||
*/ |
||||
protected String messageReceiverAddress; |
||||
|
||||
protected long messageSendTime; |
||||
|
||||
protected BaseCommand(String messageSenderAddress, String messageReceiverAddress, long messageSendTime) { |
||||
this.messageSenderAddress = messageSenderAddress; |
||||
this.messageReceiverAddress = messageReceiverAddress; |
||||
this.messageSendTime = messageSendTime; |
||||
} |
||||
} |
@ -1,212 +0,0 @@
|
||||
/* |
||||
* 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.remote.command; |
||||
|
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils; |
||||
|
||||
import java.io.Serializable; |
||||
import java.util.Date; |
||||
|
||||
/** |
||||
* execute task response command |
||||
*/ |
||||
public class TaskExecuteResponseCommand implements Serializable { |
||||
|
||||
public TaskExecuteResponseCommand() { |
||||
} |
||||
|
||||
public TaskExecuteResponseCommand(int taskInstanceId, int processInstanceId) { |
||||
this.taskInstanceId = taskInstanceId; |
||||
this.processInstanceId = processInstanceId; |
||||
} |
||||
|
||||
/** |
||||
* task instance id |
||||
*/ |
||||
private int taskInstanceId; |
||||
|
||||
/** |
||||
* process instance id |
||||
*/ |
||||
private int processInstanceId; |
||||
|
||||
/** |
||||
* status |
||||
*/ |
||||
private int status; |
||||
|
||||
/** |
||||
* startTime |
||||
*/ |
||||
private Date startTime; |
||||
|
||||
/** |
||||
* host |
||||
*/ |
||||
private String host; |
||||
|
||||
/** |
||||
* logPath |
||||
*/ |
||||
private String logPath; |
||||
|
||||
/** |
||||
* executePath |
||||
*/ |
||||
private String executePath; |
||||
|
||||
|
||||
/** |
||||
* end time |
||||
*/ |
||||
private Date endTime; |
||||
|
||||
|
||||
/** |
||||
* processId |
||||
*/ |
||||
private int processId; |
||||
|
||||
/** |
||||
* appIds |
||||
*/ |
||||
private String appIds; |
||||
|
||||
/** |
||||
* varPool string |
||||
*/ |
||||
private String varPool; |
||||
|
||||
public Date getStartTime() { |
||||
return startTime; |
||||
} |
||||
|
||||
public void setStartTime(Date startTime) { |
||||
this.startTime = startTime; |
||||
} |
||||
|
||||
public String getHost() { |
||||
return host; |
||||
} |
||||
|
||||
public void setHost(String host) { |
||||
this.host = host; |
||||
} |
||||
|
||||
public String getLogPath() { |
||||
return logPath; |
||||
} |
||||
|
||||
public void setLogPath(String logPath) { |
||||
this.logPath = logPath; |
||||
} |
||||
|
||||
public String getExecutePath() { |
||||
return executePath; |
||||
} |
||||
|
||||
public void setExecutePath(String executePath) { |
||||
this.executePath = executePath; |
||||
} |
||||
|
||||
public void setVarPool(String varPool) { |
||||
this.varPool = varPool; |
||||
} |
||||
|
||||
public String getVarPool() { |
||||
return varPool; |
||||
} |
||||
|
||||
public int getTaskInstanceId() { |
||||
return taskInstanceId; |
||||
} |
||||
|
||||
public void setTaskInstanceId(int taskInstanceId) { |
||||
this.taskInstanceId = taskInstanceId; |
||||
} |
||||
|
||||
public int getStatus() { |
||||
return status; |
||||
} |
||||
|
||||
public void setStatus(int status) { |
||||
this.status = status; |
||||
} |
||||
|
||||
public Date getEndTime() { |
||||
return endTime; |
||||
} |
||||
|
||||
public void setEndTime(Date endTime) { |
||||
this.endTime = endTime; |
||||
} |
||||
|
||||
public int getProcessId() { |
||||
return processId; |
||||
} |
||||
|
||||
public void setProcessId(int processId) { |
||||
this.processId = processId; |
||||
} |
||||
|
||||
public String getAppIds() { |
||||
return appIds; |
||||
} |
||||
|
||||
public void setAppIds(String appIds) { |
||||
this.appIds = appIds; |
||||
} |
||||
|
||||
/** |
||||
* package response command |
||||
* |
||||
* @return command |
||||
*/ |
||||
public Command convert2Command() { |
||||
Command command = new Command(); |
||||
command.setType(CommandType.TASK_EXECUTE_RESPONSE); |
||||
byte[] body = JSONUtils.toJsonByteArray(this); |
||||
command.setBody(body); |
||||
return command; |
||||
} |
||||
|
||||
@Override |
||||
public String toString() { |
||||
return "TaskExecuteResponseCommand{" |
||||
+ "taskInstanceId=" + taskInstanceId |
||||
+ ", processInstanceId=" + processInstanceId |
||||
+ ", status=" + status |
||||
+ ", startTime=" + startTime |
||||
+ ", endTime=" + endTime |
||||
+ ", host=" + host |
||||
+ ", logPath=" + logPath |
||||
+ ", executePath=" + executePath |
||||
+ ", processId=" + processId |
||||
+ ", appIds='" + appIds + '\'' |
||||
+ ", varPool=" + varPool |
||||
+ '}'; |
||||
} |
||||
|
||||
public int getProcessInstanceId() { |
||||
return processInstanceId; |
||||
} |
||||
|
||||
public void setProcessInstanceId(int processInstanceId) { |
||||
this.processInstanceId = processInstanceId; |
||||
} |
||||
} |
@ -0,0 +1,111 @@
|
||||
/* |
||||
* 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.remote.command; |
||||
|
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils; |
||||
|
||||
import java.util.Date; |
||||
|
||||
import lombok.Data; |
||||
import lombok.EqualsAndHashCode; |
||||
import lombok.NoArgsConstructor; |
||||
import lombok.ToString; |
||||
|
||||
/** |
||||
* execute task response command |
||||
*/ |
||||
@Data |
||||
@NoArgsConstructor |
||||
@ToString(callSuper = true) |
||||
@EqualsAndHashCode(callSuper = true) |
||||
public class TaskExecuteResultCommand extends BaseCommand { |
||||
|
||||
public TaskExecuteResultCommand(String messageSenderAddress, String messageReceiverAddress, long messageSendTime) { |
||||
super(messageSenderAddress, messageReceiverAddress, messageSendTime); |
||||
} |
||||
|
||||
/** |
||||
* task instance id |
||||
*/ |
||||
private int taskInstanceId; |
||||
|
||||
/** |
||||
* process instance id |
||||
*/ |
||||
private int processInstanceId; |
||||
|
||||
/** |
||||
* status |
||||
*/ |
||||
private int status; |
||||
|
||||
/** |
||||
* startTime |
||||
*/ |
||||
private Date startTime; |
||||
|
||||
/** |
||||
* host |
||||
*/ |
||||
private String host; |
||||
|
||||
/** |
||||
* logPath |
||||
*/ |
||||
private String logPath; |
||||
|
||||
/** |
||||
* executePath |
||||
*/ |
||||
private String executePath; |
||||
|
||||
|
||||
/** |
||||
* end time |
||||
*/ |
||||
private Date endTime; |
||||
|
||||
|
||||
/** |
||||
* processId |
||||
*/ |
||||
private int processId; |
||||
|
||||
/** |
||||
* appIds |
||||
*/ |
||||
private String appIds; |
||||
|
||||
/** |
||||
* varPool string |
||||
*/ |
||||
private String varPool; |
||||
|
||||
/** |
||||
* package response command |
||||
* |
||||
* @return command |
||||
*/ |
||||
public Command convert2Command() { |
||||
Command command = new Command(); |
||||
command.setType(CommandType.TASK_EXECUTE_RESULT); |
||||
byte[] body = JSONUtils.toJsonByteArray(this); |
||||
command.setBody(body); |
||||
return command; |
||||
} |
||||
} |
@ -1,115 +0,0 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.worker.cache; |
||||
|
||||
import org.apache.dolphinscheduler.common.enums.TaskEventType; |
||||
import org.apache.dolphinscheduler.remote.command.Command; |
||||
|
||||
import java.util.Map; |
||||
import java.util.concurrent.ConcurrentHashMap; |
||||
|
||||
/** |
||||
* Response Cache : cache worker send master result |
||||
*/ |
||||
public class ResponseCache { |
||||
|
||||
private static final ResponseCache instance = new ResponseCache(); |
||||
|
||||
private ResponseCache() { |
||||
} |
||||
|
||||
public static ResponseCache get() { |
||||
return instance; |
||||
} |
||||
|
||||
private final Map<Integer, Command> runningCache = new ConcurrentHashMap<>(); |
||||
private final Map<Integer, Command> responseCache = new ConcurrentHashMap<>(); |
||||
private final Map<Integer,Command> recallCache = new ConcurrentHashMap<>(); |
||||
|
||||
/** |
||||
* cache response |
||||
* |
||||
* @param taskInstanceId taskInstanceId |
||||
* @param command command |
||||
* @param event event ACK/RESULT |
||||
*/ |
||||
public void cache(Integer taskInstanceId, Command command, TaskEventType event) { |
||||
switch (event) { |
||||
case RUNNING: |
||||
runningCache.put(taskInstanceId, command); |
||||
break; |
||||
case RESULT: |
||||
responseCache.put(taskInstanceId, command); |
||||
break; |
||||
case WORKER_REJECT: |
||||
recallCache.put(taskInstanceId, command); |
||||
break; |
||||
default: |
||||
throw new IllegalArgumentException("invalid event type : " + event); |
||||
} |
||||
} |
||||
|
||||
/** |
||||
* recall response cache |
||||
* |
||||
* @param taskInstanceId taskInstanceId |
||||
*/ |
||||
public void removeRecallCache(Integer taskInstanceId) { |
||||
recallCache.remove(taskInstanceId); |
||||
} |
||||
|
||||
public Map<Integer, Command> getRecallCache() { |
||||
return recallCache; |
||||
} |
||||
|
||||
/** |
||||
* remove running cache |
||||
* |
||||
* @param taskInstanceId taskInstanceId |
||||
*/ |
||||
public void removeRunningCache(Integer taskInstanceId) { |
||||
runningCache.remove(taskInstanceId); |
||||
} |
||||
|
||||
/** |
||||
* remove response cache |
||||
* |
||||
* @param taskInstanceId taskInstanceId |
||||
*/ |
||||
public void removeResponseCache(Integer taskInstanceId) { |
||||
responseCache.remove(taskInstanceId); |
||||
} |
||||
|
||||
/** |
||||
* get running cache |
||||
* |
||||
* @return getAckCache |
||||
*/ |
||||
public Map<Integer, Command> getRunningCache() { |
||||
return runningCache; |
||||
} |
||||
|
||||
/** |
||||
* getResponseCache |
||||
* |
||||
* @return getResponseCache |
||||
*/ |
||||
public Map<Integer, Command> getResponseCache() { |
||||
return responseCache; |
||||
} |
||||
} |
@ -0,0 +1,139 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.worker.message; |
||||
|
||||
import org.apache.dolphinscheduler.common.Constants; |
||||
import org.apache.dolphinscheduler.common.thread.BaseDaemonThread; |
||||
import org.apache.dolphinscheduler.common.thread.Stopper; |
||||
import org.apache.dolphinscheduler.common.utils.LoggerUtils; |
||||
import org.apache.dolphinscheduler.remote.command.BaseCommand; |
||||
import org.apache.dolphinscheduler.remote.command.CommandType; |
||||
|
||||
import java.time.Duration; |
||||
import java.util.HashMap; |
||||
import java.util.Map; |
||||
import java.util.concurrent.ConcurrentHashMap; |
||||
|
||||
import javax.annotation.PostConstruct; |
||||
|
||||
import org.slf4j.Logger; |
||||
import org.slf4j.LoggerFactory; |
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.context.ApplicationContext; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
import lombok.NonNull; |
||||
|
||||
@Component |
||||
public class MessageRetryRunner extends BaseDaemonThread { |
||||
|
||||
private final Logger logger = LoggerFactory.getLogger(MessageRetryRunner.class); |
||||
|
||||
protected MessageRetryRunner() { |
||||
super("WorkerMessageRetryRunnerThread"); |
||||
} |
||||
|
||||
private static long MESSAGE_RETRY_WINDOW = Duration.ofMinutes(5L).toMillis(); |
||||
|
||||
@Autowired |
||||
private ApplicationContext applicationContext; |
||||
|
||||
private Map<CommandType, MessageSender<BaseCommand>> messageSenderMap = new HashMap<>(); |
||||
|
||||
private Map<Integer, Map<CommandType, BaseCommand>> needToRetryMessages = new ConcurrentHashMap<>(); |
||||
|
||||
@PostConstruct |
||||
public void init() { |
||||
Map<String, MessageSender> messageSenders = applicationContext.getBeansOfType(MessageSender.class); |
||||
messageSenders.values().forEach(messageSender -> { |
||||
messageSenderMap.put(messageSender.getMessageType(), messageSender); |
||||
logger.info("Injected message sender: {}", messageSender.getClass().getName()); |
||||
}); |
||||
} |
||||
|
||||
@Override |
||||
public synchronized void start() { |
||||
logger.info("Message retry runner staring"); |
||||
super.start(); |
||||
logger.info("Message retry runner started"); |
||||
} |
||||
|
||||
public void addRetryMessage(int taskInstanceId, @NonNull CommandType messageType, BaseCommand baseCommand) { |
||||
needToRetryMessages.computeIfAbsent(taskInstanceId, k -> new ConcurrentHashMap<>()).put(messageType, |
||||
baseCommand); |
||||
} |
||||
|
||||
public void removeRetryMessage(int taskInstanceId, @NonNull CommandType messageType) { |
||||
Map<CommandType, BaseCommand> retryMessages = needToRetryMessages.get(taskInstanceId); |
||||
if (retryMessages != null) { |
||||
retryMessages.remove(messageType); |
||||
} |
||||
} |
||||
|
||||
public void removeRetryMessages(int taskInstanceId) { |
||||
needToRetryMessages.remove(taskInstanceId); |
||||
} |
||||
|
||||
public void updateMessageHost(int taskInstanceId, String messageReceiverHost) { |
||||
Map<CommandType, BaseCommand> needToRetryMessages = this.needToRetryMessages.get(taskInstanceId); |
||||
if (needToRetryMessages != null) { |
||||
needToRetryMessages.values().forEach(baseMessage -> { |
||||
baseMessage.setMessageReceiverAddress(messageReceiverHost); |
||||
}); |
||||
} |
||||
} |
||||
|
||||
public void run() { |
||||
while (Stopper.isRunning()) { |
||||
try { |
||||
if (needToRetryMessages.isEmpty()) { |
||||
Thread.sleep(MESSAGE_RETRY_WINDOW); |
||||
} |
||||
|
||||
long now = System.currentTimeMillis(); |
||||
for (Map.Entry<Integer, Map<CommandType, BaseCommand>> taskEntry : needToRetryMessages.entrySet()) { |
||||
Integer taskInstanceId = taskEntry.getKey(); |
||||
LoggerUtils.setTaskInstanceIdMDC(taskInstanceId); |
||||
try { |
||||
for (Map.Entry<CommandType, BaseCommand> messageEntry : taskEntry.getValue().entrySet()) { |
||||
CommandType messageType = messageEntry.getKey(); |
||||
BaseCommand message = messageEntry.getValue(); |
||||
if (now - message.getMessageSendTime() > MESSAGE_RETRY_WINDOW) { |
||||
logger.info("Begin retry send message to master, message: {}", message); |
||||
message.setMessageSendTime(now); |
||||
messageSenderMap.get(messageType).sendMessage(message); |
||||
logger.info("Success send message to master, message: {}", message); |
||||
} |
||||
} |
||||
} catch (Exception e) { |
||||
logger.warn("Retry send message to master error", e); |
||||
} finally { |
||||
LoggerUtils.removeTaskInstanceIdMDC(); |
||||
} |
||||
} |
||||
Thread.sleep(Constants.SLEEP_TIME_MILLIS); |
||||
} catch (InterruptedException instance) { |
||||
logger.warn("The message retry thread is interrupted, will break this loop", instance); |
||||
Thread.currentThread().interrupt(); |
||||
break; |
||||
} catch (Exception ex) { |
||||
logger.error("Retry send message failed, get an known exception.", ex); |
||||
} |
||||
} |
||||
} |
||||
} |
@ -0,0 +1,43 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.worker.message; |
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.remote.command.BaseCommand; |
||||
import org.apache.dolphinscheduler.remote.command.CommandType; |
||||
import org.apache.dolphinscheduler.remote.exceptions.RemotingException; |
||||
|
||||
public interface MessageSender<T extends BaseCommand> { |
||||
|
||||
/** |
||||
* Send the message |
||||
* |
||||
* @throws RemotingException Cannot connect to the target host. |
||||
*/ |
||||
void sendMessage(T message) throws RemotingException; |
||||
|
||||
/** |
||||
* Build the message from task context and message received address. |
||||
*/ |
||||
T buildMessage(TaskExecutionContext taskExecutionContext, String messageReceiverAddress); |
||||
|
||||
/** |
||||
* The message type can be sent by this sender. |
||||
*/ |
||||
CommandType getMessageType(); |
||||
} |
@ -0,0 +1,70 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.worker.message; |
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.remote.command.CommandType; |
||||
import org.apache.dolphinscheduler.remote.command.TaskExecuteResultCommand; |
||||
import org.apache.dolphinscheduler.remote.exceptions.RemotingException; |
||||
import org.apache.dolphinscheduler.remote.utils.Host; |
||||
import org.apache.dolphinscheduler.server.worker.config.WorkerConfig; |
||||
import org.apache.dolphinscheduler.server.worker.rpc.WorkerRpcClient; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Component |
||||
public class TaskExecuteResultMessageSender implements MessageSender<TaskExecuteResultCommand> { |
||||
|
||||
@Autowired |
||||
private WorkerConfig workerConfig; |
||||
|
||||
@Autowired |
||||
private WorkerRpcClient workerRpcClient; |
||||
|
||||
@Override |
||||
public void sendMessage(TaskExecuteResultCommand message) throws RemotingException { |
||||
workerRpcClient.send(Host.of(message.getMessageReceiverAddress()), message.convert2Command()); |
||||
} |
||||
|
||||
public TaskExecuteResultCommand buildMessage(TaskExecutionContext taskExecutionContext, |
||||
String messageReceiverAddress) { |
||||
TaskExecuteResultCommand taskExecuteResultMessage |
||||
= new TaskExecuteResultCommand(workerConfig.getWorkerAddress(), |
||||
messageReceiverAddress, |
||||
System.currentTimeMillis()); |
||||
taskExecuteResultMessage.setProcessInstanceId(taskExecutionContext.getProcessInstanceId()); |
||||
taskExecuteResultMessage.setTaskInstanceId(taskExecutionContext.getTaskInstanceId()); |
||||
taskExecuteResultMessage.setStatus(taskExecutionContext.getCurrentExecutionStatus().getCode()); |
||||
taskExecuteResultMessage.setLogPath(taskExecutionContext.getLogPath()); |
||||
taskExecuteResultMessage.setExecutePath(taskExecutionContext.getExecutePath()); |
||||
taskExecuteResultMessage.setAppIds(taskExecutionContext.getAppIds()); |
||||
taskExecuteResultMessage.setProcessId(taskExecutionContext.getProcessId()); |
||||
taskExecuteResultMessage.setHost(taskExecutionContext.getHost()); |
||||
taskExecuteResultMessage.setStartTime(taskExecutionContext.getStartTime()); |
||||
taskExecuteResultMessage.setEndTime(taskExecutionContext.getEndTime()); |
||||
taskExecuteResultMessage.setVarPool(taskExecutionContext.getVarPool()); |
||||
taskExecuteResultMessage.setExecutePath(taskExecutionContext.getExecutePath()); |
||||
return taskExecuteResultMessage; |
||||
} |
||||
|
||||
@Override |
||||
public CommandType getMessageType() { |
||||
return CommandType.TASK_EXECUTE_RESULT; |
||||
} |
||||
} |
@ -0,0 +1,67 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.worker.message; |
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.remote.command.CommandType; |
||||
import org.apache.dolphinscheduler.remote.command.TaskExecuteRunningCommand; |
||||
import org.apache.dolphinscheduler.remote.exceptions.RemotingException; |
||||
import org.apache.dolphinscheduler.remote.utils.Host; |
||||
import org.apache.dolphinscheduler.server.worker.config.WorkerConfig; |
||||
import org.apache.dolphinscheduler.server.worker.rpc.WorkerRpcClient; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
import lombok.NonNull; |
||||
|
||||
@Component |
||||
public class TaskExecuteRunningMessageSender implements MessageSender<TaskExecuteRunningCommand> { |
||||
|
||||
@Autowired |
||||
private WorkerRpcClient workerRpcClient; |
||||
|
||||
@Autowired |
||||
private WorkerConfig workerConfig; |
||||
|
||||
@Override |
||||
public void sendMessage(TaskExecuteRunningCommand message) throws RemotingException { |
||||
workerRpcClient.send(Host.of(message.getMessageReceiverAddress()), message.convert2Command()); |
||||
} |
||||
|
||||
public TaskExecuteRunningCommand buildMessage(@NonNull TaskExecutionContext taskExecutionContext, |
||||
@NonNull String messageReceiverAddress) { |
||||
TaskExecuteRunningCommand taskExecuteRunningMessage |
||||
= new TaskExecuteRunningCommand(workerConfig.getWorkerAddress(), |
||||
messageReceiverAddress, |
||||
System.currentTimeMillis()); |
||||
taskExecuteRunningMessage.setTaskInstanceId(taskExecutionContext.getTaskInstanceId()); |
||||
taskExecuteRunningMessage.setProcessInstanceId(taskExecutionContext.getProcessInstanceId()); |
||||
taskExecuteRunningMessage.setStatus(taskExecutionContext.getCurrentExecutionStatus().getCode()); |
||||
taskExecuteRunningMessage.setLogPath(taskExecutionContext.getLogPath()); |
||||
taskExecuteRunningMessage.setHost(taskExecutionContext.getHost()); |
||||
taskExecuteRunningMessage.setStartTime(taskExecutionContext.getStartTime()); |
||||
taskExecuteRunningMessage.setExecutePath(taskExecutionContext.getExecutePath()); |
||||
return taskExecuteRunningMessage; |
||||
} |
||||
|
||||
@Override |
||||
public CommandType getMessageType() { |
||||
return CommandType.TASK_EXECUTE_RUNNING; |
||||
} |
||||
} |
@ -0,0 +1,59 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.worker.message; |
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.remote.command.CommandType; |
||||
import org.apache.dolphinscheduler.remote.command.TaskRejectCommand; |
||||
import org.apache.dolphinscheduler.remote.exceptions.RemotingException; |
||||
import org.apache.dolphinscheduler.remote.utils.Host; |
||||
import org.apache.dolphinscheduler.server.worker.config.WorkerConfig; |
||||
import org.apache.dolphinscheduler.server.worker.rpc.WorkerRpcClient; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Component |
||||
public class TaskRejectMessageSender implements MessageSender<TaskRejectCommand> { |
||||
|
||||
@Autowired |
||||
private WorkerRpcClient workerRpcClient; |
||||
|
||||
@Autowired |
||||
private WorkerConfig workerConfig; |
||||
|
||||
@Override |
||||
public void sendMessage(TaskRejectCommand message) throws RemotingException { |
||||
workerRpcClient.send(Host.of(message.getMessageReceiverAddress()), message.convert2Command()); |
||||
} |
||||
|
||||
public TaskRejectCommand buildMessage(TaskExecutionContext taskExecutionContext, String masterAddress) { |
||||
TaskRejectCommand taskRejectMessage = new TaskRejectCommand(workerConfig.getWorkerAddress(), |
||||
masterAddress, |
||||
System.currentTimeMillis()); |
||||
taskRejectMessage.setTaskInstanceId(taskExecutionContext.getTaskInstanceId()); |
||||
taskRejectMessage.setProcessInstanceId(taskExecutionContext.getProcessInstanceId()); |
||||
taskRejectMessage.setHost(taskExecutionContext.getHost()); |
||||
return taskRejectMessage; |
||||
} |
||||
|
||||
@Override |
||||
public CommandType getMessageType() { |
||||
return CommandType.TASK_REJECT; |
||||
} |
||||
} |
@ -1,276 +0,0 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.worker.processor; |
||||
|
||||
import static org.apache.dolphinscheduler.common.Constants.SLEEP_TIME_MILLIS; |
||||
|
||||
import org.apache.dolphinscheduler.common.enums.TaskEventType; |
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskConstants; |
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.remote.NettyRemotingClient; |
||||
import org.apache.dolphinscheduler.remote.command.Command; |
||||
import org.apache.dolphinscheduler.remote.command.CommandType; |
||||
import org.apache.dolphinscheduler.remote.command.TaskExecuteResponseCommand; |
||||
import org.apache.dolphinscheduler.remote.command.TaskExecuteRunningCommand; |
||||
import org.apache.dolphinscheduler.remote.command.TaskKillResponseCommand; |
||||
import org.apache.dolphinscheduler.remote.command.TaskRecallCommand; |
||||
import org.apache.dolphinscheduler.remote.config.NettyClientConfig; |
||||
import org.apache.dolphinscheduler.remote.processor.NettyRemoteChannel; |
||||
import org.apache.dolphinscheduler.server.worker.cache.ResponseCache; |
||||
|
||||
import java.util.Arrays; |
||||
import java.util.Optional; |
||||
import java.util.concurrent.ConcurrentHashMap; |
||||
|
||||
import org.slf4j.Logger; |
||||
import org.slf4j.LoggerFactory; |
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Service; |
||||
|
||||
import io.netty.channel.Channel; |
||||
import io.netty.channel.ChannelFuture; |
||||
import io.netty.channel.ChannelFutureListener; |
||||
|
||||
|
||||
/** |
||||
* task callback service |
||||
*/ |
||||
@Service |
||||
public class TaskCallbackService { |
||||
|
||||
private final Logger logger = LoggerFactory.getLogger(TaskCallbackService.class); |
||||
private static final int[] RETRY_BACKOFF = {1, 2, 3, 5, 10, 20, 40, 100, 100, 100, 100, 200, 200, 200}; |
||||
|
||||
@Autowired |
||||
private TaskExecuteRunningAckProcessor taskExecuteRunningProcessor; |
||||
|
||||
@Autowired |
||||
private TaskExecuteResponseAckProcessor taskExecuteResponseAckProcessor; |
||||
|
||||
/** |
||||
* remote channels |
||||
*/ |
||||
private static final ConcurrentHashMap<Integer, NettyRemoteChannel> REMOTE_CHANNELS = new ConcurrentHashMap<>(); |
||||
|
||||
/** |
||||
* netty remoting client |
||||
*/ |
||||
private final NettyRemotingClient nettyRemotingClient; |
||||
|
||||
public TaskCallbackService() { |
||||
final NettyClientConfig clientConfig = new NettyClientConfig(); |
||||
this.nettyRemotingClient = new NettyRemotingClient(clientConfig); |
||||
this.nettyRemotingClient.registerProcessor(CommandType.TASK_EXECUTE_RUNNING_ACK, taskExecuteRunningProcessor); |
||||
this.nettyRemotingClient.registerProcessor(CommandType.TASK_EXECUTE_RESPONSE_ACK, taskExecuteResponseAckProcessor); |
||||
} |
||||
|
||||
/** |
||||
* add callback channel |
||||
* |
||||
* @param taskInstanceId taskInstanceId |
||||
* @param channel channel |
||||
*/ |
||||
public void addRemoteChannel(int taskInstanceId, NettyRemoteChannel channel) { |
||||
REMOTE_CHANNELS.put(taskInstanceId, channel); |
||||
} |
||||
|
||||
/** |
||||
* change remote channel |
||||
*/ |
||||
public void changeRemoteChannel(int taskInstanceId, NettyRemoteChannel channel) { |
||||
REMOTE_CHANNELS.put(taskInstanceId, channel); |
||||
} |
||||
|
||||
/** |
||||
* get callback channel |
||||
* |
||||
* @param taskInstanceId taskInstanceId |
||||
* @return callback channel |
||||
*/ |
||||
private Optional<NettyRemoteChannel> getRemoteChannel(int taskInstanceId) { |
||||
Channel newChannel; |
||||
NettyRemoteChannel nettyRemoteChannel = REMOTE_CHANNELS.get(taskInstanceId); |
||||
if (nettyRemoteChannel != null) { |
||||
if (nettyRemoteChannel.isActive()) { |
||||
return Optional.of(nettyRemoteChannel); |
||||
} |
||||
newChannel = nettyRemotingClient.getChannel(nettyRemoteChannel.getHost()); |
||||
if (newChannel != null) { |
||||
return Optional.of(getRemoteChannel(newChannel, nettyRemoteChannel.getOpaque(), taskInstanceId)); |
||||
} |
||||
} |
||||
return Optional.empty(); |
||||
} |
||||
|
||||
public long pause(int ntries) { |
||||
return SLEEP_TIME_MILLIS * RETRY_BACKOFF[ntries % RETRY_BACKOFF.length]; |
||||
} |
||||
|
||||
private NettyRemoteChannel getRemoteChannel(Channel newChannel, long opaque, int taskInstanceId) { |
||||
NettyRemoteChannel remoteChannel = new NettyRemoteChannel(newChannel, opaque); |
||||
addRemoteChannel(taskInstanceId, remoteChannel); |
||||
return remoteChannel; |
||||
} |
||||
|
||||
private NettyRemoteChannel getRemoteChannel(Channel newChannel, int taskInstanceId) { |
||||
NettyRemoteChannel remoteChannel = new NettyRemoteChannel(newChannel); |
||||
addRemoteChannel(taskInstanceId, remoteChannel); |
||||
return remoteChannel; |
||||
} |
||||
|
||||
/** |
||||
* remove callback channels |
||||
* |
||||
* @param taskInstanceId taskInstanceId |
||||
*/ |
||||
public static void remove(int taskInstanceId) { |
||||
REMOTE_CHANNELS.remove(taskInstanceId); |
||||
} |
||||
|
||||
/** |
||||
* send result |
||||
* |
||||
* @param taskInstanceId taskInstanceId |
||||
* @param command command |
||||
*/ |
||||
public void send(int taskInstanceId, Command command) { |
||||
Optional<NettyRemoteChannel> nettyRemoteChannel = getRemoteChannel(taskInstanceId); |
||||
if (nettyRemoteChannel.isPresent()) { |
||||
nettyRemoteChannel.get().writeAndFlush(command).addListener(new ChannelFutureListener() { |
||||
@Override |
||||
public void operationComplete(ChannelFuture future) { |
||||
if (!future.isSuccess()) { |
||||
logger.error("Send callback command error, taskInstanceId: {}, command: {}", taskInstanceId, command); |
||||
} |
||||
} |
||||
}); |
||||
} else { |
||||
logger.warn("Remote channel of taskInstanceId is null: {}, cannot send command: {}", taskInstanceId, command); |
||||
} |
||||
} |
||||
|
||||
/** |
||||
* build task execute running command |
||||
* |
||||
* @param taskExecutionContext taskExecutionContext |
||||
* @return TaskExecuteAckCommand |
||||
*/ |
||||
private TaskExecuteRunningCommand buildTaskExecuteRunningCommand(TaskExecutionContext taskExecutionContext) { |
||||
TaskExecuteRunningCommand command = new TaskExecuteRunningCommand(); |
||||
command.setTaskInstanceId(taskExecutionContext.getTaskInstanceId()); |
||||
command.setProcessInstanceId(taskExecutionContext.getProcessInstanceId()); |
||||
command.setStatus(taskExecutionContext.getCurrentExecutionStatus().getCode()); |
||||
command.setLogPath(taskExecutionContext.getLogPath()); |
||||
command.setHost(taskExecutionContext.getHost()); |
||||
command.setStartTime(taskExecutionContext.getStartTime()); |
||||
command.setExecutePath(taskExecutionContext.getExecutePath()); |
||||
return command; |
||||
} |
||||
|
||||
/** |
||||
* build task execute response command |
||||
* |
||||
* @param taskExecutionContext taskExecutionContext |
||||
* @return TaskExecuteResponseCommand |
||||
*/ |
||||
private TaskExecuteResponseCommand buildTaskExecuteResponseCommand(TaskExecutionContext taskExecutionContext) { |
||||
TaskExecuteResponseCommand command = new TaskExecuteResponseCommand(); |
||||
command.setProcessInstanceId(taskExecutionContext.getProcessInstanceId()); |
||||
command.setTaskInstanceId(taskExecutionContext.getTaskInstanceId()); |
||||
command.setStatus(taskExecutionContext.getCurrentExecutionStatus().getCode()); |
||||
command.setLogPath(taskExecutionContext.getLogPath()); |
||||
command.setExecutePath(taskExecutionContext.getExecutePath()); |
||||
command.setAppIds(taskExecutionContext.getAppIds()); |
||||
command.setProcessId(taskExecutionContext.getProcessId()); |
||||
command.setHost(taskExecutionContext.getHost()); |
||||
command.setStartTime(taskExecutionContext.getStartTime()); |
||||
command.setEndTime(taskExecutionContext.getEndTime()); |
||||
command.setVarPool(taskExecutionContext.getVarPool()); |
||||
command.setExecutePath(taskExecutionContext.getExecutePath()); |
||||
return command; |
||||
} |
||||
|
||||
/** |
||||
* build TaskKillResponseCommand |
||||
* |
||||
* @param taskExecutionContext taskExecutionContext |
||||
* @return build TaskKillResponseCommand |
||||
*/ |
||||
private TaskKillResponseCommand buildKillTaskResponseCommand(TaskExecutionContext taskExecutionContext) { |
||||
TaskKillResponseCommand taskKillResponseCommand = new TaskKillResponseCommand(); |
||||
taskKillResponseCommand.setStatus(taskExecutionContext.getCurrentExecutionStatus().getCode()); |
||||
taskKillResponseCommand.setAppIds(Arrays.asList(taskExecutionContext.getAppIds().split(TaskConstants.COMMA))); |
||||
taskKillResponseCommand.setTaskInstanceId(taskExecutionContext.getTaskInstanceId()); |
||||
taskKillResponseCommand.setHost(taskExecutionContext.getHost()); |
||||
taskKillResponseCommand.setProcessId(taskExecutionContext.getProcessId()); |
||||
return taskKillResponseCommand; |
||||
} |
||||
|
||||
private TaskRecallCommand buildRecallCommand(TaskExecutionContext taskExecutionContext) { |
||||
TaskRecallCommand taskRecallCommand = new TaskRecallCommand(); |
||||
taskRecallCommand.setTaskInstanceId(taskExecutionContext.getTaskInstanceId()); |
||||
taskRecallCommand.setProcessInstanceId(taskExecutionContext.getProcessInstanceId()); |
||||
taskRecallCommand.setHost(taskExecutionContext.getHost()); |
||||
return taskRecallCommand; |
||||
} |
||||
|
||||
/** |
||||
* send task execute running command |
||||
* todo unified callback command |
||||
*/ |
||||
public void sendTaskExecuteRunningCommand(TaskExecutionContext taskExecutionContext) { |
||||
TaskExecuteRunningCommand command = buildTaskExecuteRunningCommand(taskExecutionContext); |
||||
// add response cache
|
||||
ResponseCache.get().cache(taskExecutionContext.getTaskInstanceId(), command.convert2Command(), TaskEventType.RUNNING); |
||||
send(taskExecutionContext.getTaskInstanceId(), command.convert2Command()); |
||||
} |
||||
|
||||
/** |
||||
* send task execute delay command |
||||
* todo unified callback command |
||||
*/ |
||||
public void sendTaskExecuteDelayCommand(TaskExecutionContext taskExecutionContext) { |
||||
TaskExecuteRunningCommand command = buildTaskExecuteRunningCommand(taskExecutionContext); |
||||
send(taskExecutionContext.getTaskInstanceId(), command.convert2Command()); |
||||
} |
||||
|
||||
/** |
||||
* send task execute response command |
||||
* todo unified callback command |
||||
*/ |
||||
public void sendTaskExecuteResponseCommand(TaskExecutionContext taskExecutionContext) { |
||||
TaskExecuteResponseCommand command = buildTaskExecuteResponseCommand(taskExecutionContext); |
||||
// add response cache
|
||||
ResponseCache.get().cache(taskExecutionContext.getTaskInstanceId(), command.convert2Command(), TaskEventType.RESULT); |
||||
send(taskExecutionContext.getTaskInstanceId(), command.convert2Command()); |
||||
} |
||||
|
||||
public void sendTaskKillResponseCommand(TaskExecutionContext taskExecutionContext) { |
||||
TaskKillResponseCommand taskKillResponseCommand = buildKillTaskResponseCommand(taskExecutionContext); |
||||
send(taskExecutionContext.getTaskInstanceId(), taskKillResponseCommand.convert2Command()); |
||||
} |
||||
|
||||
/** |
||||
* send task execute response command |
||||
*/ |
||||
public void sendRecallCommand(TaskExecutionContext taskExecutionContext) { |
||||
TaskRecallCommand taskRecallCommand = buildRecallCommand(taskExecutionContext); |
||||
ResponseCache.get().cache(taskExecutionContext.getTaskInstanceId(), taskRecallCommand.convert2Command(), TaskEventType.WORKER_REJECT); |
||||
send(taskExecutionContext.getTaskInstanceId(), taskRecallCommand.convert2Command()); |
||||
} |
||||
} |
@ -1,71 +0,0 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.worker.processor; |
||||
|
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils; |
||||
import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus; |
||||
import org.apache.dolphinscheduler.remote.command.Command; |
||||
import org.apache.dolphinscheduler.remote.command.CommandType; |
||||
import org.apache.dolphinscheduler.remote.command.TaskExecuteResponseAckCommand; |
||||
import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; |
||||
import org.apache.dolphinscheduler.server.worker.cache.ResponseCache; |
||||
|
||||
import org.slf4j.Logger; |
||||
import org.slf4j.LoggerFactory; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
import com.google.common.base.Preconditions; |
||||
|
||||
import io.netty.channel.Channel; |
||||
|
||||
/** |
||||
* task execute running ack, from master to worker |
||||
*/ |
||||
@Component |
||||
public class TaskExecuteResponseAckProcessor implements NettyRequestProcessor { |
||||
|
||||
private final Logger logger = LoggerFactory.getLogger(TaskExecuteResponseAckProcessor.class); |
||||
|
||||
@Override |
||||
public void process(Channel channel, Command command) { |
||||
Preconditions.checkArgument(CommandType.TASK_EXECUTE_RESPONSE_ACK == command.getType(), |
||||
String.format("invalid command type : %s", command.getType())); |
||||
|
||||
TaskExecuteResponseAckCommand taskExecuteResponseAckCommand = JSONUtils.parseObject( |
||||
command.getBody(), TaskExecuteResponseAckCommand.class); |
||||
|
||||
if (taskExecuteResponseAckCommand == null) { |
||||
logger.error("task execute response ack command is null"); |
||||
return; |
||||
} |
||||
logger.info("task execute response ack command : {}", taskExecuteResponseAckCommand); |
||||
|
||||
if (taskExecuteResponseAckCommand.getStatus() == ExecutionStatus.SUCCESS.getCode()) { |
||||
ResponseCache.get().removeResponseCache(taskExecuteResponseAckCommand.getTaskInstanceId()); |
||||
TaskCallbackService.remove(taskExecuteResponseAckCommand.getTaskInstanceId()); |
||||
logger.debug("remove REMOTE_CHANNELS, task instance id:{}", |
||||
taskExecuteResponseAckCommand.getTaskInstanceId()); |
||||
} else if (taskExecuteResponseAckCommand.getStatus() == ExecutionStatus.FAILURE.getCode()) { |
||||
// master handle worker response error, will still retry
|
||||
} else { |
||||
throw new IllegalArgumentException("Invalid task execute response ack status: " |
||||
+ taskExecuteResponseAckCommand.getStatus()); |
||||
} |
||||
} |
||||
|
||||
} |
@ -0,0 +1,83 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.worker.processor; |
||||
|
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils; |
||||
import org.apache.dolphinscheduler.common.utils.LoggerUtils; |
||||
import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus; |
||||
import org.apache.dolphinscheduler.remote.command.Command; |
||||
import org.apache.dolphinscheduler.remote.command.CommandType; |
||||
import org.apache.dolphinscheduler.remote.command.TaskExecuteAckCommand; |
||||
import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; |
||||
import org.apache.dolphinscheduler.server.worker.message.MessageRetryRunner; |
||||
|
||||
import org.slf4j.Logger; |
||||
import org.slf4j.LoggerFactory; |
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
import com.google.common.base.Preconditions; |
||||
|
||||
import io.netty.channel.Channel; |
||||
|
||||
/** |
||||
* task execute running ack, from master to worker |
||||
*/ |
||||
@Component |
||||
public class TaskExecuteResultAckProcessor implements NettyRequestProcessor { |
||||
|
||||
private final Logger logger = LoggerFactory.getLogger(TaskExecuteResultAckProcessor.class); |
||||
|
||||
@Autowired |
||||
private MessageRetryRunner messageRetryRunner; |
||||
|
||||
@Override |
||||
public void process(Channel channel, Command command) { |
||||
Preconditions.checkArgument(CommandType.TASK_EXECUTE_RESULT_ACK == command.getType(), |
||||
String.format("invalid command type : %s", command.getType())); |
||||
|
||||
TaskExecuteAckCommand taskExecuteAckMessage = JSONUtils.parseObject(command.getBody(), |
||||
TaskExecuteAckCommand.class); |
||||
|
||||
if (taskExecuteAckMessage == null) { |
||||
logger.error("task execute response ack command is null"); |
||||
return; |
||||
} |
||||
logger.info("task execute response ack command : {}", taskExecuteAckMessage); |
||||
|
||||
try { |
||||
LoggerUtils.setTaskInstanceIdMDC(taskExecuteAckMessage.getTaskInstanceId()); |
||||
if (taskExecuteAckMessage.getStatus() == ExecutionStatus.SUCCESS.getCode()) { |
||||
messageRetryRunner.removeRetryMessage(taskExecuteAckMessage.getTaskInstanceId(), |
||||
CommandType.TASK_EXECUTE_RESULT); |
||||
logger.debug("remove REMOTE_CHANNELS, task instance id:{}", taskExecuteAckMessage.getTaskInstanceId()); |
||||
} else if (taskExecuteAckMessage.getStatus() == ExecutionStatus.FAILURE.getCode()) { |
||||
// master handle worker response error, will still retry
|
||||
logger.error("Receive task execute result ack message, the message status is not success, message: {}", |
||||
taskExecuteAckMessage); |
||||
} else { |
||||
throw new IllegalArgumentException("Invalid task execute response ack status: " |
||||
+ taskExecuteAckMessage.getStatus()); |
||||
} |
||||
} finally { |
||||
LoggerUtils.removeTaskInstanceIdMDC(); |
||||
|
||||
} |
||||
} |
||||
|
||||
} |
@ -0,0 +1,92 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.worker.rpc; |
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.remote.command.BaseCommand; |
||||
import org.apache.dolphinscheduler.remote.command.CommandType; |
||||
import org.apache.dolphinscheduler.remote.exceptions.RemotingException; |
||||
import org.apache.dolphinscheduler.server.worker.message.MessageRetryRunner; |
||||
import org.apache.dolphinscheduler.server.worker.message.MessageSender; |
||||
|
||||
import java.util.HashMap; |
||||
import java.util.Map; |
||||
|
||||
import javax.annotation.PostConstruct; |
||||
|
||||
import org.slf4j.Logger; |
||||
import org.slf4j.LoggerFactory; |
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.context.ApplicationContext; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
import lombok.NonNull; |
||||
|
||||
@Component |
||||
public class WorkerMessageSender { |
||||
|
||||
private final Logger logger = LoggerFactory.getLogger(WorkerMessageSender.class); |
||||
|
||||
@Autowired |
||||
private MessageRetryRunner messageRetryRunner; |
||||
|
||||
@Autowired |
||||
private ApplicationContext applicationContext; |
||||
|
||||
private Map<CommandType, MessageSender> messageSenderMap = new HashMap<>(); |
||||
|
||||
@PostConstruct |
||||
public void init() { |
||||
Map<String, MessageSender> messageSenders = applicationContext.getBeansOfType(MessageSender.class); |
||||
messageSenders.values().forEach(messageSender -> messageSenderMap.put(messageSender.getMessageType(), |
||||
messageSender)); |
||||
} |
||||
|
||||
// todo: use message rather than context
|
||||
public void sendMessageWithRetry(@NonNull TaskExecutionContext taskExecutionContext, |
||||
@NonNull String messageReceiverAddress, |
||||
@NonNull CommandType messageType) { |
||||
MessageSender messageSender = messageSenderMap.get(messageType); |
||||
if (messageSender == null) { |
||||
throw new IllegalArgumentException("The messageType is invalidated, messageType: " + messageType); |
||||
} |
||||
BaseCommand baseCommand = messageSender.buildMessage(taskExecutionContext, messageReceiverAddress); |
||||
try { |
||||
messageRetryRunner.addRetryMessage(taskExecutionContext.getTaskInstanceId(), messageType, baseCommand); |
||||
messageSender.sendMessage(baseCommand); |
||||
} catch (RemotingException e) { |
||||
logger.error("Send message error, messageType: {}, message: {}", messageType, baseCommand); |
||||
} |
||||
} |
||||
|
||||
public void sendMessage(@NonNull TaskExecutionContext taskExecutionContext, |
||||
@NonNull String messageReceiverAddress, |
||||
@NonNull CommandType messageType) { |
||||
MessageSender messageSender = messageSenderMap.get(messageType); |
||||
if (messageSender == null) { |
||||
throw new IllegalArgumentException("The messageType is invalidated, messageType: " + messageType); |
||||
} |
||||
BaseCommand baseCommand = messageSender.buildMessage(taskExecutionContext, messageReceiverAddress); |
||||
try { |
||||
messageSender.sendMessage(baseCommand); |
||||
} catch (RemotingException e) { |
||||
logger.error("Send message error, messageType: {}, message: {}", messageType, baseCommand); |
||||
} |
||||
} |
||||
|
||||
} |
@ -0,0 +1,75 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.worker.rpc; |
||||
|
||||
import org.apache.dolphinscheduler.remote.NettyRemotingClient; |
||||
import org.apache.dolphinscheduler.remote.command.Command; |
||||
import org.apache.dolphinscheduler.remote.command.CommandType; |
||||
import org.apache.dolphinscheduler.remote.config.NettyClientConfig; |
||||
import org.apache.dolphinscheduler.remote.exceptions.RemotingException; |
||||
import org.apache.dolphinscheduler.remote.utils.Host; |
||||
import org.apache.dolphinscheduler.server.worker.processor.TaskExecuteResultAckProcessor; |
||||
import org.apache.dolphinscheduler.server.worker.processor.TaskExecuteRunningAckProcessor; |
||||
import org.apache.dolphinscheduler.server.worker.processor.TaskRejectAckProcessor; |
||||
|
||||
import org.slf4j.Logger; |
||||
import org.slf4j.LoggerFactory; |
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
/** |
||||
* This rpc client is only used to send message, will not receive message, all response message should send to {@link WorkerRpcServer}. |
||||
*/ |
||||
@Component |
||||
public class WorkerRpcClient implements AutoCloseable { |
||||
|
||||
private final Logger logger = LoggerFactory.getLogger(WorkerRpcClient.class); |
||||
|
||||
@Autowired |
||||
private TaskExecuteRunningAckProcessor taskExecuteRunningAckProcessor; |
||||
|
||||
@Autowired |
||||
private TaskExecuteResultAckProcessor taskExecuteResultAckProcessor; |
||||
|
||||
@Autowired |
||||
private TaskRejectAckProcessor taskRejectAckProcessor; |
||||
|
||||
private NettyRemotingClient nettyRemotingClient; |
||||
|
||||
public void start() { |
||||
logger.info("Worker rpc client starting"); |
||||
NettyClientConfig nettyClientConfig = new NettyClientConfig(); |
||||
this.nettyRemotingClient = new NettyRemotingClient(nettyClientConfig); |
||||
// we only use the client to handle the ack message, we can optimize this, send ack to the nettyServer.
|
||||
this.nettyRemotingClient.registerProcessor(CommandType.TASK_EXECUTE_RUNNING_ACK, |
||||
taskExecuteRunningAckProcessor); |
||||
this.nettyRemotingClient.registerProcessor(CommandType.TASK_EXECUTE_RESULT_ACK, taskExecuteResultAckProcessor); |
||||
this.nettyRemotingClient.registerProcessor(CommandType.TASK_REJECT_ACK, taskRejectAckProcessor); |
||||
logger.info("Worker rpc client started"); |
||||
} |
||||
|
||||
public void send(Host host, Command command) throws RemotingException { |
||||
nettyRemotingClient.send(host, command); |
||||
} |
||||
|
||||
public void close() { |
||||
logger.info("Worker rpc client closing"); |
||||
nettyRemotingClient.close(); |
||||
logger.info("Worker rpc client closed"); |
||||
} |
||||
} |
@ -1,134 +0,0 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.worker.runner; |
||||
|
||||
import org.apache.dolphinscheduler.common.thread.BaseDaemonThread; |
||||
import org.apache.dolphinscheduler.common.thread.Stopper; |
||||
import org.apache.dolphinscheduler.common.thread.ThreadUtils; |
||||
import org.apache.dolphinscheduler.remote.command.Command; |
||||
import org.apache.dolphinscheduler.server.worker.cache.ResponseCache; |
||||
import org.apache.dolphinscheduler.server.worker.processor.TaskCallbackService; |
||||
|
||||
import java.util.Map; |
||||
|
||||
import org.slf4j.Logger; |
||||
import org.slf4j.LoggerFactory; |
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
/** |
||||
* Retry Report Task Status Thread |
||||
*/ |
||||
@Component |
||||
public class RetryReportTaskStatusThread extends BaseDaemonThread { |
||||
|
||||
private final Logger logger = LoggerFactory.getLogger(RetryReportTaskStatusThread.class); |
||||
|
||||
/** |
||||
* every 5 minutes |
||||
*/ |
||||
private static long RETRY_REPORT_TASK_STATUS_INTERVAL = 5 * 60 * 1000L; |
||||
|
||||
@Autowired |
||||
private TaskCallbackService taskCallbackService; |
||||
|
||||
protected RetryReportTaskStatusThread() { |
||||
super("RetryReportTaskStatusThread"); |
||||
} |
||||
|
||||
@Override |
||||
public synchronized void start() { |
||||
logger.info("Retry report task status thread starting"); |
||||
super.start(); |
||||
logger.info("Retry report task status thread started"); |
||||
} |
||||
|
||||
/** |
||||
* retry ack/response |
||||
*/ |
||||
@Override |
||||
public void run() { |
||||
final ResponseCache instance = ResponseCache.get(); |
||||
|
||||
while (Stopper.isRunning()) { |
||||
|
||||
// sleep 5 minutes
|
||||
ThreadUtils.sleep(RETRY_REPORT_TASK_STATUS_INTERVAL); |
||||
|
||||
try { |
||||
// todo: Only retry the send failed command
|
||||
retryRunningCommand(instance); |
||||
retryResponseCommand(instance); |
||||
retryRecallCommand(instance); |
||||
} catch (Exception e) { |
||||
logger.warn("Retry report task status error", e); |
||||
} |
||||
} |
||||
} |
||||
|
||||
private void retryRunningCommand(ResponseCache instance) { |
||||
if (!instance.getRunningCache().isEmpty()) { |
||||
Map<Integer, Command> runningCache = instance.getRunningCache(); |
||||
logger.info("Send task running retry command starting, waiting to retry size: {}", runningCache.size()); |
||||
for (Map.Entry<Integer, Command> entry : runningCache.entrySet()) { |
||||
Integer taskInstanceId = entry.getKey(); |
||||
Command runningCommand = entry.getValue(); |
||||
try { |
||||
taskCallbackService.send(taskInstanceId, runningCommand); |
||||
} catch (Exception ex) { |
||||
logger.error("Retry send running command to master error, taskInstanceId: {}, command: {}", taskInstanceId, runningCommand); |
||||
} |
||||
} |
||||
logger.info("Send task running retry command finished, waiting to retry size: {}", runningCache.size()); |
||||
} |
||||
} |
||||
|
||||
private void retryResponseCommand(ResponseCache instance) { |
||||
Map<Integer, Command> responseCache = instance.getResponseCache(); |
||||
if (!responseCache.isEmpty()) { |
||||
logger.info("Send task response retry command starting, waiting to retry size: {}", responseCache.size()); |
||||
for (Map.Entry<Integer, Command> entry : responseCache.entrySet()) { |
||||
Integer taskInstanceId = entry.getKey(); |
||||
Command responseCommand = entry.getValue(); |
||||
try { |
||||
taskCallbackService.send(taskInstanceId, responseCommand); |
||||
} catch (Exception ex) { |
||||
logger.error("Retry send response command to master error, taskInstanceId: {}, command: {}", taskInstanceId, responseCommand); |
||||
} |
||||
} |
||||
logger.info("Send task response retry command finished, waiting to retry size: {}", responseCache.size()); |
||||
} |
||||
} |
||||
|
||||
private void retryRecallCommand(ResponseCache instance) { |
||||
Map<Integer, Command> recallCache = instance.getRecallCache(); |
||||
if (!recallCache.isEmpty()) { |
||||
logger.info("Send task recall retry command starting, waiting to retry size: {}", recallCache.size()); |
||||
for (Map.Entry<Integer, Command> entry : recallCache.entrySet()) { |
||||
Integer taskInstanceId = entry.getKey(); |
||||
Command responseCommand = entry.getValue(); |
||||
try { |
||||
taskCallbackService.send(taskInstanceId, responseCommand); |
||||
} catch (Exception ex) { |
||||
logger.error("Retry send recall command to master error, taskInstanceId: {}, command: {}", taskInstanceId, responseCommand); |
||||
} |
||||
} |
||||
logger.info("Send task recall retry command finished, waiting to retry size: {}", recallCache.size()); |
||||
} |
||||
} |
||||
} |
Loading…
Reference in new issue