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

Fix create consumer on partitioned topic while disable topic auto creation. #5572

Merged
merged 6 commits into from
Jan 9, 2020
Merged
Show file tree
Hide file tree
Changes from 2 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 @@ -692,6 +692,10 @@ protected void handleSubscribe(final CommandSubscribe subscribe) {
boolean createTopicIfDoesNotExist = forceTopicCreation
&& service.pulsar().getConfig().isAllowAutoTopicCreation();

if (topicName.isPartitioned()) {
createTopicIfDoesNotExist = true;
}

service.getTopic(topicName.toString(), createTopicIfDoesNotExist)
.thenCompose(optTopic -> {
if (!optTopic.isPresent()) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,78 @@
/**
* 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.client.api;

import org.apache.pulsar.client.admin.PulsarAdminException;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;

public class ConsumerCreationTest extends ProducerConsumerBase {

@BeforeClass
@Override
protected void setup() throws Exception {
conf.setManagedLedgerCacheEvictionFrequency(0.1);
super.internalSetup();
super.producerBaseSetup();
}

@AfterClass
@Override
protected void cleanup() throws Exception {
super.internalCleanup();
}

@Test
public void testCreateConsumerForPartitionedTopicWhenDisableTopicAutoCreation() throws PulsarAdminException, PulsarClientException {
conf.setAllowAutoTopicCreation(false);
final String topic = "testCreateConsumerWhenDisableTopicAutoCreation";
admin.topics().createPartitionedTopic(topic, 3);
Assert.assertNotNull(pulsarClient.newConsumer().topic(topic).subscriptionName("sub-1").subscribe());
}

@Test
public void testCreateConsumerForNonPartitionedTopicWhenDisableTopicAutoCreation() throws PulsarClientException {
conf.setAllowAutoTopicCreation(false);
final String topic = "testCreateConsumerForNonPartitionedTopicWhenDisableTopicAutoCreation";
try {
pulsarClient.newConsumer().topic(topic).subscriptionName("sub-1").subscribe();
Assert.fail("should be failed");
} catch (PulsarClientException.TopicDoesNotExistException e) {
//ok
}
}

@Test
public void testCreateConsumerForPartitionedTopicWhenEnableTopicAutoCreation() throws PulsarAdminException, PulsarClientException {
conf.setAllowAutoTopicCreation(true);
final String topic = "testCreateConsumerForPartitionedTopicWhenEnableTopicAutoCreation";
admin.topics().createPartitionedTopic(topic, 3);
Assert.assertNotNull(pulsarClient.newConsumer().topic(topic).subscriptionName("sub-1").subscribe());
}

@Test
public void testCreateConsumerForNonPartitionedTopicWhenEnableTopicAutoCreation() throws PulsarClientException {
conf.setAllowAutoTopicCreation(true);
final String topic = "testCreateConsumerForNonPartitionedTopicWhenEnableTopicAutoCreation";
Assert.assertNotNull(pulsarClient.newConsumer().topic(topic).subscriptionName("sub-1").subscribe());
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -644,6 +644,8 @@ public static PulsarClientException unwrap(Throwable t) {
return new ChecksumException(msg);
} else if (cause instanceof CryptoException) {
return new CryptoException(msg);
} else if (cause instanceof TopicDoesNotExistException) {
return new TopicDoesNotExistException(msg);
} else {
return new PulsarClientException(t);
}
Expand Down