forked from apache/pulsar
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Support tenant and namespace (apache#206)
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
1 parent
6108d36
commit 04b6164
Showing
11 changed files
with
488 additions
and
116 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
140 changes: 87 additions & 53 deletions
140
kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/KafkaRequestHandler.java
Large diffs are not rendered by default.
Oops, something went wrong.
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
82 changes: 82 additions & 0 deletions
82
kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/utils/KopTopic.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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()); | ||
} | ||
} |
77 changes: 77 additions & 0 deletions
77
kafka-impl/src/test/java/io/streamnative/pulsar/handlers/kop/utils/KopTopicTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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")); | ||
} | ||
} | ||
} |
Oops, something went wrong.