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

Improve error handling logic for effectively once #5271

Merged
merged 13 commits into from
Oct 5, 2019

Conversation

jerrypeng
Copy link
Contributor

Motivation

As a part of solving #5218

Modifications

When there are BK write errors we need to fence the topic and reset highestSequencedPushed -> highestSequencedPersisted

@jerrypeng jerrypeng added the type/bug The PR fixed a bug or issue reported a bug label Sep 24, 2019
@jerrypeng jerrypeng added this to the 2.4.2 milestone Sep 24, 2019
@jerrypeng jerrypeng self-assigned this Sep 24, 2019
@@ -125,7 +129,8 @@
public class PersistentTopic extends AbstractTopic implements Topic, AddEntryCallback {

// Managed ledger associated with the topic
protected final ManagedLedger ledger;
@VisibleForTesting
ManagedLedger ledger;
Copy link
Contributor

Choose a reason for hiding this comment

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

could we retain the final?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

will change

// close all producers
List<CompletableFuture<Void>> futures = Lists.newArrayList();
producers.forEach(producer -> futures.add(producer.disconnect()));
FutureUtil.waitForAll(futures);
Copy link
Contributor

Choose a reason for hiding this comment

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

This will just return a new future that tracks all the futures in the list, without blocking (which is actually what we want).

To ensure we decrement only after all the connections are actually closed, we'd need to do like:

FutureUtil.waitForAll(futures).thenHandle((ex, v) -> {
   decrementPendingWriteOpsAndCheck();
});

Copy link
Contributor Author

Choose a reason for hiding this comment

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

will change

callback.completed(new PersistenceException(exception), -1, -1);
}

long pending = pendingWriteOps.decrementAndGet();
Copy link
Contributor

Choose a reason for hiding this comment

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

As above, call decrementPendingWriteOpsAndCheck() from the future callback instead of here

@jerrypeng
Copy link
Contributor Author

rerun cpp tests

1 similar comment
@jerrypeng
Copy link
Contributor Author

rerun cpp tests

