|
@@ -23,15 +23,15 @@ import org.apache.hadoop.scm.container.common.helpers.Pipeline;
|
|
import org.apache.ratis.client.ClientFactory;
|
|
import org.apache.ratis.client.ClientFactory;
|
|
import org.apache.ratis.client.RaftClient;
|
|
import org.apache.ratis.client.RaftClient;
|
|
import org.apache.ratis.conf.RaftProperties;
|
|
import org.apache.ratis.conf.RaftProperties;
|
|
|
|
+import org.apache.ratis.protocol.RaftGroup;
|
|
|
|
+import org.apache.ratis.protocol.RaftGroupId;
|
|
import org.apache.ratis.protocol.RaftPeer;
|
|
import org.apache.ratis.protocol.RaftPeer;
|
|
import org.apache.ratis.protocol.RaftPeerId;
|
|
import org.apache.ratis.protocol.RaftPeerId;
|
|
import org.apache.ratis.rpc.RpcType;
|
|
import org.apache.ratis.rpc.RpcType;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.LoggerFactory;
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
-import java.util.ArrayList;
|
|
|
|
-import java.util.Arrays;
|
|
|
|
-import java.util.List;
|
|
|
|
|
|
+import java.util.*;
|
|
import java.util.stream.Collectors;
|
|
import java.util.stream.Collectors;
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -41,6 +41,10 @@ public interface RatisHelper {
|
|
Logger LOG = LoggerFactory.getLogger(RatisHelper.class);
|
|
Logger LOG = LoggerFactory.getLogger(RatisHelper.class);
|
|
|
|
|
|
static String toRaftPeerIdString(DatanodeID id) {
|
|
static String toRaftPeerIdString(DatanodeID id) {
|
|
|
|
+ return id.getIpAddr() + "_" + id.getRatisPort();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ static String toRaftPeerAddressString(DatanodeID id) {
|
|
return id.getIpAddr() + ":" + id.getRatisPort();
|
|
return id.getIpAddr() + ":" + id.getRatisPort();
|
|
}
|
|
}
|
|
|
|
|
|
@@ -48,44 +52,59 @@ public interface RatisHelper {
|
|
return RaftPeerId.valueOf(toRaftPeerIdString(id));
|
|
return RaftPeerId.valueOf(toRaftPeerIdString(id));
|
|
}
|
|
}
|
|
|
|
|
|
- static RaftPeer toRaftPeer(String id) {
|
|
|
|
- return new RaftPeer(RaftPeerId.valueOf(id), id);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
static RaftPeer toRaftPeer(DatanodeID id) {
|
|
static RaftPeer toRaftPeer(DatanodeID id) {
|
|
- return toRaftPeer(toRaftPeerIdString(id));
|
|
|
|
|
|
+ return new RaftPeer(toRaftPeerId(id), toRaftPeerAddressString(id));
|
|
}
|
|
}
|
|
|
|
|
|
static List<RaftPeer> toRaftPeers(Pipeline pipeline) {
|
|
static List<RaftPeer> toRaftPeers(Pipeline pipeline) {
|
|
- return pipeline.getMachines().stream()
|
|
|
|
- .map(RatisHelper::toRaftPeer)
|
|
|
|
|
|
+ return toRaftPeers(pipeline.getMachines());
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ static <E extends DatanodeID> List<RaftPeer> toRaftPeers(List<E> datanodes) {
|
|
|
|
+ return datanodes.stream().map(RatisHelper::toRaftPeer)
|
|
.collect(Collectors.toList());
|
|
.collect(Collectors.toList());
|
|
}
|
|
}
|
|
|
|
|
|
- static RaftPeer[] toRaftPeerArray(Pipeline pipeline) {
|
|
|
|
- return toRaftPeers(pipeline).toArray(RaftPeer.EMPTY_PEERS);
|
|
|
|
|
|
+ /* TODO: use a dummy id for all groups for the moment.
|
|
|
|
+ * It should be changed to a unique id for each group.
|
|
|
|
+ */
|
|
|
|
+ RaftGroupId DUMMY_GROUP_ID = RaftGroupId.randomId();
|
|
|
|
+
|
|
|
|
+ RaftGroup EMPTY_GROUP = new RaftGroup(DUMMY_GROUP_ID,
|
|
|
|
+ Collections.emptyList());
|
|
|
|
+
|
|
|
|
+ static RaftGroup emptyRaftGroup() {
|
|
|
|
+ return EMPTY_GROUP;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ static RaftGroup newRaftGroup(Collection<RaftPeer> peers) {
|
|
|
|
+ return peers.isEmpty()? emptyRaftGroup()
|
|
|
|
+ : new RaftGroup(DUMMY_GROUP_ID, peers);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ static RaftGroup newRaftGroup(Pipeline pipeline) {
|
|
|
|
+ return newRaftGroup(toRaftPeers(pipeline));
|
|
}
|
|
}
|
|
|
|
|
|
static RaftClient newRaftClient(RpcType rpcType, Pipeline pipeline) {
|
|
static RaftClient newRaftClient(RpcType rpcType, Pipeline pipeline) {
|
|
return newRaftClient(rpcType, toRaftPeerId(pipeline.getLeader()),
|
|
return newRaftClient(rpcType, toRaftPeerId(pipeline.getLeader()),
|
|
- toRaftPeers(pipeline));
|
|
|
|
|
|
+ newRaftGroup(pipeline));
|
|
}
|
|
}
|
|
|
|
|
|
static RaftClient newRaftClient(RpcType rpcType, RaftPeer leader) {
|
|
static RaftClient newRaftClient(RpcType rpcType, RaftPeer leader) {
|
|
return newRaftClient(rpcType, leader.getId(),
|
|
return newRaftClient(rpcType, leader.getId(),
|
|
- new ArrayList<>(Arrays.asList(leader)));
|
|
|
|
|
|
+ newRaftGroup(new ArrayList<>(Arrays.asList(leader))));
|
|
}
|
|
}
|
|
|
|
|
|
static RaftClient newRaftClient(
|
|
static RaftClient newRaftClient(
|
|
- RpcType rpcType, RaftPeerId leader, List<RaftPeer> peers) {
|
|
|
|
- LOG.trace("newRaftClient: {}, leader={}, peers={}", rpcType, leader, peers);
|
|
|
|
|
|
+ RpcType rpcType, RaftPeerId leader, RaftGroup group) {
|
|
|
|
+ LOG.trace("newRaftClient: {}, leader={}, group={}", rpcType, leader, group);
|
|
final RaftProperties properties = new RaftProperties();
|
|
final RaftProperties properties = new RaftProperties();
|
|
- final ClientFactory factory = ClientFactory.cast(rpcType.newFactory(
|
|
|
|
- properties, null));
|
|
|
|
|
|
+ final ClientFactory factory = ClientFactory.cast(rpcType.newFactory(null));
|
|
|
|
|
|
return RaftClient.newBuilder()
|
|
return RaftClient.newBuilder()
|
|
.setClientRpc(factory.newRaftClientRpc())
|
|
.setClientRpc(factory.newRaftClientRpc())
|
|
- .setServers(peers)
|
|
|
|
|
|
+ .setRaftGroup(group)
|
|
.setLeaderId(leader)
|
|
.setLeaderId(leader)
|
|
.setProperties(properties)
|
|
.setProperties(properties)
|
|
.build();
|
|
.build();
|