-
Notifications
You must be signed in to change notification settings - Fork 3.6k
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
[fix][broker] Consumer stuck when delete subscription __compaction failed #23980
base: master
Are you sure you want to change the base?
Changes from all commits
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 |
---|---|---|
|
@@ -243,8 +243,9 @@ public static boolean isDedupCursorName(String name) { | |
protected final MessageDeduplication messageDeduplication; | ||
|
||
private static final Long COMPACTION_NEVER_RUN = -0xfebecffeL; | ||
private volatile CompletableFuture<Long> currentCompaction = CompletableFuture.completedFuture( | ||
volatile CompletableFuture<Long> currentCompaction = CompletableFuture.completedFuture( | ||
COMPACTION_NEVER_RUN); | ||
final AtomicBoolean disablingCompaction = new AtomicBoolean(false); | ||
private TopicCompactionService topicCompactionService; | ||
|
||
// TODO: Create compaction strategy from topic policy when exposing strategic compaction to users. | ||
|
@@ -1340,18 +1341,42 @@ private void asyncDeleteCursorWithCleanCompactionLedger(PersistentSubscription s | |
return; | ||
} | ||
|
||
currentCompaction.handle((__, e) -> { | ||
if (e != null) { | ||
log.warn("[{}][{}] Last compaction task failed", topic, subscriptionName); | ||
// Avoid concurrently execute compaction and unsubscribing. | ||
synchronized (this) { | ||
if (!disablingCompaction.compareAndSet(false, true)) { | ||
unsubscribeFuture.completeExceptionally( | ||
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. this should be protected with 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. Could you mention the code line? I could not find it. |
||
new SubscriptionBusyException("the subscription is deleting by another task")); | ||
return; | ||
} | ||
return ((PulsarCompactorSubscription) subscription).cleanCompactedLedger(); | ||
}).whenComplete((__, ex) -> { | ||
if (ex != null) { | ||
log.error("[{}][{}] Error cleaning compacted ledger", topic, subscriptionName, ex); | ||
unsubscribeFuture.completeExceptionally(ex); | ||
} | ||
// Unsubscribe compaction cursor and delete compacted ledger. | ||
currentCompaction.thenCompose(__ -> { | ||
asyncDeleteCursor(subscriptionName, unsubscribeFuture); | ||
return unsubscribeFuture; | ||
}).thenAccept(__ -> { | ||
try { | ||
((PulsarCompactorSubscription) subscription).cleanCompactedLedger(); | ||
} catch (Exception ex) { | ||
Long compactedLedger = null; | ||
Optional<CompactedTopicContext> compactedTopicContext = getCompactedTopicContext(); | ||
if (compactedTopicContext.isPresent() && compactedTopicContext.get().getLedger() != null) { | ||
compactedLedger = compactedTopicContext.get().getLedger().getId(); | ||
} | ||
log.error("[{}][{}][{}] Error cleaning compacted ledger", topic, subscriptionName, compactedLedger, ex); | ||
} finally { | ||
// Reset the variable: disablingCompaction, | ||
disablingCompaction.compareAndSet(true, false); | ||
} | ||
}).exceptionally(ex -> { | ||
if (currentCompaction.isCompletedExceptionally()) { | ||
log.warn("[{}][{}] Last compaction task failed", topic, subscriptionName); | ||
} else { | ||
asyncDeleteCursor(subscriptionName, unsubscribeFuture); | ||
log.warn("[{}][{}] Failed to delete cursor task failed", topic, subscriptionName); | ||
} | ||
// Reset the variable: disablingCompaction, | ||
disablingCompaction.compareAndSet(true, false); | ||
unsubscribeFuture.completeExceptionally(ex); | ||
return null; | ||
}); | ||
} | ||
|
||
|
@@ -3934,6 +3959,10 @@ public synchronized void triggerCompaction() | |
log.info("[{}] Topic is closing or deleting, skip triggering compaction", topic); | ||
return; | ||
} | ||
if (disablingCompaction.get()) { | ||
log.info("[{}] Compaction is disabling, skip triggering compaction", topic); | ||
return; | ||
} | ||
|
||
if (strategicCompactionMap.containsKey(topic)) { | ||
currentCompaction = brokerService.pulsar().getStrategicCompactor() | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,151 @@ | ||
/* | ||
* 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.service.persistent; | ||
|
||
import static org.testng.Assert.assertEquals; | ||
import static org.testng.Assert.assertTrue; | ||
import static org.testng.Assert.fail; | ||
import java.util.concurrent.CompletableFuture; | ||
import java.util.concurrent.atomic.AtomicBoolean; | ||
import java.util.concurrent.atomic.AtomicInteger; | ||
import java.util.concurrent.atomic.AtomicReference; | ||
import org.apache.bookkeeper.mledger.Position; | ||
import org.apache.pulsar.broker.BrokerTestUtil; | ||
import org.apache.pulsar.client.admin.PulsarAdminException; | ||
import org.apache.pulsar.client.api.MessageId; | ||
import org.apache.pulsar.client.api.ProducerConsumerBase; | ||
import org.apache.pulsar.client.api.Schema; | ||
import org.apache.pulsar.common.naming.TopicName; | ||
import org.apache.pulsar.common.util.FutureUtil; | ||
import org.apache.pulsar.compaction.Compactor; | ||
import org.apache.zookeeper.MockZooKeeper; | ||
import org.awaitility.Awaitility; | ||
import org.awaitility.reflect.WhiteboxImpl; | ||
import org.testng.annotations.AfterClass; | ||
import org.testng.annotations.BeforeClass; | ||
import org.testng.annotations.Test; | ||
|
||
@Test(groups = "broker") | ||
public class CompactionConcurrencyTest extends ProducerConsumerBase { | ||
|
||
@BeforeClass | ||
@Override | ||
protected void setup() throws Exception { | ||
super.internalSetup(); | ||
super.producerBaseSetup(); | ||
} | ||
|
||
@AfterClass | ||
@Override | ||
protected void cleanup() throws Exception { | ||
super.internalCleanup(); | ||
} | ||
|
||
@Override | ||
protected void doInitConf() throws Exception { | ||
super.doInitConf(); | ||
// Disable the scheduled task: compaction. | ||
conf.setBrokerServiceCompactionMonitorIntervalInSeconds(Integer.MAX_VALUE); | ||
// Disable the scheduled task: retention. | ||
conf.setRetentionCheckIntervalInSeconds(Integer.MAX_VALUE); | ||
} | ||
|
||
private void triggerCompactionAndWait(String topicName) throws Exception { | ||
PersistentTopic persistentTopic = | ||
(PersistentTopic) pulsar.getBrokerService().getTopic(topicName, false).get().get(); | ||
persistentTopic.triggerCompaction(); | ||
Awaitility.await().untilAsserted(() -> { | ||
Position lastConfirmPos = persistentTopic.getManagedLedger().getLastConfirmedEntry(); | ||
Position markDeletePos = persistentTopic | ||
.getSubscription(Compactor.COMPACTION_SUBSCRIPTION).getCursor().getMarkDeletedPosition(); | ||
assertEquals(markDeletePos.getLedgerId(), lastConfirmPos.getLedgerId()); | ||
assertEquals(markDeletePos.getEntryId(), lastConfirmPos.getEntryId()); | ||
}); | ||
} | ||
|
||
@Test | ||
public void testDisableCompactionConcurrently() throws Exception { | ||
String topicName = "persistent://public/default/" + BrokerTestUtil.newUniqueName("tp"); | ||
admin.topics().createNonPartitionedTopic(topicName); | ||
admin.topicPolicies().setCompactionThreshold(topicName, 1); | ||
admin.topics().createSubscription(topicName, "s1", MessageId.earliest); | ||
var producer = pulsarClient.newProducer(Schema.STRING).topic(topicName).enableBatching(false).create(); | ||
producer.newMessage().key("k0").value("v0").send(); | ||
triggerCompactionAndWait(topicName); | ||
admin.topics().deleteSubscription(topicName, "s1"); | ||
PersistentTopic persistentTopic = | ||
(PersistentTopic) pulsar.getBrokerService().getTopic(topicName, false).get().get(); | ||
AtomicBoolean disablingCompaction = persistentTopic.disablingCompaction; | ||
|
||
// Disable compaction. | ||
// Inject a delay when the first time of deleting cursor. | ||
AtomicInteger times = new AtomicInteger(); | ||
String cursorPath = String.format("/managed-ledgers/%s/__compaction", | ||
TopicName.get(topicName).getPersistenceNamingEncoding()); | ||
admin.topicPolicies().removeCompactionThreshold(topicName); | ||
mockZooKeeper.delay(5000, (op, path) -> { | ||
return op == MockZooKeeper.Op.DELETE && cursorPath.equals(path) && times.incrementAndGet() == 1; | ||
}); | ||
mockZooKeeperGlobal.delay(5000, (op, path) -> { | ||
return op == MockZooKeeper.Op.DELETE && cursorPath.equals(path) && times.incrementAndGet() == 1; | ||
}); | ||
AtomicReference<CompletableFuture<Void>> f1 = new AtomicReference<CompletableFuture<Void>>(); | ||
AtomicReference<CompletableFuture<Void>> f2 = new AtomicReference<CompletableFuture<Void>>(); | ||
new Thread(() -> { | ||
f1.set(admin.topics().deleteSubscriptionAsync(topicName, "__compaction")); | ||
}).start(); | ||
new Thread(() -> { | ||
f2.set(admin.topics().deleteSubscriptionAsync(topicName, "__compaction")); | ||
}).start(); | ||
|
||
// Verify: the next compaction will be skipped. | ||
Awaitility.await().untilAsserted(() -> { | ||
assertTrue(disablingCompaction.get()); | ||
}); | ||
producer.newMessage().key("k1").value("v1").send(); | ||
producer.newMessage().key("k2").value("v2").send(); | ||
CompletableFuture<Long> currentCompaction1 = persistentTopic.currentCompaction; | ||
WhiteboxImpl.getInternalState(persistentTopic, "currentCompaction"); | ||
persistentTopic.triggerCompaction(); | ||
CompletableFuture<Long> currentCompaction2 = persistentTopic.currentCompaction; | ||
assertTrue(currentCompaction1 == currentCompaction2); | ||
|
||
// Verify: one of the requests should fail. | ||
Awaitility.await().untilAsserted(() -> { | ||
assertTrue(f1.get() != null); | ||
assertTrue(f2.get() != null); | ||
assertTrue(f1.get().isDone()); | ||
assertTrue(f2.get().isDone()); | ||
assertTrue(f1.get().isCompletedExceptionally() || f2.get().isCompletedExceptionally()); | ||
assertTrue(!f1.get().isCompletedExceptionally() || !f2.get().isCompletedExceptionally()); | ||
}); | ||
try { | ||
f1.get().join(); | ||
f2.get().join(); | ||
fail("Should fail"); | ||
} catch (Exception ex) { | ||
Throwable actEx = FutureUtil.unwrapCompletionException(ex); | ||
assertTrue(actEx instanceof PulsarAdminException.PreconditionFailedException); | ||
} | ||
|
||
// cleanup. | ||
producer.close(); | ||
admin.topics().delete(topicName, false); | ||
} | ||
} |
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.
Is there really a need for a synchronized block here?
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.
Yes, it is. It prevents the modifying of the variable
currentCompaction
.