Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HDDS-10384. RPC client Reusing thread resources. #6326

Merged
merged 7 commits into from
Mar 6, 2024

Conversation

xichen01
Copy link
Contributor

@xichen01 xichen01 commented Mar 4, 2024

What changes were proposed in this pull request?

The old PR #6270 has been reverted due to a bug. This PR fixed the bug and recreated the PR.

root cause

The root cause of the bug in the old PR is that the future put into the executePutBlock of CommitWatcher#futureMap may be overwritten by the last closed executePutBlock. Therefore, the buffer cannot be released correctly.

Fix

Change the CommitWatcher#futureMap from ConcurrentMap<Long, CompletableFuture<xxx>> to ConcurrentMap<Long, List<CompletableFuture<xxx>>> Records and releases all futures with the same key, so all the future can be released.

A detail log about this bug:

2024-03-03 18:49:16,741 [Thread-955] ERROR storage.BlockOutputStream (BlockOutputStream.java:executePutBlock(512)) - executePutBlock putFlushFuture flushPos 4194304, flushFuture java.util.concurrent.CompletableFuture@8bc6bca[Not completed], close false, force false
2024-03-03 18:49:16,741 [Thread-955] ERROR storage.BlockOutputStream (RatisBlockOutputStream.java:putFlushFuture(120)) - putFlushFuture flushPos 4194304 flushFuture java.util.concurrent.CompletableFuture@8bc6bca[Not completed]

In the next log, we can see the entry {4194304, CompletableFuture@8bc6bca} has been overwritten by the {4194304, CompletableFuture@4230f10c}

2024-03-03 18:49:16,741 [Thread-955] ERROR storage.BlockOutputStream (BlockOutputStream.java:executePutBlock(512)) - executePutBlock putFlushFuture flushPos 4194304, flushFuture java.util.concurrent.CompletableFuture@4230f10c[Not completed], close true, force true
2024-03-03 18:49:16,741 [Thread-955] ERROR storage.BlockOutputStream (RatisBlockOutputStream.java:putFlushFuture(120)) - putFlushFuture flushPos 4194304 flushFuture java.util.concurrent.CompletableFuture@4230f10c[Not completed]
2024-03-03 18:49:16,741 [Thread-955] ERROR storage.BlockOutputStream (RatisBlockOutputStream.java:waitOnFlushFutures(127)) - waitOnFlushFutures getFutureMap keySet [4194304] keySet Value [java.util.concurrent.CompletableFuture@4230f10c[Not completed]]

What is the link to the Apache JIRA

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

How was this patch tested?

Twice 10x10 tests for TestSecureOzoneRpcClient,TestFreonWithPipelineDestroy,TestOzoneRpcClientWithRatis#ALL all passed. (In order to make all tests pass, this test code includes a fix for the unstable test testParallelDeleteBucketAndCreateKey HDDS-10143

https://github.com/xichen01/ozone/actions/runs/8138828724/attempts/1
https://github.com/xichen01/ozone/actions/runs/8138828724

@SaketaChalamchala
Copy link
Contributor

@szetszwo could you please take a look?

@szetszwo
Copy link
Contributor

szetszwo commented Mar 4, 2024

@xichen01 , thanks for digging out the root cause!

... the future put into the executePutBlock of CommitWatcher#futureMap may be overwritten by the last closed executePutBlock. ...

Is this an existing bug? It seems the previous PR does not cause the bug.

... Change the CommitWatcher#futureMap ...

Let's don't change it. We may chain the futures as below:

+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/RatisBlockOutputStream.java
@@ -115,7 +115,9 @@ void updateCommitInfo(XceiverClientReply reply, List<ChunkBuffer> buffers) {
   @Override
   void putFlushFuture(long flushPos,
       CompletableFuture<ContainerCommandResponseProto> flushFuture) {
-    commitWatcher.getFutureMap().put(flushPos, flushFuture);
+    commitWatcher.getFutureMap().compute(flushPos,
+        (key, previous) -> previous == null? flushFuture
+            : previous.thenCombine(flushFuture, (prev, curr) -> curr));
   }

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 adoroszlai merged commit 2f05353 into apache:master Mar 6, 2024
35 checks passed
@adoroszlai
Copy link
Contributor

Thanks @xichen01 for continuing work on this, @szetszwo for the review.

jojochuang pushed a commit to jojochuang/ozone that referenced this pull request Mar 15, 2024
(cherry picked from commit 2f05353)
Change-Id: I5134395348242e595ffe8f001aacc19bda0e3d4a
xichen01 added a commit to xichen01/ozone that referenced this pull request Apr 11, 2024
xichen01 added a commit to xichen01/ozone that referenced this pull request Apr 11, 2024
xichen01 added a commit that referenced this pull request Apr 16, 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.

4 participants