Details
-
Improvement
-
Status: Resolved
-
Major
-
Resolution: Implemented
-
None
Description
In https://issues.apache.org/jira/browse/RATIS-1947, it was found that there might be cases where Datanodes in the same pipeline are closed hours apart.
# dn1 2023-11-29 15:22:59,477 [Command processor thread] INFO org.apache.hadoop.ozone.container.common.statemachine.commandhandler.ClosePipelineCommandHandler: Close Pipeline PipelineID=23e46782-6b48-4559-b3ac-0f95993cf0bc command on datanode 1669a7e6-fe3c-4f7e-8fcb-ec5d5027b0eb. #dn5 2023-11-29 14:07:55,442 [Command processor thread] INFO org.apache.hadoop.ozone.container.common.statemachine.commandhandler.ClosePipelineCommandHandler: Close Pipeline PipelineID=23e46782-6b48-4559-b3ac-0f95993cf0bc command on datanode bd1e72ab-cfd5-4cc1-8fbf-6ec9d9654c98. # dn8 2023-11-29 16:57:53,894 [Command processor thread] INFO org.apache.hadoop.ozone.container.common.statemachine.commandhandler.ClosePipelineCommandHandler: Close Pipeline PipelineID=23e46782-6b48-4559-b3ac-0f95993cf0bc command on datanode 4a23d1e8-d526-4a4d-8ed1-13ffbab3a5cc.
This might happen when there are a lot of commands in some of the Datanode commandQueue, causing some pipeline close commands to be handled by datanodes later than the others.
Furthermore, Ratis group remove operation is local to the Raft server and is not propagated to the other Raft peers in the same group. Therefore, datanodes that have not received the group remove operation will keep operating (e.g. sending RequestVote / AppendEntries RPCs), although the pipeline (Raft group) is supposed to be closed. The might affect the client communicating with the Raft peer in datanodes that have not been closed yet.
Therefore, similar to CreatePipelineCommandHandler, the first datanode that receives the close pipeline command needs to propagate the group remove command to the other datanodes (Raft peers) in the same pipeline. This will close the pipeline immediately on all the datanodes. The subsequent close pipeline commands will be ignored silently as the pipeline has been successfully closed.
Attachments
Issue Links
- causes
-
HDDS-10875 XceiverRatisServer#getRaftPeersInPipeline should be called before XceiverRatisServer#removeGroup
- Resolved
- relates to
-
RATIS-1947 TimeoutIOException in WriteLog might not release Pending Requests
- Resolved
- links to