Skip to content
This repository was archived by the owner on Jan 24, 2024. It is now read-only.
Closed
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 @@ -34,10 +34,13 @@
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;

@Slf4j
class AdminManager {
Expand All @@ -49,9 +52,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 @@ -87,9 +92,24 @@ CompletableFuture<Map<String, ApiError>> createTopicsAsync(Map<String, TopicDeta
kopTopic = new KopTopic(topic);
} catch (RuntimeException e) {
errorFuture.complete(ApiError.fromThrowable(e));
if (numTopics.decrementAndGet() == 0) {
complete.run();
}
return;
}
admin.topics().createPartitionedTopicAsync(kopTopic.getFullName(), detail.numPartitions)
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(), numPartitions)
.whenComplete((ignored, e) -> {
if (e == null) {
if (log.isDebugEnabled()) {
Expand All @@ -98,13 +118,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 Expand Up @@ -175,4 +197,20 @@ CompletableFuture<Map<ConfigResource, DescribeConfigsResponse.Config>> describeC
});
return resultFuture;
}

public Map<String, Errors> deleteTopics(Set<String> topicsToDelete) {
Map<String, Errors> result = new ConcurrentHashMap<>();
topicsToDelete.forEach(topic -> {
try {
String topicFullName = new KopTopic(topic).getFullName();
admin.topics().deletePartitionedTopic(topicFullName);
result.put(topic, Errors.NONE);
log.info("delete topic {} successfully.", topicFullName);
} catch (PulsarAdminException e) {
log.error("delete topic {} failed, exception: ", topic, e);
result.put(topic, Errors.UNKNOWN_TOPIC_OR_PARTITION);
}
});
return result;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -235,6 +235,9 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception
case DESCRIBE_CONFIGS:
handleDescribeConfigs(kafkaHeaderAndRequest, responseFuture);
break;
case DELETE_TOPICS:
handleDeleteTopics(kafkaHeaderAndRequest, responseFuture);
break;
default:
handleError(kafkaHeaderAndRequest, responseFuture);
}
Expand Down Expand Up @@ -372,6 +375,9 @@ protected void writeAndFlushWhenInactiveChannel(Channel channel) {
protected abstract void
handleDescribeConfigs(KafkaHeaderAndRequest kafkaHeaderAndRequest, CompletableFuture<AbstractResponse> response);

protected abstract void
handleDeleteTopics(KafkaHeaderAndRequest kafkaHeaderAndRequest, CompletableFuture<AbstractResponse> response);

static class KafkaHeaderAndRequest implements Closeable {

private static final String DEFAULT_CLIENT_HOST = "";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -241,7 +241,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 @@ -77,6 +77,8 @@
import org.apache.kafka.common.requests.CreateTopicsResponse;
import org.apache.kafka.common.requests.DeleteGroupsRequest;
import org.apache.kafka.common.requests.DeleteGroupsResponse;
import org.apache.kafka.common.requests.DeleteTopicsRequest;
import org.apache.kafka.common.requests.DeleteTopicsResponse;
import org.apache.kafka.common.requests.DescribeConfigsRequest;
import org.apache.kafka.common.requests.DescribeConfigsResponse;
import org.apache.kafka.common.requests.DescribeGroupsRequest;
Expand Down Expand Up @@ -1207,6 +1209,15 @@ protected void handleDescribeConfigs(KafkaHeaderAndRequest describeConfigs,
});
}

@Override
protected void handleDeleteTopics(KafkaHeaderAndRequest deleteTopics,
CompletableFuture<AbstractResponse> resultFuture) {
checkArgument(deleteTopics.getRequest() instanceof DeleteTopicsRequest);
DeleteTopicsRequest request = (DeleteTopicsRequest) deleteTopics.getRequest();
Set<String> topicsToDelete = request.topics();
resultFuture.complete(new DeleteTopicsResponse(adminManager.deleteTopics(topicsToDelete)));
}

private SaslHandshakeResponse checkSaslMechanism(String mechanism) {
if (getKafkaConfig().getSaslAllowedMechanisms().contains(mechanism)) {
return new SaslHandshakeResponse(Errors.NONE, getKafkaConfig().getSaslAllowedMechanisms());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -141,7 +141,7 @@ protected void setup() throws Exception {
ProtocolHandler handler = pulsar.getProtocolHandlers().protocol("kafka");
GroupCoordinator groupCoordinator = ((KafkaProtocolHandler) handler).getGroupCoordinator();

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 @@ -39,6 +39,7 @@
import java.util.HashSet;
import java.util.Map;
import java.util.Properties;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
Expand All @@ -57,13 +58,15 @@
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.config.ConfigResource;
import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.errors.InvalidRequestException;
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.ApiVersionsRequest;
import org.apache.kafka.common.requests.ApiVersionsResponse;
import org.apache.kafka.common.requests.CreateTopicsRequest;
import org.apache.kafka.common.requests.MetadataResponse.PartitionMetadata;
import org.apache.kafka.common.requests.RequestHeader;
import org.apache.kafka.common.requests.ResponseHeader;
Expand Down Expand Up @@ -96,6 +99,7 @@ public class KafkaRequestHandlerTest extends KopProtocolHandlerTestBase {
@BeforeMethod
@Override
protected void setup() throws Exception {
conf.setDefaultNumPartitions(2);
super.internalSetup();
log.info("success internal setup");

Expand Down Expand Up @@ -137,7 +141,7 @@ protected void setup() throws Exception {
ProtocolHandler handler1 = pulsar.getProtocolHandlers().protocol("kafka");
GroupCoordinator groupCoordinator = ((KafkaProtocolHandler) handler1).getGroupCoordinator();

adminManager = new AdminManager(pulsar.getAdminClient());
adminManager = new AdminManager(pulsar.getAdminClient(), conf);
handler = new KafkaRequestHandler(
pulsar,
(KafkaServiceConfiguration) conf,
Expand Down Expand Up @@ -316,7 +320,7 @@ public void testGetKafkaTopicNameFromPulsarTopicName() {
assertEquals(localName, getKafkaTopicNameFromPulsarTopicname(topicNamePartition));
}

void createTopicsByKafkaAdmin(AdminClient admin, Map<String, Integer> topicToNumPartitions)
private void createTopicsByKafkaAdmin(AdminClient admin, Map<String, Integer> topicToNumPartitions)
throws ExecutionException, InterruptedException {
final short replicationFactor = 1; // replication factor will be ignored
admin.createTopics(topicToNumPartitions.entrySet().stream().map(entry -> {
Expand All @@ -326,7 +330,7 @@ void createTopicsByKafkaAdmin(AdminClient admin, Map<String, Integer> topicToNum
}).collect(Collectors.toList())).all().get();
}

void verifyTopicsByPulsarAdmin(Map<String, Integer> topicToNumPartitions)
private void verifyTopicsCreatedByPulsarAdmin(Map<String, Integer> topicToNumPartitions)
throws PulsarAdminException {
for (Map.Entry<String, Integer> entry : topicToNumPartitions.entrySet()) {
final String topic = entry.getKey();
Expand All @@ -335,8 +339,22 @@ void verifyTopicsByPulsarAdmin(Map<String, Integer> topicToNumPartitions)
}
}

private void verifyTopicsDeletedByPulsarAdmin(Map<String, Integer> topicToNumPartitions)
throws PulsarAdminException {
for (Map.Entry<String, Integer> entry : topicToNumPartitions.entrySet()) {
final String topic = entry.getKey();
assertEquals(this.admin.topics().getPartitionedTopicMetadata(topic).partitions, 0);
}
}

private void deleteTopicsByKafkaAdmin(AdminClient admin, Set<String> topicsToDelete)
throws ExecutionException, InterruptedException {
admin.deleteTopics(topicsToDelete).all().get();
}


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

Expand All @@ -348,28 +366,96 @@ public void testCreateTopics() throws Exception {
put("my-tenant/my-ns/testCreateTopics-2", 1);
put("persistent://my-tenant/my-ns/testCreateTopics-3", 5);
}};
// create
createTopicsByKafkaAdmin(kafkaAdmin, topicToNumPartitions);
verifyTopicsByPulsarAdmin(topicToNumPartitions);
verifyTopicsCreatedByPulsarAdmin(topicToNumPartitions);
// delete
deleteTopicsByKafkaAdmin(kafkaAdmin, topicToNumPartitions.keySet());
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());
final Throwable cause = e.getCause();
assertTrue(cause instanceof TimeoutException || cause instanceof UnknownServerException);
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());
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();
props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:" + getKafkaBrokerPort());

@Cleanup
AdminClient kafkaAdmin = AdminClient.create(props);
Set<String> topics = new HashSet<>();
topics.add("testDeleteNotExistedTopics");
try {
deleteTopicsByKafkaAdmin(kafkaAdmin, topics);
fail();
} catch (ExecutionException e) {
assertTrue(e.getCause() instanceof UnknownTopicOrPartitionException);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ protected void setup() throws Exception {
ProtocolHandler handler = pulsar.getProtocolHandlers().protocol("kafka");
GroupCoordinator groupCoordinator = ((KafkaProtocolHandler) handler).getGroupCoordinator();

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