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

Pending ack set managed ledger config true #11494

Merged
merged 20 commits into from
Aug 10, 2021
Merged

Pending ack set managed ledger config true #11494

merged 20 commits into from
Aug 10, 2021

Conversation

liangyepianzhou
Copy link
Contributor

@liangyepianzhou liangyepianzhou commented Jul 29, 2021

Modivation

Fix the Issue of #11481
In standalone mode, pulsar 2.8.0 cannot be used normally when the transaction is started

CauseBy

getTopicwas executed twice when FunctionWorkService .
getTopicIfExistsmake createIfMissing = false When the execution ends.
PersistentSubscription will create a ledger for the subscription when transaction was turned on.
new MetadataNotFoundException("Managed ledger not found")was thrown when calling MetaStoreImpl::getManagedLedgerInfo

implement

Create a separate ManagerLedgerConfig for PendingAck

verify

Add testSubscriptionRecreateTopic in TransactionTest

Documentation

  • no-need-Documentation
    This is just a bug fix

admin.topics().unload(topicName);
try {
admin.topics().createNonPartitionedTopic(topicName);
} catch (Exception e){
Copy link
Contributor

Choose a reason for hiding this comment

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

you can use catch (PulsarAdminException.ConflictException) and let other exceptions be not caught

@@ -51,7 +52,8 @@
PersistentTopic originPersistentTopic = (PersistentTopic) subscription.getTopic();
String pendingAckTopicName = MLPendingAckStore
.getTransactionPendingAckStoreSuffix(originPersistentTopic.getName(), subscription.getName());

ManagedLedgerConfig managedLedgerConfig = originPersistentTopic.getManagedLedger().getConfig();
managedLedgerConfig.setCreateIfMissing(true);
Copy link
Contributor

Choose a reason for hiding this comment

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

this is not good, we cannot alter the configuration of another component this way.

if "createIfMissing" is required we have to set it upstream

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Thank you for pointing out.
I made some changes, please take a look again

@@ -150,6 +151,8 @@ public PersistentSubscription(PersistentTopic topic, String subscriptionName, Ma
this.setReplicated(replicated);
if (topic.getBrokerService().getPulsar().getConfig().isTransactionCoordinatorEnabled()
&& !checkTopicIsEventsNames(TopicName.get(topicName))) {
ManagedLedgerConfig config = this.topic.getManagedLedger().getConfig();
config.setCreateIfMissing(true);
Copy link
Contributor

Choose a reason for hiding this comment

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

this is still not good.
you have to apply this setting elsewhere

maybe @congbobo184 or @codelipenghui can help more

Copy link
Contributor Author

@liangyepianzhou liangyepianzhou Aug 3, 2021

Choose a reason for hiding this comment

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

Thank you very much for your advice. After discussing with Congbo, the changes have been made. Could you please take a look again?

@codelipenghui
Copy link
Contributor

@liangyepianzhou Could you please provide the root cause of #11481? before fixing the issue, we'd better point out where the problem is so that the reviewers can get more context to provide more suggestions.

@codelipenghui codelipenghui added area/transaction type/bug The PR fixed a bug or issue reported a bug labels Jul 29, 2021
@codelipenghui codelipenghui added this to the 2.9.0 milestone Jul 29, 2021
@liangyepianzhou
Copy link
Contributor Author

Thank you for pointing it @codelipenghui
I have given some context above.
If there is a problem, hope you can point it out,
it would be better if you can provide some help.

@liangyepianzhou
Copy link
Contributor Author

/pulsarbot run-failure-checks

1 similar comment
@liangyepianzhou
Copy link
Contributor Author

/pulsarbot run-failure-checks

@@ -1232,6 +1232,7 @@ public void openLedgerComplete(ManagedLedger ledger, Object ctx) {
preCreateSubForCompaction = ((SystemTopic) persistentTopic)
.preCreateSubForCompactionIfNeeded();
}
managedLedgerConfig.setCreateIfMissing(true);
Copy link
Contributor

Choose a reason for hiding this comment

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

Maybe set createIfMissing as true directly is not reasonable, this will break the service configuration?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This may depend on the original design intention of getTopicIfExists. Calling getTopicIfExists will directly set createIfMissing to false, which will affect the configuration. I don't know if this is intentional or ignored.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

However, I did make a mistake and did not consider its initial configuration. I think I should keep the configuration first, and then restore the configuration

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Thank you for pointing out.
Can you see if there are any problems now?

Copy link
Contributor Author

@liangyepianzhou liangyepianzhou Aug 3, 2021

Choose a reason for hiding this comment

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

Sorry, Ran. The above is my ill-conceived.
I have another discussion with congbo.
After this topic has been created, the field will have no meaning. And each subscription of transaction needs createIfMsiing=true to create PendingAck. So it is also feasible to set it to true directly in Completely

@Anonymitaet
Copy link
Member

Thanks for your contribution. For this PR, do we need to update docs?

(The PR template contains info about doc, which helps others know more about the changes. Can you provide doc-related info in this and future PR descriptions? Thanks)

@liangyepianzhou
Copy link
Contributor Author

Thanks for your contribution. For this PR, do we need to update docs?

(The PR template contains info about doc, which helps others know more about the changes. Can you provide doc-related info in this and future PR descriptions? Thanks)

Thank you for your reminder, I will indicate it in the comment in the future.
This is just a bug fix. No documentation required

@liangyepianzhou
Copy link
Contributor Author

/pulsarbot run-failure-checks

Copy link
Contributor

@congbobo184 congbobo184 left a comment

Choose a reason for hiding this comment

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

LGTM! left some comment. Could you please add more details? It might be better to give steps to reproduce

ConcurrentHashMap<String, CompletableFuture<ManagedLedgerImpl>> ledgers =
(ConcurrentHashMap<String, CompletableFuture<ManagedLedgerImpl>>)field.get(managedLedgerFactory);
ledgers.remove(TopicName.get(topic).getPersistenceNamingEncoding());
admin.topics().unload(topic);
Copy link
Contributor

Choose a reason for hiding this comment

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

dont need unload.

admin.topics().createNonPartitionedTopic(topic);
Assert.fail();
} catch (PulsarAdminException.ConflictException e){
log.info("Cann`t create topic again");
Copy link
Contributor

Choose a reason for hiding this comment

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

should Assert this exception

@hangc0276
Copy link
Contributor

@liangyepianzhou I am cutting 2.8.1, would you please address @codelipenghui @congbobo184 's comments?

@liangyepianzhou
Copy link
Contributor Author

@liangyepianzhou I am cutting 2.8.1, would you please address @codelipenghui @congbobo184 's comments?

I am waiting for review from @codelipenghui @congbobo184

@hangc0276
Copy link
Contributor

/pulsarbot run-failure-checks

@hangc0276
Copy link
Contributor

@liangyepianzhou I am cutting 2.8.1, would you please address @codelipenghui @congbobo184 's comments?

I am waiting for review from @codelipenghui @congbobo184

OK,@codelipenghui @congbobo184 Please help review this PR, thanks.

pendingAckStoreFuture
.complete(new MLPendingAckStore(ledger, cursor,
subscription.getCursor()));
log.info("{},{} open MLPendingAckStore cursor success",
Copy link
Contributor

Choose a reason for hiding this comment

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

probably this logger should be moved to level debug

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.

Lgtm

Copy link
Contributor

@congbobo184 congbobo184 left a comment

Choose a reason for hiding this comment

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

LGTM! good work.

@codelipenghui codelipenghui merged commit daf457d into apache:master Aug 10, 2021
hangc0276 pushed a commit that referenced this pull request Aug 12, 2021
## Modivation
Fix the Issue of #11481
In standalone mode, pulsar 2.8.0 cannot be used normally when the transaction is started

## CauseBy
```getTopic```was executed  twice when FunctionWorkService .
```getTopicIfExists```make ```createIfMissing = false``` When the execution ends.
```PersistentSubscription```  will create a ledger for the subscription  when transaction was turned on.
```new MetadataNotFoundException("Managed ledger not found")```was thrown when calling ```MetaStoreImpl::getManagedLedgerInfo```
## implement
Create a separate ManagerLedgerConfig for PendingAck
## verify
Add testSubscriptionRecreateTopic in TransactionTest

(cherry picked from commit daf457d)
@hangc0276 hangc0276 added the cherry-picked/branch-2.8 Archived: 2.8 is end of life label Aug 12, 2021
bharanic-dev pushed a commit to bharanic-dev/pulsar that referenced this pull request Mar 18, 2022
## Modivation
Fix the Issue of apache#11481
In standalone mode, pulsar 2.8.0 cannot be used normally when the transaction is started

## CauseBy
```getTopic```was executed  twice when FunctionWorkService .
```getTopicIfExists```make ```createIfMissing = false``` When the execution ends.
```PersistentSubscription```  will create a ledger for the subscription  when transaction was turned on. 
```new MetadataNotFoundException("Managed ledger not found")```was thrown when calling ```MetaStoreImpl::getManagedLedgerInfo``` 
## implement   
Create a separate ManagerLedgerConfig for PendingAck
## verify 
Add testSubscriptionRecreateTopic in TransactionTest
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area/transaction cherry-picked/branch-2.8 Archived: 2.8 is end of life doc-not-needed Your PR changes do not impact docs release/2.8.1 type/bug The PR fixed a bug or issue reported a bug
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

7 participants