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

[pulsar-client] Avoid subscribing the same topic again #7823

Merged
merged 6 commits into from Aug 17, 2020

Conversation

BewareMyPower
Copy link
Contributor

@BewareMyPower BewareMyPower commented Aug 16, 2020

Motivation

The current key of MultiTopicsConsumerImpl.topics is the topic name passed by user. The topicNameValid method checks if the name is valid and topics doesn't contain the key.

However, if a multi topics consumer subscribed a partition of a subscribed partitioned topic, subscribeAsync succeed and a new ConsumerImpl of the same partition was created, which is redundant.

Also, if a multi topics consumer subscribed public/default/topic or persistent://public/default/topic, while the initial subscribed topic is topic, the redundant consumers would be created.

Modifications

  • Use full topic name as key of MultiTopicsConsumerImpl.topics
  • Check both full topic name and full partitioned topic name not exist in MultiTopicsConsumerImpl.topics when subscribeAsync is called
  • Throw a different exception to differ topic is invalid and topic is already subscribed
  • Add a unit test for subscribing a partition of a subscribed partitioned topic or the same topic with prefix

}

// subscribe one more given topic
public CompletableFuture<Void> subscribeAsync(String topicName, boolean createTopicIfDoesNotExist) {
if (!topicNameValid(topicName)) {
TopicName topicNameInstance = getTopicName(topicName);
Copy link
Contributor

Choose a reason for hiding this comment

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

Why remove !topicNameValid(topicName) ? I think we should return Topic name not valid when the topic name does not valid and return Already subscribe to {} for the duplicate topic name.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I validate it in line 746 to 748. Because TopicName could be reused, if I wrap the validation in topicNameValid, I have to create TopicName to get full topic name and partitioned topic name again. I can't think an elegant way to do it so I create a TopicName instance at first of subscribeAsync and do checks

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@codelipenghui Sorry I haven't noticed the later part of your review before. I didn't change the original behavior in this PR because it needs relative changes with unit tests. I'll do it soon.

Comment on lines +723 to +740
private TopicName getTopicName(String topic) {
try {
return TopicName.get(topic);
} catch (Exception ignored) {
return null;
}
}

private String getFullTopicName(String topic) {
TopicName topicName = getTopicName(topic);
return (topicName != null) ? topicName.toString() : null;
}

private void removeTopic(String topic) {
String fullTopicName = getFullTopicName(topic);
if (fullTopicName != null) {
topics.remove(topic);
}
Copy link
Contributor

Choose a reason for hiding this comment

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

It seems that we do not need to add these methods?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

getTopicName and getFullTopicName are just for ignoring the exceptions because we will throw a PulsarClientException later. removeTopic is to ensure that the key to remove is a full topic name.

@BewareMyPower
Copy link
Contributor Author

/pulsarbot run-failure-checks

@BewareMyPower
Copy link
Contributor Author

/pulsarbot run-failure-checks

@codelipenghui codelipenghui merged commit e1b76a3 into apache:master Aug 17, 2020
huangdx0726 pushed a commit to huangdx0726/pulsar that referenced this pull request Aug 24, 2020
### Motivation

The current key of `MultiTopicsConsumerImpl.topics` is the topic name passed by user. The `topicNameValid` method checks if the name is valid and `topics` doesn't contain the key.

However, if a multi topics consumer subscribed a partition of a subscribed partitioned topic,  `subscribeAsync` succeed and a new `ConsumerImpl` of the same partition was created, which is redundant.

Also, if a multi topics consumer subscribed `public/default/topic` or `persistent://public/default/topic`, while the initial subscribed topic is `topic`, the redundant consumers would be created.

### Modifications

- Use full topic name as key of `MultiTopicsConsumerImpl.topics`
- Check both full topic name and full partitioned topic name not exist in `MultiTopicsConsumerImpl.topics` when `subscribeAsync` is called
- Throw a different exception to differ topic is invalid and topic is already subscribed
- Add a unit test for subscribing a partition of a subscribed partitioned topic or the same topic with prefix
lbenc135 pushed a commit to lbenc135/pulsar that referenced this pull request Sep 5, 2020
### Motivation

The current key of `MultiTopicsConsumerImpl.topics` is the topic name passed by user. The `topicNameValid` method checks if the name is valid and `topics` doesn't contain the key.

However, if a multi topics consumer subscribed a partition of a subscribed partitioned topic,  `subscribeAsync` succeed and a new `ConsumerImpl` of the same partition was created, which is redundant.

Also, if a multi topics consumer subscribed `public/default/topic` or `persistent://public/default/topic`, while the initial subscribed topic is `topic`, the redundant consumers would be created.

### Modifications

- Use full topic name as key of `MultiTopicsConsumerImpl.topics`
- Check both full topic name and full partitioned topic name not exist in `MultiTopicsConsumerImpl.topics` when `subscribeAsync` is called
- Throw a different exception to differ topic is invalid and topic is already subscribed
- Add a unit test for subscribing a partition of a subscribed partitioned topic or the same topic with prefix
lbenc135 pushed a commit to lbenc135/pulsar that referenced this pull request Sep 5, 2020
### Motivation

The current key of `MultiTopicsConsumerImpl.topics` is the topic name passed by user. The `topicNameValid` method checks if the name is valid and `topics` doesn't contain the key.

However, if a multi topics consumer subscribed a partition of a subscribed partitioned topic,  `subscribeAsync` succeed and a new `ConsumerImpl` of the same partition was created, which is redundant.

Also, if a multi topics consumer subscribed `public/default/topic` or `persistent://public/default/topic`, while the initial subscribed topic is `topic`, the redundant consumers would be created.

### Modifications

- Use full topic name as key of `MultiTopicsConsumerImpl.topics`
- Check both full topic name and full partitioned topic name not exist in `MultiTopicsConsumerImpl.topics` when `subscribeAsync` is called
- Throw a different exception to differ topic is invalid and topic is already subscribed
- Add a unit test for subscribing a partition of a subscribed partitioned topic or the same topic with prefix
lbenc135 pushed a commit to lbenc135/pulsar that referenced this pull request Sep 5, 2020
### Motivation

The current key of `MultiTopicsConsumerImpl.topics` is the topic name passed by user. The `topicNameValid` method checks if the name is valid and `topics` doesn't contain the key.

However, if a multi topics consumer subscribed a partition of a subscribed partitioned topic,  `subscribeAsync` succeed and a new `ConsumerImpl` of the same partition was created, which is redundant.

Also, if a multi topics consumer subscribed `public/default/topic` or `persistent://public/default/topic`, while the initial subscribed topic is `topic`, the redundant consumers would be created.

### Modifications

- Use full topic name as key of `MultiTopicsConsumerImpl.topics`
- Check both full topic name and full partitioned topic name not exist in `MultiTopicsConsumerImpl.topics` when `subscribeAsync` is called
- Throw a different exception to differ topic is invalid and topic is already subscribed
- Add a unit test for subscribing a partition of a subscribed partitioned topic or the same topic with prefix
wolfstudy pushed a commit that referenced this pull request Oct 30, 2020
### Motivation

The current key of `MultiTopicsConsumerImpl.topics` is the topic name passed by user. The `topicNameValid` method checks if the name is valid and `topics` doesn't contain the key.

However, if a multi topics consumer subscribed a partition of a subscribed partitioned topic,  `subscribeAsync` succeed and a new `ConsumerImpl` of the same partition was created, which is redundant.

Also, if a multi topics consumer subscribed `public/default/topic` or `persistent://public/default/topic`, while the initial subscribed topic is `topic`, the redundant consumers would be created.

### Modifications

- Use full topic name as key of `MultiTopicsConsumerImpl.topics`
- Check both full topic name and full partitioned topic name not exist in `MultiTopicsConsumerImpl.topics` when `subscribeAsync` is called
- Throw a different exception to differ topic is invalid and topic is already subscribed
- Add a unit test for subscribing a partition of a subscribed partitioned topic or the same topic with prefix

(cherry picked from commit e1b76a3)
merlimat pushed a commit to merlimat/pulsar that referenced this pull request Dec 19, 2020
### Motivation

The current key of `MultiTopicsConsumerImpl.topics` is the topic name passed by user. The `topicNameValid` method checks if the name is valid and `topics` doesn't contain the key.

However, if a multi topics consumer subscribed a partition of a subscribed partitioned topic,  `subscribeAsync` succeed and a new `ConsumerImpl` of the same partition was created, which is redundant.

Also, if a multi topics consumer subscribed `public/default/topic` or `persistent://public/default/topic`, while the initial subscribed topic is `topic`, the redundant consumers would be created.

### Modifications

- Use full topic name as key of `MultiTopicsConsumerImpl.topics`
- Check both full topic name and full partitioned topic name not exist in `MultiTopicsConsumerImpl.topics` when `subscribeAsync` is called
- Throw a different exception to differ topic is invalid and topic is already subscribed
- Add a unit test for subscribing a partition of a subscribed partitioned topic or the same topic with prefix
@BewareMyPower BewareMyPower deleted the multi-consumer-sub-dev branch September 16, 2022 07:41
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.

None yet

3 participants