Skip to content
This repository was archived by the owner on Jan 24, 2024. It is now read-only.
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 @@ -16,6 +16,7 @@
import static io.streamnative.pulsar.handlers.kop.utils.delayed.DelayedOperationKey.TopicKey;
import static org.apache.kafka.common.requests.CreateTopicsRequest.TopicDetails;

import io.streamnative.pulsar.handlers.kop.exceptions.KoPTopicException;
import io.streamnative.pulsar.handlers.kop.utils.KopTopic;
import io.streamnative.pulsar.handlers.kop.utils.delayed.DelayedOperation;
import io.streamnative.pulsar.handlers.kop.utils.delayed.DelayedOperationPurgatory;
Expand All @@ -32,8 +33,10 @@
import lombok.extern.slf4j.Slf4j;
import org.apache.kafka.common.config.ConfigResource;
import org.apache.kafka.common.errors.InvalidRequestException;
import org.apache.kafka.common.errors.TopicExistsException;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.requests.ApiError;
import org.apache.kafka.common.requests.CreateTopicsRequest;
import org.apache.kafka.common.requests.DescribeConfigsResponse;
import org.apache.pulsar.client.admin.PulsarAdmin;
import org.apache.pulsar.client.admin.PulsarAdminException;
Expand All @@ -48,9 +51,11 @@ class AdminManager {
.build();

private final PulsarAdmin admin;
private final int defaultNumPartitions;

AdminManager(PulsarAdmin admin) {
public AdminManager(PulsarAdmin admin, KafkaServiceConfiguration conf) {
this.admin = admin;
this.defaultNumPartitions = conf.getDefaultNumPartitions();
}

public void shutdown() {
Expand Down Expand Up @@ -84,11 +89,26 @@ CompletableFuture<Map<String, ApiError>> createTopicsAsync(Map<String, TopicDeta
KopTopic kopTopic;
try {
kopTopic = new KopTopic(topic);
} catch (RuntimeException e) {
} catch (KoPTopicException e) {
errorFuture.complete(ApiError.fromThrowable(e));
if (numTopics.decrementAndGet() == 0) {
complete.run();
}
return;
}
int numPartitions = detail.numPartitions;
if (numPartitions == CreateTopicsRequest.NO_NUM_PARTITIONS) {
numPartitions = defaultNumPartitions;
}
if (numPartitions < 0) {
errorFuture.complete(ApiError.fromThrowable(
new InvalidRequestException("The partition '" + numPartitions + "' is negative")));
if (numTopics.decrementAndGet() == 0) {
complete.run();
}
return;
}
admin.topics().createPartitionedTopicAsync(kopTopic.getFullName(), detail.numPartitions)
admin.topics().createPartitionedTopicAsync(kopTopic.getFullName(), numPartitions)
.whenComplete((ignored, e) -> {
if (e == null) {
if (log.isDebugEnabled()) {
Expand All @@ -97,13 +117,15 @@ CompletableFuture<Map<String, ApiError>> createTopicsAsync(Map<String, TopicDeta
} else {
log.error("Failed to create topic '{}': {}", topic, e);
}

int restNumTopics = numTopics.decrementAndGet();
if (restNumTopics < 0) {
return;
if (e == null) {
errorFuture.complete(ApiError.NONE);
} else if (e instanceof PulsarAdminException.ConflictException) {
errorFuture.complete(ApiError.fromThrowable(
new TopicExistsException("Topic '" + topic + "' already exists.")));
} else {
errorFuture.complete(ApiError.fromThrowable(e));
}
errorFuture.complete((e == null) ? ApiError.NONE : ApiError.fromThrowable(e));
if (restNumTopics == 0) {
if (numTopics.decrementAndGet() == 0) {
complete.run();
}
});
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -262,7 +262,7 @@ public void start(BrokerService service) {
KopVersion.getBuildTime());

try {
adminManager = new AdminManager(brokerService.getPulsar().getAdminClient());
adminManager = new AdminManager(brokerService.getPulsar().getAdminClient(), kafkaConfig);
} catch (PulsarServerException e) {
log.error("Failed to create PulsarAdmin: {}", e.getMessage());
throw new IllegalStateException(e);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ protected void setup() throws Exception {
GroupCoordinator groupCoordinator = ((KafkaProtocolHandler) handler).getGroupCoordinator();
TransactionCoordinator transactionCoordinator = ((KafkaProtocolHandler) handler).getTransactionCoordinator();

adminManager = new AdminManager(pulsar.getAdminClient());
adminManager = new AdminManager(pulsar.getAdminClient(), conf);
kafkaRequestHandler = new KafkaRequestHandler(
pulsar,
(KafkaServiceConfiguration) conf,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -119,7 +119,7 @@ protected void setup() throws Exception {
GroupCoordinator groupCoordinator = ((KafkaProtocolHandler) handler).getGroupCoordinator();
TransactionCoordinator transactionCoordinator = ((KafkaProtocolHandler) handler).getTransactionCoordinator();

adminManager = new AdminManager(pulsar.getAdminClient());
adminManager = new AdminManager(pulsar.getAdminClient(), conf);
kafkaRequestHandler = new KafkaRequestHandler(
pulsar,
(KafkaServiceConfiguration) conf,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,15 +64,17 @@
import org.apache.kafka.common.Node;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.config.ConfigResource;
import org.apache.kafka.common.errors.InvalidRequestException;
import org.apache.kafka.common.errors.InvalidTopicException;
import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.errors.TopicExistsException;
import org.apache.kafka.common.errors.UnknownServerException;
import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.requests.AbstractResponse;
import org.apache.kafka.common.requests.ApiVersionsRequest;
import org.apache.kafka.common.requests.ApiVersionsResponse;
import org.apache.kafka.common.requests.CreateTopicsRequest;
import org.apache.kafka.common.requests.IsolationLevel;
import org.apache.kafka.common.requests.ListOffsetRequest;
import org.apache.kafka.common.requests.ListOffsetResponse;
Expand Down Expand Up @@ -114,6 +116,7 @@ public static Object[][] metadataVersions() {
@BeforeClass
@Override
protected void setup() throws Exception {
conf.setDefaultNumPartitions(2);
super.internalSetup();
log.info("success internal setup");

Expand All @@ -137,7 +140,7 @@ protected void setup() throws Exception {
GroupCoordinator groupCoordinator = ((KafkaProtocolHandler) handler1).getGroupCoordinator();
TransactionCoordinator transactionCoordinator = ((KafkaProtocolHandler) handler1).getTransactionCoordinator();

adminManager = new AdminManager(pulsar.getAdminClient());
adminManager = new AdminManager(pulsar.getAdminClient(), conf);
handler = new KafkaRequestHandler(
pulsar,
(KafkaServiceConfiguration) conf,
Expand Down Expand Up @@ -350,27 +353,74 @@ public void testCreateAndDeleteTopics() throws Exception {
verifyTopicsDeletedByPulsarAdmin(topicToNumPartitions);
}

@Test(timeOut = 10000)
@Test(timeOut = 20000)
public void testCreateInvalidTopics() {
Properties props = new Properties();
props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:" + getKafkaBrokerPort());
props.put(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, 5000);

@Cleanup
AdminClient kafkaAdmin = AdminClient.create(props);
Map<String, Integer> topicToNumPartitions = new HashMap<String, Integer>(){{
put("xxx/testCreateInvalidTopics-0", 1);
}};
Map<String, Integer> topicToNumPartitions = Collections.singletonMap("xxx/testCreateInvalidTopics-0", 1);
try {
createTopicsByKafkaAdmin(kafkaAdmin, topicToNumPartitions);
fail("create a invalid topic should fail");
} catch (Exception e) {
log.info("Failed to create topics: {} caused by {}", topicToNumPartitions, e.getCause());
assertTrue(e.getCause() instanceof UnknownServerException);
}
topicToNumPartitions = Collections.singletonMap("testCreateInvalidTopics-1", -1234);
try {
createTopicsByKafkaAdmin(kafkaAdmin, topicToNumPartitions);
fail("create a invalid topic should fail");
} catch (Exception e) {
log.info("Failed to create topics: {} caused by {}", topicToNumPartitions, e.getCause());
final Throwable cause = e.getCause();
assertTrue(cause instanceof TimeoutException || cause instanceof UnknownServerException);
assertTrue(e.getCause() instanceof InvalidRequestException);
}
}

@Test(timeOut = 10000)
public void testCreateExistedTopic() {
Properties props = new Properties();
props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:" + getKafkaBrokerPort());
props.put(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, 5000);

@Cleanup
AdminClient kafkaAdmin = AdminClient.create(props);
final Map<String, Integer> topicToNumPartitions = Collections.singletonMap("testCreatedExistedTopic", 1);
try {
createTopicsByKafkaAdmin(kafkaAdmin, topicToNumPartitions);
} catch (ExecutionException | InterruptedException e) {
fail(e.getMessage());
}
try {
createTopicsByKafkaAdmin(kafkaAdmin, topicToNumPartitions);
fail("Create the existed topic should fail");
} catch (ExecutionException e) {
log.info("Failed to create existed topic: {}", e.getMessage());
assertTrue(e.getCause() instanceof TopicExistsException);
} catch (InterruptedException e) {
fail(e.getMessage());
}
}

@Test(timeOut = 10000)
public void testCreateTopicWithDefaultPartitions() throws Exception {
Properties props = new Properties();
props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:" + getKafkaBrokerPort());
props.put(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, 5000);

final String topic = "testCreatedTopicWithDefaultPartitions";

@Cleanup
AdminClient kafkaAdmin = AdminClient.create(props);
final Map<String, Integer> topicToNumPartitions = Collections.singletonMap(
topic,
CreateTopicsRequest.NO_NUM_PARTITIONS);
createTopicsByKafkaAdmin(kafkaAdmin, topicToNumPartitions);
assertEquals(admin.topics().getPartitionedTopicMetadata(topic).partitions, conf.getDefaultNumPartitions());
}

@Test(timeOut = 10000)
public void testDeleteNotExistedTopics() throws Exception {
Properties props = new Properties();
Expand Down Expand Up @@ -464,6 +514,8 @@ public void testProduceCallback() throws Exception {
final int numMessages = 10;
final String messagePrefix = "msg-";

admin.topics().createPartitionedTopic(topic, 1);

final Properties props = new Properties();
props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:" + getKafkaBrokerPort());
props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -68,7 +68,7 @@ protected void setup() throws Exception {
GroupCoordinator groupCoordinator = ((KafkaProtocolHandler) handler).getGroupCoordinator();
TransactionCoordinator transactionCoordinator = ((KafkaProtocolHandler) handler).getTransactionCoordinator();

adminManager = new AdminManager(pulsar.getAdminClient());
adminManager = new AdminManager(pulsar.getAdminClient(), conf);
kafkaRequestHandler = new KafkaRequestHandler(
pulsar,
(KafkaServiceConfiguration) conf,
Expand Down