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] clean up topic that failed to unload from the cache #8968

Merged
merged 1 commit into from
Dec 16, 2020
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -268,8 +268,8 @@ public CompletableFuture<Optional<URL>> getWebServiceUrlAsync(ServiceUnitId suNa
* If the service unit is not owned, return an empty optional
*/
public Optional<URL> getWebServiceUrl(ServiceUnitId suName, LookupOptions options) throws Exception {
return getWebServiceUrlAsync(suName, options).get(
pulsar.getConfiguration().getZooKeeperOperationTimeoutSeconds(), SECONDS);
return getWebServiceUrlAsync(suName, options)
.get(pulsar.getConfiguration().getZooKeeperOperationTimeoutSeconds(), SECONDS);
}

private CompletableFuture<Optional<URL>> internalGetWebServiceUrl(NamespaceBundle bundle, LookupOptions options) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -132,6 +132,8 @@ public CompletableFuture<Void> handleUnloadRequest(PulsarService pulsar, long ti
} else {
unloadedTopics.set(numUnloadedTopics);
}
// clean up topics that failed to unload from the broker ownership cache
pulsar.getBrokerService().cleanUnloadedTopicFromCache(bundle);
return null;
})
.thenCompose(v -> {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1569,6 +1569,15 @@ private CompletableFuture<Integer> unloadServiceUnit(NamespaceBundle serviceUnit
return FutureUtil.waitForAll(closeFutures).thenApply(v -> closeFutures.size());
}

public void cleanUnloadedTopicFromCache(NamespaceBundle serviceUnit) {
topics.forEach((name, topicFuture) -> {
TopicName topicName = TopicName.get(name);
if (serviceUnit.includes(topicName)) {
pulsar.getBrokerService().removeTopicFromCache(topicName.toString());
}
});
}

public AuthorizationService getAuthorizationService() {
return authorizationService;
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,87 @@
/**
* 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.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.doNothing;
import static org.mockito.Mockito.mock;
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertNotNull;

import java.lang.reflect.Field;
import java.util.concurrent.TimeUnit;

import org.apache.bookkeeper.client.LedgerHandle;
import org.apache.bookkeeper.mledger.ManagedLedger;
import org.apache.pulsar.broker.service.BrokerTestBase;
import org.apache.pulsar.client.api.MessageRoutingMode;
import org.apache.pulsar.client.api.Producer;
import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.naming.TopicName;
import org.testng.annotations.AfterMethod;
import org.testng.annotations.BeforeMethod;
import org.testng.annotations.Test;

/**
*/
public class PersistentTopicTest extends BrokerTestBase {
@BeforeMethod
@Override
protected void setup() throws Exception {
super.baseSetup();
}

@AfterMethod(alwaysRun = true)
@Override
protected void cleanup() throws Exception {
super.internalCleanup();
}

/**
* Test validates that broker cleans up topic which failed to unload while bundle unloading.
*
* @throws Exception
*/
@Test
public void testCleanFailedUnloadTopic() throws Exception {
final String topicName = "persistent://prop/ns-abc/failedUnload";

// 1. producer connect
Producer<byte[]> producer = pulsarClient.newProducer().topic(topicName).enableBatching(false)
.messageRoutingMode(MessageRoutingMode.SinglePartition).create();

PersistentTopic topicRef = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topicName).get();
assertNotNull(topicRef);

ManagedLedger ml = topicRef.ledger;
LedgerHandle ledger = mock(LedgerHandle.class);
Field handleField = ml.getClass().getDeclaredField("currentLedger");
handleField.setAccessible(true);
handleField.set(ml, ledger);
doNothing().when(ledger).asyncClose(any(), any());

NamespaceBundle bundle = pulsar.getNamespaceService().getBundle(TopicName.get(topicName));
pulsar.getNamespaceService().unloadNamespaceBundle(bundle, 5, TimeUnit.SECONDS).get();

retryStrategically((test) -> !pulsar.getBrokerService().getTopicReference(topicName).isPresent(), 5, 500);
assertFalse(pulsar.getBrokerService().getTopicReference(topicName).isPresent());

producer.close();
}
}