-
Notifications
You must be signed in to change notification settings - Fork 3.7k
[fix][broker] Fix PendingAckHandleImpl when replay failed. #18886
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
Changes from all commits
9c7ca2d
f81902b
a39723e
938dfdf
6b7fe43
5e3b841
4fd2e3d
8bdb3ba
5ceecea
348b6bd
28b2c8e
5e7ac1e
0bb9cc2
436aab6
8ac4924
10bbd1d
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -116,6 +116,7 @@ | |
| import org.apache.pulsar.broker.stats.ReplicationMetrics; | ||
| import org.apache.pulsar.broker.transaction.buffer.TransactionBuffer; | ||
| import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferDisable; | ||
| import org.apache.pulsar.broker.transaction.pendingack.exceptions.PendingAckHandleReplayException; | ||
| import org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStore; | ||
| import org.apache.pulsar.client.admin.LongRunningProcessStatus; | ||
| import org.apache.pulsar.client.admin.OffloadProcessStatus; | ||
|
|
@@ -298,12 +299,23 @@ public PersistentTopic(String topic, ManagedLedger ledger, BrokerService brokerS | |
| // ignore it for now and let the message dedup logic to take care of it | ||
| } else { | ||
| final String subscriptionName = Codec.decode(cursor.getName()); | ||
| subscriptions.put(subscriptionName, createPersistentSubscription(subscriptionName, cursor, | ||
| PersistentSubscription subscription = createPersistentSubscription(subscriptionName, cursor, | ||
| PersistentSubscription.isCursorFromReplicatedSubscription(cursor), | ||
| cursor.getCursorProperties())); | ||
| // subscription-cursor gets activated by default: deactivate as there is no active subscription right | ||
| // now | ||
| subscriptions.get(subscriptionName).deactivateCursor(); | ||
| cursor.getCursorProperties()); | ||
| subscriptions.put(subscriptionName, subscription); | ||
| subscription.getPendingAckHandle() | ||
| .pendingAckHandleFuture() | ||
| .exceptionally(t -> { | ||
| log.warn("PersistentSubscription [{}] pendingAckHandleImpl relay failed " | ||
| + "when initialize topic [{}].", subscriptionName, topic, t); | ||
| if (subscriptions.remove(subscriptionName, subscription)) { | ||
| subscription.retryClose(); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. why should add this method? persistentTopic init sub then remove will return false?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think it may happen. will be executed after the topic created. If a consumer connected the topic and unsubscribe the topic, |
||
| } else { | ||
| log.warn("[{}] Remove subscription {} from subscriptions failed.", | ||
| topic, subscriptionName); | ||
| } | ||
| return null; | ||
| }); | ||
| } | ||
| } | ||
| this.messageDeduplication = new MessageDeduplication(brokerService.pulsar(), this, ledger); | ||
|
|
@@ -887,6 +899,13 @@ private CompletableFuture<Consumer> internalSubscribe(final TransportCnx cnx, St | |
| } else if (ex.getCause() instanceof BrokerServiceException.SubscriptionFencedException | ||
| && isCompactionSubscription(subscriptionName)) { | ||
| log.warn("[{}] Failed to create compaction subscription: {}", topic, ex.getMessage()); | ||
| } else if (ex.getCause() instanceof PendingAckHandleReplayException) { | ||
| PersistentSubscription subscription = subscriptions.remove(subscriptionName); | ||
| if (subscription != null) { | ||
| subscription.retryClose(); | ||
| } | ||
| log.warn("[{}] Failed to create subscription {} due to PendingAckHandle recover failed.", | ||
| topic, subscriptionName, ex); | ||
| } else { | ||
| log.error("[{}] Failed to create subscription: {}", topic, subscriptionName, ex); | ||
| } | ||
|
|
@@ -910,7 +929,8 @@ public CompletableFuture<Consumer> subscribe(final TransportCnx cnx, String subs | |
| replicatedSubscriptionStateArg, keySharedMeta, null, DEFAULT_CONSUMER_EPOCH); | ||
| } | ||
|
|
||
| private CompletableFuture<Subscription> getDurableSubscription(String subscriptionName, | ||
| @VisibleForTesting | ||
| public CompletableFuture<Subscription> getDurableSubscription(String subscriptionName, | ||
| InitialPosition initialPosition, long startMessageRollbackDurationSec, boolean replicated, | ||
| Map<String, String> subscriptionProperties) { | ||
| CompletableFuture<Subscription> subscriptionFuture = new CompletableFuture<>(); | ||
|
|
@@ -970,7 +990,8 @@ public void openCursorFailed(ManagedLedgerException exception, Object ctx) { | |
| return subscriptionFuture; | ||
| } | ||
|
|
||
| private CompletableFuture<? extends Subscription> getNonDurableSubscription(String subscriptionName, | ||
| @VisibleForTesting | ||
| public CompletableFuture<? extends Subscription> getNonDurableSubscription(String subscriptionName, | ||
| MessageId startMessageId, InitialPosition initialPosition, long startMessageRollbackDurationSec, | ||
| boolean isReadCompacted, Map<String, String> subscriptionProperties) { | ||
| log.info("[{}][{}] Creating non-durable subscription at msg id {} - {}", | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,25 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, | ||
| * software distributed under the License is distributed on an | ||
| * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
| * KIND, either express or implied. See the License for the | ||
| * specific language governing permissions and limitations | ||
| * under the License. | ||
| */ | ||
| package org.apache.pulsar.broker.transaction.pendingack.exceptions; | ||
|
|
||
| public class PendingAckHandleReplayException extends Exception { | ||
| public PendingAckHandleReplayException(Throwable t) { | ||
| super(t); | ||
| } | ||
| } |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
20 seconds ?
Probably we have to log at WARN level something like
log.warn("Re-scheduling closing of subscription {} in 20 seconds")