Skip to content

Commit

Permalink
Support tenant and namespace (apache#206)
Browse files Browse the repository at this point in the history
Currently KoP only support produce or consume topics in the configured namespace by `kafkaTenant` and `kafkaNamespace`. If we want to make use of Pulsar's multi-tenancy, we need to enable SASL authentication and encode the namespace info into the `sasl.jaas.config` property.

This PR is to use Pulsar topic name as Kafka topic name, the topic mapping from topic in Kafka client to the real Pulsar topic is:

- my-topic => `persistent://<kafkaTenant>/<kafkaNamespace>/my-topic`
- my-tenant/my-ns/my-topic => `persistent://my-tenant/my-ns/my-topic`

This work is done by the new added `KopTopic`.

Besides, if client want to fetch the metadata of all topics, before this PR, only the topics of `<kafkaTenant>/<kafkaNamespace>` would be returned, now the topics of all namespaces would be returned.

Following unit tests are added:

- `KopTopicTest`: test the basic functions of `KopTopic`;
- `DifferentNamespaceTest`: test the newly supported topic name version to specify a custom namespace
* Add KopTopic

* Support parse tenant/namespace from topic name

* Fix auto topic creation error

* Fix topic metadata response error

* Add test for specify topics with different namespaces

* Fix test error of testBrokerPublishByteThrotting

* Remove redundant check for numPartitions

* Make get all topics async

* Fix checkstyle failure

* Add test for commit offset in another namespace

* Add a TODO to the authentication issue after multi-tenancy was supported

* Rollback the refactor of handleTopicMetadataRequest

* Rollback indent style changes
  • Loading branch information
BewareMyPower authored Oct 29, 2020
1 parent 6108d36 commit 04b6164
Show file tree
Hide file tree
Showing 11 changed files with 488 additions and 116 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import io.streamnative.pulsar.handlers.kop.coordinator.group.GroupCoordinator;
import io.streamnative.pulsar.handlers.kop.coordinator.group.OffsetConfig;
import io.streamnative.pulsar.handlers.kop.utils.ConfigurationUtils;
import io.streamnative.pulsar.handlers.kop.utils.KopTopic;
import io.streamnative.pulsar.handlers.kop.utils.MetadataUtils;
import io.streamnative.pulsar.handlers.kop.utils.timer.SystemTimer;
import java.net.InetSocketAddress;
Expand Down Expand Up @@ -197,6 +198,7 @@ public void initialize(ServiceConfiguration conf) throws Exception {
kafkaConfig.setBindAddress(conf.getBindAddress());
}
this.bindAddress = ServiceConfigurationUtils.getDefaultOrConfiguredAddress(kafkaConfig.getBindAddress());
KopTopic.initialize(kafkaConfig.getKafkaTenant() + "/" + kafkaConfig.getKafkaNamespace());
}

// This method is called after initialize
Expand Down

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -14,13 +14,13 @@
package io.streamnative.pulsar.handlers.kop;

import static io.streamnative.pulsar.handlers.kop.utils.MessageRecordUtils.entriesToRecords;
import static io.streamnative.pulsar.handlers.kop.utils.TopicNameUtils.pulsarTopicName;
import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;

import com.google.common.collect.Lists;
import io.netty.util.Recycler;
import io.netty.util.Recycler.Handle;
import io.streamnative.pulsar.handlers.kop.KafkaCommandDecoder.KafkaHeaderAndRequest;
import io.streamnative.pulsar.handlers.kop.utils.KopTopic;
import io.streamnative.pulsar.handlers.kop.utils.MessageIdUtils;
import java.util.Date;
import java.util.LinkedHashMap;
Expand Down Expand Up @@ -48,7 +48,6 @@
import org.apache.kafka.common.requests.FetchRequest;
import org.apache.kafka.common.requests.FetchResponse;
import org.apache.kafka.common.requests.FetchResponse.PartitionData;
import org.apache.pulsar.common.naming.TopicName;

