---初始化项目
This commit is contained in:
563
powerjob-remote/powerjob-remote-impl-mu/DESIGN.md
Normal file
563
powerjob-remote/powerjob-remote-impl-mu/DESIGN.md
Normal file
@ -0,0 +1,563 @@
|
||||
# PowerJob Mu 协议技术设计方案
|
||||
|
||||
## 1. 项目背景
|
||||
|
||||
### 1.1 业务场景
|
||||
PowerJob 是一个分布式任务调度框架,采用 Server-Worker 架构。在实际部署中,经常遇到以下网络环境限制:
|
||||
- Worker 节点部署在内网环境,只能单向访问公网的 Server
|
||||
- Server 无法直接访问 Worker 的内网 IP 地址
|
||||
- 业务需要支持 Server 向 Worker 的主动通讯(任务下发、状态查询等)
|
||||
|
||||
### 1.2 现有协议局限性
|
||||
- **HTTP 协议**:基于请求-响应模式,Server 向 Worker 推送需要 Worker 轮询,实时性差
|
||||
- **AKKA 协议**:功能强大但配置复杂,在 NAT 环境下需要复杂的网络配置
|
||||
|
||||
### 1.3 设计目标
|
||||
1. 支持 Worker 仅可出站访问的网络环境
|
||||
2. 实现 Server 与 Worker 的双向通讯
|
||||
3. 支持完整的节点间通讯矩阵(Worker↔Server, Worker↔Worker, Server↔Server)
|
||||
4. 保持高性能和低延迟
|
||||
5. 简化网络配置和部署复杂度
|
||||
|
||||
## 2. 总体设计
|
||||
|
||||
### 2.1 核心设计原则
|
||||
|
||||
#### 连接复用 (Connection Reuse)
|
||||
- Worker 主动建立到 Server 的长连接
|
||||
- Server 通过注册机制维护 Worker 连接映射
|
||||
- Server 复用 Worker 建立的连接进行反向通讯
|
||||
|
||||
#### 延迟连接 (Lazy Connection)
|
||||
- 节点启动时不立即建立所有连接
|
||||
- 首次通讯时才建立目标连接
|
||||
- 避免启动时的网络依赖和连接失败
|
||||
|
||||
#### 统一架构 (Unified Architecture)
|
||||
- 所有节点都具备 Server 和 Client 双重能力
|
||||
- 统一的消息格式和处理流程
|
||||
- 支持任意节点间的直接通讯
|
||||
|
||||
### 2.2 架构图
|
||||
|
||||
```
|
||||
PowerJob Mu Protocol Architecture
|
||||
|
||||
┌──────────────────────────────────────────────────────────────────┐
|
||||
│ Application Layer │
|
||||
├──────────────────────────────────────────────────────────────────┤
|
||||
│ Actor System (Handler Registration & Message Routing) │
|
||||
├──────────────────────────────────────────────────────────────────┤
|
||||
│ Transport Layer │
|
||||
│ ┌─────────────┐ ┌─────────────┐ ┌─────────────┐ │
|
||||
│ │MuTransporter│ │ChannelMgr │ │ConnectionMgr│ │
|
||||
│ │ │ │ │ │ │ │
|
||||
│ │- tell() │ │- Worker Reg │ │- Lazy Conn │ │
|
||||
│ │- ask() │ │- Ask/Resp │ │- Conn Pool │ │
|
||||
│ └─────────────┘ └─────────────┘ └─────────────┘ │
|
||||
├──────────────────────────────────────────────────────────────────┤
|
||||
│ Protocol Layer │
|
||||
│ ┌─────────────┐ ┌─────────────┐ ┌─────────────┐ │
|
||||
│ │ MuMessage │ │MuMessageCodec│ │Message Types│ │
|
||||
│ │ │ │ │ │ │ │
|
||||
│ │- Type │ │- Encode │ │- TELL │ │
|
||||
│ │- RequestId │ │- Decode │ │- ASK │ │
|
||||
│ │- Path │ │- Length │ │- RESPONSE │ │
|
||||
│ │- Payload │ │- JSON │ │- HEARTBEAT │ │
|
||||
│ └─────────────┘ └─────────────┘ └─────────────┘ │
|
||||
├──────────────────────────────────────────────────────────────────┤
|
||||
│ Network Layer │
|
||||
│ Netty Framework │
|
||||
│ ┌─────────────┐ ┌─────────────┐ ┌─────────────┐ │
|
||||
│ │EventLoopGroup│ │ Bootstrap │ │ChannelPipe │ │
|
||||
│ │ │ │ │ │ │ │
|
||||
│ │- Boss │ │- Server │ │- Codec │ │
|
||||
│ │- Worker │ │- Client │ │- Handler │ │
|
||||
│ └─────────────┘ └─────────────┘ └─────────────┘ │
|
||||
└──────────────────────────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
## 3. 详细设计
|
||||
|
||||
### 3.1 通讯流程设计
|
||||
|
||||
#### 3.1.1 Worker 注册流程
|
||||
```mermaid
|
||||
sequenceDiagram
|
||||
participant W as Worker
|
||||
participant S as Server
|
||||
|
||||
W->>S: 1. 建立TCP连接
|
||||
W->>S: 2. 发送HEARTBEAT消息 (包含Worker地址)
|
||||
S->>S: 3. 注册Worker连接到ChannelManager
|
||||
S->>S: 4. 建立 workerAddress -> Channel 映射
|
||||
|
||||
Note over W,S: 连接建立完成,可以双向通讯
|
||||
|
||||
W->>S: 5. 定期发送HEARTBEAT (保活)
|
||||
S->>W: 6. 可以通过已注册连接发送消息
|
||||
```
|
||||
|
||||
#### 3.1.2 延迟连接流程
|
||||
```mermaid
|
||||
sequenceDiagram
|
||||
participant W1 as Worker1
|
||||
participant W2 as Worker2
|
||||
|
||||
W1->>W1: 1. 需要向Worker2发送消息
|
||||
W1->>W1: 2. 检查连接池,无现有连接
|
||||
W1->>W2: 3. 建立新的TCP连接
|
||||
W1->>W2: 4. 发送消息
|
||||
W1->>W1: 5. 连接加入连接池,供后续复用
|
||||
```
|
||||
|
||||
### 3.2 核心组件设计
|
||||
|
||||
#### 3.2.1 MuCSInitializer
|
||||
```java
|
||||
public class MuCSInitializer implements CSInitializer {
|
||||
|
||||
// 根据节点类型初始化不同服务
|
||||
public void init(CSInitializerConfig config) {
|
||||
if (config.getServerType() == ServerType.SERVER) {
|
||||
initServer(); // 启动Netty服务端 + 连接管理器
|
||||
} else {
|
||||
initWorker(); // 启动Netty服务端 + 连接管理器
|
||||
}
|
||||
}
|
||||
|
||||
private void initServer() {
|
||||
// 1. 创建ServerHandler
|
||||
// 2. 启动Netty服务端监听
|
||||
// 3. 初始化连接管理器
|
||||
}
|
||||
|
||||
private void initWorker() {
|
||||
// 1. 创建WorkerHandler
|
||||
// 2. 启动Netty服务端监听 (支持Worker间通讯)
|
||||
// 3. 初始化连接管理器
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
#### 3.2.2 MuTransporter
|
||||
```java
|
||||
public class MuTransporter implements Transporter {
|
||||
|
||||
public void tell(URL url, PowerSerializable request) {
|
||||
if (当前节点是Worker) {
|
||||
// 使用连接管理器建立到目标的连接
|
||||
connectionManager.getOrCreateConnection(url.getAddress())
|
||||
.thenAccept(channel -> channel.writeAndFlush(message));
|
||||
} else {
|
||||
// Server端需要区分目标类型
|
||||
if (目标是Worker) {
|
||||
// 使用已注册的Worker连接
|
||||
Channel channel = channelManager.getWorkerChannel(url.getAddress());
|
||||
channel.writeAndFlush(message);
|
||||
} else {
|
||||
// 使用连接管理器连接到目标Server
|
||||
connectionManager.getOrCreateConnection(url.getAddress())
|
||||
.thenAccept(channel -> channel.writeAndFlush(message));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public <T> CompletionStage<T> ask(URL url, PowerSerializable request, Class<T> clz) {
|
||||
// 1. 生成唯一的requestId
|
||||
// 2. 注册Future到ChannelManager
|
||||
// 3. 按照tell的逻辑发送ASK消息
|
||||
// 4. 返回Future等待响应
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
#### 3.2.3 ChannelManager
|
||||
```java
|
||||
public class ChannelManager {
|
||||
|
||||
// Worker地址到连接的映射 (Server端用)
|
||||
private final ConcurrentMap<String, Channel> workerChannels;
|
||||
|
||||
// 请求ID到Future的映射 (Ask模式用)
|
||||
private final ConcurrentMap<String, CompletableFuture<Object>> pendingRequests;
|
||||
|
||||
// 请求ID到响应类型的映射 (类型转换用)
|
||||
private final ConcurrentMap<String, Class<?>> requestResponseTypes;
|
||||
|
||||
public void registerWorkerChannel(Address workerAddress, Channel channel) {
|
||||
String key = workerAddress.getHost() + ":" + workerAddress.getPort();
|
||||
workerChannels.put(key, channel);
|
||||
// 监听连接关闭,自动清理映射
|
||||
}
|
||||
|
||||
public void completePendingRequest(String requestId, Object response) {
|
||||
CompletableFuture<Object> future = pendingRequests.remove(requestId);
|
||||
Class<?> responseType = requestResponseTypes.remove(requestId);
|
||||
|
||||
// 类型转换,解决LinkedHashMap问题
|
||||
Object convertedResponse = convertResponse(response, responseType);
|
||||
future.complete(convertedResponse);
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
#### 3.2.4 MuConnectionManager
|
||||
```java
|
||||
public class MuConnectionManager {
|
||||
|
||||
// 目标地址到连接的映射
|
||||
private final ConcurrentMap<String, Channel> connections;
|
||||
|
||||
// 正在建立的连接
|
||||
private final ConcurrentMap<String, CompletableFuture<Channel>> pendingConnections;
|
||||
|
||||
public CompletableFuture<Channel> getOrCreateConnection(Address targetAddress) {
|
||||
String key = targetAddress.getHost() + ":" + targetAddress.getPort();
|
||||
|
||||
// 1. 检查现有连接
|
||||
Channel existingChannel = connections.get(key);
|
||||
if (existingChannel != null && existingChannel.isActive()) {
|
||||
return CompletableFuture.completedFuture(existingChannel);
|
||||
}
|
||||
|
||||
// 2. 检查正在建立的连接
|
||||
CompletableFuture<Channel> pendingConnection = pendingConnections.get(key);
|
||||
if (pendingConnection != null) {
|
||||
return pendingConnection;
|
||||
}
|
||||
|
||||
// 3. 建立新连接
|
||||
return createNewConnection(targetAddress);
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
### 3.3 消息协议设计
|
||||
|
||||
#### 3.3.1 消息格式
|
||||
```java
|
||||
public class MuMessage implements PowerSerializable {
|
||||
private MessageType messageType; // 消息类型
|
||||
private String requestId; // 请求ID (Ask模式)
|
||||
private String path; // 处理器路径
|
||||
private Address senderAddress; // 发送方地址 (注册用)
|
||||
private Object payload; // 消息载荷
|
||||
private String errorMessage; // 错误信息
|
||||
}
|
||||
|
||||
public enum MessageType {
|
||||
TELL, // 单向消息
|
||||
ASK, // 请求消息
|
||||
RESPONSE, // 响应消息
|
||||
ERROR, // 错误响应
|
||||
HEARTBEAT // 心跳消息
|
||||
}
|
||||
```
|
||||
|
||||
#### 3.3.2 编解码器
|
||||
```java
|
||||
public class MuMessageCodec extends ByteToMessageCodec<MuMessage> {
|
||||
|
||||
@Override
|
||||
protected void encode(ChannelHandlerContext ctx, MuMessage msg, ByteBuf out) {
|
||||
byte[] data = OBJECT_MAPPER.writeValueAsBytes(msg);
|
||||
out.writeInt(data.length); // 长度前缀
|
||||
out.writeBytes(data); // 消息内容
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out) {
|
||||
if (in.readableBytes() < 4) return; // 长度不足
|
||||
|
||||
int length = in.readInt();
|
||||
if (in.readableBytes() < length) {
|
||||
in.resetReaderIndex(); // 重置读取位置
|
||||
return;
|
||||
}
|
||||
|
||||
byte[] data = new byte[length];
|
||||
in.readBytes(data);
|
||||
MuMessage message = OBJECT_MAPPER.readValue(data, MuMessage.class);
|
||||
out.add(message);
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
## 4. 关键技术方案
|
||||
|
||||
### 4.1 连接复用机制
|
||||
|
||||
#### 问题描述
|
||||
在单向网络环境中,Server 无法主动连接到 Worker,但需要向 Worker 发送消息。
|
||||
|
||||
#### 解决方案
|
||||
1. **Worker 主动注册**:Worker 启动时主动连接到 Server,并发送包含自身地址的心跳消息
|
||||
2. **连接映射维护**:Server 维护 Worker 地址到连接通道的映射关系
|
||||
3. **反向通讯**:Server 需要向 Worker 发送消息时,从映射中查找对应的连接通道
|
||||
|
||||
```java
|
||||
// Worker端:发送心跳注册
|
||||
MuMessage heartbeat = new MuMessage(
|
||||
MessageType.HEARTBEAT, null, null,
|
||||
workerAddress, // 关键:携带Worker地址
|
||||
null, null
|
||||
);
|
||||
channel.writeAndFlush(heartbeat);
|
||||
|
||||
// Server端:处理心跳并注册
|
||||
public void handleHeartbeat(ChannelHandlerContext ctx, MuMessage msg) {
|
||||
if (msg.getSenderAddress() != null) {
|
||||
channelManager.registerWorkerChannel(msg.getSenderAddress(), ctx.channel());
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
### 4.2 延迟连接机制
|
||||
|
||||
#### 问题描述
|
||||
节点启动时立即建立所有连接会导致:
|
||||
- 启动时间长
|
||||
- 网络故障影响启动
|
||||
- 不必要的资源占用
|
||||
|
||||
#### 解决方案
|
||||
1. **按需连接**:只在首次需要通讯时才建立连接
|
||||
2. **连接缓存**:建立的连接保存在连接池中供后续复用
|
||||
3. **并发控制**:避免同时建立到同一目标的多个连接
|
||||
|
||||
```java
|
||||
public CompletableFuture<Channel> getOrCreateConnection(Address targetAddress) {
|
||||
// 1. 检查缓存连接
|
||||
Channel cached = connectionCache.get(addressKey);
|
||||
if (cached != null && cached.isActive()) {
|
||||
return CompletableFuture.completedFuture(cached);
|
||||
}
|
||||
|
||||
// 2. 检查正在建立的连接,避免重复建立
|
||||
CompletableFuture<Channel> pending = pendingConnections.get(addressKey);
|
||||
if (pending != null) {
|
||||
return pending;
|
||||
}
|
||||
|
||||
// 3. 建立新连接
|
||||
return createNewConnection(targetAddress);
|
||||
}
|
||||
```
|
||||
|
||||
### 4.3 消息路由机制
|
||||
|
||||
#### 问题描述
|
||||
不同的通讯场景需要使用不同的连接方式,需要智能路由。
|
||||
|
||||
#### 解决方案
|
||||
基于调用方类型和目标类型的二维路由表:
|
||||
|
||||
```java
|
||||
public void routeMessage(URL url, MuMessage message) {
|
||||
if (currentNodeType == ServerType.WORKER) {
|
||||
// Worker作为发送方:统一使用连接管理器
|
||||
connectionManager.getOrCreateConnection(url.getAddress())
|
||||
.thenAccept(channel -> channel.writeAndFlush(message));
|
||||
} else {
|
||||
// Server作为发送方:根据目标类型选择策略
|
||||
if (url.getServerType() == ServerType.WORKER) {
|
||||
// 目标是Worker:使用已注册的连接
|
||||
Channel workerChannel = channelManager.getWorkerChannel(url.getAddress());
|
||||
workerChannel.writeAndFlush(message);
|
||||
} else {
|
||||
// 目标是Server:使用连接管理器
|
||||
connectionManager.getOrCreateConnection(url.getAddress())
|
||||
.thenAccept(channel -> channel.writeAndFlush(message));
|
||||
}
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
### 4.4 类型转换机制
|
||||
|
||||
#### 问题描述
|
||||
Jackson 反序列化时可能将对象反序列化为 LinkedHashMap,导致类型转换异常。
|
||||
|
||||
#### 解决方案
|
||||
1. **类型映射**:Ask 请求时记录期望的响应类型
|
||||
2. **智能转换**:响应时根据期望类型进行转换
|
||||
|
||||
```java
|
||||
// 发送Ask请求时记录类型
|
||||
public <T> CompletionStage<T> ask(URL url, PowerSerializable request, Class<T> clz) {
|
||||
String requestId = UUID.randomUUID().toString();
|
||||
channelManager.registerPendingRequest(requestId, future, clz); // 记录期望类型
|
||||
// ... 发送消息
|
||||
}
|
||||
|
||||
// 接收响应时转换类型
|
||||
public void completePendingRequest(String requestId, Object response) {
|
||||
Class<?> expectedType = requestResponseTypes.remove(requestId);
|
||||
Object convertedResponse = JsonUtils.toJavaObject(response, expectedType);
|
||||
future.complete(convertedResponse);
|
||||
}
|
||||
```
|
||||
|
||||
## 5. 性能优化
|
||||
|
||||
### 5.1 连接池管理
|
||||
- **连接复用**:相同目标的多次请求复用连接
|
||||
- **连接清理**:自动检测和清理失效连接
|
||||
- **连接限制**:控制最大连接数,避免资源耗尽
|
||||
|
||||
### 5.2 异步处理
|
||||
- **非阻塞IO**:基于Netty的异步IO模型
|
||||
- **事件驱动**:消息处理采用事件驱动方式
|
||||
- **线程池优化**:合理配置EventLoopGroup大小
|
||||
|
||||
### 5.3 内存管理
|
||||
- **零拷贝**:利用Netty的零拷贝特性
|
||||
- **对象池**:复用消息对象,减少GC压力
|
||||
- **缓冲区管理**:合理设置接收和发送缓冲区大小
|
||||
|
||||
## 6. 容错设计
|
||||
|
||||
### 6.1 连接故障处理
|
||||
```java
|
||||
// 连接断开监听
|
||||
channel.closeFuture().addListener(future -> {
|
||||
// 清理连接映射
|
||||
connectionCache.remove(addressKey);
|
||||
workerChannels.remove(workerKey);
|
||||
|
||||
// 失败所有待处理的请求
|
||||
failPendingRequests(channel);
|
||||
});
|
||||
|
||||
// 自动重连
|
||||
public CompletableFuture<Channel> reconnect(Address address) {
|
||||
return CompletableFuture
|
||||
.runAsync(() -> Thread.sleep(retryInterval))
|
||||
.thenCompose(v -> createConnection(address));
|
||||
}
|
||||
```
|
||||
|
||||
### 6.2 消息超时处理
|
||||
```java
|
||||
// Ask请求超时 (JDK8兼容方式)
|
||||
ScheduledExecutorService scheduler = Executors.newSingleThreadScheduledExecutor();
|
||||
scheduler.schedule(() -> {
|
||||
if (!future.isDone()) {
|
||||
channelManager.removePendingRequest(requestId);
|
||||
future.completeExceptionally(new TimeoutException("Request timeout"));
|
||||
}
|
||||
}, timeout, TimeUnit.SECONDS);
|
||||
```
|
||||
|
||||
### 6.3 异常传播
|
||||
- **网络异常**:连接失败、超时等网络层异常
|
||||
- **协议异常**:消息格式错误、编解码异常
|
||||
- **业务异常**:处理器执行异常
|
||||
|
||||
## 7. 监控与运维
|
||||
|
||||
### 7.1 关键指标
|
||||
- **连接数统计**:活跃连接数、总连接数
|
||||
- **消息统计**:发送/接收消息数、消息大小
|
||||
- **性能指标**:响应时间、吞吐量
|
||||
- **错误统计**:连接失败、消息失败、超时次数
|
||||
|
||||
### 7.2 日志记录
|
||||
```java
|
||||
// 连接事件
|
||||
log.info("[MuConnectionManager] Connected to {}", targetAddress);
|
||||
log.warn("[MuConnectionManager] Connection failed to {}", targetAddress);
|
||||
|
||||
// 消息事件
|
||||
log.debug("[MuTransporter] Sent {} message to {}", messageType, url);
|
||||
log.error("[MuHandler] Failed to process message", exception);
|
||||
```
|
||||
|
||||
### 7.3 健康检查
|
||||
- **连接健康**:定期检查连接状态
|
||||
- **心跳监控**:监控Worker心跳状态
|
||||
- **性能监控**:监控关键性能指标
|
||||
|
||||
## 8. 测试方案
|
||||
|
||||
### 8.1 单元测试
|
||||
- **组件测试**:各核心组件的独立测试
|
||||
- **协议测试**:消息编解码正确性测试
|
||||
- **异常测试**:各种异常场景的处理测试
|
||||
|
||||
### 8.2 集成测试
|
||||
- **通讯测试**:各种通讯场景的端到端测试
|
||||
- **故障测试**:网络故障、节点故障的恢复测试
|
||||
- **性能测试**:高并发、大数据量的性能测试
|
||||
|
||||
### 8.3 场景测试
|
||||
```java
|
||||
// Worker到Server通讯测试
|
||||
@Test
|
||||
public void testWorkerToServerCommunication() {
|
||||
// 1. 启动Server
|
||||
// 2. 启动Worker并连接到Server
|
||||
// 3. Worker发送消息到Server
|
||||
// 4. 验证Server收到消息
|
||||
}
|
||||
|
||||
// 网络故障恢复测试
|
||||
@Test
|
||||
public void testNetworkFailureRecovery() {
|
||||
// 1. 建立正常连接
|
||||
// 2. 模拟网络中断
|
||||
// 3. 恢复网络连接
|
||||
// 4. 验证通讯自动恢复
|
||||
}
|
||||
```
|
||||
|
||||
## 9. 部署指南
|
||||
|
||||
### 9.1 网络配置
|
||||
```yaml
|
||||
# Server配置
|
||||
server:
|
||||
host: 0.0.0.0 # 监听所有网卡
|
||||
port: 7700 # 监听端口
|
||||
external_host: 公网IP # 外部访问地址
|
||||
|
||||
# Worker配置
|
||||
worker:
|
||||
host: 0.0.0.0 # 本地监听地址
|
||||
port: 27777 # 本地监听端口
|
||||
server_host: 公网IP # Server地址
|
||||
server_port: 7700 # Server端口
|
||||
```
|
||||
|
||||
### 9.2 防火墙配置
|
||||
```bash
|
||||
# Server端:开放监听端口
|
||||
iptables -A INPUT -p tcp --dport 7700 -j ACCEPT
|
||||
|
||||
# Worker端:确保可以访问Server端口
|
||||
# 通常不需要额外配置,确保出站不受限即可
|
||||
```
|
||||
|
||||
### 9.3 高可用部署
|
||||
- **Server集群**:多个Server实例负载均衡
|
||||
- **Worker多连接**:Worker配置多个Server地址
|
||||
- **故障转移**:自动检测和切换故障节点
|
||||
|
||||
## 10. 总结
|
||||
|
||||
PowerJob Mu 协议通过创新的连接复用和延迟连接机制,成功解决了单向网络环境下的双向通讯问题。主要技术特点包括:
|
||||
|
||||
1. **网络适应性**:专为受限网络环境设计,无需复杂的网络配置
|
||||
2. **高性能**:基于Netty的异步IO,支持高并发和低延迟
|
||||
3. **可靠性**:完善的故障检测和恢复机制
|
||||
4. **扩展性**:统一的架构设计,支持多种通讯场景
|
||||
5. **易部署**:简化的配置和部署流程
|
||||
|
||||
该协议特别适合云原生、容器化部署以及混合云环境,为PowerJob在复杂网络环境下提供了可靠的通讯基础。
|
||||
|
||||
---
|
||||
|
||||
*PowerJob Mu Protocol Technical Design*
|
||||
*Version: 1.0*
|
||||
*Date: January 2025*
|
||||
42
powerjob-remote/powerjob-remote-impl-mu/pom.xml
Normal file
42
powerjob-remote/powerjob-remote-impl-mu/pom.xml
Normal file
@ -0,0 +1,42 @@
|
||||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0"
|
||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<parent>
|
||||
<artifactId>powerjob-remote</artifactId>
|
||||
<groupId>tech.powerjob</groupId>
|
||||
<version>5.1.2</version>
|
||||
</parent>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
<artifactId>powerjob-remote-impl-mu</artifactId>
|
||||
<name>powerjob-remote-impl-mu</name>
|
||||
<version>5.1.2</version>
|
||||
|
||||
<properties>
|
||||
<maven.compiler.source>8</maven.compiler.source>
|
||||
<maven.compiler.target>8</maven.compiler.target>
|
||||
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
|
||||
|
||||
<netty.version>4.1.104.Final</netty.version>
|
||||
<powerjob-remote-framework.version>5.1.2</powerjob-remote-framework.version>
|
||||
</properties>
|
||||
|
||||
<dependencies>
|
||||
|
||||
<dependency>
|
||||
<groupId>tech.powerjob</groupId>
|
||||
<artifactId>powerjob-remote-framework</artifactId>
|
||||
<version>${powerjob-remote-framework.version}</version>
|
||||
</dependency>
|
||||
|
||||
<!-- Netty for NIO communication -->
|
||||
<dependency>
|
||||
<groupId>io.netty</groupId>
|
||||
<artifactId>netty-all</artifactId>
|
||||
<version>${netty.version}</version>
|
||||
</dependency>
|
||||
|
||||
</dependencies>
|
||||
|
||||
</project>
|
||||
@ -0,0 +1,122 @@
|
||||
package tech.powerjob.remote.mu;
|
||||
|
||||
import io.netty.channel.Channel;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import tech.powerjob.common.serialize.JsonUtils;
|
||||
import tech.powerjob.remote.framework.base.Address;
|
||||
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
|
||||
/**
|
||||
* Channel manager for maintaining worker address to channel mapping
|
||||
* Supports both tell and ask modes for reverse communication
|
||||
*
|
||||
* @author claude
|
||||
* @since 2025/1/1
|
||||
*/
|
||||
@Slf4j
|
||||
public class ChannelManager {
|
||||
|
||||
private final ConcurrentMap<String, Channel> workerChannels = new ConcurrentHashMap<>();
|
||||
private final ConcurrentMap<String, CompletableFuture<Object>> pendingRequests = new ConcurrentHashMap<>();
|
||||
private final ConcurrentMap<String, Class<?>> requestResponseTypes = new ConcurrentHashMap<>();
|
||||
|
||||
/**
|
||||
* Register a worker channel
|
||||
* @param workerAddress worker address
|
||||
* @param channel Netty channel
|
||||
*/
|
||||
public void registerWorkerChannel(Address workerAddress, Channel channel) {
|
||||
String key = workerAddress.getHost() + ":" + workerAddress.getPort();
|
||||
workerChannels.put(key, channel);
|
||||
log.info("[ChannelManager] Registered worker channel: {}", key);
|
||||
|
||||
// Remove channel when it becomes inactive
|
||||
channel.closeFuture().addListener(future -> {
|
||||
workerChannels.remove(key);
|
||||
log.info("[ChannelManager] Removed inactive worker channel: {}", key);
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Get channel for worker
|
||||
* @param workerAddress worker address
|
||||
* @return Channel or null if not found
|
||||
*/
|
||||
public Channel getWorkerChannel(Address workerAddress) {
|
||||
String key = workerAddress.getHost() + ":" + workerAddress.getPort();
|
||||
return workerChannels.get(key);
|
||||
}
|
||||
|
||||
/**
|
||||
* Store pending request for ask mode
|
||||
* @param requestId request ID
|
||||
* @param future future to complete when response received
|
||||
* @param responseType expected response type
|
||||
*/
|
||||
public void registerPendingRequest(String requestId, CompletableFuture<Object> future, Class<?> responseType) {
|
||||
pendingRequests.put(requestId, future);
|
||||
requestResponseTypes.put(requestId, responseType);
|
||||
}
|
||||
|
||||
/**
|
||||
* Complete pending request with response
|
||||
* @param requestId request ID
|
||||
* @param response response object
|
||||
*/
|
||||
public void completePendingRequest(String requestId, Object response) {
|
||||
CompletableFuture<Object> future = pendingRequests.remove(requestId);
|
||||
Class<?> responseType = requestResponseTypes.remove(requestId);
|
||||
|
||||
if (future != null) {
|
||||
Object convertedResponse = convertResponse(response, responseType);
|
||||
future.complete(convertedResponse);
|
||||
} else {
|
||||
log.warn("[ChannelManager] No pending request found for ID: {}", requestId);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Complete pending request with exception
|
||||
* @param requestId request ID
|
||||
* @param exception exception
|
||||
*/
|
||||
public void completePendingRequestExceptionally(String requestId, Throwable exception) {
|
||||
CompletableFuture<Object> future = pendingRequests.remove(requestId);
|
||||
requestResponseTypes.remove(requestId); // Clean up response type mapping
|
||||
if (future != null) {
|
||||
future.completeExceptionally(exception);
|
||||
} else {
|
||||
log.warn("[ChannelManager] No pending request found for ID: {}", requestId);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove pending request (timeout cleanup)
|
||||
* @param requestId request ID
|
||||
*/
|
||||
public void removePendingRequest(String requestId) {
|
||||
pendingRequests.remove(requestId);
|
||||
requestResponseTypes.remove(requestId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert response to expected type
|
||||
* @param response raw response object
|
||||
* @param responseType expected response type
|
||||
* @return converted response
|
||||
*/
|
||||
private Object convertResponse(Object response, Class<?> responseType) {
|
||||
if (response == null || responseType == null) {
|
||||
return response;
|
||||
}
|
||||
|
||||
if (responseType.isInstance(response)) {
|
||||
return response;
|
||||
}
|
||||
|
||||
return JsonUtils.toJavaObject(response, responseType);
|
||||
}
|
||||
}
|
||||
@ -0,0 +1,179 @@
|
||||
package tech.powerjob.remote.mu;
|
||||
|
||||
import io.netty.bootstrap.ServerBootstrap;
|
||||
import io.netty.channel.*;
|
||||
import io.netty.channel.nio.NioEventLoopGroup;
|
||||
import io.netty.channel.socket.SocketChannel;
|
||||
import io.netty.channel.socket.nio.NioServerSocketChannel;
|
||||
import io.netty.handler.timeout.IdleStateHandler;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import tech.powerjob.remote.framework.actor.ActorInfo;
|
||||
import tech.powerjob.remote.framework.base.ServerType;
|
||||
import tech.powerjob.remote.framework.cs.CSInitializer;
|
||||
import tech.powerjob.remote.framework.cs.CSInitializerConfig;
|
||||
import tech.powerjob.remote.framework.transporter.Transporter;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
* Mu CSInitializer implementation using Netty
|
||||
* Supports bidirectional communication with worker-only outbound connectivity
|
||||
*
|
||||
* @author claude
|
||||
* @since 2025/1/1
|
||||
*/
|
||||
@Slf4j
|
||||
public class MuCSInitializer implements CSInitializer {
|
||||
|
||||
private EventLoopGroup bossGroup;
|
||||
private EventLoopGroup workerGroup;
|
||||
private Channel serverChannel;
|
||||
private CSInitializerConfig config;
|
||||
private final ChannelManager channelManager = new ChannelManager();
|
||||
|
||||
private MuServerHandler serverHandler;
|
||||
private MuWorkerHandler workerHandler;
|
||||
private MuConnectionManager connectionManager;
|
||||
|
||||
@Override
|
||||
public String type() {
|
||||
return "MU";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void init(CSInitializerConfig config) {
|
||||
this.config = config;
|
||||
|
||||
bossGroup = new NioEventLoopGroup(1);
|
||||
workerGroup = new NioEventLoopGroup();
|
||||
|
||||
if (config.getServerType() == ServerType.SERVER) {
|
||||
initServer();
|
||||
} else {
|
||||
initWorker();
|
||||
}
|
||||
}
|
||||
|
||||
private void initServer() {
|
||||
try {
|
||||
serverHandler = new MuServerHandler(channelManager);
|
||||
|
||||
ServerBootstrap bootstrap = new ServerBootstrap();
|
||||
bootstrap.group(bossGroup, workerGroup)
|
||||
.channel(NioServerSocketChannel.class)
|
||||
.option(ChannelOption.SO_BACKLOG, 128)
|
||||
.childOption(ChannelOption.SO_KEEPALIVE, true)
|
||||
.childOption(ChannelOption.TCP_NODELAY, true)
|
||||
.childHandler(new ChannelInitializer<SocketChannel>() {
|
||||
@Override
|
||||
protected void initChannel(SocketChannel ch) {
|
||||
ch.pipeline()
|
||||
.addLast(new IdleStateHandler(0, 0, 60, TimeUnit.SECONDS))
|
||||
.addLast(new MuMessageCodec())
|
||||
.addLast(serverHandler);
|
||||
}
|
||||
});
|
||||
|
||||
ChannelFuture future = bootstrap.bind(
|
||||
config.getBindAddress().getHost(),
|
||||
config.getBindAddress().getPort()
|
||||
).sync();
|
||||
|
||||
serverChannel = future.channel();
|
||||
log.info("[MuCSInitializer] Server started on {}:{}",
|
||||
config.getBindAddress().getHost(),
|
||||
config.getBindAddress().getPort());
|
||||
|
||||
// 初始化连接管理器,用于Server连接到其他Server
|
||||
connectionManager = new MuConnectionManager(workerGroup, channelManager, serverHandler, config.getBindAddress());
|
||||
log.info("[MuCSInitializer] Server initialized with client capabilities for server-to-server communication");
|
||||
} catch (Exception e) {
|
||||
log.error("[MuCSInitializer] Failed to start server", e);
|
||||
throw new RuntimeException("Failed to start Mu server", e);
|
||||
}
|
||||
}
|
||||
|
||||
private void initWorker() {
|
||||
try {
|
||||
// Worker需要同时具备服务端和客户端能力
|
||||
// 服务端:接受其他Worker的连接
|
||||
// 客户端:连接到Server或其他Worker
|
||||
|
||||
// 初始化handler
|
||||
workerHandler = new MuWorkerHandler(channelManager);
|
||||
|
||||
// 启动服务端,接受其他Worker的连接
|
||||
ServerBootstrap serverBootstrap = new ServerBootstrap();
|
||||
serverBootstrap.group(bossGroup, workerGroup)
|
||||
.channel(NioServerSocketChannel.class)
|
||||
.option(ChannelOption.SO_BACKLOG, 128)
|
||||
.childOption(ChannelOption.SO_KEEPALIVE, true)
|
||||
.childOption(ChannelOption.TCP_NODELAY, true)
|
||||
.childHandler(new ChannelInitializer<SocketChannel>() {
|
||||
@Override
|
||||
protected void initChannel(SocketChannel ch) {
|
||||
ch.pipeline()
|
||||
.addLast(new IdleStateHandler(0, 0, 60, TimeUnit.SECONDS))
|
||||
.addLast(new MuMessageCodec())
|
||||
.addLast(workerHandler);
|
||||
}
|
||||
});
|
||||
|
||||
ChannelFuture serverFuture = serverBootstrap.bind(
|
||||
config.getBindAddress().getHost(),
|
||||
config.getBindAddress().getPort()
|
||||
).sync();
|
||||
|
||||
serverChannel = serverFuture.channel();
|
||||
log.info("[MuCSInitializer] Worker server started on {}:{}",
|
||||
config.getBindAddress().getHost(),
|
||||
config.getBindAddress().getPort());
|
||||
|
||||
// 初始化连接管理器,用于连接到其他节点
|
||||
connectionManager = new MuConnectionManager(workerGroup, channelManager, workerHandler, config.getBindAddress());
|
||||
log.info("[MuCSInitializer] Worker initialized with server and client capabilities");
|
||||
} catch (Exception e) {
|
||||
log.error("[MuCSInitializer] Failed to initialize worker", e);
|
||||
throw new RuntimeException("Failed to initialize Mu worker", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Transporter buildTransporter() {
|
||||
return new MuTransporter(channelManager, config.getServerType(), connectionManager);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void bindHandlers(List<ActorInfo> actorInfos) {
|
||||
if (config.getServerType() == ServerType.SERVER && serverHandler != null) {
|
||||
serverHandler.bindHandlers(actorInfos);
|
||||
} else if (config.getServerType() == ServerType.WORKER && workerHandler != null) {
|
||||
workerHandler.bindHandlers(actorInfos);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
if (serverChannel != null) {
|
||||
serverChannel.close().sync();
|
||||
}
|
||||
if (connectionManager != null) {
|
||||
connectionManager.closeAllConnections();
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
log.warn("[MuCSInitializer] Interrupted while closing channels", e);
|
||||
} finally {
|
||||
if (bossGroup != null) {
|
||||
bossGroup.shutdownGracefully();
|
||||
}
|
||||
if (workerGroup != null) {
|
||||
workerGroup.shutdownGracefully();
|
||||
}
|
||||
}
|
||||
log.info("[MuCSInitializer] Mu CSInitializer closed");
|
||||
}
|
||||
}
|
||||
@ -0,0 +1,136 @@
|
||||
package tech.powerjob.remote.mu;
|
||||
|
||||
import io.netty.bootstrap.Bootstrap;
|
||||
import io.netty.channel.Channel;
|
||||
import io.netty.channel.ChannelFuture;
|
||||
import io.netty.channel.ChannelInitializer;
|
||||
import io.netty.channel.ChannelOption;
|
||||
import io.netty.channel.EventLoopGroup;
|
||||
import io.netty.channel.socket.SocketChannel;
|
||||
import io.netty.channel.socket.nio.NioSocketChannel;
|
||||
import io.netty.handler.timeout.IdleStateHandler;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import tech.powerjob.remote.framework.base.Address;
|
||||
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
* Connection manager for worker-side lazy connection to server
|
||||
*
|
||||
* @author claude
|
||||
* @since 2025/1/1
|
||||
*/
|
||||
@Slf4j
|
||||
public class MuConnectionManager {
|
||||
|
||||
private final EventLoopGroup workerGroup;
|
||||
private final ChannelManager channelManager;
|
||||
private final Object messageHandler; // Can be MuWorkerHandler or MuServerHandler
|
||||
private final Address localAddress;
|
||||
|
||||
private final ConcurrentMap<String, Channel> serverConnections = new ConcurrentHashMap<>();
|
||||
private final ConcurrentMap<String, CompletableFuture<Channel>> pendingConnections = new ConcurrentHashMap<>();
|
||||
|
||||
public MuConnectionManager(EventLoopGroup workerGroup, ChannelManager channelManager,
|
||||
Object messageHandler, Address localAddress) {
|
||||
this.workerGroup = workerGroup;
|
||||
this.channelManager = channelManager;
|
||||
this.messageHandler = messageHandler;
|
||||
this.localAddress = localAddress;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get or create connection to target address
|
||||
* @param targetAddress target address
|
||||
* @return CompletableFuture of channel
|
||||
*/
|
||||
public CompletableFuture<Channel> getOrCreateConnection(Address targetAddress) {
|
||||
String key = targetAddress.getHost() + ":" + targetAddress.getPort();
|
||||
|
||||
// Check if we already have an active connection
|
||||
Channel existingChannel = serverConnections.get(key);
|
||||
if (existingChannel != null && existingChannel.isActive()) {
|
||||
return CompletableFuture.completedFuture(existingChannel);
|
||||
}
|
||||
|
||||
// Check if there's already a pending connection
|
||||
CompletableFuture<Channel> pendingConnection = pendingConnections.get(key);
|
||||
if (pendingConnection != null) {
|
||||
return pendingConnection;
|
||||
}
|
||||
|
||||
// Create new connection
|
||||
CompletableFuture<Channel> connectionFuture = new CompletableFuture<>();
|
||||
pendingConnections.put(key, connectionFuture);
|
||||
|
||||
try {
|
||||
Bootstrap bootstrap = new Bootstrap();
|
||||
bootstrap.group(workerGroup)
|
||||
.channel(NioSocketChannel.class)
|
||||
.option(ChannelOption.SO_KEEPALIVE, true)
|
||||
.option(ChannelOption.TCP_NODELAY, true)
|
||||
.handler(new ChannelInitializer<SocketChannel>() {
|
||||
@Override
|
||||
protected void initChannel(SocketChannel ch) {
|
||||
ch.pipeline()
|
||||
.addLast(new IdleStateHandler(0, 30, 0, TimeUnit.SECONDS))
|
||||
.addLast(new MuMessageCodec())
|
||||
.addLast((io.netty.channel.ChannelHandler) messageHandler);
|
||||
|
||||
// Only add heartbeat handler for Worker connections
|
||||
if (messageHandler instanceof MuWorkerHandler) {
|
||||
ch.pipeline().addLast(new MuWorkerHeartbeatHandler(localAddress));
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
ChannelFuture future = bootstrap.connect(targetAddress.getHost(), targetAddress.getPort());
|
||||
future.addListener(f -> {
|
||||
pendingConnections.remove(key);
|
||||
if (f.isSuccess()) {
|
||||
Channel channel = future.channel();
|
||||
serverConnections.put(key, channel);
|
||||
|
||||
// Remove connection when it becomes inactive
|
||||
channel.closeFuture().addListener(closeFuture -> {
|
||||
serverConnections.remove(key);
|
||||
log.info("[MuConnectionManager] Removed inactive server connection: {}", key);
|
||||
});
|
||||
|
||||
connectionFuture.complete(channel);
|
||||
log.info("[MuConnectionManager] Connected to server: {}", key);
|
||||
} else {
|
||||
connectionFuture.completeExceptionally(f.cause());
|
||||
log.error("[MuConnectionManager] Failed to connect to server: {}", key, f.cause());
|
||||
}
|
||||
});
|
||||
} catch (Exception e) {
|
||||
pendingConnections.remove(key);
|
||||
connectionFuture.completeExceptionally(e);
|
||||
log.error("[MuConnectionManager] Error creating connection to server: {}", key, e);
|
||||
}
|
||||
|
||||
return connectionFuture;
|
||||
}
|
||||
|
||||
/**
|
||||
* Close all connections
|
||||
*/
|
||||
public void closeAllConnections() {
|
||||
for (Channel channel : serverConnections.values()) {
|
||||
if (channel.isActive()) {
|
||||
channel.close();
|
||||
}
|
||||
}
|
||||
serverConnections.clear();
|
||||
|
||||
// Complete all pending connections with exception
|
||||
for (CompletableFuture<Channel> future : pendingConnections.values()) {
|
||||
future.completeExceptionally(new RuntimeException("Connection manager is closing"));
|
||||
}
|
||||
pendingConnections.clear();
|
||||
}
|
||||
}
|
||||
@ -0,0 +1,37 @@
|
||||
package tech.powerjob.remote.mu;
|
||||
|
||||
import lombok.AllArgsConstructor;
|
||||
import lombok.Data;
|
||||
import lombok.NoArgsConstructor;
|
||||
import tech.powerjob.common.PowerSerializable;
|
||||
import tech.powerjob.remote.framework.base.Address;
|
||||
|
||||
/**
|
||||
* Mu protocol message format
|
||||
*
|
||||
* @author claude
|
||||
* @since 2025/1/1
|
||||
*/
|
||||
@Data
|
||||
@NoArgsConstructor
|
||||
@AllArgsConstructor
|
||||
public class MuMessage implements PowerSerializable {
|
||||
|
||||
/**
|
||||
* Message types
|
||||
*/
|
||||
public enum MessageType {
|
||||
TELL, // Fire-and-forget
|
||||
ASK, // Request-response
|
||||
RESPONSE, // Response to ASK
|
||||
HEARTBEAT, // Worker heartbeat/registration
|
||||
ERROR // Error response
|
||||
}
|
||||
|
||||
private MessageType messageType;
|
||||
private String requestId; // Unique ID for ask/response correlation
|
||||
private String path; // Handler path
|
||||
private Address senderAddress; // Sender address for registration
|
||||
private Object payload; // Actual message payload
|
||||
private String errorMessage; // Error message for ERROR type
|
||||
}
|
||||
@ -0,0 +1,75 @@
|
||||
package tech.powerjob.remote.mu;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.handler.codec.ByteToMessageCodec;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Mu message codec for encoding/decoding messages over Netty
|
||||
*
|
||||
* @author claude
|
||||
* @since 2025/1/1
|
||||
*/
|
||||
@Slf4j
|
||||
public class MuMessageCodec extends ByteToMessageCodec<MuMessage> {
|
||||
|
||||
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
||||
private static final int MAX_MESSAGE_SIZE = 64 * 1024 * 1024; // 64MB
|
||||
|
||||
@Override
|
||||
protected void encode(ChannelHandlerContext ctx, MuMessage msg, ByteBuf out) throws Exception {
|
||||
try {
|
||||
byte[] data = OBJECT_MAPPER.writeValueAsBytes(msg);
|
||||
if (data.length > MAX_MESSAGE_SIZE) {
|
||||
throw new IllegalArgumentException("Message too large: " + data.length + " bytes");
|
||||
}
|
||||
|
||||
// Write message length followed by message data
|
||||
out.writeInt(data.length);
|
||||
out.writeBytes(data);
|
||||
} catch (Exception e) {
|
||||
log.error("[MuMessageCodec] Failed to encode message", e);
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out) throws Exception {
|
||||
try {
|
||||
// Need at least 4 bytes for length
|
||||
if (in.readableBytes() < 4) {
|
||||
return;
|
||||
}
|
||||
|
||||
// Mark reader index to reset if not enough data
|
||||
in.markReaderIndex();
|
||||
|
||||
// Read message length
|
||||
int length = in.readInt();
|
||||
|
||||
if (length <= 0 || length > MAX_MESSAGE_SIZE) {
|
||||
throw new IllegalArgumentException("Invalid message length: " + length);
|
||||
}
|
||||
|
||||
// Check if we have enough bytes for the full message
|
||||
if (in.readableBytes() < length) {
|
||||
in.resetReaderIndex();
|
||||
return;
|
||||
}
|
||||
|
||||
// Read and decode message
|
||||
byte[] data = new byte[length];
|
||||
in.readBytes(data);
|
||||
|
||||
MuMessage message = OBJECT_MAPPER.readValue(data, MuMessage.class);
|
||||
out.add(message);
|
||||
} catch (Exception e) {
|
||||
log.error("[MuMessageCodec] Failed to decode message", e);
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -0,0 +1,18 @@
|
||||
package tech.powerjob.remote.mu;
|
||||
|
||||
import tech.powerjob.remote.framework.transporter.Protocol;
|
||||
|
||||
/**
|
||||
* Mu Protocol implementation using Netty for bidirectional communication
|
||||
* with support for worker-only outbound connectivity
|
||||
*
|
||||
* @author claude
|
||||
* @since 2025/1/1
|
||||
*/
|
||||
public class MuProtocol implements Protocol {
|
||||
|
||||
@Override
|
||||
public String name() {
|
||||
return tech.powerjob.common.enums.Protocol.MU.name();
|
||||
}
|
||||
}
|
||||
@ -0,0 +1,185 @@
|
||||
package tech.powerjob.remote.mu;
|
||||
|
||||
import io.netty.channel.ChannelHandler;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.channel.SimpleChannelInboundHandler;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import tech.powerjob.common.serialize.JsonUtils;
|
||||
import tech.powerjob.remote.framework.actor.ActorInfo;
|
||||
import tech.powerjob.remote.framework.actor.HandlerInfo;
|
||||
import tech.powerjob.remote.framework.utils.RemoteUtils;
|
||||
|
||||
import java.lang.reflect.Method;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
/**
|
||||
* Server-side message handler for Mu protocol
|
||||
* Handles incoming messages from workers and manages channel registration
|
||||
*
|
||||
* @author claude
|
||||
* @since 2025/1/1
|
||||
*/
|
||||
@Slf4j
|
||||
@ChannelHandler.Sharable
|
||||
public class MuServerHandler extends SimpleChannelInboundHandler<MuMessage> {
|
||||
|
||||
private final ChannelManager channelManager;
|
||||
private final Map<String, ActorInfo> handlerMap = new ConcurrentHashMap<>();
|
||||
|
||||
public MuServerHandler(ChannelManager channelManager) {
|
||||
this.channelManager = channelManager;
|
||||
}
|
||||
|
||||
public void bindHandlers(List<ActorInfo> actorInfos) {
|
||||
for (ActorInfo actorInfo : actorInfos) {
|
||||
if (actorInfo.getHandlerInfos() != null) {
|
||||
for (HandlerInfo handlerInfo : actorInfo.getHandlerInfos()) {
|
||||
String path = handlerInfo.getLocation().toPath();
|
||||
handlerMap.put(path, actorInfo);
|
||||
log.info("[MuServerHandler] Bound handler: {}", path);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void channelRead0(ChannelHandlerContext ctx, MuMessage msg) throws Exception {
|
||||
try {
|
||||
switch (msg.getMessageType()) {
|
||||
case HEARTBEAT:
|
||||
handleHeartbeat(ctx, msg);
|
||||
break;
|
||||
case TELL:
|
||||
handleTell(ctx, msg);
|
||||
break;
|
||||
case ASK:
|
||||
handleAsk(ctx, msg);
|
||||
break;
|
||||
case RESPONSE:
|
||||
handleResponse(ctx, msg);
|
||||
break;
|
||||
case ERROR:
|
||||
handleError(ctx, msg);
|
||||
break;
|
||||
default:
|
||||
log.warn("[MuServerHandler] Unknown message type: {}", msg.getMessageType());
|
||||
}
|
||||
} catch (Exception e) {
|
||||
log.error("[MuServerHandler] Error processing message", e);
|
||||
if (msg.getMessageType() == MuMessage.MessageType.ASK) {
|
||||
// Send error response for ASK messages
|
||||
MuMessage errorResponse = new MuMessage(
|
||||
MuMessage.MessageType.ERROR,
|
||||
msg.getRequestId(),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
"Internal server error: " + e.getMessage()
|
||||
);
|
||||
ctx.writeAndFlush(errorResponse);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void handleHeartbeat(ChannelHandlerContext ctx, MuMessage msg) {
|
||||
if (msg.getSenderAddress() != null) {
|
||||
channelManager.registerWorkerChannel(msg.getSenderAddress(), ctx.channel());
|
||||
log.debug("[MuServerHandler] Registered worker: {}", msg.getSenderAddress());
|
||||
}
|
||||
}
|
||||
|
||||
private void handleTell(ChannelHandlerContext ctx, MuMessage msg) {
|
||||
invokeHandler(msg, false, ctx);
|
||||
}
|
||||
|
||||
private void handleAsk(ChannelHandlerContext ctx, MuMessage msg) {
|
||||
Object response = invokeHandler(msg, true, ctx);
|
||||
|
||||
MuMessage.MessageType responseType = response != null ?
|
||||
MuMessage.MessageType.RESPONSE : MuMessage.MessageType.ERROR;
|
||||
String errorMessage = response == null ? "Handler returned null" : null;
|
||||
|
||||
MuMessage responseMsg = new MuMessage(
|
||||
responseType,
|
||||
msg.getRequestId(),
|
||||
null,
|
||||
null,
|
||||
response,
|
||||
errorMessage
|
||||
);
|
||||
|
||||
ctx.writeAndFlush(responseMsg);
|
||||
}
|
||||
|
||||
private void handleResponse(ChannelHandlerContext ctx, MuMessage msg) {
|
||||
channelManager.completePendingRequest(msg.getRequestId(), msg.getPayload());
|
||||
}
|
||||
|
||||
private void handleError(ChannelHandlerContext ctx, MuMessage msg) {
|
||||
Exception exception = new RuntimeException(msg.getErrorMessage());
|
||||
channelManager.completePendingRequestExceptionally(msg.getRequestId(), exception);
|
||||
}
|
||||
|
||||
private Object invokeHandler(MuMessage msg, boolean needResponse, ChannelHandlerContext ctx) {
|
||||
try {
|
||||
String path = msg.getPath();
|
||||
ActorInfo actorInfo = handlerMap.get(path);
|
||||
|
||||
if (actorInfo == null) {
|
||||
log.warn("[MuServerHandler] No handler found for path: {}", path);
|
||||
return null;
|
||||
}
|
||||
|
||||
HandlerInfo handlerInfo = actorInfo.getHandlerInfos().stream()
|
||||
.filter(h -> h.getLocation().toPath().equals(path))
|
||||
.findFirst()
|
||||
.orElse(null);
|
||||
|
||||
if (handlerInfo == null) {
|
||||
log.warn("[MuServerHandler] Handler info not found for path: {}", path);
|
||||
return null;
|
||||
}
|
||||
|
||||
Method method = handlerInfo.getMethod();
|
||||
Optional<Class<?>> powerSerializeClz = RemoteUtils.findPowerSerialize(method.getParameterTypes());
|
||||
|
||||
if (!powerSerializeClz.isPresent()) {
|
||||
log.error("[MuServerHandler] No PowerSerializable parameter found for handler: {}", path);
|
||||
return null;
|
||||
}
|
||||
|
||||
Object convertedPayload = convertPayload(msg.getPayload(), powerSerializeClz.get());
|
||||
Object response = method.invoke(actorInfo.getActor(), convertedPayload);
|
||||
|
||||
if (needResponse) {
|
||||
return response;
|
||||
}
|
||||
|
||||
return null;
|
||||
} catch (Exception e) {
|
||||
log.error("[MuServerHandler] Failed to invoke handler for path: {}", msg.getPath(), e);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
private Object convertPayload(Object payload, Class<?> targetClass) {
|
||||
if (payload == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
if (targetClass.isInstance(payload)) {
|
||||
return payload;
|
||||
}
|
||||
|
||||
return JsonUtils.toJavaObject(payload, targetClass);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
|
||||
log.error("[MuServerHandler] Channel exception", cause);
|
||||
ctx.close();
|
||||
}
|
||||
}
|
||||
@ -0,0 +1,195 @@
|
||||
package tech.powerjob.remote.mu;
|
||||
|
||||
import io.netty.channel.Channel;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import tech.powerjob.common.PowerSerializable;
|
||||
import tech.powerjob.remote.framework.base.RemotingException;
|
||||
import tech.powerjob.remote.framework.base.ServerType;
|
||||
import tech.powerjob.remote.framework.base.URL;
|
||||
import tech.powerjob.remote.framework.transporter.Protocol;
|
||||
import tech.powerjob.remote.framework.transporter.Transporter;
|
||||
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
* Mu protocol transporter implementation
|
||||
* Handles both client-side (worker) and reverse (server-to-worker) communication
|
||||
*
|
||||
* @author claude
|
||||
* @since 2025/1/1
|
||||
*/
|
||||
@Slf4j
|
||||
public class MuTransporter implements Transporter {
|
||||
|
||||
private static final Protocol PROTOCOL = new MuProtocol();
|
||||
private static final long ASK_TIMEOUT_SECONDS = 30;
|
||||
|
||||
private final ChannelManager channelManager;
|
||||
private final ServerType serverType;
|
||||
private final MuConnectionManager connectionManager; // For worker-side lazy connection
|
||||
|
||||
public MuTransporter(ChannelManager channelManager, ServerType serverType, MuConnectionManager connectionManager) {
|
||||
this.channelManager = channelManager;
|
||||
this.serverType = serverType;
|
||||
this.connectionManager = connectionManager;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Protocol getProtocol() {
|
||||
return PROTOCOL;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void tell(URL url, PowerSerializable request) {
|
||||
try {
|
||||
MuMessage message = new MuMessage(
|
||||
MuMessage.MessageType.TELL,
|
||||
null,
|
||||
url.getLocation().toPath(),
|
||||
null,
|
||||
request,
|
||||
null
|
||||
);
|
||||
|
||||
if (serverType == ServerType.WORKER) {
|
||||
// Worker to server/worker: use connection manager for lazy connection
|
||||
connectionManager.getOrCreateConnection(url.getAddress())
|
||||
.thenAccept(channel -> {
|
||||
if (channel.isActive()) {
|
||||
channel.writeAndFlush(message);
|
||||
log.debug("[MuTransporter] Sent TELL message to {}", url);
|
||||
} else {
|
||||
log.error("[MuTransporter] Channel is not active for {}", url);
|
||||
}
|
||||
})
|
||||
.exceptionally(throwable -> {
|
||||
log.error("[MuTransporter] Failed to get connection for TELL to {}", url, throwable);
|
||||
return null;
|
||||
});
|
||||
} else {
|
||||
// Server side: distinguish between worker and server targets
|
||||
if (url.getServerType() == ServerType.WORKER) {
|
||||
// Server to worker: use stored channel from worker registration
|
||||
Channel channel = channelManager.getWorkerChannel(url.getAddress());
|
||||
if (channel != null && channel.isActive()) {
|
||||
channel.writeAndFlush(message);
|
||||
log.debug("[MuTransporter] Sent TELL message to worker {}", url);
|
||||
} else {
|
||||
log.error("[MuTransporter] No active channel available for worker {}", url);
|
||||
throw new RemotingException("No active channel available for " + url);
|
||||
}
|
||||
} else {
|
||||
// Server to server: use connection manager for direct connection
|
||||
connectionManager.getOrCreateConnection(url.getAddress())
|
||||
.thenAccept(channel -> {
|
||||
if (channel.isActive()) {
|
||||
channel.writeAndFlush(message);
|
||||
log.debug("[MuTransporter] Sent TELL message to server {}", url);
|
||||
} else {
|
||||
log.error("[MuTransporter] Channel is not active for server {}", url);
|
||||
}
|
||||
})
|
||||
.exceptionally(throwable -> {
|
||||
log.error("[MuTransporter] Failed to get connection for TELL to server {}", url, throwable);
|
||||
return null;
|
||||
});
|
||||
}
|
||||
}
|
||||
} catch (Exception e) {
|
||||
log.error("[MuTransporter] Failed to send TELL message to {}", url, e);
|
||||
throw new RemotingException("Failed to send TELL message", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> CompletionStage<T> ask(URL url, PowerSerializable request, Class<T> clz) throws RemotingException {
|
||||
try {
|
||||
String requestId = UUID.randomUUID().toString();
|
||||
CompletableFuture<T> future = new CompletableFuture<>();
|
||||
|
||||
// Register the future for response handling
|
||||
channelManager.registerPendingRequest(requestId, (CompletableFuture<Object>) future, clz);
|
||||
|
||||
// Set timeout for the request (JDK8 compatible)
|
||||
future.whenComplete((result, throwable) -> {
|
||||
if (throwable != null) {
|
||||
channelManager.removePendingRequest(requestId);
|
||||
}
|
||||
});
|
||||
|
||||
// Schedule timeout manually for JDK8 compatibility
|
||||
java.util.concurrent.Executors.newSingleThreadScheduledExecutor().schedule(() -> {
|
||||
if (!future.isDone()) {
|
||||
channelManager.removePendingRequest(requestId);
|
||||
future.completeExceptionally(new java.util.concurrent.TimeoutException("Request timeout after " + ASK_TIMEOUT_SECONDS + " seconds"));
|
||||
}
|
||||
}, ASK_TIMEOUT_SECONDS, TimeUnit.SECONDS);
|
||||
|
||||
MuMessage message = new MuMessage(
|
||||
MuMessage.MessageType.ASK,
|
||||
requestId,
|
||||
url.getLocation().toPath(),
|
||||
null,
|
||||
request,
|
||||
null
|
||||
);
|
||||
|
||||
if (serverType == ServerType.WORKER) {
|
||||
// Worker to server/worker: use connection manager for lazy connection
|
||||
connectionManager.getOrCreateConnection(url.getAddress())
|
||||
.thenAccept(channel -> {
|
||||
if (channel.isActive()) {
|
||||
channel.writeAndFlush(message);
|
||||
log.debug("[MuTransporter] Sent ASK message to {} with requestId {}", url, requestId);
|
||||
} else {
|
||||
channelManager.removePendingRequest(requestId);
|
||||
future.completeExceptionally(new RemotingException("Channel is not active for " + url));
|
||||
}
|
||||
})
|
||||
.exceptionally(throwable -> {
|
||||
channelManager.removePendingRequest(requestId);
|
||||
future.completeExceptionally(new RemotingException("Failed to get connection for ASK to " + url, throwable));
|
||||
return null;
|
||||
});
|
||||
} else {
|
||||
// Server side: distinguish between worker and server targets
|
||||
if (url.getServerType() == ServerType.WORKER) {
|
||||
// Server to worker: use stored channel from worker registration
|
||||
Channel channel = channelManager.getWorkerChannel(url.getAddress());
|
||||
if (channel != null && channel.isActive()) {
|
||||
channel.writeAndFlush(message);
|
||||
log.debug("[MuTransporter] Sent ASK message to worker {} with requestId {}", url, requestId);
|
||||
} else {
|
||||
channelManager.removePendingRequest(requestId);
|
||||
future.completeExceptionally(new RemotingException("No active channel available for " + url));
|
||||
}
|
||||
} else {
|
||||
// Server to server: use connection manager for direct connection
|
||||
connectionManager.getOrCreateConnection(url.getAddress())
|
||||
.thenAccept(channel -> {
|
||||
if (channel.isActive()) {
|
||||
channel.writeAndFlush(message);
|
||||
log.debug("[MuTransporter] Sent ASK message to server {} with requestId {}", url, requestId);
|
||||
} else {
|
||||
channelManager.removePendingRequest(requestId);
|
||||
future.completeExceptionally(new RemotingException("Channel is not active for server " + url));
|
||||
}
|
||||
})
|
||||
.exceptionally(throwable -> {
|
||||
channelManager.removePendingRequest(requestId);
|
||||
future.completeExceptionally(new RemotingException("Failed to get connection for ASK to server " + url, throwable));
|
||||
return null;
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
return future;
|
||||
} catch (Exception e) {
|
||||
log.error("[MuTransporter] Failed to send ASK message to {}", url, e);
|
||||
throw new RemotingException("Failed to send ASK message", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -0,0 +1,190 @@
|
||||
package tech.powerjob.remote.mu;
|
||||
|
||||
import io.netty.channel.ChannelHandler;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.channel.SimpleChannelInboundHandler;
|
||||
import lombok.SneakyThrows;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import tech.powerjob.common.serialize.JsonUtils;
|
||||
import tech.powerjob.remote.framework.actor.ActorInfo;
|
||||
import tech.powerjob.remote.framework.actor.HandlerInfo;
|
||||
import tech.powerjob.remote.framework.utils.RemoteUtils;
|
||||
|
||||
import java.lang.reflect.Method;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
/**
|
||||
* Worker-side message handler for Mu protocol
|
||||
* Handles incoming messages from server and processes local handlers
|
||||
*
|
||||
* @author claude
|
||||
* @since 2025/1/1
|
||||
*/
|
||||
@Slf4j
|
||||
@ChannelHandler.Sharable
|
||||
public class MuWorkerHandler extends SimpleChannelInboundHandler<MuMessage> {
|
||||
|
||||
private final Map<String, ActorInfo> handlerMap = new ConcurrentHashMap<>();
|
||||
private final ChannelManager channelManager;
|
||||
|
||||
public MuWorkerHandler(ChannelManager channelManager) {
|
||||
this.channelManager = channelManager;
|
||||
}
|
||||
|
||||
public void bindHandlers(List<ActorInfo> actorInfos) {
|
||||
for (ActorInfo actorInfo : actorInfos) {
|
||||
if (actorInfo.getHandlerInfos() != null) {
|
||||
for (HandlerInfo handlerInfo : actorInfo.getHandlerInfos()) {
|
||||
String path = handlerInfo.getLocation().toPath();
|
||||
handlerMap.put(path, actorInfo);
|
||||
log.info("[MuWorkerHandler] Bound handler: {}", path);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void channelRead0(ChannelHandlerContext ctx, MuMessage msg) throws Exception {
|
||||
try {
|
||||
switch (msg.getMessageType()) {
|
||||
case TELL:
|
||||
handleTell(ctx, msg);
|
||||
break;
|
||||
case ASK:
|
||||
handleAsk(ctx, msg);
|
||||
break;
|
||||
case RESPONSE:
|
||||
handleResponse(ctx, msg);
|
||||
break;
|
||||
case ERROR:
|
||||
handleError(ctx, msg);
|
||||
break;
|
||||
case HEARTBEAT:
|
||||
handleHeartbeat(ctx, msg);
|
||||
break;
|
||||
default:
|
||||
log.warn("[MuWorkerHandler] Unknown message type: {}", msg.getMessageType());
|
||||
}
|
||||
} catch (Exception e) {
|
||||
log.error("[MuWorkerHandler] Error processing message", e);
|
||||
if (msg.getMessageType() == MuMessage.MessageType.ASK) {
|
||||
// Send error response for ASK messages
|
||||
MuMessage errorResponse = new MuMessage(
|
||||
MuMessage.MessageType.ERROR,
|
||||
msg.getRequestId(),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
"Internal worker error: " + e.getMessage()
|
||||
);
|
||||
ctx.writeAndFlush(errorResponse);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void handleTell(ChannelHandlerContext ctx, MuMessage msg) {
|
||||
invokeHandler(msg, false, ctx);
|
||||
}
|
||||
|
||||
private void handleAsk(ChannelHandlerContext ctx, MuMessage msg) {
|
||||
Object response = invokeHandler(msg, true, ctx);
|
||||
|
||||
MuMessage.MessageType responseType = response != null ?
|
||||
MuMessage.MessageType.RESPONSE : MuMessage.MessageType.ERROR;
|
||||
String errorMessage = response == null ? "Handler returned null" : null;
|
||||
|
||||
MuMessage responseMsg = new MuMessage(
|
||||
responseType,
|
||||
msg.getRequestId(),
|
||||
null,
|
||||
null,
|
||||
response,
|
||||
errorMessage
|
||||
);
|
||||
|
||||
ctx.writeAndFlush(responseMsg);
|
||||
}
|
||||
|
||||
private void handleResponse(ChannelHandlerContext ctx, MuMessage msg) {
|
||||
channelManager.completePendingRequest(msg.getRequestId(), msg.getPayload());
|
||||
}
|
||||
|
||||
private void handleError(ChannelHandlerContext ctx, MuMessage msg) {
|
||||
Exception exception = new RuntimeException(msg.getErrorMessage());
|
||||
channelManager.completePendingRequestExceptionally(msg.getRequestId(), exception);
|
||||
}
|
||||
|
||||
private void handleHeartbeat(ChannelHandlerContext ctx, MuMessage msg) {
|
||||
// Worker接收到心跳消息时,通常不需要特殊处理
|
||||
// 但记录一下调试信息,表明收到了心跳
|
||||
if (msg.getSenderAddress() != null) {
|
||||
log.debug("[MuWorkerHandler] Received heartbeat from: {}", msg.getSenderAddress());
|
||||
} else {
|
||||
log.debug("[MuWorkerHandler] Received heartbeat");
|
||||
}
|
||||
}
|
||||
|
||||
private Object invokeHandler(MuMessage msg, boolean needResponse, ChannelHandlerContext ctx) {
|
||||
try {
|
||||
String path = msg.getPath();
|
||||
ActorInfo actorInfo = handlerMap.get(path);
|
||||
|
||||
if (actorInfo == null) {
|
||||
log.warn("[MuWorkerHandler] No handler found for path: {}", path);
|
||||
return null;
|
||||
}
|
||||
|
||||
HandlerInfo handlerInfo = actorInfo.getHandlerInfos().stream()
|
||||
.filter(h -> h.getLocation().toPath().equals(path))
|
||||
.findFirst()
|
||||
.orElse(null);
|
||||
|
||||
if (handlerInfo == null) {
|
||||
log.warn("[MuWorkerHandler] Handler info not found for path: {}", path);
|
||||
return null;
|
||||
}
|
||||
|
||||
Method method = handlerInfo.getMethod();
|
||||
Optional<Class<?>> powerSerializeClz = RemoteUtils.findPowerSerialize(method.getParameterTypes());
|
||||
|
||||
if (!powerSerializeClz.isPresent()) {
|
||||
log.error("[MuWorkerHandler] No PowerSerializable parameter found for handler: {}", path);
|
||||
return null;
|
||||
}
|
||||
|
||||
Object convertedPayload = convertPayload(msg.getPayload(), powerSerializeClz.get());
|
||||
Object response = method.invoke(actorInfo.getActor(), convertedPayload);
|
||||
|
||||
if (needResponse) {
|
||||
return response;
|
||||
}
|
||||
|
||||
return null;
|
||||
} catch (Exception e) {
|
||||
log.error("[MuWorkerHandler] Failed to invoke handler for path: {}", msg.getPath(), e);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
@SneakyThrows
|
||||
private Object convertPayload(Object payload, Class<?> targetClass) {
|
||||
if (payload == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
if (targetClass.isInstance(payload)) {
|
||||
return payload;
|
||||
}
|
||||
|
||||
return JsonUtils.toJavaObject(payload, targetClass);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
|
||||
log.error("[MuWorkerHandler] Channel exception", cause);
|
||||
ctx.close();
|
||||
}
|
||||
}
|
||||
@ -0,0 +1,61 @@
|
||||
package tech.powerjob.remote.mu;
|
||||
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.channel.ChannelInboundHandlerAdapter;
|
||||
import io.netty.handler.timeout.IdleState;
|
||||
import io.netty.handler.timeout.IdleStateEvent;
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
import tech.powerjob.remote.framework.base.Address;
|
||||
|
||||
/**
|
||||
* Worker heartbeat handler for maintaining connection and registration
|
||||
*
|
||||
* @author claude
|
||||
* @since 2025/1/1
|
||||
*/
|
||||
@Slf4j
|
||||
public class MuWorkerHeartbeatHandler extends ChannelInboundHandlerAdapter {
|
||||
|
||||
private final Address workerAddress;
|
||||
|
||||
public MuWorkerHeartbeatHandler(Address workerAddress) {
|
||||
this.workerAddress = workerAddress;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void channelActive(ChannelHandlerContext ctx) throws Exception {
|
||||
super.channelActive(ctx);
|
||||
sendHeartbeat(ctx);
|
||||
log.info("[MuWorkerHeartbeatHandler] Worker connected and sent initial heartbeat");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception {
|
||||
if (evt instanceof IdleStateEvent) {
|
||||
IdleStateEvent event = (IdleStateEvent) evt;
|
||||
if (event.state() == IdleState.WRITER_IDLE) {
|
||||
sendHeartbeat(ctx);
|
||||
log.debug("[MuWorkerHeartbeatHandler] Sent heartbeat");
|
||||
}
|
||||
}
|
||||
super.userEventTriggered(ctx, evt);
|
||||
}
|
||||
|
||||
private void sendHeartbeat(ChannelHandlerContext ctx) {
|
||||
MuMessage heartbeat = new MuMessage(
|
||||
MuMessage.MessageType.HEARTBEAT,
|
||||
null,
|
||||
null,
|
||||
workerAddress,
|
||||
null,
|
||||
null
|
||||
);
|
||||
ctx.writeAndFlush(heartbeat);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
|
||||
log.error("[MuWorkerHeartbeatHandler] Exception in heartbeat handler", cause);
|
||||
ctx.close();
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user