-
Notifications
You must be signed in to change notification settings - Fork 14.2k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
KAFKA-18225: ClientQuotaCallback#updateClusterMetadata is unsupported by kraft #18196
Changes from 48 commits
5fe6d1a
b3edea5
15d7dbb
4123c70
765d7f1
513b381
74b1006
d1cfe97
a4758e7
dee9a65
cd853b6
48a8b41
6bf41a7
3fe1d7c
ed50ea8
f87db91
57fad78
f468e55
0c136b7
ac9f55b
909d8f9
5e88ee1
b3365f0
8e36e2f
977a603
de16d0e
9f04283
830c99c
3a3e409
e48ca88
6f679f1
be432e6
af4cee5
39f542e
0bab1c7
1ea0067
929e87a
101cbdc
9c28db2
9514964
498ab89
c5fb945
c986339
9721c48
294cb93
3877941
2b527b4
3dc5afb
9469bff
71123fb
d902055
3673d78
21cf98e
9451d69
ee23d92
7924c03
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -23,7 +23,7 @@ | |
import java.util.Map; | ||
|
||
/** | ||
* Quota callback interface for brokers that enables customization of client quota computation. | ||
* Quota callback interface for brokers and controllers that enables customization of client quota computation. | ||
*/ | ||
public interface ClientQuotaCallback extends Configurable { | ||
|
||
|
@@ -89,11 +89,11 @@ public interface ClientQuotaCallback extends Configurable { | |
boolean quotaResetRequired(ClientQuotaType quotaType); | ||
|
||
/** | ||
* Metadata update callback that is invoked whenever UpdateMetadata request is received from | ||
* the controller. This is useful if quota computation takes partitions into account. | ||
* Metadata update callback that is invoked whenever the topic and cluster delta changed. | ||
* This is useful if quota computation takes partitions into account. | ||
* Topics that are being deleted will not be included in `cluster`. | ||
* | ||
* @param cluster Cluster metadata including partitions and their leaders if known | ||
* @param cluster Cluster metadata including topic and cluster | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm not sure I understand this change. I prefer the previous message. |
||
* @return true if quotas have changed and metric configs may need to be updated | ||
*/ | ||
boolean updateClusterMetadata(Cluster cluster); | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -19,15 +19,20 @@ package kafka.server | |
|
||
import kafka.server.metadata.{ConfigRepository, KRaftMetadataCache} | ||
import org.apache.kafka.admin.BrokerMetadata | ||
import org.apache.kafka.common.internals.Topic | ||
import org.apache.kafka.common.message.{DescribeClientQuotasRequestData, DescribeClientQuotasResponseData, DescribeTopicPartitionsResponseData, DescribeUserScramCredentialsRequestData, DescribeUserScramCredentialsResponseData, MetadataResponseData} | ||
import org.apache.kafka.common.network.ListenerName | ||
import org.apache.kafka.common.{Cluster, Node, TopicPartition, Uuid} | ||
import org.apache.kafka.metadata.LeaderAndIsr | ||
import org.apache.kafka.common.{Cluster, Node, PartitionInfo, TopicPartition, Uuid} | ||
import org.apache.kafka.image.MetadataImage | ||
import org.apache.kafka.metadata.{BrokerRegistration, LeaderAndIsr, PartitionRegistration} | ||
import org.apache.kafka.server.common.{FinalizedFeatures, KRaftVersion, MetadataVersion} | ||
|
||
import java.util | ||
import java.util.Collections | ||
import java.util.concurrent.ThreadLocalRandom | ||
import java.util.function.Supplier | ||
import scala.collection._ | ||
import scala.jdk.CollectionConverters.CollectionHasAsScala | ||
|
||
trait MetadataCache extends ConfigRepository { | ||
/** | ||
|
@@ -144,4 +149,95 @@ object MetadataCache { | |
): KRaftMetadataCache = { | ||
new KRaftMetadataCache(brokerId, kraftVersionSupplier) | ||
} | ||
|
||
def toCluster(clusterId: String, image: MetadataImage): Cluster = { | ||
val brokerToNodes = new util.HashMap[Integer, java.util.List[Node]] | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We can drop the |
||
image.cluster().brokers() | ||
.values().stream() | ||
.filter(broker => !broker.fenced()) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. -> There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, I want to keep this because we need to convert it to a Java future. |
||
.forEach { broker => brokerToNodes.put(broker.id(), broker.nodes()) } | ||
|
||
def getNodes(id: Int): java.util.List[Node] = brokerToNodes.get(id) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Again drop the |
||
|
||
val partitionInfos = new util.ArrayList[PartitionInfo] | ||
val internalTopics = new util.HashSet[String] | ||
|
||
def toArray(replicas: Array[Int]): Array[Node] = { | ||
util.Arrays.stream(replicas) | ||
.mapToObj(replica => getNodes(replica)) | ||
.flatMap(replica => replica.stream()).toArray(size => new Array[Node](size)) | ||
} | ||
|
||
val topicImages = image.topics().topicsByName().values() | ||
if (topicImages != null) { | ||
topicImages.forEach { topic => | ||
topic.partitions().forEach { (key, value) => | ||
val partitionId = key | ||
val partition = value | ||
val nodes = getNodes(partition.leader) | ||
if (nodes != null) { | ||
nodes.forEach(node => { | ||
partitionInfos.add(new PartitionInfo(topic.name(), | ||
partitionId, | ||
node, | ||
toArray(partition.replicas), | ||
toArray(partition.isr), | ||
getOfflineReplicas(image, partition).stream() | ||
.map(replica => getNodes(replica)) | ||
.flatMap(replica => replica.stream()).toArray(size => new Array[Node](size)))) | ||
}) | ||
if (Topic.isInternal(topic.name())) { | ||
internalTopics.add(topic.name()) | ||
} | ||
} | ||
} | ||
} | ||
} | ||
|
||
val controllerNode = getNodes(getRandomAliveBroker(image).getOrElse(-1)) match { | ||
case null => Node.noNode() | ||
case nodes => nodes.get(0) | ||
} | ||
// Note: the constructor of Cluster does not allow us to reference unregistered nodes. | ||
// So, for example, if partition foo-0 has replicas [1, 2] but broker 2 is not | ||
// registered, we pass its replicas as [1, -1]. This doesn't make a lot of sense, but | ||
// we are duplicating the behavior of ZkMetadataCache, for now. | ||
new Cluster(clusterId, brokerToNodes.values().stream().flatMap(n => n.stream()).collect(util.stream.Collectors.toList()), | ||
partitionInfos, Collections.emptySet(), internalTopics, controllerNode) | ||
} | ||
|
||
private def getOfflineReplicas(image: MetadataImage, | ||
partition: PartitionRegistration, | ||
listenerName: ListenerName = null): util.List[Integer] = { | ||
val offlineReplicas = new util.ArrayList[Integer](0) | ||
for (brokerId <- partition.replicas) { | ||
Option(image.cluster().broker(brokerId)) match { | ||
case None => offlineReplicas.add(brokerId) | ||
case Some(broker) => if (listenerName == null || isReplicaOffline(partition, listenerName, broker)) { | ||
offlineReplicas.add(brokerId) | ||
} | ||
} | ||
} | ||
offlineReplicas | ||
} | ||
|
||
private def isReplicaOffline(partition: PartitionRegistration, listenerName: ListenerName, broker: BrokerRegistration) = | ||
broker.fenced() || !broker.listeners().containsKey(listenerName.value()) || isReplicaInOfflineDir(broker, partition) | ||
|
||
private def isReplicaInOfflineDir(broker: BrokerRegistration, partition: PartitionRegistration): Boolean = | ||
!broker.hasOnlineDir(partition.directory(broker.id())) | ||
|
||
private def getRandomAliveBroker(image: MetadataImage): Option[Int] = { | ||
val aliveBrokers = getAliveBrokers(image).toList | ||
if (aliveBrokers.isEmpty) { | ||
None | ||
} else { | ||
Some(aliveBrokers(ThreadLocalRandom.current().nextInt(aliveBrokers.size)).id) | ||
} | ||
} | ||
|
||
private def getAliveBrokers(image: MetadataImage): Iterable[BrokerMetadata] = { | ||
image.cluster().brokers().values().asScala.filterNot(_.fenced()). | ||
map(b => new BrokerMetadata(b.id, b.rack)) | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,69 @@ | ||
/** | ||
* 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 kafka.server.metadata | ||
|
||
import kafka.server.{KafkaConfig, MetadataCache} | ||
import kafka.server.QuotaFactory.QuotaManagers | ||
import kafka.utils.Logging | ||
import org.apache.kafka.image.{MetadataDelta, MetadataImage} | ||
import org.apache.kafka.image.loader.LoaderManifest | ||
import org.apache.kafka.server.fault.FaultHandler | ||
|
||
/** | ||
* Publishing dynamic topic or cluster changes to the client quota manager. | ||
* Temporary solution since Cluster objects are immutable and costly to update for every metadata change. | ||
* See KAFKA-18239 to trace the issue. | ||
*/ | ||
class DynamicTopicClusterQuotaPublisher ( | ||
m1a2st marked this conversation as resolved.
Show resolved
Hide resolved
|
||
clusterId: String, | ||
conf: KafkaConfig, | ||
faultHandler: FaultHandler, | ||
nodeType: String, | ||
quotaManagers: QuotaManagers | ||
) extends Logging with org.apache.kafka.image.publisher.MetadataPublisher { | ||
logIdent = s"[${name()}] " | ||
|
||
override def name(): String = s"DynamicTopicClusterQuotaPublisher $nodeType id=${conf.nodeId}" | ||
|
||
override def onMetadataUpdate( | ||
delta: MetadataDelta, | ||
newImage: MetadataImage, | ||
manifest: LoaderManifest | ||
): Unit = { | ||
onMetadataUpdate(delta, newImage) | ||
} | ||
|
||
def onMetadataUpdate( | ||
delta: MetadataDelta, | ||
newImage: MetadataImage, | ||
): Unit = { | ||
val deltaName = s"MetadataDelta up to ${newImage.highestOffsetAndEpoch().offset}" | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can we only compute this val in the |
||
try { | ||
quotaManagers.clientQuotaCallback().ifPresent(clientQuotaCallback => { | ||
if (delta.topicsDelta() != null || delta.clusterDelta() != null) { | ||
val cluster = MetadataCache.toCluster(clusterId, newImage) | ||
if (clientQuotaCallback.updateClusterMetadata(cluster)) { | ||
quotaManagers.fetch.updateQuotaMetricConfigs() | ||
quotaManagers.produce.updateQuotaMetricConfigs() | ||
quotaManagers.request.updateQuotaMetricConfigs() | ||
quotaManagers.controllerMutation.updateQuotaMetricConfigs() | ||
} | ||
} | ||
}) | ||
} catch { | ||
case t: Throwable => faultHandler.handleFault("Uncaught exception while " + | ||
s"publishing dynamic topic or cluster changes from $deltaName", t) | ||
} | ||
} | ||
} | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Nit: missing new line |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'm not sure we want to mention cluster/topic deltas here, this is a public API and part of our javadoc. I think we can say something like:
This callback is invoked whenever the cluster metadata changes. This includes brokers added or removed, topics created or deleted, and partition leadership changes.