/**
* MessageFetchContext handling FetchRequest .
Expand Down Expand Up @@ -96,10 +95,8 @@ public CompletableFuture<AbstractResponse> handleFetch(CompletableFuture<Abstrac
((FetchRequest) fetchRequest.getRequest())
.fetchData().entrySet().stream()
.map(entry -> {
TopicName topicName = pulsarTopicName(entry.getKey(), requestHandler.getNamespace());

CompletableFuture<KafkaTopicConsumerManager> consumerManager =
requestHandler.getTopicManager().getTopicConsumerManager(topicName.toString());
requestHandler.getTopicManager().getTopicConsumerManager(KopTopic.toString(entry.getKey()));

return Pair.of(
entry.getKey(),
Expand Down Expand Up @@ -221,9 +218,8 @@ private void readMessagesInternal(KafkaHeaderAndRequest fetch,
fetch.getHeader(), kafkaTopic, e);

// delete related cursor in TCM
TopicName pulsarTopicName = pulsarTopicName(kafkaTopic, requestHandler.getNamespace());
requestHandler.getTopicManager()
.getTopicConsumerManager(pulsarTopicName.toString())
.getTopicConsumerManager(KopTopic.toString(kafkaTopic))
.thenAccept(cm -> {
// Notice, channel may be close, then TCM would be null.
if (cm != null) {
Expand Down Expand Up @@ -282,10 +278,9 @@ private void readMessagesInternal(KafkaHeaderAndRequest fetch,
TopicPartition kafkaPartition = responseEntrys.getKey();
List<Entry> entries = responseEntrys.getValue();
// Add cursor and offset back to TCM when all the read completed.
TopicName pulsarTopicName = pulsarTopicName(kafkaPartition, requestHandler.getNamespace());
Pair<ManagedCursor, Long> pair = cursors.get(kafkaPartition);
requestHandler.getTopicManager()
.getTopicConsumerManager(pulsarTopicName.toString())
.getTopicConsumerManager(KopTopic.toString(kafkaPartition))
.thenAccept(cm -> {
// Notice, channel may be close, then TCM would be null.
if (cm != null) {
Expand Down Expand Up @@ -375,9 +370,7 @@ private Map<TopicPartition, CompletableFuture<Entry>> readAllCursorOnce(
new ReadEntriesCallback() {
@Override
public void readEntriesComplete(List<Entry> list, Object o) {
TopicName topicName = pulsarTopicName(
cursorOffsetPair.getKey(),
requestHandler.getNamespace());
String fullPartitionName = KopTopic.toString(cursorOffsetPair.getKey());

Entry entry = null;
if (!list.isEmpty()) {
Expand All @@ -403,7 +396,7 @@ public void readEntriesComplete(List<Entry> list, Object o) {
log.debug("Topic {} success read entry: ledgerId: {}, entryId: {}, size: {},"
+ " ConsumerManager original offset: {}, entryOffset: {} - {}, "
+ "nextOffset: {} - {}",
topicName.toString(), entry.getLedgerId(), entry.getEntryId(),
fullPartitionName, entry.getLedgerId(), entry.getEntryId(),
entry.getLength(), currentOffset, offset, currentPosition,
nextOffset, nextPosition);
}
Expand All @@ -414,8 +407,7 @@ public void readEntriesComplete(List<Entry> list, Object o) {

@Override
public void readEntriesFailed(ManagedLedgerException e, Object o) {
log.error("Error read entry for topic: {}",
pulsarTopicName(cursorOffsetPair.getKey(), requestHandler.getNamespace()));
log.error("Error read entry for topic: {}", KopTopic.toString(cursorOffsetPair.getKey()));

readFuture.completeExceptionally(e);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -119,7 +119,7 @@ public static GroupCoordinator of(
.timeoutTimer(timer)
.build();

OffsetAcker offsetAcker = new OffsetAcker(pulsarClient, kafkaServiceConfiguration);
OffsetAcker offsetAcker = new OffsetAcker(pulsarClient);
return new GroupCoordinator(
groupConfig,
metadataManager,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,10 +13,9 @@
*/
package io.streamnative.pulsar.handlers.kop.coordinator.group;

import io.streamnative.pulsar.handlers.kop.KafkaServiceConfiguration;
import io.streamnative.pulsar.handlers.kop.offset.OffsetAndMetadata;
import io.streamnative.pulsar.handlers.kop.utils.KopTopic;
import io.streamnative.pulsar.handlers.kop.utils.MessageIdUtils;
import io.streamnative.pulsar.handlers.kop.utils.TopicNameUtils;
import java.io.Closeable;
import java.nio.ByteBuffer;
import java.util.Map;
Expand All @@ -32,8 +31,6 @@
import org.apache.pulsar.client.api.MessageId;
import org.apache.pulsar.client.api.SubscriptionInitialPosition;
import org.apache.pulsar.client.impl.PulsarClientImpl;
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.naming.TopicName;

