-
Notifications
You must be signed in to change notification settings - Fork 14.1k
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
base: trunk
Are you sure you want to change the base?
Changes from 36 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
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 |
---|---|---|
|
@@ -22,7 +22,7 @@ import kafka.raft.KafkaRaftManager | |
import kafka.server.QuotaFactory.QuotaManagers | ||
|
||
import scala.collection.immutable | ||
import kafka.server.metadata.{AclPublisher, ClientQuotaMetadataManager, DelegationTokenPublisher, DynamicClientQuotaPublisher, DynamicConfigPublisher, KRaftMetadataCache, KRaftMetadataCachePublisher, ScramPublisher} | ||
import kafka.server.metadata.{AclPublisher, ClientQuotaMetadataManager, DelegationTokenPublisher, DynamicClientQuotaPublisher, DynamicConfigPublisher, DynamicTopicClusterQuotaPublisher, KRaftMetadataCache, KRaftMetadataCachePublisher, ScramPublisher} | ||
import kafka.utils.{CoreUtils, Logging} | ||
import org.apache.kafka.common.message.ApiMessageType.ListenerType | ||
import org.apache.kafka.common.network.ListenerName | ||
|
@@ -332,7 +332,17 @@ class ControllerServer( | |
config, | ||
sharedServer.metadataPublishingFaultHandler, | ||
"controller", | ||
clientQuotaMetadataManager)) | ||
clientQuotaMetadataManager | ||
)) | ||
|
||
// Set up the DynamicTopicClusterQuotaPublisher. This will enable quotas for the cluster and topics. | ||
metadataPublishers.add(new DynamicTopicClusterQuotaPublisher( | ||
clusterId, | ||
config, | ||
sharedServer.metadataPublishingFaultHandler, | ||
"broker", | ||
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. Controller? |
||
quotaManagers, | ||
)) | ||
|
||
// Set up the SCRAM publisher. | ||
metadataPublishers.add(new ScramPublisher( | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,64 @@ | ||
/** | ||
* 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 | ||
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 | ||
|
||
class DynamicTopicClusterQuotaPublisher ( | ||
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. Please add comments to this new publisher. we should emphasize this approach is temporary and there is a follow-up jira |
||
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}" | ||
try { | ||
quotaManagers.clientQuotaCallback().ifPresent(clientQuotaCallback => { | ||
if (delta.topicsDelta() != null || delta.clusterDelta() != null) { | ||
val cluster = KRaftMetadataCache.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) | ||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -18,6 +18,7 @@ | |
package kafka.server.metadata | ||
|
||
import kafka.server.MetadataCache | ||
import kafka.server.metadata.KRaftMetadataCache.{getOfflineReplicas, getRandomAliveBroker} | ||
import kafka.utils.Logging | ||
import org.apache.kafka.admin.BrokerMetadata | ||
import org.apache.kafka.common._ | ||
|
@@ -206,28 +207,7 @@ class KRaftMetadataCache( | |
} | ||
} | ||
} | ||
|
||
private def getOfflineReplicas(image: MetadataImage, | ||
partition: PartitionRegistration, | ||
listenerName: ListenerName): 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 (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())) | ||
|
||
|
||
/** | ||
* Get the endpoint matching the provided listener if the broker is alive. Note that listeners can | ||
* be added dynamically, so a broker with a missing listener could be a transient error. | ||
|
@@ -361,12 +341,7 @@ class KRaftMetadataCache( | |
Option(_currentImage.cluster.broker(brokerId)).count(_.inControlledShutdown) == 1 | ||
} | ||
|
||
override def getAliveBrokers(): Iterable[BrokerMetadata] = getAliveBrokers(_currentImage) | ||
|
||
private def getAliveBrokers(image: MetadataImage): Iterable[BrokerMetadata] = { | ||
image.cluster().brokers().values().asScala.filterNot(_.fenced()). | ||
map(b => new BrokerMetadata(b.id, b.rack)) | ||
} | ||
override def getAliveBrokers(): Iterable[BrokerMetadata] = KRaftMetadataCache.getAliveBrokers(_currentImage) | ||
|
||
override def getAliveBrokerNode(brokerId: Int, listenerName: ListenerName): Option[Node] = { | ||
Option(_currentImage.cluster().broker(brokerId)).filterNot(_.fenced()). | ||
|
@@ -443,15 +418,6 @@ class KRaftMetadataCache( | |
getRandomAliveBroker(_currentImage) | ||
} | ||
|
||
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) | ||
} | ||
} | ||
|
||
def getAliveBrokerEpoch(brokerId: Int): Option[Long] = { | ||
Option(_currentImage.cluster().broker(brokerId)).filterNot(_.fenced()). | ||
map(brokerRegistration => brokerRegistration.epoch()) | ||
|
@@ -545,3 +511,96 @@ class KRaftMetadataCache( | |
} | ||
} | ||
|
||
object KRaftMetadataCache { | ||
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. #18632 is trying to remove reference of |
||
|
||
def toCluster(clusterId: String, image: MetadataImage): 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. This result is different with |
||
val brokerToNodes = new util.HashMap[Integer, java.util.List[Node]] | ||
image.cluster().brokers() | ||
.values().stream() | ||
.filter(broker => !broker.fenced()) | ||
.forEach { broker => brokerToNodes.put(broker.id(), broker.nodes()) } | ||
|
||
def getNodes(id: Int): java.util.List[Node] = brokerToNodes.get(id) | ||
|
||
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)) | ||
} | ||
} |
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.
Please add test for controller