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

[fix] [client] Messages lost when consumer reconnect #20695

Conversation

poorbarcode
Copy link
Contributor

@poorbarcode poorbarcode commented Jun 30, 2023

Reopen #20591

Motivation

Background of consumer reconnects

  • grab connection:
    • lookup the broker which owned the topic
    • get the existing connection by the broker; create one if it does not exist
  • clear the messages in memory
  • send CMD-subscribe to the broker
  • send flow permits to broker to increment availablePermits

Background of scenarios that could trigger reconnection:

Background of the response of broker received subscribe request
The broker only response success if it receives a second subscribe request of the same consumer

https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java#L1197


I wanted to prevent this by adding a validation below to prevent the messages which in memory is cleared if the method grabCnx execute after the subscribe is finished, which result in messages being lost.

class consumerImpl {

  public void connectionOpened(){
    // the new check.
    if (this.state == Ready){
      return.
    }
  }
}

But I notice that there is a check consumer.cnx == null in the method grabCnx, because if the state of the consumer is Ready, the variable cnx of the consumer must not be null, so this test can be effective as the check consumer.state != Ready.

And I notice another issue below:

Issue-1

If the method grab connection is executed multi times, it will lose some messages due to a race condition, for example:

time grab connection 1 grab connection 2
1 Check the variable cnx of the consumer is null Check the variable cnx of the consumer is null
2 Compare and set the variable duringConnect to true
3 Get the connection from the pool
4 Call consumer.connectionOpened()
4-1 Clear messages in memory
4-2 Set consumer.cnx
4-3 Do subscribe and increase availablePermits
5 Set the variable duringConnect to false
6 receive 712 messages
7 Compare and set the variable duringConnect to true
8 Get connection from the pool
9 Call consumer.connectionOpened()
9-1 Clear messages in memory.(Highlight) 712 messages were lost
9-2 Set consumer.cnx
9-3 Do subscribe and increase availablePermits
10 Set the variable duringConnect to false

We should make the check consumer.cnx == null execute after the check compare and set duringConnect.


Issue-2

After we fixed the issue-1, the check consumer.cnx == null and the check duringConnect == false switch the order, the new issue occurs: Since we use the variable duringConnectto prevent multigrabCnxrunning at the same time, we should makeset consumer.cnx to nullbeing executed beforeset duringConnect to true` when the subscribe request fails. This can avoid the issue below:

time reconnect later grab connection
1 Compare and set the variable duringConnect to false
2 Compare and set the variable duringConnect to true
3 Check the variable cnx of the consumer is null
4 set consumer.cnx to null

Modifications

  • Make the check consumer.cnx != null execute after the check compare and set duringConnect.
  • Make set consumer.cnx to null being executed before set duringConnect to true when the subscribe request fails
  • Change the log level to WARN if one consumer subscribe more than one times(the original log level is INFO), because it is not an expected behavior

Documentation

  • doc
  • doc-required
  • doc-not-needed
  • doc-complete

Matching PR in forked repository

PR in forked repository: x

@github-actions github-actions bot added the doc-not-needed Your PR changes do not impact docs label Jun 30, 2023
@poorbarcode poorbarcode self-assigned this Jun 30, 2023
@poorbarcode poorbarcode added release/2.10.5 release/2.11.2 release/3.0.2 type/bug The PR fixed a bug or issue reported a bug labels Jun 30, 2023
@poorbarcode poorbarcode added this to the 3.1.0 milestone Jun 30, 2023
// responsible for reconnection. And the variable "duringConnect" will prevent the concurrent execution.
if (getState() == State.Ready) {
return CompletableFuture.completedFuture(null);
}
Copy link
Contributor

@JooHyukKim JooHyukKim Jun 30, 2023

Choose a reason for hiding this comment

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

By duringConnect, did you mean duringSeek in the current method at line 785? I am assuming this because I couldn't find the word duringConnect within the method or the class itself.

If duringConnect is outside, maybe providing JavaDoc with {@link } tag will help readers to follow.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fixed, thanks

@poorbarcode poorbarcode requested a review from JooHyukKim June 30, 2023 09:31
Copy link
Member

@tisonkun tisonkun left a comment

Choose a reason for hiding this comment

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

A clear patch! LGTM.

@mattisonchao
Copy link
Member

Good catch!

@codelipenghui codelipenghui merged commit 09c89cd into apache:master Jul 4, 2023
liangyepianzhou pushed a commit that referenced this pull request Jul 8, 2023
nicoloboschi pushed a commit to datastax/pulsar that referenced this pull request Jul 17, 2023
(cherry picked from commit 09c89cd)
(cherry picked from commit 5d9f764)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

8 participants