/**
* This class used to track all the partition offset commit position.
Expand All @@ -42,21 +39,12 @@
public class OffsetAcker implements Closeable {

private final ConsumerBuilder<byte[]> consumerBuilder;
private KafkaServiceConfiguration kafkaServiceConfiguration;


public OffsetAcker(PulsarClientImpl pulsarClient) {
this.consumerBuilder = pulsarClient.newConsumer()
.receiverQueueSize(0)
.subscriptionInitialPosition(SubscriptionInitialPosition.Earliest);
}
public OffsetAcker(PulsarClientImpl pulsarClient, KafkaServiceConfiguration kafkaServiceConfiguration) {
this.consumerBuilder = pulsarClient.newConsumer()
.receiverQueueSize(0)
.subscriptionInitialPosition(SubscriptionInitialPosition.Earliest);
this.kafkaServiceConfiguration = kafkaServiceConfiguration;
}


// map off consumser: <groupId, consumers>
Map<String, Map<TopicPartition, CompletableFuture<Consumer<byte[]>>>> consumers = new ConcurrentHashMap<>();
Expand Down Expand Up @@ -124,12 +112,9 @@ private CompletableFuture<Consumer<byte[]>> getConsumer(String groupId, TopicPar
}

private CompletableFuture<Consumer<byte[]>> createConsumer(String groupId, TopicPartition topicPartition) {
NamespaceName nameSpace = NamespaceName.get(
kafkaServiceConfiguration.getKafkaTenant(),
kafkaServiceConfiguration.getKafkaNamespace());
TopicName pulsarTopicName = TopicNameUtils.pulsarTopicName(topicPartition, nameSpace);
KopTopic kopTopic = new KopTopic(topicPartition.topic());
return consumerBuilder.clone()
.topic(pulsarTopicName.toString())
.topic(kopTopic.getPartitionName(topicPartition.partition()))
.subscriptionName(groupId)
.subscribeAsync();
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,82 @@
/**
* Licensed 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 io.streamnative.pulsar.handlers.kop.utils;

import static org.apache.pulsar.common.naming.TopicName.PARTITIONED_TOPIC_SUFFIX;

import lombok.Getter;
import org.apache.kafka.common.TopicPartition;

/**
* KopTopic maintains two topic name, one is the original topic name, the other is the full topic name used in Pulsar.
* We shouldn't use the original topic name directly in KoP source code. Instead, we should
* 1. getOriginalName() when read a Kafka request from client or write a Kafka response to client.
* 2. getFullName() when access Pulsar resources.
*/
public class KopTopic {

private static final String persistentDomain = "persistent://";
private static volatile String namespacePrefix; // the full namespace prefix, e.g. "public/default"

public static void initialize(String namespace) {
if (namespace.split("/").length != 2) {
throw new IllegalArgumentException("Invalid namespace: " + namespace);
}
KopTopic.namespacePrefix = namespace;
}

@Getter
private final String originalName;
@Getter
private final String fullName;

public KopTopic(String topic) {
if (namespacePrefix == null) {
throw new RuntimeException("KopTopic is not initialized");
}
originalName = topic;
fullName = expandToFullName(topic);
}

private String expandToFullName(String topic) {
if (topic.startsWith(persistentDomain)) {
if (topic.substring(persistentDomain.length()).split("/").length != 3) {
throw new IllegalArgumentException("Invalid topic name '" + topic + "', it should be "
+ " persistent://<tenant>/<namespace>/<topic>");
}
return topic;
}

String[] parts = topic.split("/");
if (parts.length == 3) {
return persistentDomain + topic;
} else if (parts.length == 1) {
return persistentDomain + namespacePrefix + "/" + topic;
} else {
throw new IllegalArgumentException("Invalid short topic name '" + topic + "', it should be in the format"
+ " of <tenant>/<namespace>/<topic> or <topic>");
}
}

public String getPartitionName(int partition) {
if (partition < 0) {
throw new IllegalArgumentException("Invalid partition " + partition + ", it should be non-negative number");
}
return fullName + PARTITIONED_TOPIC_SUFFIX + partition;
}

public static String toString(TopicPartition topicPartition) {
return (new KopTopic(topicPartition.topic())).getPartitionName(topicPartition.partition());
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
/**
* Licensed 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 io.streamnative.pulsar.handlers.kop.utils;

import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertTrue;
import static org.testng.Assert.fail;

import org.testng.annotations.Test;

/**
* Test for KopTopic.
*/
public class KopTopicTest {

@Test
public void testConstructor() {
KopTopic topic;
try {
topic = new KopTopic("my-topic");
fail();
} catch (RuntimeException e) {
assertEquals(e.getMessage(), "KopTopic is not initialized");
}

KopTopic.initialize("my-tenant/my-ns");

topic = new KopTopic("my-topic");
assertEquals(topic.getOriginalName(), "my-topic");
assertEquals(topic.getFullName(), "persistent://my-tenant/my-ns/my-topic");

topic = new KopTopic("my-tenant-2/my-ns-2/my-topic");
assertEquals(topic.getOriginalName(), "my-tenant-2/my-ns-2/my-topic");
assertEquals(topic.getFullName(), "persistent://my-tenant-2/my-ns-2/my-topic");

topic = new KopTopic("persistent://my-tenant-3/my-ns-3/my-topic");
assertEquals(topic.getOriginalName(), "persistent://my-tenant-3/my-ns-3/my-topic");
assertEquals(topic.getFullName(), topic.getOriginalName());

try {
topic = new KopTopic("my-ns/my-topic");
fail();
} catch (IllegalArgumentException e) {
assertTrue(e.getMessage().contains("Invalid short topic name"));
}
try {
topic = new KopTopic("persistent://my-topic");
} catch (IllegalArgumentException e) {
assertTrue(e.getMessage().contains("Invalid topic name"));
}
}

@Test
public void testGetPartitionName() {
KopTopic.initialize("my-tenant/my-ns");
KopTopic topic = new KopTopic("my-topic");
assertEquals(topic.getPartitionName(0), "persistent://my-tenant/my-ns/my-topic-partition-0");
assertEquals(topic.getPartitionName(12), "persistent://my-tenant/my-ns/my-topic-partition-12");
try {
topic.getPartitionName(-1);
fail();
} catch (IllegalArgumentException e) {
assertTrue(e.getMessage().contains("Invalid partition"));
}
}
}
Loading

0 comments on commit 04b6164

Please sign in to comment.