Skip to content

Conversation

@ivandika3
Copy link
Contributor

@ivandika3 ivandika3 commented Dec 19, 2023

What changes were proposed in this pull request?

In https://issues.apache.org/jira/browse/RATIS-1947, it was found that there might be cases where RaftServer for 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.

Note: There seems to be an unused methods in RatisPipelineUtils related to pipeline destroy (RatisPipelineUtils#destroyPipeline) which were re-introduced in #538 although they were not really used. However, it seems the intent of the methods are for the SCM to destroy the pipeline directly, without going through the command queue. We can delete in this patch if needed.

What is the link to the Apache JIRA

https://issues.apache.org/jira/browse/HDDS-9959

How was this patch tested?

Unit test for ClosePipelineCommandHandler.

CI run: https://github.com/ivandika3/ozone/actions/runs/7258608368/job/19774884832

@ivandika3 ivandika3 marked this pull request as ready for review December 19, 2023 14:27
@ivandika3
Copy link
Contributor Author

@sumitagrawl @DaveTeng0 @szetszwo Could you help to review if you have time?

Copy link
Contributor

@szetszwo szetszwo left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@ivandika3 , thanks for working on this! Please see the comments inlined.

... There seems to be an unused methods in RatisPipelineUtils related to pipeline destroy (RatisPipelineUtils#destroyPipeline) ...

Sure, let's remove the unused methods.

final Collection<RaftPeer> peers = ratisServer.getRaftPeersInPipeline(pipelineID);
final boolean shouldDeleteRatisLogDirectory = ratisServer.getShouldDeleteRatisLogDirectory();
peers.stream()
.filter(peer -> peer.getId() != ratisServer.getServer().getId())
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should use equals(..) instead of !=.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updated, thank you for catching it.

// ignore silently since this means that the group has been closed by earlier close pipeline
// command in another datanode
} catch (IOException ioe) {
LOG.warn("Remove group failed for peer {}", peer.getId(), ioe);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's include the group id.

                    LOG.warn("Failed to remove group {} for peer {}", raftGroupId, peer.getId(), ioe);

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updated. PTAL.

Copy link
Contributor

@szetszwo szetszwo left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 the change looks good.

@adoroszlai
Copy link
Contributor

Test failure in TestBlockOutputStreamWithFailures.test2DatanodesFailure seems related, and it has not been flaky before.

https://github.com/apache/ozone/actions/runs/7283278738/job/19847237481?pr=5827#step:6:132

Copy link
Contributor

@adoroszlai adoroszlai left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @ivandika3 for the patch.

/**
* Test cases to verify ClosePipelineCommandHandler.
*/
@ExtendWith(MockitoExtension.class)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

MockitoExtension is unnecessary, there are no variables annotated as @Mock.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you for the review. Updated.

We might also need to remove this for TestCreatePipelineCommandHandler.

@BeforeEach
public void setup() throws Exception {
conf = new OzoneConfiguration();
ozoneContainer = Mockito.mock(OzoneContainer.class);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: Please add static import for mock(), when(), any(), etc.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you for the review. Updated.

@adoroszlai adoroszlai merged commit ac68072 into apache:master Jan 4, 2024
@adoroszlai
Copy link
Contributor

Thanks @ivandika3 for the patch, @szetszwo for the review.

@ivandika3 ivandika3 deleted the HDDS-9959 branch January 25, 2024 04:24
@ivandika3 ivandika3 self-assigned this Apr 23, 2024
@szetszwo
Copy link
Contributor

@ivandika3 , TestECContainerRecovery is failing with GroupMismatchException. The failure seems to be related to this change. See @duongkame 's comment.

@ivandika3
Copy link
Contributor Author

@szetszwo Thanks for the info. Let me take a look.

xichen01 pushed a commit to xichen01/ozone that referenced this pull request Jul 17, 2024
xichen01 pushed a commit to xichen01/ozone that referenced this pull request Jul 17, 2024
xichen01 pushed a commit to xichen01/ozone that referenced this pull request Jul 17, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants