KAFKA-18407: Remove ZkAdminManager, DelayedCreatePartitions, CreatePartitionsMetadata, ZkConfigRepository, DelayedDeleteTopics (#18574)

Reviewers: Mickael Maison <mickael.maison@gmail.com>
This commit is contained in:
Ken Huang 2025-01-17 02:32:22 +08:00 committed by GitHub
parent 60cc2a0dea
commit cabbb613a2
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
5 changed files with 0 additions and 1307 deletions

View File

@ -1,21 +0,0 @@
/**
* 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.common
class TopicAlreadyMarkedForDeletionException(message: String) extends RuntimeException(message) {
}

View File

@ -1,104 +0,0 @@
/**
* 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.utils.Logging
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.requests.ApiError
import org.apache.kafka.metadata.LeaderAndIsr
import org.apache.kafka.server.purgatory.DelayedOperation
import scala.collection._
/**
* The create metadata maintained by the delayed create topic or create partitions operations.
*/
case class CreatePartitionsMetadata(topic: String, partitions: Set[Int], error: ApiError)
object CreatePartitionsMetadata {
def apply(topic: String, partitions: Set[Int]): CreatePartitionsMetadata = {
CreatePartitionsMetadata(topic, partitions, ApiError.NONE)
}
def apply(topic: String, error: Errors): CreatePartitionsMetadata = {
CreatePartitionsMetadata(topic, Set.empty, new ApiError(error, null))
}
def apply(topic: String, throwable: Throwable): CreatePartitionsMetadata = {
CreatePartitionsMetadata(topic, Set.empty, ApiError.fromThrowable(throwable))
}
}
/**
* A delayed create topic or create partitions operation that is stored in the topic purgatory.
*/
class DelayedCreatePartitions(delayMs: Long,
createMetadata: Seq[CreatePartitionsMetadata],
adminManager: ZkAdminManager,
responseCallback: Map[String, ApiError] => Unit)
extends DelayedOperation(delayMs) with Logging {
/**
* The operation can be completed if all of the topics that do not have an error exist and every partition has a
* leader in the controller.
* See KafkaController.onNewTopicCreation
*/
override def tryComplete() : Boolean = {
trace(s"Trying to complete operation for $createMetadata")
val leaderlessPartitionCount = createMetadata.filter(_.error.isSuccess).foldLeft(0) { case (topicCounter, metadata) =>
topicCounter + missingLeaderCount(metadata.topic, metadata.partitions)
}
if (leaderlessPartitionCount == 0) {
trace("All partitions have a leader, completing the delayed operation")
forceComplete()
} else {
trace(s"$leaderlessPartitionCount partitions do not have a leader, not completing the delayed operation")
false
}
}
/**
* Check for partitions that are still missing a leader, update their error code and call the responseCallback
*/
override def onComplete(): Unit = {
trace(s"Completing operation for $createMetadata")
val results = createMetadata.map { metadata =>
// ignore topics that already have errors
if (metadata.error.isSuccess && missingLeaderCount(metadata.topic, metadata.partitions) > 0)
(metadata.topic, new ApiError(Errors.REQUEST_TIMED_OUT, null))
else
(metadata.topic, metadata.error)
}.toMap
responseCallback(results)
}
override def onExpiration(): Unit = {}
private def missingLeaderCount(topic: String, partitions: Set[Int]): Int = {
partitions.foldLeft(0) { case (counter, partition) =>
if (isMissingLeader(topic, partition)) counter + 1 else counter
}
}
private def isMissingLeader(topic: String, partition: Int): Boolean = {
val partitionInfo = adminManager.metadataCache.getPartitionInfo(topic, partition)
partitionInfo.forall(_.leader == LeaderAndIsr.NO_LEADER)
}
}

View File

@ -1,85 +0,0 @@
/**
* 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.utils.Logging
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.server.purgatory.DelayedOperation
import scala.collection._
/**
* The delete metadata maintained by the delayed delete operation
*/
case class DeleteTopicMetadata(topic: String, error: Errors)
object DeleteTopicMetadata {
def apply(topic: String, throwable: Throwable): DeleteTopicMetadata = {
DeleteTopicMetadata(topic, Errors.forException(throwable))
}
}
/**
* A delayed delete topics operation that can be created by the admin manager and watched
* in the topic purgatory
*/
class DelayedDeleteTopics(delayMs: Long,
deleteMetadata: Seq[DeleteTopicMetadata],
adminManager: ZkAdminManager,
responseCallback: Map[String, Errors] => Unit)
extends DelayedOperation(delayMs) with Logging {
/**
* The operation can be completed if all of the topics not in error have been removed
*/
override def tryComplete() : Boolean = {
trace(s"Trying to complete operation for $deleteMetadata")
// Ignore topics that already have errors
val existingTopics = deleteMetadata.count { metadata => metadata.error == Errors.NONE && topicExists(metadata.topic) }
if (existingTopics == 0) {
trace("All topics have been deleted or have errors, completing the delayed operation")
forceComplete()
} else {
trace(s"$existingTopics topics still exist, not completing the delayed operation")
false
}
}
/**
* Check for partitions that still exist, update their error code and call the responseCallback
*/
override def onComplete(): Unit = {
trace(s"Completing operation for $deleteMetadata")
val results = deleteMetadata.map { metadata =>
// ignore topics that already have errors
if (metadata.error == Errors.NONE && topicExists(metadata.topic))
(metadata.topic, Errors.REQUEST_TIMED_OUT)
else
(metadata.topic, metadata.error)
}.toMap
responseCallback(results)
}
override def onExpiration(): Unit = { }
private def topicExists(topic: String): Boolean = {
adminManager.metadataCache.contains(topic)
}
}

File diff suppressed because it is too large Load Diff

View File

@ -1,51 +0,0 @@
/**
* 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.metadata
import java.util.Properties
import kafka.zk.{AdminZkClient, KafkaZkClient}
import org.apache.kafka.common.config.ConfigResource
import org.apache.kafka.common.config.ConfigResource.Type
import org.apache.kafka.common.errors.InvalidRequestException
import org.apache.kafka.server.config.{ConfigType, ZooKeeperInternals}
object ZkConfigRepository {
def apply(zkClient: KafkaZkClient): ZkConfigRepository =
new ZkConfigRepository(new AdminZkClient(zkClient))
}
class ZkConfigRepository(adminZkClient: AdminZkClient) extends ConfigRepository {
override def config(configResource: ConfigResource): Properties = {
val configTypeForZk = configResource.`type` match {
case Type.TOPIC => ConfigType.TOPIC
case Type.BROKER => ConfigType.BROKER
case Type.CLIENT_METRICS => throw new InvalidRequestException("Config type client-metrics is only supported on KRaft clusters")
case Type.GROUP => throw new InvalidRequestException("Config type groups is only supported on KRaft clusters")
case tpe => throw new IllegalArgumentException(s"Unsupported config type: $tpe")
}
// ZK stores cluster configs under "<default>".
val effectiveName = if (configResource.`type`.equals(Type.BROKER) &&
configResource.name.isEmpty) {
ZooKeeperInternals.DEFAULT_STRING
} else {
configResource.name
}
adminZkClient.fetchEntityConfig(configTypeForZk, effectiveName)
}
}