@@ -272,17 +290,41 @@ private PersistentSubscription createPersistentSubscription(String subscriptionN

@Override
public void publishMessage(ByteBuf headersAndPayload, PublishContext publishContext) {
pendingWriteOps.incrementAndGet();
Copy link
Contributor

Choose a reason for hiding this comment

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

We should add a comment here with the logic behind the "increment then check the fence status" operation, because it will not be evident to a reader here.

Copy link
Contributor

@ivankelly ivankelly left a comment

Choose a reason for hiding this comment

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

Isn't there a fundamental problem here.

What if the client produces [M1, seq:1],[M2, seq:2],[M3, seq:3] asynchronously. M1 succeeds, M2 fails with a BK error, the managed ledger recovers from the error, then M3 hits the broker and is persisted. At this point, M2 can retry, but the message is lost because seq:2 is lower than seq:3.

List<CompletableFuture<Void>> futures = Lists.newArrayList();
producers.forEach(producer -> futures.add(producer.disconnect()));
FutureUtil.waitForAll(futures).handle((BiFunction<Void, Throwable, Void>) (aVoid, throwable) -> {
decrementPendingWriteOpsAndCheck();
Copy link
Contributor

Choose a reason for hiding this comment

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

Add a comment here that the write op being decremented is the one incremented in the call that eventually triggered addFailed. Otherwise it looks like you're decrementing for each producer closed.

@jiazhai
Copy link
Member

jiazhai commented Sep 25, 2019

run integration tests

@merlimat
Copy link
Contributor

What if the client produces [M1, seq:1],[M2, seq:2],[M3, seq:3] asynchronously. M1 succeeds, M2 fails with a BK error, the managed ledger recovers from the error, then M3 hits the broker and is persisted. At this point, M2 can retry, but the message is lost because seq:2 is lower than seq:3.

The current guard against this scenario is that managed ledger will reject all the writes for a period of 10sec. In practical terms, this should avoid all races between threads (for non-blocking ops), though of course it does not give 100% proof.

The next step is to have managed ledger to stay in "error mode" after write failure, until we manually set it back into normal mode, after all the pending ops are done and we got the chance of resetting the topic.

@jerrypeng
Copy link
Contributor Author

rerun integration tests

2 similar comments
@jerrypeng
Copy link
Contributor Author

rerun integration tests

@jerrypeng
Copy link
Contributor Author

rerun integration tests

@ivankelly
Copy link
Contributor

What if the client produces [M1, seq:1],[M2, seq:2],[M3, seq:3] asynchronously. M1 succeeds, M2 fails with a BK error, the managed ledger recovers from the error, then M3 hits the broker and is persisted. At this point, M2 can retry, but the message is lost because seq:2 is lower than seq:3.

The current guard against this scenario is that managed ledger will reject all the writes for a period of 10sec. In practical terms, this should avoid all races between threads (for non-blocking ops), though of course it does not give 100% proof.

The next step is to have managed ledger to stay in "error mode" after write failure, until we manually set it back into normal mode, after all the pending ops are done and we got the chance of resetting the topic.

Maybe we need to rebrand our "exactly-once" again from "effectively-once" to "probably-once".
I think there needs to be some cooperation with the client w.r.t. failures. A client could have a write pending during the time that the error occurs on a previous write and that error being handled, so when the write hits the broker it proceeds as normal, losing the previous write forever.

Maybe we should have some sort of epoch to represent the client <-> producer relationship? When an error occurs on a write, all subsequent writes from that epoch should fail. The error should be kicked back to the client, which should then have to reestablish it's current position before preceeding.

@merlimat
Copy link
Contributor

In this PR, the sequence of handling errors will be:

  1. Get a write error
  2. "fence" topic
  3. disconnect all producers
    • Producers will fail to reconnect because of the topic state
    • This will discard any other publish request in the pipe
  4. Wait until all pending write ops are completed (with failure)
  5. "un-fence" the topic

@jerrypeng
Copy link
Contributor Author

rerun integration tests

7 similar comments
@jerrypeng
Copy link
Contributor Author

rerun integration tests

@jerrypeng
Copy link
Contributor Author

rerun integration tests

@jerrypeng
Copy link
Contributor Author

rerun integration tests

@jerrypeng
Copy link
Contributor Author

rerun integration tests

@jerrypeng
Copy link
Contributor Author

rerun integration tests

@jerrypeng
Copy link
Contributor Author

rerun integration tests

@jerrypeng
Copy link
Contributor Author

rerun integration tests

@jerrypeng jerrypeng merged commit 8e95f43 into apache:master Oct 5, 2019
wolfstudy pushed a commit that referenced this pull request Nov 20, 2019
* Bug in Message Deduplication that may cause incorrect behavior

* add tests

* fix error message

* fix client backoff

* fix tests

* cleaning up

* Fix handling of BK write failures for message dedup

* tests and clean up

* refactoring code

* fixing bugs

* addressing comments

* add missing license header

(cherry picked from commit 8e95f43)
sijie pushed a commit that referenced this pull request Aug 6, 2020
…s unloaded (#7735)

### Motivation

When a topic is unloaded and moved to another broker, the producer for geo-replication often remains unclosed. Because of this, geo-replication is not possible on the broker to which the topic was moved and messages accumulate in the replication backlog.

```
18:56:55.166 [pulsar-io-21-6] ERROR o.a.pulsar.client.impl.ProducerImpl  - [persistent://xxx/yyy/zzz] [pulsar.repl.dc2] Failed to create producer: Producer with name 'pulsar.repl.dc2' is already connected to topic
```

When this issue occurs, the following log is output on the broker where the topic is unloaded.

```
17:14:36.424 [bookkeeper-ml-workers-OrderedExecutor-18-0] INFO  o.a.p.b.s.persistent.PersistentTopic - [persistent://xxx/yyy/zzz] Un-fencing topic...
```

Unloaded topics are usually fenced to prevent new clients from connecting. In this case, however, the producers reconnected to the topic because it had been unfenced, and the replicator was restarted.

I think this is due to #5271. If a topic is fenced to close or delete, we should not unfence it.

### Modifications

When closing or deleting the `PersistentTopic` instance, set the `isClosingOrDeleting` flag to true. If `isClosingOrDeleting` is true, do not unfence the topic unless closing or deleting fails.
huangdx0726 pushed a commit to huangdx0726/pulsar that referenced this pull request Aug 24, 2020
…s unloaded (apache#7735)

### Motivation

When a topic is unloaded and moved to another broker, the producer for geo-replication often remains unclosed. Because of this, geo-replication is not possible on the broker to which the topic was moved and messages accumulate in the replication backlog.

```
18:56:55.166 [pulsar-io-21-6] ERROR o.a.pulsar.client.impl.ProducerImpl  - [persistent://xxx/yyy/zzz] [pulsar.repl.dc2] Failed to create producer: Producer with name 'pulsar.repl.dc2' is already connected to topic
```

When this issue occurs, the following log is output on the broker where the topic is unloaded.

```
17:14:36.424 [bookkeeper-ml-workers-OrderedExecutor-18-0] INFO  o.a.p.b.s.persistent.PersistentTopic - [persistent://xxx/yyy/zzz] Un-fencing topic...
```

Unloaded topics are usually fenced to prevent new clients from connecting. In this case, however, the producers reconnected to the topic because it had been unfenced, and the replicator was restarted.

I think this is due to apache#5271. If a topic is fenced to close or delete, we should not unfence it.

### Modifications

When closing or deleting the `PersistentTopic` instance, set the `isClosingOrDeleting` flag to true. If `isClosingOrDeleting` is true, do not unfence the topic unless closing or deleting fails.
lbenc135 pushed a commit to lbenc135/pulsar that referenced this pull request Sep 5, 2020
…s unloaded (apache#7735)

### Motivation

When a topic is unloaded and moved to another broker, the producer for geo-replication often remains unclosed. Because of this, geo-replication is not possible on the broker to which the topic was moved and messages accumulate in the replication backlog.

```
18:56:55.166 [pulsar-io-21-6] ERROR o.a.pulsar.client.impl.ProducerImpl  - [persistent://xxx/yyy/zzz] [pulsar.repl.dc2] Failed to create producer: Producer with name 'pulsar.repl.dc2' is already connected to topic
```

When this issue occurs, the following log is output on the broker where the topic is unloaded.

```
17:14:36.424 [bookkeeper-ml-workers-OrderedExecutor-18-0] INFO  o.a.p.b.s.persistent.PersistentTopic - [persistent://xxx/yyy/zzz] Un-fencing topic...
```

Unloaded topics are usually fenced to prevent new clients from connecting. In this case, however, the producers reconnected to the topic because it had been unfenced, and the replicator was restarted.

I think this is due to apache#5271. If a topic is fenced to close or delete, we should not unfence it.

### Modifications

When closing or deleting the `PersistentTopic` instance, set the `isClosingOrDeleting` flag to true. If `isClosingOrDeleting` is true, do not unfence the topic unless closing or deleting fails.
lbenc135 pushed a commit to lbenc135/pulsar that referenced this pull request Sep 5, 2020
…s unloaded (apache#7735)

### Motivation

When a topic is unloaded and moved to another broker, the producer for geo-replication often remains unclosed. Because of this, geo-replication is not possible on the broker to which the topic was moved and messages accumulate in the replication backlog.

```
18:56:55.166 [pulsar-io-21-6] ERROR o.a.pulsar.client.impl.ProducerImpl  - [persistent://xxx/yyy/zzz] [pulsar.repl.dc2] Failed to create producer: Producer with name 'pulsar.repl.dc2' is already connected to topic
```

When this issue occurs, the following log is output on the broker where the topic is unloaded.

```
17:14:36.424 [bookkeeper-ml-workers-OrderedExecutor-18-0] INFO  o.a.p.b.s.persistent.PersistentTopic - [persistent://xxx/yyy/zzz] Un-fencing topic...
```

Unloaded topics are usually fenced to prevent new clients from connecting. In this case, however, the producers reconnected to the topic because it had been unfenced, and the replicator was restarted.

I think this is due to apache#5271. If a topic is fenced to close or delete, we should not unfence it.

### Modifications

When closing or deleting the `PersistentTopic` instance, set the `isClosingOrDeleting` flag to true. If `isClosingOrDeleting` is true, do not unfence the topic unless closing or deleting fails.
lbenc135 pushed a commit to lbenc135/pulsar that referenced this pull request Sep 5, 2020
…s unloaded (apache#7735)

### Motivation

When a topic is unloaded and moved to another broker, the producer for geo-replication often remains unclosed. Because of this, geo-replication is not possible on the broker to which the topic was moved and messages accumulate in the replication backlog.

```
18:56:55.166 [pulsar-io-21-6] ERROR o.a.pulsar.client.impl.ProducerImpl  - [persistent://xxx/yyy/zzz] [pulsar.repl.dc2] Failed to create producer: Producer with name 'pulsar.repl.dc2' is already connected to topic
```

When this issue occurs, the following log is output on the broker where the topic is unloaded.

```
17:14:36.424 [bookkeeper-ml-workers-OrderedExecutor-18-0] INFO  o.a.p.b.s.persistent.PersistentTopic - [persistent://xxx/yyy/zzz] Un-fencing topic...
```

Unloaded topics are usually fenced to prevent new clients from connecting. In this case, however, the producers reconnected to the topic because it had been unfenced, and the replicator was restarted.

I think this is due to apache#5271. If a topic is fenced to close or delete, we should not unfence it.

### Modifications

When closing or deleting the `PersistentTopic` instance, set the `isClosingOrDeleting` flag to true. If `isClosingOrDeleting` is true, do not unfence the topic unless closing or deleting fails.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
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

4 participants