IGNITE-20852 Opposite connection attempts may cause connection failure#2850
IGNITE-20852 Opposite connection attempts may cause connection failure#2850rpuch wants to merge 16 commits intoapache:mainfrom
Conversation
…s a ChannelAlreadyExistsException
…s either directly or after a retry; never throw ChannelAlreadyExistsException to the users of NettyClient
…DefaultMessagingService
| * | ||
| * @return Final future that represents the handshake operation. | ||
| */ | ||
| CompletionStage<NettySender> finalHandshakeFuture(); |
There was a problem hiding this comment.
How about different name - globalHandshakeFuture?
There was a problem hiding this comment.
'final' says that this is the result we are interested in. 'ultimate' seems to be ok as well, but it seems too loud.
'global' would be about a different property: not about the 'final result we want to obtain', but that it's common for everyone, and this is not true (another side would have its own future).
| boolean ignorable = stopping.get() || !msg.reason().critical(); | ||
|
|
||
| if (ignorable) { | ||
| LOG.debug("Handshake rejected by server: {}", msg.message()); |
There was a problem hiding this comment.
Maybe check for debug enabled?
There was a problem hiding this comment.
Inside, there is a check, so if debug level is not enabled, nothing will be logged. If we add the check here, we'll save one method call to message() (negligible) and one allocation to create a vararg array. This saving seems to be not important as this code is not hot, we don't handle a million handshakes per second. But we'll have to pay with one line for this. I'm not sure it's worth it.
...rc/main/java/org/apache/ignite/internal/network/recovery/RecoveryClientHandshakeManager.java
Outdated
Show resolved
Hide resolved
| * Master future used to complete the handshake either with the results of this handshake of the competing one | ||
| * (in the opposite direction), if it wins. | ||
| */ | ||
| private final CompletableFuture<CompletionStage<NettySender>> masterHandshakeCompleteFuture = new CompletableFuture<>(); |
There was a problem hiding this comment.
And here it could be final, terminal or resulting - WDYT?
There was a problem hiding this comment.
Changed it to resulting
...rc/main/java/org/apache/ignite/internal/network/recovery/RecoveryClientHandshakeManager.java
Outdated
Show resolved
Hide resolved
...rc/main/java/org/apache/ignite/internal/network/recovery/RecoveryClientHandshakeManager.java
Outdated
Show resolved
Hide resolved
| ); | ||
|
|
||
| DescriptorAcquiry myAcquiry = descriptor.holder(); | ||
| assert myAcquiry != null; |
There was a problem hiding this comment.
I maybe a bit paranoid here but I really would prefer to have some sort of IDs on recovery descriptors. I cannot imaging a scenario when we get a HandshakeRejectedMessage out of thin air but if we do we'll fail on these asserts immediately.
Or these messages could be constructed maliciously e.g. to fail a node so this code could be a security vulnerability.
What do you think about these ideas? However this is not a blocker for this improvement right now.
There was a problem hiding this comment.
We cannot do anything here 'easily', and the protocol seems to be designed around trust to the other side. If this has to be changed, we'll redesign the protocol, but I think this should be solved by other means (firewall and TLS auth)
| if (oldAcquiry != null && oldAcquiry.channel() == ctx.channel()) { | ||
| // We have successfully released the descriptor. | ||
| // Let's mark the clinch resolved just in case. | ||
| oldAcquiry.markClinchResolved(); |
There was a problem hiding this comment.
Can we check if the acquiry is in clinch state? AFAIK it should be a wrong state for acquiry here, so this fact deserves to be logged for further investigation.
There was a problem hiding this comment.
An acquiry is a local thing, and a clinch is a distributed state, so we cannot see whether there is (was) a clinch or not.
Here, it's just a cleanup procedure to make sure that we always release the clinch (if it existed).
| * Returns {@code true} iff the rejection is not expected and should be treated as a critical failure (requiring | ||
| * the rejected node to restart). | ||
| */ | ||
| public boolean critical() { |
There was a problem hiding this comment.
How about renaming critical to something like hazardous to make it clearer that we'd better send this node into the FailureHandler mouth?
There was a problem hiding this comment.
Renamed it to restartRequired()
…ay cause connection failure (#2850) * Handshake protocol is extended to allow a node losing a clinch notify its origin * As a result of a handshake, the caller always gets a NettySender, even if the caller lost the clinch * If, during a handshake, a party cannot obtain a lock at its side, it gives the competitor way unconditionally (as the competitor has advanced further) Signed-off-by: Sergey Chugunov <sergey.chugunov@gmail.com>
https://issues.apache.org/jira/browse/IGNITE-20852
Thank you for submitting the pull request.
To streamline the review process of the patch and ensure better code quality
we ask both an author and a reviewer to verify the following:
The Review Checklist
- There is a single JIRA ticket related to the pull request.
- The web-link to the pull request is attached to the JIRA ticket.
- The JIRA ticket has the Patch Available state.
- The description of the JIRA ticket explains WHAT was made, WHY and HOW.
- The pull request title is treated as the final commit message. The following pattern must be used: IGNITE-XXXX Change summary where XXXX - number of JIRA issue.
Notes