Skip to content

ZOOKEEPER-4293: Lock Contention in ClientCnxnSocketNetty#1917

Merged
anmolnar merged 1 commit intoapache:masterfrom
MikeEdgar:ZOOKEEPER-4293
Sep 19, 2024
Merged

ZOOKEEPER-4293: Lock Contention in ClientCnxnSocketNetty#1917
anmolnar merged 1 commit intoapache:masterfrom
MikeEdgar:ZOOKEEPER-4293

Conversation

@MikeEdgar
Copy link
Contributor

@MikeEdgar MikeEdgar commented Aug 18, 2022

Check for failed/cancelled ChannelFutures before acquiring connectLock. This prevents lock contention where cleanup is unable to complete.

Replaces #1713 (target master rather than branch-3.5)

@sonatype-lift
Copy link

sonatype-lift bot commented Oct 21, 2022

⚠️ 52 God Classes were detected by Lift in this project. Visit the Lift web console for more details.

@MikeEdgar MikeEdgar force-pushed the ZOOKEEPER-4293 branch 2 times, most recently from b4797a9 to 052cce2 Compare October 21, 2022 17:10
@MikeEdgar
Copy link
Contributor Author

@eolivelli , @maoling - please take a look when you have a chance. This replaces #1713 that you looked at last year.

Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

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

good catch

nice work

LGTM

@anmolnar @phunt @symat PTAL

@eolivelli
Copy link
Contributor

it looks some test related to this patch failed on CI

image

can you please take a look @MikeEdgar ?

@MikeEdgar
Copy link
Contributor Author

@eolivelli , I've made a change that should address the NPE in the new test.

Question.. I see errors locally around org.apache.zookeeper.server.FollowerRequestProcessorTest having a bad package declaration. Is this intended?

@MikeEdgar
Copy link
Contributor Author

@eolivelli , please take another look.

@MikeEdgar
Copy link
Contributor Author

@eolivelli @anmolnar @phunt @symat PTAL. The CI has not yet run with the test fix.

Please also note my earlier question regarding FollowerRequestProcessorTest. The package declaration does not match the source directory. Is this intended?

Copy link
Member

@kezhuw kezhuw left a comment

Choose a reason for hiding this comment

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

I submit a reproducible test in kezhuw@0f2610b. From the test case, I think we should fix two sub issues:

  • Deadlock due to synchronous close and only one event loop thread.
  • Prevent completion of old connection from interfering current connection.

I have left comments for these two.

I hope this test case could help in reviewing.

