Skip to content
This repository has been archived by the owner on Jan 24, 2024. It is now read-only.

Add inner topic protection #451

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 @@ -78,6 +78,7 @@
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.ApiException;
import org.apache.kafka.common.errors.CorruptRecordException;
import org.apache.kafka.common.errors.InvalidTopicException;
import org.apache.kafka.common.errors.LeaderNotAvailableException;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
Expand Down Expand Up @@ -651,6 +652,13 @@ protected void handleProduceRequest(KafkaHeaderAndRequest produceHar,
for (Map.Entry<TopicPartition, ? extends Records> entry : produceRequest.partitionRecordsOrFail().entrySet()) {
TopicPartition topicPartition = entry.getKey();
try {
String fullPartitionName = KopTopic.toString(topicPartition);
if (isOffsetTopic(fullPartitionName) || isTransactionTopic(fullPartitionName)) {
log.error("[{}] Request {}: not support produce message to inner topic. topic: {}",
ctx.channel(), produceHar.getHeader(), topicPartition);
throw new InvalidTopicException(Errors.INVALID_TOPIC_EXCEPTION.message());
}

MemoryRecords validRecords = validateRecords(produceHar.getHeader().apiVersion(),
topicPartition, (MemoryRecords) entry.getValue());

Expand All @@ -663,7 +671,6 @@ protected void handleProduceRequest(KafkaHeaderAndRequest produceHar,
topicPartition.topic(), topicPartition.partition(), responsesSize);
}

String fullPartitionName = KopTopic.toString(topicPartition);
PendingProduce pendingProduce = new PendingProduce(partitionResponse, topicManager, fullPartitionName,
entryFormatter, validRecords, executor, transactionCoordinator, requestStats);
PendingProduceQueue queue =
Expand Down Expand Up @@ -1692,12 +1699,20 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
return returnFuture;
}

private boolean isOffsetTopic(String topic) {
protected boolean isOffsetTopic(String topic) {
String offsetsTopic = kafkaConfig.getKafkaMetadataTenant() + "/"
+ kafkaConfig.getKafkaMetadataNamespace()
+ "/" + GROUP_METADATA_TOPIC_NAME;

return topic.contains(offsetsTopic);
return topic != null && topic.contains(offsetsTopic);
}

protected boolean isTransactionTopic(String topic) {
String transactionTopic = kafkaConfig.getKafkaMetadataTenant() + "/"
+ kafkaConfig.getKafkaMetadataNamespace()
+ "/" + TRANSACTION_STATE_TOPIC_NAME;

return topic != null && topic.contains(transactionTopic);
}

public CompletableFuture<PartitionMetadata> findBroker(TopicName topic) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -126,6 +126,12 @@ public void publishMessages() {
throw new RuntimeException(e);
}

if (persistentTopic.isSystemTopic()) {
log.error("Not support producing message to system topic: {}", persistentTopic);
responseFuture.complete(new PartitionResponse(Errors.INVALID_TOPIC_EXCEPTION));
return;
}

if (log.isDebugEnabled()) {
log.debug("publishMessages for topic partition: {}, records size is {}", partitionName, numMessages);
}
Expand Down
8 changes: 8 additions & 0 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@
<os-maven-plugin.version>1.4.1.Final</os-maven-plugin.version>
<puppycrawl.checkstyle.version>6.19</puppycrawl.checkstyle.version>
<spotbugs-maven-plugin.version>3.1.8</spotbugs-maven-plugin.version>
<avro.version>1.10.2</avro.version>
</properties>

<licenses>
Expand Down Expand Up @@ -228,6 +229,13 @@
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>${avro.version}</version>
<scope>provided</scope>
</dependency>

</dependencies>

<build>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,185 @@
/**
* 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;

import com.google.common.collect.Sets;
import java.net.URL;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.Optional;
import java.util.Properties;

import lombok.Cleanup;
import lombok.extern.slf4j.Slf4j;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.serialization.StringSerializer;
import org.apache.pulsar.client.admin.PulsarAdminException;
import org.apache.pulsar.common.policies.data.ClusterData;
import org.apache.pulsar.common.policies.data.RetentionPolicies;
import org.apache.pulsar.common.policies.data.TenantInfo;
import org.testng.Assert;
import org.testng.annotations.AfterMethod;
import org.testng.annotations.BeforeMethod;
import org.testng.annotations.Test;

/**
* Inner topic protection test.
*/
@Slf4j
public class InnerTopicProtectionTest extends KopProtocolHandlerTestBase {

protected int offsetsTopicNumPartitions;

protected KafkaServiceConfiguration resetConfig(int brokerPort, int webPort, int kafkaPort) {
KafkaServiceConfiguration kConfig = new KafkaServiceConfiguration();
kConfig.setBrokerServicePort(Optional.ofNullable(brokerPort));
kConfig.setWebServicePort(Optional.ofNullable(webPort));
kConfig.setListeners(PLAINTEXT_PREFIX + "localhost:" + kafkaPort);

kConfig.setOffsetsTopicNumPartitions(offsetsTopicNumPartitions);
kConfig.setEnableGroupCoordinator(true);

kConfig.setAdvertisedAddress("localhost");
kConfig.setClusterName(configClusterName);
kConfig.setManagedLedgerCacheSizeMB(8);
kConfig.setActiveConsumerFailoverDelayTimeMillis(0);
kConfig.setDefaultNumberOfNamespaceBundles(2);
kConfig.setZookeeperServers("localhost:2181");
kConfig.setConfigurationStoreServers("localhost:3181");
kConfig.setEnableGroupCoordinator(true);
kConfig.setAuthenticationEnabled(false);
kConfig.setAuthorizationEnabled(false);
kConfig.setAllowAutoTopicCreation(true);
kConfig.setAllowAutoTopicCreationType("partitioned");
kConfig.setBrokerDeleteInactiveTopicsEnabled(false);
kConfig.setSystemTopicEnabled(true);
kConfig.setTopicLevelPoliciesEnabled(true);

// set protocol related config
URL testHandlerUrl = this.getClass().getClassLoader().getResource("test-protocol-handler.nar");
Path handlerPath;
try {
handlerPath = Paths.get(testHandlerUrl.toURI());
} catch (Exception e) {
log.error("failed to get handler Path, handlerUrl: {}. Exception: ", testHandlerUrl, e);
return null;
}

String protocolHandlerDir = handlerPath.toFile().getParent();

kConfig.setProtocolHandlerDirectory(
protocolHandlerDir
);
kConfig.setMessagingProtocols(Sets.newHashSet("kafka"));

return kConfig;
}

@Override
protected void resetConfig() {
offsetsTopicNumPartitions = 16;
conf = resetConfig(
brokerPort,
brokerWebservicePort,
kafkaBrokerPort);

log.info("Ports -- broker: {}, brokerWeb:{}, kafka: {}",
brokerPort, brokerWebservicePort, kafkaBrokerPort);
}
BewareMyPower marked this conversation as resolved.
Show resolved Hide resolved

@BeforeMethod
@Override
protected void setup() throws Exception {
super.internalSetup();
log.info("success internal setup");

if (!admin.clusters().getClusters().contains(configClusterName)) {
// so that clients can test short names
admin.clusters().createCluster(configClusterName,
new ClusterData("http://127.0.0.1:" + brokerWebservicePort));
} else {
admin.clusters().updateCluster(configClusterName,
new ClusterData("http://127.0.0.1:" + brokerWebservicePort));
}

if (!admin.tenants().getTenants().contains("public")) {
admin.tenants().createTenant("public",
new TenantInfo(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test")));
} else {
admin.tenants().updateTenant("public",
new TenantInfo(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test")));
}
if (!admin.namespaces().getNamespaces("public").contains("public/default")) {
admin.namespaces().createNamespace("public/default");
admin.namespaces().setNamespaceReplicationClusters("public/default", Sets.newHashSet("test"));
admin.namespaces().setRetention("public/default",
new RetentionPolicies(60, 1000));
}
if (!admin.namespaces().getNamespaces("public").contains("public/__kafka")) {
admin.namespaces().createNamespace("public/__kafka");
admin.namespaces().setNamespaceReplicationClusters("public/__kafka", Sets.newHashSet("test"));
admin.namespaces().setRetention("public/__kafka",
new RetentionPolicies(-1, -1));
}
}

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

@Test(timeOut = 30000)
public void testInnerTopicProduce() throws PulsarAdminException {
final String offsetTopic = "public/__kafka/__consumer_offsets";
final String transactionTopic = "public/__kafka/__transaction_state";
final String systemTopic = "__change_events";
final String commonTopic = "normal-topic";

admin.topics().createPartitionedTopic(commonTopic, 3);
// test inner topic produce
@Cleanup
final KafkaProducer<String, String> kafkaProducer = newKafkaProducer();
final String msg = "test-inner-topic-produce-and-consume";
assertProduceMessage(kafkaProducer, offsetTopic, msg, true);
assertProduceMessage(kafkaProducer, transactionTopic, msg, true);
assertProduceMessage(kafkaProducer, systemTopic, msg, true);
assertProduceMessage(kafkaProducer, commonTopic, msg, false);
}

private void assertProduceMessage(KafkaProducer producer, final String topic, final String value,
boolean assertException) {
try {
producer.send(new ProducerRecord<>(topic, value)).get();
} catch (Exception e) {
if (assertException) {
Assert.assertEquals(e.getCause().getMessage(),
"The request attempted to perform an operation on an invalid topic.");
} else {
Assert.fail();
}
}
}

protected KafkaProducer<String, String> newKafkaProducer() {
final Properties props = new Properties();
props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:" + getKafkaBrokerPort());
props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
props.put(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG, 1000);
return new KafkaProducer<>(props);
}
}