Skip to content

Commit

Permalink
Ensure deleting a partitioned-topic on a non existing namespace retur…
Browse files Browse the repository at this point in the history
…ns 404 (apache#7777)
  • Loading branch information
merlimat authored Aug 7, 2020
1 parent 3e13b9b commit d1070b7
Show file tree
Hide file tree
Showing 2 changed files with 59 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -613,7 +613,7 @@ protected void internalDeletePartitionedTopic(AsyncResponse asyncResponse, boole
return;
}
final CompletableFuture<Void> future = new CompletableFuture<>();
getPartitionedTopicMetadataAsync(topicName, authoritative, false).thenAccept(partitionMeta -> {
pulsar().getBrokerService().fetchPartitionedTopicMetadataAsync(topicName).thenAccept(partitionMeta -> {
final int numPartitions = partitionMeta.partitions;
if (numPartitions > 0) {
final AtomicInteger count = new AtomicInteger(numPartitions);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,7 @@
import org.apache.pulsar.broker.web.PulsarWebResource;
import org.apache.pulsar.broker.web.RestException;
import org.apache.pulsar.client.admin.PulsarAdminException;
import org.apache.pulsar.client.admin.PulsarAdminException.NotFoundException;
import org.apache.pulsar.client.api.Consumer;
import org.apache.pulsar.client.api.Producer;
import org.apache.pulsar.client.api.SubscriptionType;
Expand Down Expand Up @@ -1298,4 +1299,61 @@ private void mockWebUrl(URL localWebServiceUrl, NamespaceName namespace) throws
doReturn(true).when(nsSvc)
.isServiceUnitOwned(Mockito.argThat(bundle -> bundle.getNamespaceObject().equals(namespace)));
}

@Test
public void testDeleteNonPartitionedTopicMultipleTimes() throws Exception {
String namespace = this.testTenant + "/namespace-" + System.nanoTime();
String topic = namespace + "/topic";

admin.namespaces().createNamespace(namespace, Sets.newHashSet(testLocalCluster));

admin.topics().createNonPartitionedTopic(topic);

admin.topics().delete(topic);

try {
admin.topics().delete(topic);
fail("should have failed");
} catch (NotFoundException e) {
// Expected
}

admin.namespaces().deleteNamespace(namespace);

try {
admin.topics().delete(topic);
fail("should have failed");
} catch (NotFoundException e) {
// Expected
}
}

@Test
public void testDeletePartitionedTopicMultipleTimes() throws Exception {
String namespace = this.testTenant + "/namespace-" + System.nanoTime();
String topic = namespace + "/topic";

admin.namespaces().createNamespace(namespace, Sets.newHashSet(testLocalCluster));

admin.topics().createPartitionedTopic(topic, 3);
assertEquals(admin.topics().getPartitionedTopicMetadata(topic).partitions, 3);

admin.topics().deletePartitionedTopic(topic);

try {
admin.topics().deletePartitionedTopic(topic);
fail("should have failed");
} catch (NotFoundException e) {
// Expected
}

admin.namespaces().deleteNamespace(namespace);

try {
admin.topics().deletePartitionedTopic(topic);
fail("should have failed");
} catch (NotFoundException e) {
// Expected
}
}
}

0 comments on commit d1070b7

Please sign in to comment.