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-broker] Handle multiple topic creation for same topic-name in broker #10847

Merged
merged 1 commit into from Jun 18, 2021

Conversation

rdhabalia
Copy link
Contributor

Motivation

When the broker takes a longer time to load the topic and times out before completing the topic future, then the broker keeps multiple topics opened , doesn't clean up timed out topic, fail to create replicator producer on successfully created topic with error: repl-producer is already connected to topic, builds replication backlog.

19:16:10.107 [pulsar-ordered-OrderedExecutor-5-0] INFO  org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - Opening managed ledger myProp/global/myNs/persistent/myTopic
:
9:17:10.953 [BookKeeperClientWorker-OrderedExecutor-2-0] INFO  org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl - [myProp/global/myNs/persistent/myTopic] Successfully initialize managed ledger
:
19:17:10.065 [pulsar-io-23-30] ERROR org.apache.pulsar.broker.service.ServerCnx - [/10.196.133.62:47278] Failed to create topic persistent://myProp/global/myNs/myTopic, producerId=382
:
19:17:10.954 [BookKeeperClientWorker-OrderedExecutor-2-0] INFO  org.apache.pulsar.broker.service.persistent.PersistentReplicator - [persistent://myProp/global/myNs/myTopic][west1 -> west2] Starting open cursor for replicator
:
19:17:10.955 [BookKeeperClientWorker-OrderedExecutor-2-0] INFO  org.apache.pulsar.broker.service.BrokerService - Created topic persistent://myProp/global/myNs/myTopic - dedup is disabled
:
19:17:51.532 [pulsar-ordered-OrderedExecutor-5-0] INFO  org.apache.pulsar.broker.service.BrokerService - Created topic persistent://myProp/global/myNs/myTopic - dedup is disabled
:
19:17:51.530 [pulsar-ordered-OrderedExecutor-5-0] INFO  org.apache.pulsar.broker.service.persistent.PersistentReplicator - [persistent://myProp/global/myNs/myTopic][west1 -> west2] Starting open cursor for replicator
:
07:25:51.377 [pulsar-io-23-5] ERROR org.apache.pulsar.client.impl.ProducerImpl - [persistent://myProp/global/myNs/myTopic] [pulsar.repl.west1] Failed to create producer: Producer with name 'pulsarrepl.west1' is already connected to topic

Modification

  • Stopped replicator for failed and timed-out topic
  • Clean up failed topic

Result

  • Successfully create replicator producer for the topic and avoid creating replication backlog

@rdhabalia rdhabalia added this to the 2.8.0 milestone Jun 7, 2021
@rdhabalia rdhabalia self-assigned this Jun 7, 2021
@codelipenghui
Copy link
Contributor

@rdhabalia we have started the 2.8.0 release, can we move it 2.8.1 or 2.7.3?

@rdhabalia
Copy link
Contributor Author

yes, we can move to 2.8.1

@merlimat merlimat added the type/bug The PR fixed a bug or issue reported a bug label Jun 7, 2021
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.

@@ -162,4 +164,33 @@ public synchronized Throwable fillInStackTrace() {
return this;
}
}

public static <T> CompletableFuture<T> futureWithDeadline(ScheduledExecutorService executor, Long delay,
Copy link
Contributor

Choose a reason for hiding this comment

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

What is the difference between this and addTimeoutHandling? why we need add this method?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

it is an existing method just moved to FutureUtil class. futureWithDeadline completes future with exception after the time whereas timeout handling something gets trigger once future is completed with exception.

@@ -970,7 +970,7 @@ public void deleteLedgerFailed(ManagedLedgerException exception, Object ctx) {
}

private CompletableFuture<Optional<Topic>> createNonPersistentTopic(String topic) {
CompletableFuture<Optional<Topic>> topicFuture = futureWithDeadline();
CompletableFuture<Optional<Topic>> topicFuture = FutureUtil.futureWithDeadline(executor());
Copy link
Contributor

Choose a reason for hiding this comment

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

why need change futureWithDeadline() to FutureUtil.futureWithDeadline(executor())?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It is a utility method for Future so, it should be part of FutureUtil

@Anonymitaet
Copy link
Member

@rdhabalia thanks for your contribution. For this PR, do we need to update docs?

@rdhabalia
Copy link
Contributor Author

@Anonymitaet No, this is a bug fix and we don't need documentation for this one.

@sijie sijie merged commit 1447e6b into apache:master Jun 18, 2021
yangl pushed a commit to yangl/pulsar that referenced this pull request Jun 23, 2021
… broker (apache#10847)

### Motivation

When the broker takes a longer time to load the topic and times out before completing the topic future, then the broker keeps multiple topics opened , doesn't clean up timed out topic, fail to create replicator producer on successfully created topic with error: `repl-producer is already connected to topic`, builds replication backlog.

```
19:16:10.107 [pulsar-ordered-OrderedExecutor-5-0] INFO  org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - Opening managed ledger myProp/global/myNs/persistent/myTopic
:
9:17:10.953 [BookKeeperClientWorker-OrderedExecutor-2-0] INFO  org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl - [myProp/global/myNs/persistent/myTopic] Successfully initialize managed ledger
:
19:17:10.065 [pulsar-io-23-30] ERROR org.apache.pulsar.broker.service.ServerCnx - [/10.196.133.62:47278] Failed to create topic persistent://myProp/global/myNs/myTopic, producerId=382
:
19:17:10.954 [BookKeeperClientWorker-OrderedExecutor-2-0] INFO  org.apache.pulsar.broker.service.persistent.PersistentReplicator - [persistent://myProp/global/myNs/myTopic][west1 -> west2] Starting open cursor for replicator
:
19:17:10.955 [BookKeeperClientWorker-OrderedExecutor-2-0] INFO  org.apache.pulsar.broker.service.BrokerService - Created topic persistent://myProp/global/myNs/myTopic - dedup is disabled
:
19:17:51.532 [pulsar-ordered-OrderedExecutor-5-0] INFO  org.apache.pulsar.broker.service.BrokerService - Created topic persistent://myProp/global/myNs/myTopic - dedup is disabled
:
19:17:51.530 [pulsar-ordered-OrderedExecutor-5-0] INFO  org.apache.pulsar.broker.service.persistent.PersistentReplicator - [persistent://myProp/global/myNs/myTopic][west1 -> west2] Starting open cursor for replicator
:
07:25:51.377 [pulsar-io-23-5] ERROR org.apache.pulsar.client.impl.ProducerImpl - [persistent://myProp/global/myNs/myTopic] [pulsar.repl.west1] Failed to create producer: Producer with name 'pulsarrepl.west1' is already connected to topic
```

### Modification
- Stopped replicator for failed and timed-out topic
- Clean up failed topic

### Result
- Successfully create replicator producer for the topic and avoid creating replication backlog
codelipenghui pushed a commit that referenced this pull request Jun 25, 2021
… broker (#10847)

### Motivation

When the broker takes a longer time to load the topic and times out before completing the topic future, then the broker keeps multiple topics opened , doesn't clean up timed out topic, fail to create replicator producer on successfully created topic with error: `repl-producer is already connected to topic`, builds replication backlog.

```
19:16:10.107 [pulsar-ordered-OrderedExecutor-5-0] INFO  org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - Opening managed ledger myProp/global/myNs/persistent/myTopic
:
9:17:10.953 [BookKeeperClientWorker-OrderedExecutor-2-0] INFO  org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl - [myProp/global/myNs/persistent/myTopic] Successfully initialize managed ledger
:
19:17:10.065 [pulsar-io-23-30] ERROR org.apache.pulsar.broker.service.ServerCnx - [/10.196.133.62:47278] Failed to create topic persistent://myProp/global/myNs/myTopic, producerId=382
:
19:17:10.954 [BookKeeperClientWorker-OrderedExecutor-2-0] INFO  org.apache.pulsar.broker.service.persistent.PersistentReplicator - [persistent://myProp/global/myNs/myTopic][west1 -> west2] Starting open cursor for replicator
:
19:17:10.955 [BookKeeperClientWorker-OrderedExecutor-2-0] INFO  org.apache.pulsar.broker.service.BrokerService - Created topic persistent://myProp/global/myNs/myTopic - dedup is disabled
:
19:17:51.532 [pulsar-ordered-OrderedExecutor-5-0] INFO  org.apache.pulsar.broker.service.BrokerService - Created topic persistent://myProp/global/myNs/myTopic - dedup is disabled
:
19:17:51.530 [pulsar-ordered-OrderedExecutor-5-0] INFO  org.apache.pulsar.broker.service.persistent.PersistentReplicator - [persistent://myProp/global/myNs/myTopic][west1 -> west2] Starting open cursor for replicator
:
07:25:51.377 [pulsar-io-23-5] ERROR org.apache.pulsar.client.impl.ProducerImpl - [persistent://myProp/global/myNs/myTopic] [pulsar.repl.west1] Failed to create producer: Producer with name 'pulsarrepl.west1' is already connected to topic
```

### Modification
- Stopped replicator for failed and timed-out topic
- Clean up failed topic

### Result
- Successfully create replicator producer for the topic and avoid creating replication backlog

(cherry picked from commit 1447e6b)
@codelipenghui codelipenghui added the cherry-picked/branch-2.8 Archived: 2.8 is end of life label Jun 25, 2021
codelipenghui pushed a commit that referenced this pull request Jun 27, 2021
… broker (#10847)

When the broker takes a longer time to load the topic and times out before completing the topic future, then the broker keeps multiple topics opened , doesn't clean up timed out topic, fail to create replicator producer on successfully created topic with error: `repl-producer is already connected to topic`, builds replication backlog.

```
19:16:10.107 [pulsar-ordered-OrderedExecutor-5-0] INFO  org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - Opening managed ledger myProp/global/myNs/persistent/myTopic
:
9:17:10.953 [BookKeeperClientWorker-OrderedExecutor-2-0] INFO  org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl - [myProp/global/myNs/persistent/myTopic] Successfully initialize managed ledger
:
19:17:10.065 [pulsar-io-23-30] ERROR org.apache.pulsar.broker.service.ServerCnx - [/10.196.133.62:47278] Failed to create topic persistent://myProp/global/myNs/myTopic, producerId=382
:
19:17:10.954 [BookKeeperClientWorker-OrderedExecutor-2-0] INFO  org.apache.pulsar.broker.service.persistent.PersistentReplicator - [persistent://myProp/global/myNs/myTopic][west1 -> west2] Starting open cursor for replicator
:
19:17:10.955 [BookKeeperClientWorker-OrderedExecutor-2-0] INFO  org.apache.pulsar.broker.service.BrokerService - Created topic persistent://myProp/global/myNs/myTopic - dedup is disabled
:
19:17:51.532 [pulsar-ordered-OrderedExecutor-5-0] INFO  org.apache.pulsar.broker.service.BrokerService - Created topic persistent://myProp/global/myNs/myTopic - dedup is disabled
:
19:17:51.530 [pulsar-ordered-OrderedExecutor-5-0] INFO  org.apache.pulsar.broker.service.persistent.PersistentReplicator - [persistent://myProp/global/myNs/myTopic][west1 -> west2] Starting open cursor for replicator
:
07:25:51.377 [pulsar-io-23-5] ERROR org.apache.pulsar.client.impl.ProducerImpl - [persistent://myProp/global/myNs/myTopic] [pulsar.repl.west1] Failed to create producer: Producer with name 'pulsarrepl.west1' is already connected to topic
```

- Stopped replicator for failed and timed-out topic
- Clean up failed topic

- Successfully create replicator producer for the topic and avoid creating replication backlog

(cherry picked from commit 1447e6b)
@codelipenghui codelipenghui added the cherry-picked/branch-2.7 Archived: 2.7 is end of life label Jun 27, 2021
@rdhabalia rdhabalia deleted the topic_clear branch October 8, 2021 17:58
bharanic-dev pushed a commit to bharanic-dev/pulsar that referenced this pull request Mar 18, 2022
… broker (apache#10847)

### Motivation

When the broker takes a longer time to load the topic and times out before completing the topic future, then the broker keeps multiple topics opened , doesn't clean up timed out topic, fail to create replicator producer on successfully created topic with error: `repl-producer is already connected to topic`, builds replication backlog.

```
19:16:10.107 [pulsar-ordered-OrderedExecutor-5-0] INFO  org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - Opening managed ledger myProp/global/myNs/persistent/myTopic
:
9:17:10.953 [BookKeeperClientWorker-OrderedExecutor-2-0] INFO  org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl - [myProp/global/myNs/persistent/myTopic] Successfully initialize managed ledger
:
19:17:10.065 [pulsar-io-23-30] ERROR org.apache.pulsar.broker.service.ServerCnx - [/10.196.133.62:47278] Failed to create topic persistent://myProp/global/myNs/myTopic, producerId=382
:
19:17:10.954 [BookKeeperClientWorker-OrderedExecutor-2-0] INFO  org.apache.pulsar.broker.service.persistent.PersistentReplicator - [persistent://myProp/global/myNs/myTopic][west1 -> west2] Starting open cursor for replicator
:
19:17:10.955 [BookKeeperClientWorker-OrderedExecutor-2-0] INFO  org.apache.pulsar.broker.service.BrokerService - Created topic persistent://myProp/global/myNs/myTopic - dedup is disabled
:
19:17:51.532 [pulsar-ordered-OrderedExecutor-5-0] INFO  org.apache.pulsar.broker.service.BrokerService - Created topic persistent://myProp/global/myNs/myTopic - dedup is disabled
:
19:17:51.530 [pulsar-ordered-OrderedExecutor-5-0] INFO  org.apache.pulsar.broker.service.persistent.PersistentReplicator - [persistent://myProp/global/myNs/myTopic][west1 -> west2] Starting open cursor for replicator
:
07:25:51.377 [pulsar-io-23-5] ERROR org.apache.pulsar.client.impl.ProducerImpl - [persistent://myProp/global/myNs/myTopic] [pulsar.repl.west1] Failed to create producer: Producer with name 'pulsarrepl.west1' is already connected to topic
```

### Modification
- Stopped replicator for failed and timed-out topic
- Clean up failed topic

### Result
- Successfully create replicator producer for the topic and avoid creating replication backlog
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area/broker cherry-picked/branch-2.7 Archived: 2.7 is end of life cherry-picked/branch-2.8 Archived: 2.8 is end of life release/2.7.3 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

6 participants