afterConnectFutureCancel();
}
if (channel != null) {
channel.close().syncUninterruptibly();
Copy link
Member

Choose a reason for hiding this comment

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

I think it is a good to move this blocking syncUninterruptibly part out of lock scope. This could also solve the deadlock issue.

connectLock.lock();
try {
if (connectFuture != null) {
connectFuture.cancel(false);
Copy link
Member

@kezhuw kezhuw May 19, 2023

Choose a reason for hiding this comment

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

After taking a look at netty code AbstractNioChannel::connect, AbstractEpollChannel::connect and AbstractKQueueChannel::connect, I think this statement connectFuture.cancel(false) cloud be the root cause.

Basically, a connecting future is uncancellable which means connectFuture.cancel(false) could be a nop. So, the old connectFuture could still pending for completion. When we cleanup new connectFuture and old connectFuture completes, they are stuck due to leak of event threads to complete channel.close().syncUninterruptibly()(invoke close in event thread).

I think we could also wait connectFuture to complete in cleanup, but I don't think it is a good. Connection timeout is a possible reason for us to fall here. It is nonsense to block next connection for the same reason.

I have watched this issue for a while, and could not image a situation where there is single connectFuture and this stuck emerged. The channel.close().syncUninterruptibly() means its connectFuture already completed successfully.

eventLoopGroup = NettyUtils.newNioOrEpollEventLoopGroup(1 /* nThreads */);

}
}

boolean cancelled(ChannelFuture channelFuture) {
Copy link
Member

Choose a reason for hiding this comment

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

I think we could concentrate more on identity of channelFuture and connectFuture. This will make it clear that channelFuture could come from a old connection. Better to document a bit.

It is also possible that new connection is not completed yet. In this case, old connection should be simply filtered out, otherwise we could introduce inconsistence with outside world.

@MikeEdgar
Copy link
Contributor Author

@eolivelli , what are you thoughts on @kezhuw's comments? Does this PR need additional updates or can it be accepted as-is?

Copy link
Member

@kezhuw kezhuw left a comment

Choose a reason for hiding this comment

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

I think the fix clould be as simple as droping syncUninterruptibly from channel.close().syncUninterruptibly(). This way we break the deadlock chain.

netty/netty#13849 probably "fixed" this. But I think it is still vital for us to realize that completed channelFuture may belong to old dangling connection. So I am tend to fix this in our side (also).

if (!channelFuture.isSuccess()) {
LOG.warn("future isn't success.", channelFuture.cause());
return;
} else if (connectFuture == null) {
Copy link
Member

Choose a reason for hiding this comment

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

How about simple channelFuture != connectFuture in earlier stage ? This should cover connectFuture == null. In case of channelFuture != connectFuture we should do nothing but simple return.

afterConnectFutureCancel();
}
if (channel != null) {
channel.close().syncUninterruptibly();
Copy link
Member

Choose a reason for hiding this comment

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

How about drop .syncUninterruptibly() part ? This way we will not wait for channel close completion callback which could deadlock with old connectFuture's completion callback.

@kezhuw
Copy link
Member

kezhuw commented Aug 26, 2024

netty/netty#13849 probably "fixed" this.

I confirmed this with kezhuw@0f2610b.

After netty/netty#13849, connectFuture.cancel(false) will fire cancellation completion callback before next connection in netty event group. So there will be no ongoing connecting future in next iteration. But I am not sure whether there are other paths/conditions leading to uncancellable connecting future.

Though, I am still prefer to fix this in ZooKeeper side.

@kezhuw kezhuw requested review from eolivelli and kezhuw August 26, 2024 14:18
@MikeEdgar
Copy link
Contributor Author

@kezhuw , @eolivelli , I'm not in a position at the moment to devote much time to reworking and testing this PR, especially in an integrated environment where the issue originally occurred for us. I wonder if it needs the adjustments proposed by @kezhuw, whether it should be a separate PR and just close this one.

Please let me know your thoughts.

@anmolnar
Copy link
Contributor

anmolnar commented Sep 11, 2024

I think the fix clould be as simple as droping syncUninterruptibly from channel.close().syncUninterruptibly(). This way we break the deadlock chain.

@MikeEdgar @kezhuw

Calling a sync method in an async operation inside a lock is a code smell I believe, so I tend to prefer this solution over what the patch suggests. Is this sync call was the root cause of the original lock contention? I mean did you see that the thread is waiting for this call in particular and cannot release the lock?

@kezhuw
Copy link
Member

kezhuw commented Sep 12, 2024

The jira ZOOKEEPER-4293 has the deadlock stack dump. Below is what kezhuw@0f2610b reproduced:

  1. Send thread which is holding the lock blocks on channel closing future.
  2. Cancellation of connectFuture from previous run blocks on acquiring lock. This occupies the sole event loop thread.
  3. Channel closing is waits for event loop thread.
graph TD;
    SendThread -- blocks on --- ChannelClosing[channel closing];
    ChannelClosing[channel closing] -- waits for event loop --- ConnectFutureCancellation[old connection cancellation];
    ConnectFutureCancellation[old connection cancellation] -- blocks on lock --- SendThread;
Loading

I think the bug is multi-folds:

  1. Netty event group uses only one thread.
  2. Channel is closed synchronous under lock.
  3. Connecting future is uncancellable. So connectFuture.cancel(false) is a nop for connecting future.

Theoretically, we can fix the deadlock by breaking any of the above. netty/netty#13849 breaks the third. I do think bump netty version cloud fix this deadlock. But I also think it might be fragile to depend on bump dependency version to fix bug as netty version could be pinned/upgraded independently. So, personally I prefer to fix it in our side.

cc @anmolnar

@anmolnar
Copy link
Contributor

Thanks for the clarification @kezhuw , I appreciate your nice diagram.
I agree with you that we should fix this on our side too, so I suggest the two above mentioned things to do:

  • Bump Netty version
  • Remove .syncUninterruptibly()

Thoughts?

@kezhuw
Copy link
Member

kezhuw commented Sep 12, 2024

@anmolnar Good to go.

@anmolnar
Copy link
Contributor

@MikeEdgar What do you think?
Would you mind modifying your patch accordingly?

@MikeEdgar
Copy link
Contributor Author

@anmolnar, @kezhuw the third commit in this PR includes the latest modifications. I think I understood what you were looking for, but please confirm. Note, bumping the Netty version to the latest 4.1 release also required a minor change to account for a deprecated method.

Copy link
Contributor

@anmolnar anmolnar left a comment

Choose a reason for hiding this comment

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

@MikeEdgar

Sorry I meant we didn't need this modification at all. Instead just do the 2 things that was mentioned in the previous comment.

Signed-off-by: Michael Edgar <medgar@redhat.com>
@MikeEdgar
Copy link
Contributor Author

Sorry I meant we didn't need this modification at all. Instead just do the 2 things that was mentioned in the previous comment.

@anmolnar - done

@MikeEdgar
Copy link
Contributor Author

@anmolnar @kezhuw let me know if this looks as you're expecting.

Copy link
Contributor

@anmolnar anmolnar left a comment

Choose a reason for hiding this comment

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

lgtm.

@eolivelli @kezhuw PTAL.

Copy link
Member

@kezhuw kezhuw left a comment

Choose a reason for hiding this comment

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

LGTM

}

if (bytesRead == litmus.length && SslHandler.isEncrypted(Unpooled.wrappedBuffer(litmus))) {
if (bytesRead == litmus.length && SslHandler.isEncrypted(Unpooled.wrappedBuffer(litmus), false)) {
Copy link
Member

Choose a reason for hiding this comment

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

This variant is brand new in netty-4.1.113.Final, so this patch will enforce netty version. Given my comments #1917 (comment), I am +1 to this. So, client will have to bump its netty version to match this patch in compilation.

netty commit: netty/netty@dc30c33#diff-963dd9b8e77e913395273812889e9b6f2f449fe55e2b748b95311b219d1c58faR1286

Copy link
Contributor

Choose a reason for hiding this comment

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

Isn't that a problem if we backport this to 3.9? (I want to)
Can we just use another variant which is backward compatible?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The only other variant is the deprecated one, which fails to compile due to the -Werror flag. This file change could possibly be left out of a backport which uses an older netty version.

Copy link
Contributor

Choose a reason for hiding this comment

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

No worries, we'll upgrade on 3.9 too.

@anmolnar
Copy link
Contributor

cppunit tests are stuck here too

@anmolnar anmolnar merged commit 9df310a into apache:master Sep 19, 2024
anmolnar pushed a commit that referenced this pull request Sep 19, 2024
…ly (#1917)

Reviewers: anmolnar, kezhuw
Author: MikeEdgar
Closes #1917 from MikeEdgar/ZOOKEEPER-4293
@anmolnar
Copy link
Contributor

Merged. Thanks @MikeEdgar !
I also added you to contributors list in Jira.

@MikeEdgar MikeEdgar deleted the ZOOKEEPER-4293 branch September 19, 2024 19:42
prabhjyotsingh pushed a commit to acceldata-io/zookeeper that referenced this pull request Feb 28, 2025
…nterruptibly (apache#1917)

Reviewers: anmolnar, kezhuw
Author: MikeEdgar
Closes apache#1917 from MikeEdgar/ZOOKEEPER-4293

(cherry picked from commit 9df310a)
shubhluck pushed a commit to acceldata-io/zookeeper that referenced this pull request May 26, 2025
…nterruptibly (apache#1917)

Reviewers: anmolnar, kezhuw
Author: MikeEdgar
Closes apache#1917 from MikeEdgar/ZOOKEEPER-4293

(cherry picked from commit 9df310a)
anmolnar pushed a commit to anmolnar/zookeeper that referenced this pull request Jul 30, 2025
…ly (apache#1917)

Reviewers: anmolnar, kezhuw
Author: MikeEdgar
Closes apache#1917 from MikeEdgar/ZOOKEEPER-4293
anmolnar added a commit that referenced this pull request Jul 30, 2025
…adlock) (branch-3.8 backport)

ZOOKEEPER-4293: Bump netty to 4.1.113.Final, remove syncUninterruptibly (#1917)
Reviewers: anmolnar, kezhuw
Author: MikeEdgar
Closes #1917 from MikeEdgar/ZOOKEEPER-4293
Reviewers: kezhuw
Author: anmolnar
Closes #2284 from anmolnar/ZOOKEEPER-4293_38
basapuram-kumar pushed a commit to acceldata-io/zookeeper that referenced this pull request Sep 23, 2025
…ly (apache#1917)

Reviewers: anmolnar, kezhuw
Author: MikeEdgar
Closes apache#1917 from MikeEdgar/ZOOKEEPER-4293
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