Skip to content
Open
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
6 changes: 5 additions & 1 deletion core/src/main/scala/kafka/server/KafkaApis.scala
Original file line number Diff line number Diff line change
Expand Up @@ -67,10 +67,11 @@ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
import org.apache.kafka.common.security.token.delegation.{DelegationToken, TokenInformation}
import org.apache.kafka.common.utils.{ProducerIdAndEpoch, Time}
import org.apache.kafka.common.{Node, TopicIdPartition, TopicPartition, Uuid}
import org.apache.kafka.controller.ReplicationControlManager
import org.apache.kafka.coordinator.group.{Group, GroupCoordinator}
import org.apache.kafka.server.ClientMetricsManager
import org.apache.kafka.server.authorizer._
import org.apache.kafka.server.common.{MetadataVersion}
import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.server.common.MetadataVersion.{IBP_0_11_0_IV0, IBP_2_3_IV0}
import org.apache.kafka.server.record.BrokerCompressionType
import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchIsolation, FetchParams, FetchPartitionData}
Expand Down Expand Up @@ -2087,6 +2088,8 @@ class KafkaApis(val requestChannel: RequestChannel,
}
sendResponseCallback(results)
}

ReplicationControlManager.validateTotalNumberOfPartitions(createTopicsRequest.data, config.numPartitions)
zkSupport.adminManager.createTopics(
createTopicsRequest.data.timeoutMs,
createTopicsRequest.data.validateOnly,
Expand Down Expand Up @@ -2139,6 +2142,7 @@ class KafkaApis(val requestChannel: RequestChannel,
unauthorized.map(_.name -> new ApiError(Errors.TOPIC_AUTHORIZATION_FAILED, "The topic authorization is failed.")) ++
queuedForDeletion.map(_.name -> new ApiError(Errors.INVALID_TOPIC_EXCEPTION, "The topic is queued for deletion."))

ReplicationControlManager.validateTotalNumberOfPartitions(createPartitionsRequest.data)
zkSupport.adminManager.createPartitions(
createPartitionsRequest.data.timeoutMs,
valid,
Expand Down
108 changes: 108 additions & 0 deletions core/src/test/java/kafka/server/TooManyPartitionsIntegrationTest.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,108 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 kafka.server;

import kafka.test.ClusterInstance;
import kafka.test.annotation.ClusterConfigProperty;
import kafka.test.annotation.ClusterTest;
import kafka.test.annotation.ClusterTestDefaults;
import kafka.test.junit.ClusterTestExtensions;
import org.apache.kafka.clients.admin.Admin;
import org.apache.kafka.clients.admin.NewPartitions;
import org.apache.kafka.clients.admin.NewTopic;
import org.apache.kafka.common.errors.PolicyViolationException;
import org.apache.kafka.server.policy.CreateTopicPolicy;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.extension.ExtendWith;

import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.ExecutionException;

import static org.junit.jupiter.api.Assertions.fail;

@ClusterTestDefaults(
brokers = 3,
serverProperties = {
@ClusterConfigProperty(key = "create.topic.policy.class.name", value = "kafka.server.TooManyPartitionsIntegrationTest$TopicPolicy"),
}
)
@ExtendWith(value = ClusterTestExtensions.class)
public class TooManyPartitionsIntegrationTest {
@ClusterTest
public void testCreateTooMany(ClusterInstance clusterInstance) throws Exception {
clusterInstance.waitForReadyBrokers();
try (Admin admin = clusterInstance.createAdminClient()) {
// policy kicks in
try {
admin.createTopics(Collections.singleton(new NewTopic("topic1K", 1_000, (short) 3))).all().get();
fail("ExecutionException expected");
} catch (ExecutionException e) {
Assertions.assertInstanceOf(PolicyViolationException.class, e.getCause());
Assertions.assertTrue(e.getCause().getMessage().contains("> 999"), e.getCause().getMessage());
}

// protection kicks in
try {
admin.createTopics(Collections.singleton(new NewTopic("topic1G", 1_000_000_000, (short) 3))).all().get();
fail("ExecutionException expected");
} catch (ExecutionException e) {
Assertions.assertInstanceOf(PolicyViolationException.class, e.getCause());
Assertions.assertTrue(e.getCause().getMessage().contains("Excessively large"), e.getCause().getMessage());
}
}
}

@ClusterTest
public void testExtendTooMany(ClusterInstance clusterInstance) throws Exception {
clusterInstance.waitForReadyBrokers();
try (Admin admin = clusterInstance.createAdminClient()) {
admin.createTopics(Collections.singleton(new NewTopic("topic", 1, (short) 3))).all().get();

// protection kicks in
try {
Map<String, NewPartitions> newPartitions = new HashMap<>();
newPartitions.put("topic", NewPartitions.increaseTo(1_000_000_000));
admin.createPartitions(newPartitions).all().get();
fail("ExecutionException expected");
} catch (ExecutionException e) {
Assertions.assertInstanceOf(PolicyViolationException.class, e.getCause());
Assertions.assertTrue(e.getCause().getMessage().contains("Excessively large"), e.getCause().getMessage());
}
}
}

public static class TopicPolicy implements CreateTopicPolicy {
@Override
public void validate(RequestMetadata requestMetadata) throws PolicyViolationException {
if (requestMetadata.numPartitions() > 999) {
throw new PolicyViolationException("Too many partitions: " + requestMetadata.numPartitions() + " > 999");
}
}

@Override
public void close() throws Exception {
}

@Override
public void configure(Map<String, ?> map) {
}
}
}

Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@
import org.apache.kafka.common.message.AssignReplicasToDirsRequestData;
import org.apache.kafka.common.message.AssignReplicasToDirsResponseData;
import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
import org.apache.kafka.common.message.CreatePartitionsRequestData;
import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic;
import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult;
import org.apache.kafka.common.message.CreateTopicsRequestData;
Expand Down Expand Up @@ -1149,7 +1150,7 @@ ControllerResult<AlterPartitionResponseData> alterPartition(
* @param defaultNumPartitions default number of partitions to assign if unspecified.
* @throws PolicyViolationException if total number of partitions exceeds {@value MAX_PARTITIONS_PER_BATCH}.
*/
static void validateTotalNumberOfPartitions(CreateTopicsRequestData request, int defaultNumPartitions) {
public static void validateTotalNumberOfPartitions(CreateTopicsRequestData request, int defaultNumPartitions) {
int totalPartitions = 0;
for (CreatableTopic topic: request.topics()) {
if (topic.assignments().isEmpty()) {
Expand All @@ -1168,6 +1169,38 @@ static void validateTotalNumberOfPartitions(CreateTopicsRequestData request, int
}
}

public static void validateTotalNumberOfPartitions(CreatePartitionsRequestData request) {
int totalPartitions = 0;
for (CreatePartitionsTopic topic: request.topics()) {
if (topic.assignments() == null || topic.assignments().isEmpty()) {
if (topic.count() > 0) {
totalPartitions += topic.count();
}
} else {
totalPartitions += topic.assignments().size();
}
}
if (totalPartitions > MAX_PARTITIONS_PER_BATCH) {
throw new PolicyViolationException("Excessively large number of partitions per request.");
}
}

static void validateTotalNumberOfPartitions(List<CreatePartitionsTopic> topics) {
int totalPartitions = 0;
for (CreatePartitionsTopic topic: topics) {
if (topic.assignments() == null || topic.assignments().isEmpty()) {
if (topic.count() > 0) {
totalPartitions += topic.count();
}
} else {
totalPartitions += topic.assignments().size();
}
}
if (totalPartitions > MAX_PARTITIONS_PER_BATCH) {
throw new PolicyViolationException("Excessively large number of partitions per request.");
}
}

/**
* Validate the partition information included in the alter partition request.
*
Expand Down Expand Up @@ -1756,6 +1789,9 @@ ControllerResult<List<CreatePartitionsTopicResult>> createPartitions(
) {
List<ApiMessageAndVersion> records = BoundedList.newArrayBacked(MAX_RECORDS_PER_USER_OP);
List<CreatePartitionsTopicResult> results = BoundedList.newArrayBacked(MAX_RECORDS_PER_USER_OP);

validateTotalNumberOfPartitions(topics);

for (CreatePartitionsTopic topic : topics) {
ApiError apiError = ApiError.NONE;
try {
Expand Down