|
@@ -26,9 +26,14 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
|
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
|
|
.ContainerCommandRequestProto;
|
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
|
|
+import org.apache.hadoop.hdds.protocol.proto
|
|
|
+ .StorageContainerDatanodeProtocolProtos.ClosePipelineInfo;
|
|
|
+import org.apache.hadoop.hdds.protocol.proto
|
|
|
+ .StorageContainerDatanodeProtocolProtos.PipelineAction;
|
|
|
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
|
|
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
|
import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
|
|
|
+import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
|
|
|
import org.apache.hadoop.ozone.container.common.transport.server
|
|
|
.XceiverServerSpi;
|
|
|
import org.apache.ratis.RaftConfigKeys;
|
|
@@ -43,10 +48,15 @@ import org.apache.ratis.protocol.RaftClientReply;
|
|
|
import org.apache.ratis.protocol.ClientId;
|
|
|
import org.apache.ratis.protocol.NotLeaderException;
|
|
|
import org.apache.ratis.protocol.StateMachineException;
|
|
|
+import org.apache.ratis.protocol.RaftPeerId;
|
|
|
+import org.apache.ratis.protocol.RaftGroup;
|
|
|
+import org.apache.ratis.protocol.RaftGroupId;
|
|
|
import org.apache.ratis.rpc.RpcType;
|
|
|
import org.apache.ratis.rpc.SupportedRpcType;
|
|
|
import org.apache.ratis.server.RaftServer;
|
|
|
import org.apache.ratis.server.RaftServerConfigKeys;
|
|
|
+import org.apache.ratis.shaded.proto.RaftProtos;
|
|
|
+import org.apache.ratis.shaded.proto.RaftProtos.RoleInfoProto;
|
|
|
import org.apache.ratis.shaded.proto.RaftProtos.ReplicationLevel;
|
|
|
import org.apache.ratis.util.SizeInBytes;
|
|
|
import org.apache.ratis.util.TimeDuration;
|
|
@@ -59,6 +69,7 @@ import java.net.InetSocketAddress;
|
|
|
import java.net.ServerSocket;
|
|
|
import java.net.SocketAddress;
|
|
|
import java.util.Objects;
|
|
|
+import java.util.UUID;
|
|
|
import java.util.concurrent.ArrayBlockingQueue;
|
|
|
import java.util.concurrent.CompletableFuture;
|
|
|
import java.util.concurrent.ThreadPoolExecutor;
|
|
@@ -81,24 +92,72 @@ public final class XceiverServerRatis implements XceiverServerSpi {
|
|
|
private final RaftServer server;
|
|
|
private ThreadPoolExecutor chunkExecutor;
|
|
|
private ClientId clientId = ClientId.randomId();
|
|
|
+ private final StateContext context;
|
|
|
+ private final ReplicationLevel replicationLevel;
|
|
|
+ private long nodeFailureTimeoutMs;
|
|
|
|
|
|
private XceiverServerRatis(DatanodeDetails dd, int port, String storageDir,
|
|
|
- ContainerDispatcher dispatcher, Configuration conf) throws IOException {
|
|
|
+ ContainerDispatcher dispatcher, Configuration conf, StateContext context)
|
|
|
+ throws IOException {
|
|
|
+ Objects.requireNonNull(dd, "id == null");
|
|
|
+ this.port = port;
|
|
|
+ RaftProperties serverProperties = newRaftProperties(conf, storageDir);
|
|
|
+ final int numWriteChunkThreads = conf.getInt(
|
|
|
+ OzoneConfigKeys.DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_KEY,
|
|
|
+ OzoneConfigKeys.DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_DEFAULT);
|
|
|
+ chunkExecutor =
|
|
|
+ new ThreadPoolExecutor(numWriteChunkThreads, numWriteChunkThreads,
|
|
|
+ 100, TimeUnit.SECONDS,
|
|
|
+ new ArrayBlockingQueue<>(1024),
|
|
|
+ new ThreadPoolExecutor.CallerRunsPolicy());
|
|
|
+ this.context = context;
|
|
|
+ this.replicationLevel =
|
|
|
+ conf.getEnum(OzoneConfigKeys.DFS_CONTAINER_RATIS_REPLICATION_LEVEL_KEY,
|
|
|
+ OzoneConfigKeys.DFS_CONTAINER_RATIS_REPLICATION_LEVEL_DEFAULT);
|
|
|
+ ContainerStateMachine stateMachine =
|
|
|
+ new ContainerStateMachine(dispatcher, chunkExecutor, this);
|
|
|
+ this.server = RaftServer.newBuilder()
|
|
|
+ .setServerId(RatisHelper.toRaftPeerId(dd))
|
|
|
+ .setGroup(RatisHelper.emptyRaftGroup())
|
|
|
+ .setProperties(serverProperties)
|
|
|
+ .setStateMachine(stateMachine)
|
|
|
+ .build();
|
|
|
+ }
|
|
|
+
|
|
|
|
|
|
+ private RaftProperties newRaftProperties(Configuration conf,
|
|
|
+ String storageDir) {
|
|
|
+ final RaftProperties properties = new RaftProperties();
|
|
|
+
|
|
|
+ // Set rpc type
|
|
|
final String rpcType = conf.get(
|
|
|
OzoneConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY,
|
|
|
OzoneConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT);
|
|
|
final RpcType rpc = SupportedRpcType.valueOfIgnoreCase(rpcType);
|
|
|
+ RaftConfigKeys.Rpc.setType(properties, rpc);
|
|
|
+
|
|
|
+ // set raft segment size
|
|
|
final int raftSegmentSize = conf.getInt(
|
|
|
OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_SIZE_KEY,
|
|
|
OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_SIZE_DEFAULT);
|
|
|
+ RaftServerConfigKeys.Log.setSegmentSizeMax(properties,
|
|
|
+ SizeInBytes.valueOf(raftSegmentSize));
|
|
|
+
|
|
|
+ // set raft segment pre-allocated size
|
|
|
final int raftSegmentPreallocatedSize = conf.getInt(
|
|
|
OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY,
|
|
|
OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT);
|
|
|
+ RaftServerConfigKeys.Log.Appender.setBufferCapacity(properties,
|
|
|
+ SizeInBytes.valueOf(raftSegmentPreallocatedSize));
|
|
|
+ RaftServerConfigKeys.Log.setPreallocatedSize(properties,
|
|
|
+ SizeInBytes.valueOf(raftSegmentPreallocatedSize));
|
|
|
+
|
|
|
+ // Set max write buffer size, which is the scm chunk size
|
|
|
final int maxChunkSize = OzoneConfigKeys.DFS_CONTAINER_CHUNK_MAX_SIZE;
|
|
|
- final int numWriteChunkThreads = conf.getInt(
|
|
|
- OzoneConfigKeys.DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_KEY,
|
|
|
- OzoneConfigKeys.DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_DEFAULT);
|
|
|
+ RaftServerConfigKeys.Log.setWriteBufferSize(properties,
|
|
|
+ SizeInBytes.valueOf(maxChunkSize));
|
|
|
+
|
|
|
+ // Set the client requestTimeout
|
|
|
TimeUnit timeUnit =
|
|
|
OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_DEFAULT
|
|
|
.getUnit();
|
|
@@ -108,6 +167,10 @@ public final class XceiverServerRatis implements XceiverServerSpi {
|
|
|
.getDuration(), timeUnit);
|
|
|
final TimeDuration clientRequestTimeout =
|
|
|
TimeDuration.valueOf(duration, timeUnit);
|
|
|
+ RaftClientConfigKeys.Rpc
|
|
|
+ .setRequestTimeout(properties, clientRequestTimeout);
|
|
|
+
|
|
|
+ // Set the server Request timeout
|
|
|
timeUnit = OzoneConfigKeys.DFS_RATIS_SERVER_REQUEST_TIMEOUT_DURATION_DEFAULT
|
|
|
.getUnit();
|
|
|
duration = conf.getTimeDuration(
|
|
@@ -116,61 +179,44 @@ public final class XceiverServerRatis implements XceiverServerSpi {
|
|
|
.getDuration(), timeUnit);
|
|
|
final TimeDuration serverRequestTimeout =
|
|
|
TimeDuration.valueOf(duration, timeUnit);
|
|
|
-
|
|
|
- Objects.requireNonNull(dd, "id == null");
|
|
|
- this.port = port;
|
|
|
- RaftProperties serverProperties =
|
|
|
- newRaftProperties(rpc, port, storageDir, maxChunkSize, raftSegmentSize,
|
|
|
- raftSegmentPreallocatedSize);
|
|
|
- setRequestTimeout(serverProperties, clientRequestTimeout,
|
|
|
- serverRequestTimeout);
|
|
|
-
|
|
|
- chunkExecutor =
|
|
|
- new ThreadPoolExecutor(numWriteChunkThreads, numWriteChunkThreads,
|
|
|
- 100, TimeUnit.SECONDS,
|
|
|
- new ArrayBlockingQueue<>(1024),
|
|
|
- new ThreadPoolExecutor.CallerRunsPolicy());
|
|
|
- ContainerStateMachine stateMachine =
|
|
|
- new ContainerStateMachine(dispatcher, chunkExecutor);
|
|
|
- this.server = RaftServer.newBuilder()
|
|
|
- .setServerId(RatisHelper.toRaftPeerId(dd))
|
|
|
- .setGroup(RatisHelper.emptyRaftGroup())
|
|
|
- .setProperties(serverProperties)
|
|
|
- .setStateMachine(stateMachine)
|
|
|
- .build();
|
|
|
- }
|
|
|
-
|
|
|
- private static void setRequestTimeout(RaftProperties serverProperties,
|
|
|
- TimeDuration clientRequestTimeout, TimeDuration serverRequestTimeout) {
|
|
|
- RaftClientConfigKeys.Rpc
|
|
|
- .setRequestTimeout(serverProperties, clientRequestTimeout);
|
|
|
RaftServerConfigKeys.Rpc
|
|
|
- .setRequestTimeout(serverProperties, serverRequestTimeout);
|
|
|
- }
|
|
|
+ .setRequestTimeout(properties, serverRequestTimeout);
|
|
|
|
|
|
- private static RaftProperties newRaftProperties(
|
|
|
- RpcType rpc, int port, String storageDir, int scmChunkSize,
|
|
|
- int raftSegmentSize, int raftSegmentPreallocatedSize) {
|
|
|
- final RaftProperties properties = new RaftProperties();
|
|
|
+ // Enable batch append on raft server
|
|
|
RaftServerConfigKeys.Log.Appender.setBatchEnabled(properties, true);
|
|
|
- RaftServerConfigKeys.Log.Appender.setBufferCapacity(properties,
|
|
|
- SizeInBytes.valueOf(raftSegmentPreallocatedSize));
|
|
|
- RaftServerConfigKeys.Log.setWriteBufferSize(properties,
|
|
|
- SizeInBytes.valueOf(scmChunkSize));
|
|
|
- RaftServerConfigKeys.Log.setPreallocatedSize(properties,
|
|
|
- SizeInBytes.valueOf(raftSegmentPreallocatedSize));
|
|
|
- RaftServerConfigKeys.Log.setSegmentSizeMax(properties,
|
|
|
- SizeInBytes.valueOf(raftSegmentSize));
|
|
|
- RaftServerConfigKeys.setStorageDir(properties, new File(storageDir));
|
|
|
- RaftConfigKeys.Rpc.setType(properties, rpc);
|
|
|
|
|
|
+ // Set the maximum cache segments
|
|
|
RaftServerConfigKeys.Log.setMaxCachedSegmentNum(properties, 2);
|
|
|
- GrpcConfigKeys.setMessageSizeMax(properties,
|
|
|
- SizeInBytes.valueOf(scmChunkSize + raftSegmentPreallocatedSize));
|
|
|
+
|
|
|
+ // Set the ratis leader election timeout
|
|
|
RaftServerConfigKeys.Rpc.setTimeoutMin(properties,
|
|
|
TimeDuration.valueOf(800, TimeUnit.MILLISECONDS));
|
|
|
RaftServerConfigKeys.Rpc.setTimeoutMax(properties,
|
|
|
TimeDuration.valueOf(1000, TimeUnit.MILLISECONDS));
|
|
|
+
|
|
|
+ // set the node failure timeout
|
|
|
+ timeUnit = OzoneConfigKeys.DFS_RATIS_SERVER_FAILURE_DURATION_DEFAULT
|
|
|
+ .getUnit();
|
|
|
+ duration = conf.getTimeDuration(
|
|
|
+ OzoneConfigKeys.DFS_RATIS_SERVER_FAILURE_DURATION_KEY,
|
|
|
+ OzoneConfigKeys.DFS_RATIS_SERVER_FAILURE_DURATION_DEFAULT
|
|
|
+ .getDuration(), timeUnit);
|
|
|
+ final TimeDuration nodeFailureTimeout =
|
|
|
+ TimeDuration.valueOf(duration, timeUnit);
|
|
|
+ RaftServerConfigKeys.setLeaderElectionTimeout(properties,
|
|
|
+ nodeFailureTimeout);
|
|
|
+ RaftServerConfigKeys.Rpc.setSlownessTimeout(properties,
|
|
|
+ nodeFailureTimeout);
|
|
|
+ nodeFailureTimeoutMs = nodeFailureTimeout.toLong(TimeUnit.MILLISECONDS);
|
|
|
+
|
|
|
+ // Set the ratis storage directory
|
|
|
+ RaftServerConfigKeys.setStorageDir(properties, new File(storageDir));
|
|
|
+
|
|
|
+ // For grpc set the maximum message size
|
|
|
+ GrpcConfigKeys.setMessageSizeMax(properties,
|
|
|
+ SizeInBytes.valueOf(maxChunkSize + raftSegmentPreallocatedSize));
|
|
|
+
|
|
|
+ // Set the ratis port number
|
|
|
if (rpc == SupportedRpcType.GRPC) {
|
|
|
GrpcConfigKeys.Server.setPort(properties, port);
|
|
|
} else if (rpc == SupportedRpcType.NETTY) {
|
|
@@ -181,7 +227,7 @@ public final class XceiverServerRatis implements XceiverServerSpi {
|
|
|
|
|
|
public static XceiverServerRatis newXceiverServerRatis(
|
|
|
DatanodeDetails datanodeDetails, Configuration ozoneConf,
|
|
|
- ContainerDispatcher dispatcher) throws IOException {
|
|
|
+ ContainerDispatcher dispatcher, StateContext context) throws IOException {
|
|
|
final String ratisDir = File.separator + "ratis";
|
|
|
int localPort = ozoneConf.getInt(
|
|
|
OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_PORT,
|
|
@@ -226,7 +272,7 @@ public final class XceiverServerRatis implements XceiverServerSpi {
|
|
|
datanodeDetails.setPort(
|
|
|
DatanodeDetails.newPort(DatanodeDetails.Port.Name.RATIS, localPort));
|
|
|
return new XceiverServerRatis(datanodeDetails, localPort, storageDir,
|
|
|
- dispatcher, ozoneConf);
|
|
|
+ dispatcher, ozoneConf, context);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -296,7 +342,7 @@ public final class XceiverServerRatis implements XceiverServerSpi {
|
|
|
// the request here are applied on all the raft servers.
|
|
|
RaftClientRequest raftClientRequest =
|
|
|
createRaftClientRequest(request, pipelineID,
|
|
|
- RaftClientRequest.writeRequestType(ReplicationLevel.ALL));
|
|
|
+ RaftClientRequest.writeRequestType(replicationLevel));
|
|
|
CompletableFuture<RaftClientReply> reply =
|
|
|
server.submitClientRequestAsync(raftClientRequest);
|
|
|
reply.thenAccept(this::processReply);
|
|
@@ -309,4 +355,57 @@ public final class XceiverServerRatis implements XceiverServerSpi {
|
|
|
PipelineID.getFromProtobuf(pipelineID).getRaftGroupID(),
|
|
|
nextCallId(), 0, Message.valueOf(request.toByteString()), type);
|
|
|
}
|
|
|
+
|
|
|
+ private void handlePipelineFailure(RaftGroupId groupId,
|
|
|
+ RoleInfoProto roleInfoProto) {
|
|
|
+ String msg;
|
|
|
+ UUID datanode = RatisHelper.toDatanodeId(roleInfoProto.getSelf());
|
|
|
+ RaftPeerId id = RaftPeerId.valueOf(roleInfoProto.getSelf().getId());
|
|
|
+ switch (roleInfoProto.getRole()) {
|
|
|
+ case CANDIDATE:
|
|
|
+ msg = datanode + " is in candidate state for " +
|
|
|
+ roleInfoProto.getCandidateInfo().getLastLeaderElapsedTimeMs() + "ms";
|
|
|
+ break;
|
|
|
+ case LEADER:
|
|
|
+ StringBuilder sb = new StringBuilder();
|
|
|
+ sb.append(datanode).append(" has not seen follower/s");
|
|
|
+ for (RaftProtos.ServerRpcProto follower : roleInfoProto.getLeaderInfo()
|
|
|
+ .getFollowerInfoList()) {
|
|
|
+ if (follower.getLastRpcElapsedTimeMs() > nodeFailureTimeoutMs) {
|
|
|
+ sb.append(" ").append(RatisHelper.toDatanodeId(follower.getId()))
|
|
|
+ .append(" for ").append(follower.getLastRpcElapsedTimeMs())
|
|
|
+ .append("ms");
|
|
|
+ }
|
|
|
+ }
|
|
|
+ msg = sb.toString();
|
|
|
+ break;
|
|
|
+ default:
|
|
|
+ LOG.error("unknown state:" + roleInfoProto.getRole());
|
|
|
+ throw new IllegalStateException("node" + id + " is in illegal role "
|
|
|
+ + roleInfoProto.getRole());
|
|
|
+ }
|
|
|
+
|
|
|
+ PipelineID pipelineID = PipelineID.valueOf(groupId);
|
|
|
+ ClosePipelineInfo.Builder closePipelineInfo =
|
|
|
+ ClosePipelineInfo.newBuilder()
|
|
|
+ .setPipelineID(pipelineID.getProtobuf())
|
|
|
+ .setReason(ClosePipelineInfo.Reason.PIPELINE_FAILED)
|
|
|
+ .setDetailedReason(msg);
|
|
|
+
|
|
|
+ PipelineAction action = PipelineAction.newBuilder()
|
|
|
+ .setClosePipeline(closePipelineInfo)
|
|
|
+ .setAction(PipelineAction.Action.CLOSE)
|
|
|
+ .build();
|
|
|
+ context.addPipelineActionIfAbsent(action);
|
|
|
+ }
|
|
|
+
|
|
|
+ void handleNodeSlowness(
|
|
|
+ RaftGroup group, RoleInfoProto roleInfoProto) {
|
|
|
+ handlePipelineFailure(group.getGroupId(), roleInfoProto);
|
|
|
+ }
|
|
|
+
|
|
|
+ void handleNoLeader(
|
|
|
+ RaftGroup group, RoleInfoProto roleInfoProto) {
|
|
|
+ handlePipelineFailure(group.getGroupId(), roleInfoProto);
|
|
|
+ }
|
|
|
}
|