Browse Source
This PR implements broker-side KRaft snapshots, including both saving and loading. The code for triggering a periodic broker-side snapshot will come in a follow-on PR. Loading should work with just this PR. It also implements reloading broker snapshots after initialization. In order to facilitate snapshots, this PR introduces the concept of MetadataImage and MetadataDelta. MetadataImage represents the metadata state retained in memory. It is basically a generalization of MetadataCache that includes a few things that MetadataCache does not (such as features and client quotas.) KRaftMetadataCache is now an accessor for the data stored in this object. Similarly, MetadataImage replaces CacheConfigRespository and ClientQuotaCache. It also subsumes kafka.server.metadata.MetadataImage and related classes. MetadataDelta represents a change to a MetadataImage. When a KRaft snapshot is loaded, we will accumulate all the changes into a MetadataDelta first, prior to applying it. If we must reload a snapshot because we fell too far behind while consuming metadata, the resulting MetadataDelta will contain all the changes needed to catch us up. During normal operation, MetadataDelta is also used to accumulate the changes of each incoming batch of metadata records. These incremental deltas should be relatively small. I have removed the logic for updating the various manager objects from BrokerMetadataListener and placed it into BrokerMetadataPublisher. This makes it easier to unit test BrokerMetadataListener. Reviewers: David Arthur <mumrah@gmail.com>, Jason Gustafson <jason@confluent.io>pull/10681/merge
Colin Patrick McCabe
3 years ago
committed by
GitHub
46 changed files with 1454 additions and 4202 deletions
@ -1,252 +0,0 @@
@@ -1,252 +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.cluster.Partition |
||||
import kafka.controller.StateChangeLogger |
||||
import kafka.log.Log |
||||
import kafka.server.checkpoints.OffsetCheckpoints |
||||
import kafka.server.metadata.{MetadataBrokers, MetadataPartition} |
||||
import kafka.utils.Implicits.MapExtensionMethods |
||||
import org.apache.kafka.common.{TopicPartition, Uuid} |
||||
import org.apache.kafka.common.errors.KafkaStorageException |
||||
|
||||
import scala.collection.{Map, Set, mutable} |
||||
|
||||
trait RaftReplicaChangeDelegateHelper { |
||||
def stateChangeLogger: StateChangeLogger |
||||
def replicaFetcherManager: ReplicaFetcherManager |
||||
def replicaAlterLogDirsManager: ReplicaAlterLogDirsManager |
||||
def markDeferred(state: HostedPartition.Deferred): Unit |
||||
def getLogDir(topicPartition: TopicPartition): Option[String] |
||||
def error(msg: => String, e: => Throwable): Unit |
||||
def markOffline(topicPartition: TopicPartition): Unit |
||||
def markOnline(partition: Partition): Unit |
||||
def completeDelayedFetchOrProduceRequests(topicPartition: TopicPartition): Unit |
||||
def isShuttingDown: Boolean |
||||
def initialFetchOffset(log: Log): Long |
||||
def config: KafkaConfig |
||||
} |
||||
|
||||
class RaftReplicaChangeDelegate(helper: RaftReplicaChangeDelegateHelper) { |
||||
def makeDeferred(partitionsNewMap: Map[Partition, Boolean], |
||||
metadataOffset: Long): Unit = { |
||||
val traceLoggingEnabled = helper.stateChangeLogger.isTraceEnabled |
||||
if (traceLoggingEnabled) |
||||
partitionsNewMap.forKeyValue { (partition, isNew) => |
||||
helper.stateChangeLogger.trace(s"Metadata batch $metadataOffset: starting the " + |
||||
s"become-deferred transition for partition ${partition.topicPartition} isNew=$isNew") |
||||
} |
||||
|
||||
// Stop fetchers for all the partitions |
||||
helper.replicaFetcherManager.removeFetcherForPartitions(partitionsNewMap.keySet.map(_.topicPartition)) |
||||
helper.stateChangeLogger.info(s"Metadata batch $metadataOffset: as part of become-deferred request, " + |
||||
s"stopped any fetchers for ${partitionsNewMap.size} partitions") |
||||
// mark all the partitions as deferred |
||||
partitionsNewMap.forKeyValue((partition, isNew) => helper.markDeferred(HostedPartition.Deferred(partition, isNew))) |
||||
|
||||
helper.replicaFetcherManager.shutdownIdleFetcherThreads() |
||||
helper.replicaAlterLogDirsManager.shutdownIdleFetcherThreads() |
||||
|
||||
if (traceLoggingEnabled) |
||||
partitionsNewMap.keys.foreach { partition => |
||||
helper.stateChangeLogger.trace(s"Completed batch $metadataOffset become-deferred " + |
||||
s"transition for partition ${partition.topicPartition}") |
||||
} |
||||
} |
||||
|
||||
def makeLeaders(prevPartitionsAlreadyExisting: Set[MetadataPartition], |
||||
partitionStates: Map[Partition, MetadataPartition], |
||||
highWatermarkCheckpoints: OffsetCheckpoints, |
||||
metadataOffset: Option[Long], |
||||
topicIds: String => Option[Uuid]): Set[Partition] = { |
||||
val partitionsMadeLeaders = mutable.Set[Partition]() |
||||
val traceLoggingEnabled = helper.stateChangeLogger.isTraceEnabled |
||||
val deferredBatches = metadataOffset.isEmpty |
||||
val topLevelLogPrefix = if (deferredBatches) |
||||
"Metadata batch <multiple deferred>" |
||||
else |
||||
s"Metadata batch ${metadataOffset.get}" |
||||
try { |
||||
// First stop fetchers for all the partitions |
||||
helper.replicaFetcherManager.removeFetcherForPartitions(partitionStates.keySet.map(_.topicPartition)) |
||||
helper.stateChangeLogger.info(s"$topLevelLogPrefix: stopped ${partitionStates.size} fetcher(s)") |
||||
// Update the partition information to be the leader |
||||
partitionStates.forKeyValue { (partition, state) => |
||||
val topicPartition = partition.topicPartition |
||||
val partitionLogMsgPrefix = if (deferredBatches) |
||||
s"Apply deferred leader partition $topicPartition" |
||||
else |
||||
s"Metadata batch ${metadataOffset.get} $topicPartition" |
||||
try { |
||||
val isrState = state.toLeaderAndIsrPartitionState( |
||||
!prevPartitionsAlreadyExisting(state)) |
||||
if (partition.makeLeader(isrState, highWatermarkCheckpoints, topicIds(partition.topic))) { |
||||
partitionsMadeLeaders += partition |
||||
if (traceLoggingEnabled) { |
||||
helper.stateChangeLogger.trace(s"$partitionLogMsgPrefix: completed the become-leader state change.") |
||||
} |
||||
} else { |
||||
helper.stateChangeLogger.info(s"$partitionLogMsgPrefix: skipped the " + |
||||
"become-leader state change since it is already the leader.") |
||||
} |
||||
} catch { |
||||
case e: KafkaStorageException => |
||||
helper.stateChangeLogger.error(s"$partitionLogMsgPrefix: unable to make " + |
||||
s"leader because the replica for the partition is offline due to disk error $e") |
||||
val dirOpt = helper.getLogDir(topicPartition) |
||||
helper.error(s"Error while making broker the leader for partition $partition in dir $dirOpt", e) |
||||
helper.markOffline(topicPartition) |
||||
} |
||||
} |
||||
} catch { |
||||
case e: Throwable => |
||||
helper.stateChangeLogger.error(s"$topLevelLogPrefix: error while processing batch.", e) |
||||
// Re-throw the exception for it to be caught in BrokerMetadataListener |
||||
throw e |
||||
} |
||||
partitionsMadeLeaders |
||||
} |
||||
|
||||
def makeFollowers(prevPartitionsAlreadyExisting: Set[MetadataPartition], |
||||
currentBrokers: MetadataBrokers, |
||||
partitionStates: Map[Partition, MetadataPartition], |
||||
highWatermarkCheckpoints: OffsetCheckpoints, |
||||
metadataOffset: Option[Long], |
||||
topicIds: String => Option[Uuid]): Set[Partition] = { |
||||
val traceLoggingEnabled = helper.stateChangeLogger.isTraceEnabled |
||||
val deferredBatches = metadataOffset.isEmpty |
||||
val topLevelLogPrefix = if (deferredBatches) |
||||
"Metadata batch <multiple deferred>" |
||||
else |
||||
s"Metadata batch ${metadataOffset.get}" |
||||
if (traceLoggingEnabled) { |
||||
partitionStates.forKeyValue { (partition, state) => |
||||
val topicPartition = partition.topicPartition |
||||
val partitionLogMsgPrefix = if (deferredBatches) |
||||
s"Apply deferred follower partition $topicPartition" |
||||
else |
||||
s"Metadata batch ${metadataOffset.get} $topicPartition" |
||||
helper.stateChangeLogger.trace(s"$partitionLogMsgPrefix: starting the " + |
||||
s"become-follower transition with leader ${state.leaderId}") |
||||
} |
||||
} |
||||
|
||||
val partitionsMadeFollower: mutable.Set[Partition] = mutable.Set() |
||||
// all brokers, including both alive and not |
||||
val acceptableLeaderBrokerIds = currentBrokers.iterator().map(broker => broker.id).toSet |
||||
val allBrokersByIdMap = currentBrokers.iterator().map(broker => broker.id -> broker).toMap |
||||
try { |
||||
partitionStates.forKeyValue { (partition, state) => |
||||
val topicPartition = partition.topicPartition |
||||
val partitionLogMsgPrefix = if (deferredBatches) |
||||
s"Apply deferred follower partition $topicPartition" |
||||
else |
||||
s"Metadata batch ${metadataOffset.get} $topicPartition" |
||||
try { |
||||
val isNew = !prevPartitionsAlreadyExisting(state) |
||||
if (!acceptableLeaderBrokerIds.contains(state.leaderId)) { |
||||
// The leader broker should always be present in the metadata cache. |
||||
// If not, we should record the error message and abort the transition process for this partition |
||||
helper.stateChangeLogger.error(s"$partitionLogMsgPrefix: cannot become follower " + |
||||
s"since the new leader ${state.leaderId} is unavailable.") |
||||
// Create the local replica even if the leader is unavailable. This is required to ensure that we include |
||||
// the partition's high watermark in the checkpoint file (see KAFKA-1647) |
||||
partition.createLogIfNotExists(isNew, isFutureReplica = false, highWatermarkCheckpoints, topicIds(partition.topic)) |
||||
} else { |
||||
val isrState = state.toLeaderAndIsrPartitionState(isNew) |
||||
if (partition.makeFollower(isrState, highWatermarkCheckpoints, topicIds(partition.topic))) { |
||||
partitionsMadeFollower += partition |
||||
if (traceLoggingEnabled) { |
||||
helper.stateChangeLogger.trace(s"$partitionLogMsgPrefix: completed the " + |
||||
s"become-follower state change with new leader ${state.leaderId}.") |
||||
} |
||||
} else { |
||||
helper.stateChangeLogger.info(s"$partitionLogMsgPrefix: skipped the " + |
||||
s"become-follower state change since " + |
||||
s"the new leader ${state.leaderId} is the same as the old leader.") |
||||
} |
||||
} |
||||
} catch { |
||||
case e: KafkaStorageException => |
||||
helper.stateChangeLogger.error(s"$partitionLogMsgPrefix: unable to complete the " + |
||||
s"become-follower state change since the " + |
||||
s"replica for the partition is offline due to disk error $e") |
||||
val dirOpt = helper.getLogDir(partition.topicPartition) |
||||
helper.error(s"Error while making broker the follower with leader ${state.leaderId} in dir $dirOpt", e) |
||||
helper.markOffline(topicPartition) |
||||
} |
||||
} |
||||
|
||||
if (partitionsMadeFollower.nonEmpty) { |
||||
helper.replicaFetcherManager.removeFetcherForPartitions(partitionsMadeFollower.map(_.topicPartition)) |
||||
helper.stateChangeLogger.info(s"$topLevelLogPrefix: stopped followers for ${partitionsMadeFollower.size} partitions") |
||||
|
||||
partitionsMadeFollower.foreach { partition => |
||||
helper.completeDelayedFetchOrProduceRequests(partition.topicPartition) |
||||
} |
||||
|
||||
if (helper.isShuttingDown) { |
||||
if (traceLoggingEnabled) { |
||||
partitionsMadeFollower.foreach { partition => |
||||
val topicPartition = partition.topicPartition |
||||
val partitionLogMsgPrefix = if (deferredBatches) |
||||
s"Apply deferred follower partition $topicPartition" |
||||
else |
||||
s"Metadata batch ${metadataOffset.get} $topicPartition" |
||||
helper.stateChangeLogger.trace(s"$partitionLogMsgPrefix: skipped the " + |
||||
s"adding-fetcher step of the become-follower state for " + |
||||
s"$topicPartition since we are shutting down.") |
||||
} |
||||
} |
||||
} else { |
||||
// we do not need to check if the leader exists again since this has been done at the beginning of this process |
||||
val partitionsToMakeFollowerWithLeaderAndOffset = partitionsMadeFollower.map { partition => |
||||
val leader = allBrokersByIdMap(partition.leaderReplicaIdOpt.get).brokerEndPoint(helper.config.interBrokerListenerName) |
||||
val log = partition.localLogOrException |
||||
val fetchOffset = helper.initialFetchOffset(log) |
||||
if (deferredBatches) { |
||||
helper.markOnline(partition) |
||||
} |
||||
partition.topicPartition -> InitialFetchState(leader, partition.getLeaderEpoch, fetchOffset) |
||||
}.toMap |
||||
|
||||
helper.replicaFetcherManager.addFetcherForPartitions(partitionsToMakeFollowerWithLeaderAndOffset) |
||||
} |
||||
} |
||||
} catch { |
||||
case e: Throwable => |
||||
helper.stateChangeLogger.error(s"$topLevelLogPrefix: error while processing batch", e) |
||||
// Re-throw the exception for it to be caught in BrokerMetadataListener |
||||
throw e |
||||
} |
||||
|
||||
if (traceLoggingEnabled) |
||||
partitionsMadeFollower.foreach { partition => |
||||
val topicPartition = partition.topicPartition |
||||
val state = partitionStates(partition) |
||||
val partitionLogMsgPrefix = if (deferredBatches) |
||||
s"Apply deferred follower partition $topicPartition" |
||||
else |
||||
s"Metadata batch ${metadataOffset.get} $topicPartition" |
||||
helper.stateChangeLogger.trace(s"$partitionLogMsgPrefix: completed become-follower " + |
||||
s"transition for partition $topicPartition with new leader ${state.leaderId}") |
||||
} |
||||
|
||||
partitionsMadeFollower |
||||
} |
||||
} |
@ -1,404 +0,0 @@
@@ -1,404 +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 java.util.concurrent.atomic.AtomicBoolean |
||||
|
||||
import kafka.cluster.Partition |
||||
import kafka.controller.StateChangeLogger |
||||
import kafka.log.{Log, LogManager} |
||||
import kafka.server.QuotaFactory.QuotaManagers |
||||
import kafka.server.checkpoints.LazyOffsetCheckpoints |
||||
import kafka.server.metadata.{ConfigRepository, MetadataImage, MetadataImageBuilder, MetadataPartition, RaftMetadataCache} |
||||
import kafka.utils.Scheduler |
||||
import org.apache.kafka.common.{TopicPartition, Uuid} |
||||
import org.apache.kafka.common.errors.{InconsistentTopicIdException, KafkaStorageException} |
||||
import org.apache.kafka.common.metrics.Metrics |
||||
import org.apache.kafka.common.utils.Time |
||||
|
||||
import scala.collection.{Set, mutable} |
||||
|
||||
class RaftReplicaManager(config: KafkaConfig, |
||||
metrics: Metrics, |
||||
time: Time, |
||||
scheduler: Scheduler, |
||||
logManager: LogManager, |
||||
isShuttingDown: AtomicBoolean, |
||||
quotaManagers: QuotaManagers, |
||||
brokerTopicStats: BrokerTopicStats, |
||||
metadataCache: RaftMetadataCache, |
||||
logDirFailureChannel: LogDirFailureChannel, |
||||
delayedProducePurgatory: DelayedOperationPurgatory[DelayedProduce], |
||||
delayedFetchPurgatory: DelayedOperationPurgatory[DelayedFetch], |
||||
delayedDeleteRecordsPurgatory: DelayedOperationPurgatory[DelayedDeleteRecords], |
||||
delayedElectLeaderPurgatory: DelayedOperationPurgatory[DelayedElectLeader], |
||||
threadNamePrefix: Option[String], |
||||
configRepository: ConfigRepository, |
||||
alterIsrManager: AlterIsrManager) extends ReplicaManager( |
||||
config, metrics, time, None, scheduler, logManager, isShuttingDown, quotaManagers, |
||||
brokerTopicStats, metadataCache, logDirFailureChannel, delayedProducePurgatory, delayedFetchPurgatory, |
||||
delayedDeleteRecordsPurgatory, delayedElectLeaderPurgatory, threadNamePrefix, configRepository, alterIsrManager) { |
||||
|
||||
def this(config: KafkaConfig, |
||||
metrics: Metrics, |
||||
time: Time, |
||||
scheduler: Scheduler, |
||||
logManager: LogManager, |
||||
isShuttingDown: AtomicBoolean, |
||||
quotaManagers: QuotaManagers, |
||||
brokerTopicStats: BrokerTopicStats, |
||||
metadataCache: RaftMetadataCache, |
||||
logDirFailureChannel: LogDirFailureChannel, |
||||
alterIsrManager: AlterIsrManager, |
||||
configRepository: ConfigRepository, |
||||
threadNamePrefix: Option[String] = None) = { |
||||
this(config, metrics, time, scheduler, logManager, isShuttingDown, |
||||
quotaManagers, brokerTopicStats, metadataCache, logDirFailureChannel, |
||||
DelayedOperationPurgatory[DelayedProduce]( |
||||
purgatoryName = "Produce", brokerId = config.brokerId, |
||||
purgeInterval = config.producerPurgatoryPurgeIntervalRequests), |
||||
DelayedOperationPurgatory[DelayedFetch]( |
||||
purgatoryName = "Fetch", brokerId = config.brokerId, |
||||
purgeInterval = config.fetchPurgatoryPurgeIntervalRequests), |
||||
DelayedOperationPurgatory[DelayedDeleteRecords]( |
||||
purgatoryName = "DeleteRecords", brokerId = config.brokerId, |
||||
purgeInterval = config.deleteRecordsPurgatoryPurgeIntervalRequests), |
||||
DelayedOperationPurgatory[DelayedElectLeader]( |
||||
purgatoryName = "ElectLeader", brokerId = config.brokerId), |
||||
threadNamePrefix, configRepository, alterIsrManager) |
||||
} |
||||
|
||||
if (config.requiresZookeeper) { |
||||
throw new IllegalStateException(s"Cannot use ${getClass.getSimpleName} when using ZooKeeper") |
||||
} |
||||
|
||||
class RaftReplicaManagerChangeDelegateHelper(raftReplicaManager: RaftReplicaManager) extends RaftReplicaChangeDelegateHelper { |
||||
override def completeDelayedFetchOrProduceRequests(topicPartition: TopicPartition): Unit = raftReplicaManager.completeDelayedFetchOrProduceRequests(topicPartition) |
||||
|
||||
override def config: KafkaConfig = raftReplicaManager.config |
||||
|
||||
override def error(msg: => String, e: => Throwable): Unit = raftReplicaManager.error(msg, e) |
||||
|
||||
override def getLogDir(topicPartition: TopicPartition): Option[String] = raftReplicaManager.getLogDir(topicPartition) |
||||
|
||||
override def initialFetchOffset(log: Log): Long = raftReplicaManager.initialFetchOffset(log) |
||||
|
||||
override def isShuttingDown: Boolean = raftReplicaManager.isShuttingDown.get |
||||
|
||||
override def markDeferred(state: HostedPartition.Deferred): Unit = raftReplicaManager.markPartitionDeferred(state) |
||||
|
||||
override def markOffline(topicPartition: TopicPartition): Unit = raftReplicaManager.markPartitionOffline(topicPartition) |
||||
|
||||
override def markOnline(partition: Partition): Unit = raftReplicaManager.allPartitions.put(partition.topicPartition, HostedPartition.Online(partition)) |
||||
|
||||
override def replicaAlterLogDirsManager: ReplicaAlterLogDirsManager = raftReplicaManager.replicaAlterLogDirsManager |
||||
|
||||
override def replicaFetcherManager: ReplicaFetcherManager = raftReplicaManager.replicaFetcherManager |
||||
|
||||
override def stateChangeLogger: StateChangeLogger = raftReplicaManager.stateChangeLogger |
||||
} |
||||
|
||||
// visible/overwriteable for testing, generally will not change otherwise |
||||
private[server] var delegate = new RaftReplicaChangeDelegate(new RaftReplicaManagerChangeDelegateHelper(this)) |
||||
|
||||
// Changes are initially deferred when using a Raft-based metadata quorum, and they may flip-flop to not |
||||
// being deferred and being deferred again thereafter as the broker (re)acquires/loses its lease. |
||||
// Changes are never deferred when using ZooKeeper. When true, this indicates that we should transition |
||||
// online partitions to the deferred state if we see a metadata update for that partition. |
||||
private var deferringMetadataChanges: Boolean = true |
||||
stateChangeLogger.debug(s"Metadata changes are initially being deferred") |
||||
|
||||
def beginMetadataChangeDeferral(): Unit = { |
||||
replicaStateChangeLock synchronized { |
||||
deferringMetadataChanges = true |
||||
stateChangeLogger.info(s"Metadata changes are now being deferred") |
||||
} |
||||
} |
||||
|
||||
def endMetadataChangeDeferral(onLeadershipChange: (Iterable[Partition], Iterable[Partition]) => Unit): Unit = { |
||||
val startMs = time.milliseconds() |
||||
var partitionsMadeFollower = Set.empty[Partition] |
||||
var partitionsMadeLeader = Set.empty[Partition] |
||||
replicaStateChangeLock synchronized { |
||||
stateChangeLogger.info(s"Applying deferred metadata changes") |
||||
val highWatermarkCheckpoints = new LazyOffsetCheckpoints(this.highWatermarkCheckpoints) |
||||
val metadataImage = metadataCache.currentImage() |
||||
val brokers = metadataImage.brokers |
||||
try { |
||||
val leaderPartitionStates = mutable.Map[Partition, MetadataPartition]() |
||||
val followerPartitionStates = mutable.Map[Partition, MetadataPartition]() |
||||
val partitionsAlreadyExisting = mutable.Set[MetadataPartition]() |
||||
deferredPartitionsIterator.foreach { deferredPartition => |
||||
val partition = deferredPartition.partition |
||||
val state = cachedState(metadataImage, partition) |
||||
if (state.leaderId == localBrokerId) { |
||||
leaderPartitionStates.put(partition, state) |
||||
} else { |
||||
followerPartitionStates.put(partition, state) |
||||
} |
||||
if (!deferredPartition.isNew) { |
||||
partitionsAlreadyExisting += state |
||||
} |
||||
} |
||||
if (leaderPartitionStates.nonEmpty) |
||||
partitionsMadeLeader = delegate.makeLeaders(partitionsAlreadyExisting, leaderPartitionStates, highWatermarkCheckpoints, None, metadataImage.topicNameToId) |
||||
if (followerPartitionStates.nonEmpty) |
||||
partitionsMadeFollower = delegate.makeFollowers(partitionsAlreadyExisting, brokers, followerPartitionStates, highWatermarkCheckpoints, None, metadataImage.topicNameToId) |
||||
|
||||
// We need to transition anything that hasn't transitioned from Deferred to Offline to the Online state. |
||||
deferredPartitionsIterator.foreach { deferredPartition => |
||||
val partition = deferredPartition.partition |
||||
allPartitions.put(partition.topicPartition, HostedPartition.Online(partition)) |
||||
} |
||||
|
||||
updateLeaderAndFollowerMetrics(partitionsMadeFollower.map(_.topic).toSet) |
||||
|
||||
maybeAddLogDirFetchers(partitionsMadeFollower, highWatermarkCheckpoints, metadataImage.topicNameToId) |
||||
|
||||
replicaFetcherManager.shutdownIdleFetcherThreads() |
||||
replicaAlterLogDirsManager.shutdownIdleFetcherThreads() |
||||
if (partitionsMadeLeader.nonEmpty || partitionsMadeFollower.nonEmpty) { |
||||
onLeadershipChange(partitionsMadeLeader, partitionsMadeFollower) |
||||
} |
||||
} catch { |
||||
case e: Throwable => |
||||
deferredPartitionsIterator.foreach { metadata => |
||||
val partition = metadata.partition |
||||
val state = cachedState(metadataImage, partition) |
||||
val topicPartition = partition.topicPartition |
||||
val leader = state.leaderId == localBrokerId |
||||
val leaderOrFollower = if (leader) "leader" else "follower" |
||||
val partitionLogMsgPrefix = s"Apply deferred $leaderOrFollower partition $topicPartition" |
||||
stateChangeLogger.error(s"$partitionLogMsgPrefix: error while applying deferred metadata.", e) |
||||
} |
||||
stateChangeLogger.info(s"Applied ${partitionsMadeLeader.size + partitionsMadeFollower.size} deferred partitions prior to the error: " + |
||||
s"${partitionsMadeLeader.size} leader(s) and ${partitionsMadeFollower.size} follower(s)") |
||||
// Re-throw the exception for it to be caught in BrokerMetadataListener |
||||
throw e |
||||
} |
||||
deferringMetadataChanges = false |
||||
} |
||||
val endMs = time.milliseconds() |
||||
val elapsedMs = endMs - startMs |
||||
stateChangeLogger.info(s"Applied ${partitionsMadeLeader.size + partitionsMadeFollower.size} deferred partitions: " + |
||||
s"${partitionsMadeLeader.size} leader(s) and ${partitionsMadeFollower.size} follower(s)" + |
||||
s"in $elapsedMs ms") |
||||
stateChangeLogger.info("Metadata changes are no longer being deferred") |
||||
} |
||||
|
||||
/** |
||||
* Handle changes made by a batch of metadata log records. |
||||
* |
||||
* @param imageBuilder The MetadataImage builder. |
||||
* @param metadataOffset The last offset in the batch of records. |
||||
* @param onLeadershipChange The callbacks to invoke when leadership changes. |
||||
*/ |
||||
def handleMetadataRecords(imageBuilder: MetadataImageBuilder, |
||||
metadataOffset: Long, |
||||
onLeadershipChange: (Iterable[Partition], Iterable[Partition]) => Unit): Unit = { |
||||
val startMs = time.milliseconds() |
||||
val builder = imageBuilder.partitionsBuilder() |
||||
replicaStateChangeLock synchronized { |
||||
stateChangeLogger.info(("Metadata batch %d: %d local partition(s) changed, %d " + |
||||
"local partition(s) removed.").format(metadataOffset, builder.localChanged().size, |
||||
builder.localRemoved().size)) |
||||
if (stateChangeLogger.isTraceEnabled) { |
||||
builder.localChanged().foreach { state => |
||||
stateChangeLogger.trace(s"Metadata batch $metadataOffset: locally changed: ${state}") |
||||
} |
||||
builder.localRemoved().foreach { state => |
||||
stateChangeLogger.trace(s"Metadata batch $metadataOffset: locally removed: ${state}") |
||||
} |
||||
} |
||||
if (deferringMetadataChanges) { |
||||
val prevPartitions = imageBuilder.prevImage.partitions |
||||
// partitionChangesToBeDeferred maps each partition to be deferred to whether it is new (i.e. existed before deferral began) |
||||
val partitionChangesToBeDeferred = mutable.HashMap[Partition, Boolean]() |
||||
builder.localChanged().foreach { currentState => |
||||
val topicPartition = currentState.toTopicPartition |
||||
val (partition, priorDeferredMetadata) = getPartition(topicPartition) match { |
||||
case HostedPartition.Offline => |
||||
stateChangeLogger.warn(s"Ignoring handlePartitionChanges at $metadataOffset " + |
||||
s"for partition $topicPartition as the local replica for the partition is " + |
||||
"in an offline log directory") |
||||
(None, None) |
||||
|
||||
case HostedPartition.Online(partition) => (Some(partition), None) |
||||
case deferred@HostedPartition.Deferred(partition, _) => (Some(partition), Some(deferred)) |
||||
|
||||
case HostedPartition.None => |
||||
// Create the partition instance since it does not yet exist |
||||
(Some(Partition(topicPartition, time, this)), None) |
||||
} |
||||
partition.foreach { partition => |
||||
checkTopicId(builder.topicNameToId(partition.topic), partition.topicId, partition.topicPartition) |
||||
val isNew = priorDeferredMetadata match { |
||||
case Some(alreadyDeferred) => alreadyDeferred.isNew |
||||
case _ => prevPartitions.topicPartition(topicPartition.topic(), topicPartition.partition()).isEmpty |
||||
} |
||||
partitionChangesToBeDeferred.put(partition, isNew) |
||||
} |
||||
} |
||||
|
||||
stateChangeLogger.info(s"Deferring metadata changes for ${partitionChangesToBeDeferred.size} partition(s)") |
||||
if (partitionChangesToBeDeferred.nonEmpty) { |
||||
delegate.makeDeferred(partitionChangesToBeDeferred, metadataOffset) |
||||
} |
||||
} else { // not deferring changes, so make leaders/followers accordingly |
||||
val partitionsToBeLeader = mutable.HashMap[Partition, MetadataPartition]() |
||||
val partitionsToBeFollower = mutable.HashMap[Partition, MetadataPartition]() |
||||
builder.localChanged().foreach { currentState => |
||||
val topicPartition = currentState.toTopicPartition |
||||
val partition = getPartition(topicPartition) match { |
||||
case HostedPartition.Offline => |
||||
stateChangeLogger.warn(s"Ignoring handlePartitionChanges at $metadataOffset " + |
||||
s"for partition $topicPartition as the local replica for the partition is " + |
||||
"in an offline log directory") |
||||
None |
||||
|
||||
case HostedPartition.Online(partition) => Some(partition) |
||||
case _: HostedPartition.Deferred => throw new IllegalStateException( |
||||
s"There should never be deferred partition metadata when we aren't deferring changes: $topicPartition") |
||||
|
||||
case HostedPartition.None => |
||||
// it's a partition that we don't know about yet, so create it and mark it online |
||||
val partition = Partition(topicPartition, time, this) |
||||
allPartitions.putIfNotExists(topicPartition, HostedPartition.Online(partition)) |
||||
Some(partition) |
||||
} |
||||
partition.foreach { partition => |
||||
checkTopicId(builder.topicNameToId(partition.topic), partition.topicId, partition.topicPartition) |
||||
if (currentState.leaderId == localBrokerId) { |
||||
partitionsToBeLeader.put(partition, currentState) |
||||
} else { |
||||
partitionsToBeFollower.put(partition, currentState) |
||||
} |
||||
} |
||||
} |
||||
|
||||
val prevPartitions = imageBuilder.prevImage.partitions |
||||
val changedPartitionsPreviouslyExisting = mutable.Set[MetadataPartition]() |
||||
builder.localChanged().foreach(metadataPartition => |
||||
prevPartitions.topicPartition(metadataPartition.topicName, metadataPartition.partitionIndex).foreach( |
||||
changedPartitionsPreviouslyExisting.add)) |
||||
val nextBrokers = imageBuilder.brokers() |
||||
val highWatermarkCheckpoints = new LazyOffsetCheckpoints(this.highWatermarkCheckpoints) |
||||
val partitionsBecomeLeader = if (partitionsToBeLeader.nonEmpty) |
||||
delegate.makeLeaders(changedPartitionsPreviouslyExisting, partitionsToBeLeader, highWatermarkCheckpoints, |
||||
Some(metadataOffset), builder.topicNameToId) |
||||
else |
||||
Set.empty[Partition] |
||||
val partitionsBecomeFollower = if (partitionsToBeFollower.nonEmpty) |
||||
delegate.makeFollowers(changedPartitionsPreviouslyExisting, nextBrokers, partitionsToBeFollower, highWatermarkCheckpoints, |
||||
Some(metadataOffset), builder.topicNameToId) |
||||
else |
||||
Set.empty[Partition] |
||||
updateLeaderAndFollowerMetrics(partitionsBecomeFollower.map(_.topic).toSet) |
||||
|
||||
builder.localChanged().foreach { state => |
||||
val topicPartition = state.toTopicPartition |
||||
/* |
||||
* If there is offline log directory, a Partition object may have been created by getOrCreatePartition() |
||||
* before getOrCreateReplica() failed to create local replica due to KafkaStorageException. |
||||
* In this case ReplicaManager.allPartitions will map this topic-partition to an empty Partition object. |
||||
* we need to map this topic-partition to OfflinePartition instead. |
||||
*/ |
||||
if (localLog(topicPartition).isEmpty) { |
||||
markPartitionOffline(topicPartition) |
||||
} |
||||
} |
||||
|
||||
maybeAddLogDirFetchers(partitionsBecomeFollower, highWatermarkCheckpoints, builder.topicNameToId) |
||||
|
||||
replicaFetcherManager.shutdownIdleFetcherThreads() |
||||
replicaAlterLogDirsManager.shutdownIdleFetcherThreads() |
||||
onLeadershipChange(partitionsBecomeLeader, partitionsBecomeFollower) |
||||
stateChangeLogger.info(s"Metadata batch $metadataOffset: applied ${partitionsBecomeLeader.size + partitionsBecomeFollower.size} partitions: " + |
||||
s"${partitionsBecomeLeader.size} leader(s) and ${partitionsBecomeFollower.size} follower(s)") |
||||
} |
||||
// TODO: we should move aside log directories which have been deleted rather than |
||||
// purging them from the disk immediately. |
||||
if (builder.localRemoved().nonEmpty) { |
||||
// we schedule removal immediately even if we are deferring changes |
||||
stopPartitions(builder.localRemoved().map(_.toTopicPartition -> true).toMap).foreach { case (topicPartition, e) => |
||||
if (e.isInstanceOf[KafkaStorageException]) { |
||||
stateChangeLogger.error(s"Metadata batch $metadataOffset: unable to delete " + |
||||
s"${topicPartition} as the local replica for the partition is in an offline " + |
||||
"log directory") |
||||
} else { |
||||
stateChangeLogger.error(s"Metadata batch $metadataOffset: unable to delete " + |
||||
s"${topicPartition} due to an unexpected ${e.getClass.getName} exception: " + |
||||
s"${e.getMessage}") |
||||
} |
||||
} |
||||
} |
||||
} |
||||
val endMs = time.milliseconds() |
||||
val elapsedMs = endMs - startMs |
||||
stateChangeLogger.info(s"Metadata batch $metadataOffset: handled replica changes " + |
||||
s"in ${elapsedMs} ms") |
||||
} |
||||
|
||||
def markPartitionDeferred(partition: Partition, isNew: Boolean): Unit = { |
||||
markPartitionDeferred(HostedPartition.Deferred(partition, isNew)) |
||||
} |
||||
|
||||
private def markPartitionDeferred(state: HostedPartition.Deferred): Unit = replicaStateChangeLock synchronized { |
||||
allPartitions.put(state.partition.topicPartition, state) |
||||
} |
||||
|
||||
// An iterator over all deferred partitions. This is a weakly consistent iterator; a partition made off/online |
||||
// after the iterator has been constructed could still be returned by this iterator. |
||||
private def deferredPartitionsIterator: Iterator[HostedPartition.Deferred] = { |
||||
allPartitions.values.iterator.flatMap { |
||||
case deferred: HostedPartition.Deferred => Some(deferred) |
||||
case _ => None |
||||
} |
||||
} |
||||
|
||||
private def cachedState(metadataImage: MetadataImage, partition: Partition): MetadataPartition = { |
||||
metadataImage.partitions.topicPartition(partition.topic, partition.partitionId).getOrElse( |
||||
throw new IllegalStateException(s"Partition has metadata changes but does not exist in the metadata cache: ${partition.topicPartition}")) |
||||
} |
||||
|
||||
/** |
||||
* Checks if the topic ID received from the MetadataPartitionsBuilder is consistent with the topic ID in the log. |
||||
* If the log does not exist, logTopicIdOpt will be None. In this case, the ID is not inconsistent. |
||||
* |
||||
* @param receivedTopicIdOpt the topic ID received from the MetadataRecords if it exists |
||||
* @param logTopicIdOpt the topic ID in the log if the log exists |
||||
* @param topicPartition the topicPartition for the Partition being checked |
||||
* @throws InconsistentTopicIdException if the topic ids are not consistent |
||||
* @throws IllegalArgumentException if the MetadataPartitionsBuilder did not have a topic ID associated with the topic |
||||
*/ |
||||
private def checkTopicId(receivedTopicIdOpt: Option[Uuid], logTopicIdOpt: Option[Uuid], topicPartition: TopicPartition): Unit = { |
||||
receivedTopicIdOpt match { |
||||
case Some(receivedTopicId) => |
||||
logTopicIdOpt.foreach { logTopicId => |
||||
if (receivedTopicId != logTopicId) { |
||||
// not sure if we need both the logger and the error thrown |
||||
stateChangeLogger.error(s"Topic ID in memory: $logTopicId does not" + |
||||
s" match the topic ID for partition $topicPartition received: " + |
||||
s"$receivedTopicId.") |
||||
throw new InconsistentTopicIdException(s"Topic partition $topicPartition had an inconsistent topic ID.") |
||||
} |
||||
} |
||||
case None => throw new IllegalStateException(s"Topic partition $topicPartition is missing a topic ID") |
||||
} |
||||
} |
||||
} |
@ -0,0 +1,249 @@
@@ -0,0 +1,249 @@
|
||||
/** |
||||
* 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 kafka.coordinator.group.GroupCoordinator |
||||
import kafka.coordinator.transaction.TransactionCoordinator |
||||
import kafka.log.{Log, LogManager} |
||||
import kafka.server.ConfigType |
||||
import kafka.server.{ConfigHandler, FinalizedFeatureCache, KafkaConfig, ReplicaManager, RequestLocal} |
||||
import kafka.utils.Logging |
||||
import org.apache.kafka.common.TopicPartition |
||||
import org.apache.kafka.common.config.ConfigResource |
||||
import org.apache.kafka.common.internals.Topic |
||||
import org.apache.kafka.image.{MetadataDelta, MetadataImage, TopicDelta} |
||||
|
||||
import scala.collection.mutable |
||||
|
||||
|
||||
object BrokerMetadataPublisher { |
||||
/** |
||||
* Given a topic name, find out if it changed. Note: if a topic named X was deleted and |
||||
* then re-created, this method will return just the re-creation. The deletion will show |
||||
* up in deletedTopicIds and must be handled separately. |
||||
* |
||||
* @param topicName The topic name. |
||||
* @param newImage The new metadata image. |
||||
* @param delta The metadata delta to search. |
||||
* |
||||
* @return The delta, or None if appropriate. |
||||
*/ |
||||
def getTopicDelta(topicName: String, |
||||
newImage: MetadataImage, |
||||
delta: MetadataDelta): Option[TopicDelta] = { |
||||
Option(newImage.topics().getTopic(topicName)).map { |
||||
topicImage => delta.topicsDelta().changedTopic(topicImage.id()) |
||||
} |
||||
} |
||||
|
||||
/** |
||||
* Find logs which should not be on the current broker, according to the metadata image. |
||||
* |
||||
* @param brokerId The ID of the current broker. |
||||
* @param newImage The metadata image. |
||||
* @param logs A collection of Log objects. |
||||
* |
||||
* @return The topic partitions which are no longer needed on this broker. |
||||
*/ |
||||
def findGhostReplicas(brokerId: Int, |
||||
newImage: MetadataImage, |
||||
logs: Iterable[Log]): Iterable[TopicPartition] = { |
||||
logs.flatMap { log => |
||||
log.topicId match { |
||||
case None => throw new RuntimeException(s"Topic ${log.name} does not have a topic ID, " + |
||||
"which is not allowed when running in KRaft mode.") |
||||
case Some(topicId) => |
||||
val partitionId = log.topicPartition.partition() |
||||
Option(newImage.topics().getPartition(topicId, partitionId)) match { |
||||
case None => None |
||||
case Some(partition) => if (partition.replicas.contains(brokerId)) { |
||||
Some(log.topicPartition) |
||||
} else { |
||||
None |
||||
} |
||||
} |
||||
} |
||||
} |
||||
} |
||||
} |
||||
|
||||
class BrokerMetadataPublisher(conf: KafkaConfig, |
||||
metadataCache: KRaftMetadataCache, |
||||
logManager: LogManager, |
||||
replicaManager: ReplicaManager, |
||||
groupCoordinator: GroupCoordinator, |
||||
txnCoordinator: TransactionCoordinator, |
||||
clientQuotaMetadataManager: ClientQuotaMetadataManager, |
||||
featureCache: FinalizedFeatureCache, |
||||
dynamicConfigHandlers: Map[String, ConfigHandler]) extends MetadataPublisher with Logging { |
||||
logIdent = s"[BrokerMetadataPublisher id=${conf.nodeId}] " |
||||
|
||||
import BrokerMetadataPublisher._ |
||||
|
||||
/** |
||||
* The broker ID. |
||||
*/ |
||||
val brokerId = conf.nodeId |
||||
|
||||
/** |
||||
* True if this is the first time we have published metadata. |
||||
*/ |
||||
var _firstPublish = true |
||||
|
||||
override def publish(newHighestMetadataOffset: Long, |
||||
delta: MetadataDelta, |
||||
newImage: MetadataImage): Unit = { |
||||
try { |
||||
// Publish the new metadata image to the metadata cache. |
||||
metadataCache.setImage(newImage) |
||||
|
||||
if (_firstPublish) { |
||||
info(s"Publishing initial metadata at offset ${newHighestMetadataOffset}.") |
||||
|
||||
// If this is the first metadata update we are applying, initialize the managers |
||||
// first (but after setting up the metadata cache). |
||||
initializeManagers() |
||||
} else if (isDebugEnabled) { |
||||
debug(s"Publishing metadata at offset ${newHighestMetadataOffset}.") |
||||
} |
||||
|
||||
// Apply feature deltas. |
||||
Option(delta.featuresDelta()).foreach { featuresDelta => |
||||
featureCache.update(featuresDelta, newHighestMetadataOffset) |
||||
} |
||||
|
||||
// Apply topic deltas. |
||||
Option(delta.topicsDelta()).foreach { topicsDelta => |
||||
// Notify the replica manager about changes to topics. |
||||
replicaManager.applyDelta(newImage, topicsDelta) |
||||
|
||||
// Handle the case where the old consumer offsets topic was deleted. |
||||
if (topicsDelta.topicWasDeleted(Topic.GROUP_METADATA_TOPIC_NAME)) { |
||||
topicsDelta.image().getTopic(Topic.GROUP_METADATA_TOPIC_NAME).partitions().entrySet().forEach { |
||||
entry => |
||||
if (entry.getValue().leader == brokerId) { |
||||
groupCoordinator.onResignation(entry.getKey(), Some(entry.getValue().leaderEpoch)) |
||||
} |
||||
} |
||||
} |
||||
// Handle the case where we have new local leaders or followers for the consumer |
||||
// offsets topic. |
||||
getTopicDelta(Topic.GROUP_METADATA_TOPIC_NAME, newImage, delta).foreach { topicDelta => |
||||
topicDelta.newLocalLeaders(brokerId).forEach { |
||||
entry => groupCoordinator.onElection(entry.getKey(), entry.getValue().leaderEpoch) |
||||
} |
||||
topicDelta.newLocalFollowers(brokerId).forEach { |
||||
entry => groupCoordinator.onResignation(entry.getKey(), Some(entry.getValue().leaderEpoch)) |
||||
} |
||||
} |
||||
|
||||
// Handle the case where the old transaction state topic was deleted. |
||||
if (topicsDelta.topicWasDeleted(Topic.TRANSACTION_STATE_TOPIC_NAME)) { |
||||
topicsDelta.image().getTopic(Topic.TRANSACTION_STATE_TOPIC_NAME).partitions().entrySet().forEach { |
||||
entry => |
||||
if (entry.getValue().leader == brokerId) { |
||||
txnCoordinator.onResignation(entry.getKey(), Some(entry.getValue().leaderEpoch)) |
||||
} |
||||
} |
||||
} |
||||
// If the transaction state topic changed in a way that's relevant to this broker, |
||||
// notify the transaction coordinator. |
||||
getTopicDelta(Topic.TRANSACTION_STATE_TOPIC_NAME, newImage, delta).foreach { topicDelta => |
||||
topicDelta.newLocalLeaders(brokerId).forEach { |
||||
entry => txnCoordinator.onElection(entry.getKey(), entry.getValue().leaderEpoch) |
||||
} |
||||
topicDelta.newLocalFollowers(brokerId).forEach { |
||||
entry => txnCoordinator.onResignation(entry.getKey(), Some(entry.getValue().leaderEpoch)) |
||||
} |
||||
} |
||||
|
||||
// Notify the group coordinator about deleted topics. |
||||
val deletedTopicPartitions = new mutable.ArrayBuffer[TopicPartition]() |
||||
topicsDelta.deletedTopicIds().forEach { id => |
||||
val topicImage = topicsDelta.image().getTopic(id) |
||||
topicImage.partitions().keySet().forEach { |
||||
id => deletedTopicPartitions += new TopicPartition(topicImage.name(), id) |
||||
} |
||||
} |
||||
if (deletedTopicPartitions.nonEmpty) { |
||||
groupCoordinator.handleDeletedPartitions(deletedTopicPartitions, RequestLocal.NoCaching) |
||||
} |
||||
} |
||||
|
||||
// Apply configuration deltas. |
||||
Option(delta.configsDelta()).foreach { configsDelta => |
||||
configsDelta.changes().keySet().forEach { configResource => |
||||
val tag = configResource.`type`() match { |
||||
case ConfigResource.Type.TOPIC => Some(ConfigType.Topic) |
||||
case ConfigResource.Type.BROKER => Some(ConfigType.Broker) |
||||
case _ => None |
||||
} |
||||
tag.foreach { t => |
||||
val newProperties = newImage.configs().configProperties(configResource) |
||||
dynamicConfigHandlers(t).processConfigChanges(configResource.name(), newProperties) |
||||
} |
||||
} |
||||
} |
||||
|
||||
// Apply client quotas delta. |
||||
Option(delta.clientQuotasDelta()).foreach { clientQuotasDelta => |
||||
clientQuotaMetadataManager.update(clientQuotasDelta) |
||||
} |
||||
|
||||
if (_firstPublish) { |
||||
finishInitializingReplicaManager(newImage) |
||||
} |
||||
} catch { |
||||
case t: Throwable => error(s"Error publishing broker metadata at ${newHighestMetadataOffset}", t) |
||||
throw t |
||||
} finally { |
||||
_firstPublish = false |
||||
} |
||||
} |
||||
|
||||
private def initializeManagers(): Unit = { |
||||
// Start log manager, which will perform (potentially lengthy) |
||||
// recovery-from-unclean-shutdown if required. |
||||
logManager.startup(metadataCache.getAllTopics()) |
||||
|
||||
// Start the replica manager. |
||||
replicaManager.startup() |
||||
|
||||
// Start the group coordinator. |
||||
groupCoordinator.startup(() => metadataCache.numPartitions( |
||||
Topic.GROUP_METADATA_TOPIC_NAME).getOrElse(conf.offsetsTopicPartitions)) |
||||
|
||||
// Start the transaction coordinator. |
||||
txnCoordinator.startup(() => metadataCache.numPartitions( |
||||
Topic.TRANSACTION_STATE_TOPIC_NAME).getOrElse(conf.transactionTopicPartitions)) |
||||
} |
||||
|
||||
private def finishInitializingReplicaManager(newImage: MetadataImage): Unit = { |
||||
// Delete log directories which we're not supposed to have, according to the |
||||
// latest metadata. This is only necessary to do when we're first starting up. If |
||||
// we have to load a snapshot later, these topics will appear in deletedTopicIds. |
||||
val ghostReplicas = findGhostReplicas(brokerId, newImage, logManager.allLogs) |
||||
if (ghostReplicas.nonEmpty) { |
||||
replicaManager.deleteGhostReplicas(ghostReplicas) |
||||
} |
||||
|
||||
// Make sure that the high water mark checkpoint thread is running for the replica |
||||
// manager. |
||||
replicaManager.startHighWatermarkCheckPointThread() |
||||
} |
||||
} |
@ -1,111 +0,0 @@
@@ -1,111 +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 |
||||
import java.util.Properties |
||||
import java.util.concurrent.ConcurrentHashMap |
||||
import java.util.function.BiFunction |
||||
|
||||
import org.apache.kafka.common.config.ConfigResource |
||||
import org.apache.kafka.common.config.ConfigResource.Type |
||||
|
||||
import scala.jdk.CollectionConverters._ |
||||
|
||||
/** |
||||
* A ConfigRepository that stores configurations locally. |
||||
*/ |
||||
class CachedConfigRepository extends ConfigRepository { |
||||
private val configMap = new ConcurrentHashMap[ConfigResource, util.HashMap[String, String]] |
||||
|
||||
/** |
||||
* Set the topic config for the given topic name and the given key to the given value. |
||||
* |
||||
* @param topicName the name of the topic for which the config will be set |
||||
* @param key the key identifying the topic config to set |
||||
* @param value the value to set for the topic config with null implying a removal |
||||
*/ |
||||
def setTopicConfig(topicName: String, key: String, value: String): Unit = { |
||||
setConfig(new ConfigResource(Type.TOPIC, topicName), key, value) |
||||
} |
||||
|
||||
/** |
||||
* Set the broker config for the given broker ID and the given key to the given value. |
||||
* |
||||
* @param brokerId the ID of the broker for which the config will be set |
||||
* @param key the key identifying the broker config to set |
||||
* @param value the value to set for the broker config with null implying a removal |
||||
*/ |
||||
def setBrokerConfig(brokerId: Int, key: String, value: String): Unit = { |
||||
setConfig(new ConfigResource(Type.BROKER, brokerId.toString()), key, value) |
||||
} |
||||
|
||||
/** |
||||
* Set the config for the given resource and the given key to the given value. |
||||
* |
||||
* @param configResource the resource for which the config will be set |
||||
* @param key the key identifying the resource config to set |
||||
* @param value the value to set for the resource config with null implying a removal |
||||
*/ |
||||
def setConfig(configResource: ConfigResource, key: String, value: String): Unit = { |
||||
configMap.compute(configResource, new BiFunction[ConfigResource, util.HashMap[String, String], util.HashMap[String, String]] { |
||||
override def apply(resource: ConfigResource, |
||||
curConfig: util.HashMap[String, String]): util.HashMap[String, String] = { |
||||
if (value == null) { |
||||
if (curConfig == null) { |
||||
null |
||||
} else { |
||||
val newConfig = new util.HashMap[String, String](curConfig) |
||||
newConfig.remove(key) |
||||
if (newConfig.isEmpty) { |
||||
null |
||||
} else { |
||||
newConfig |
||||
} |
||||
} |
||||
} else { |
||||
if (curConfig == null) { |
||||
val newConfig = new util.HashMap[String, String](1) |
||||
newConfig.put(key, value) |
||||
newConfig |
||||
} else { |
||||
val newConfig = new util.HashMap[String, String](curConfig.size() + 1) |
||||
newConfig.putAll(curConfig) |
||||
newConfig.put(key, value) |
||||
newConfig |
||||
} |
||||
} |
||||
} |
||||
}) |
||||
} |
||||
|
||||
override def config(configResource: ConfigResource): Properties = { |
||||
val properties = new Properties() |
||||
Option(configMap.get(configResource)).foreach { resourceConfigMap => |
||||
resourceConfigMap.entrySet.iterator.asScala.foreach { entry => |
||||
properties.put(entry.getKey, entry.getValue) |
||||
} |
||||
} |
||||
properties |
||||
} |
||||
|
||||
def remove(configResource: ConfigResource): Unit = { |
||||
configMap.remove(configResource) |
||||
} |
||||
|
||||
} |
@ -1,307 +0,0 @@
@@ -1,307 +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 kafka.utils.CoreUtils.{inReadLock, inWriteLock} |
||||
import org.apache.kafka.common.errors.{InvalidRequestException, UnsupportedVersionException} |
||||
import org.apache.kafka.common.quota.{ClientQuotaEntity, ClientQuotaFilterComponent} |
||||
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock |
||||
import scala.collection.mutable |
||||
import scala.jdk.CollectionConverters._ |
||||
|
||||
|
||||
// A type for the cache index keys |
||||
sealed trait CacheIndexKey |
||||
case object DefaultUser extends CacheIndexKey |
||||
case class SpecificUser(user: String) extends CacheIndexKey |
||||
case object DefaultClientId extends CacheIndexKey |
||||
case class SpecificClientId(clientId: String) extends CacheIndexKey |
||||
case object DefaultIp extends CacheIndexKey |
||||
case class SpecificIp(ip: String) extends CacheIndexKey |
||||
|
||||
|
||||
// Different types of matching constraints |
||||
sealed trait QuotaMatch |
||||
case class ExactMatch(entityName: String) extends QuotaMatch |
||||
case object DefaultMatch extends QuotaMatch |
||||
case object TypeMatch extends QuotaMatch |
||||
|
||||
|
||||
/** |
||||
* Maintains a cache of QuotaEntity and their respective quotas. |
||||
* |
||||
* The main cache is structured like: |
||||
* |
||||
* <pre> |
||||
* { |
||||
* (user:alice) -> {consumer_byte_rate: 10000}, |
||||
* (user:alice,client:x) -> {consumer_byte_rate: 8000, producer_byte_rate: 8000} |
||||
* } |
||||
* </pre> |
||||
* |
||||
* In addition to this cache, this class maintains three indexes for the three supported entity types (user, client, |
||||
* and IP). These indexes map a part of an entity to the list of all QuotaEntity which contain that entity. For example: |
||||
* |
||||
* <pre> |
||||
* { |
||||
* SpecificUser(alice) -> [(user:alice), (user:alice,client:x)], |
||||
* DefaultUser -> [(user:default), (user:default, client:x)] |
||||
* } |
||||
* </pre> |
||||
* |
||||
* These indexes exist to support the flexible lookups needed by DescribeClientQuota RPC |
||||
*/ |
||||
class ClientQuotaCache { |
||||
private type QuotaCacheIndex = mutable.HashMap[CacheIndexKey, mutable.HashSet[QuotaEntity]] |
||||
|
||||
private val quotaCache = new mutable.HashMap[QuotaEntity, mutable.Map[String, Double]] |
||||
|
||||
// We need three separate indexes because we also support wildcard lookups on entity type. |
||||
private val userEntityIndex = new QuotaCacheIndex |
||||
private val clientIdEntityIndex = new QuotaCacheIndex |
||||
private val ipEntityIndex = new QuotaCacheIndex |
||||
|
||||
private val lock = new ReentrantReadWriteLock() |
||||
|
||||
/** |
||||
* Return quota entries for a given filter. These entries are returned from an in-memory cache and may not reflect |
||||
* the latest state of the quotas according to the controller. If a filter is given for an unsupported entity type |
||||
* or an invalid combination of entity types, this method will throw an exception. |
||||
* |
||||
* @param filters A collection of quota filters (entity type and a match clause). |
||||
* @param strict True if we should only return entities which match all the filter clauses and have no |
||||
* additional unmatched parts. |
||||
* @return A mapping of quota entities along with their quota values. |
||||
*/ |
||||
def describeClientQuotas(filters: Seq[ClientQuotaFilterComponent], strict: Boolean): |
||||
Map[ClientQuotaEntity, Map[String, Double]] = inReadLock(lock) { |
||||
describeClientQuotasInternal(filters, strict).map { case (entity, value) => convertEntity(entity) -> value} |
||||
} |
||||
|
||||
// Visible for testing (QuotaEntity is nicer for assertions in test code) |
||||
private[metadata] def describeClientQuotasInternal(filters: Seq[ClientQuotaFilterComponent], strict: Boolean): |
||||
Map[QuotaEntity, Map[String, Double]] = inReadLock(lock) { |
||||
|
||||
// Do some preliminary validation of the filter types and convert them to correct QuotaMatch type |
||||
val entityFilters = mutable.HashMap.empty[String, QuotaMatch] |
||||
filters.foreach { component => |
||||
val entityType = component.entityType() |
||||
if (entityFilters.contains(entityType)) { |
||||
throw new InvalidRequestException(s"Duplicate ${entityType} filter component entity type") |
||||
} else if (entityType.isEmpty) { |
||||
throw new InvalidRequestException("Unexpected empty filter component entity type") |
||||
} else if (!ClientQuotaEntity.isValidEntityType(entityType)) { |
||||
throw new UnsupportedVersionException(s"Custom entity type ${entityType} not supported") |
||||
} |
||||
|
||||
// A present "match()" is an exact match on name, an absent "match()" is a match on the default entity, |
||||
// and a null "match()" is a match on the entity type |
||||
val entityMatch = if (component.`match`() != null && component.`match`().isPresent) { |
||||
ExactMatch(component.`match`().get()) |
||||
} else if (component.`match`() != null) { |
||||
DefaultMatch |
||||
} else { |
||||
TypeMatch |
||||
} |
||||
entityFilters.put(entityType, entityMatch) |
||||
} |
||||
|
||||
// Special case for non-strict empty filter, match everything |
||||
if (filters.isEmpty && !strict) { |
||||
val allResults: Map[QuotaEntity, Map[String, Double]] = quotaCache.map { |
||||
entry => entry._1 -> entry._2.toMap |
||||
}.toMap |
||||
return allResults |
||||
} |
||||
|
||||
if (entityFilters.isEmpty) { |
||||
return Map.empty |
||||
} |
||||
|
||||
// We do not allow IP filters to be combined with user or client filters |
||||
val matchingEntities: Set[QuotaEntity] = if (entityFilters.contains(ClientQuotaEntity.IP)) { |
||||
if (entityFilters.size > 1) { |
||||
throw new InvalidRequestException("Invalid entity filter component combination, IP filter component should " + |
||||
"not be used with User or ClientId filter component.") |
||||
} |
||||
val ipMatch = entityFilters.get(ClientQuotaEntity.IP) |
||||
ipMatch.fold(Set.empty[QuotaEntity]) { |
||||
case ExactMatch(ip) => ipEntityIndex.getOrElse(SpecificIp(ip), Set.empty).toSet |
||||
case DefaultMatch => ipEntityIndex.getOrElse(DefaultIp, Set.empty).toSet |
||||
case TypeMatch => ipEntityIndex.values.flatten.toSet |
||||
} |
||||
} else if (entityFilters.contains(ClientQuotaEntity.USER) || entityFilters.contains(ClientQuotaEntity.CLIENT_ID)) { |
||||
// If either are present, check both user and client indexes |
||||
val userMatch = entityFilters.get(ClientQuotaEntity.USER) |
||||
val userIndexMatches = userMatch.fold(Set.empty[QuotaEntity]) { |
||||
case ExactMatch(user) => userEntityIndex.getOrElse(SpecificUser(user), Set.empty).toSet |
||||
case DefaultMatch => userEntityIndex.getOrElse(DefaultUser, Set.empty).toSet |
||||
case TypeMatch => userEntityIndex.values.flatten.toSet |
||||
} |
||||
|
||||
val clientMatch = entityFilters.get(ClientQuotaEntity.CLIENT_ID) |
||||
val clientIndexMatches = clientMatch.fold(Set.empty[QuotaEntity]) { |
||||
case ExactMatch(clientId) => clientIdEntityIndex.getOrElse(SpecificClientId(clientId), Set.empty).toSet |
||||
case DefaultMatch => clientIdEntityIndex.getOrElse(DefaultClientId, Set.empty).toSet |
||||
case TypeMatch => clientIdEntityIndex.values.flatten.toSet |
||||
} |
||||
|
||||
val candidateMatches = if (userMatch.isDefined && clientMatch.isDefined) { |
||||
userIndexMatches.intersect(clientIndexMatches) |
||||
} else if (userMatch.isDefined) { |
||||
userIndexMatches |
||||
} else { |
||||
clientIndexMatches |
||||
} |
||||
|
||||
if (strict) { |
||||
// If in strict mode, we need to remove any matches with unspecified entity types. This only applies to results |
||||
// with more than one entity part (i.e., user and clientId) |
||||
candidateMatches.filter { quotaEntity => |
||||
quotaEntity match { |
||||
case ExplicitUserExplicitClientIdEntity(_, _) => userMatch.isDefined && clientMatch.isDefined |
||||
case DefaultUserExplicitClientIdEntity(_) => userMatch.isDefined && clientMatch.isDefined |
||||
case ExplicitUserDefaultClientIdEntity(_) => userMatch.isDefined && clientMatch.isDefined |
||||
case DefaultUserDefaultClientIdEntity => userMatch.isDefined && clientMatch.isDefined |
||||
case _ => true |
||||
} |
||||
} |
||||
} else { |
||||
candidateMatches |
||||
} |
||||
} else { |
||||
// ClientQuotaEntity.isValidEntityType check above should prevent any unknown entity types |
||||
throw new IllegalStateException(s"Unexpected handling of ${entityFilters} after filter validation") |
||||
} |
||||
|
||||
val resultsMap: Map[QuotaEntity, Map[String, Double]] = matchingEntities.map { |
||||
quotaEntity => { |
||||
quotaCache.get(quotaEntity) match { |
||||
case Some(quotas) => quotaEntity -> quotas.toMap |
||||
case None => quotaEntity -> Map.empty[String, Double] |
||||
} |
||||
} |
||||
}.toMap |
||||
|
||||
resultsMap |
||||
} |
||||
|
||||
private def convertEntity(entity: QuotaEntity): ClientQuotaEntity = { |
||||
val entityMap = entity match { |
||||
case IpEntity(ip) => Map(ClientQuotaEntity.IP -> ip) |
||||
case DefaultIpEntity => Map(ClientQuotaEntity.IP -> null) |
||||
case UserEntity(user) => Map(ClientQuotaEntity.USER -> user) |
||||
case DefaultUserEntity => Map(ClientQuotaEntity.USER -> null) |
||||
case ClientIdEntity(clientId) => Map(ClientQuotaEntity.CLIENT_ID -> clientId) |
||||
case DefaultClientIdEntity => Map(ClientQuotaEntity.CLIENT_ID -> null) |
||||
case ExplicitUserExplicitClientIdEntity(user, clientId) => |
||||
Map(ClientQuotaEntity.USER -> user, ClientQuotaEntity.CLIENT_ID -> clientId) |
||||
case ExplicitUserDefaultClientIdEntity(user) => |
||||
Map(ClientQuotaEntity.USER -> user, ClientQuotaEntity.CLIENT_ID -> null) |
||||
case DefaultUserExplicitClientIdEntity(clientId) => |
||||
Map(ClientQuotaEntity.USER -> null, ClientQuotaEntity.CLIENT_ID -> clientId) |
||||
case DefaultUserDefaultClientIdEntity => |
||||
Map(ClientQuotaEntity.USER -> null, ClientQuotaEntity.CLIENT_ID -> null) |
||||
} |
||||
new ClientQuotaEntity(entityMap.asJava) |
||||
} |
||||
|
||||
// Update the cache indexes |
||||
private def updateCacheIndex(quotaEntity: QuotaEntity, |
||||
remove: Boolean) |
||||
(quotaCacheIndex: QuotaCacheIndex, |
||||
key: CacheIndexKey): Unit = { |
||||
if (remove) { |
||||
val needsCleanup = quotaCacheIndex.get(key) match { |
||||
case Some(quotaEntitySet) => |
||||
quotaEntitySet.remove(quotaEntity) |
||||
quotaEntitySet.isEmpty |
||||
case None => false |
||||
} |
||||
if (needsCleanup) { |
||||
quotaCacheIndex.remove(key) |
||||
} |
||||
} else { |
||||
quotaCacheIndex.getOrElseUpdate(key, mutable.HashSet.empty).add(quotaEntity) |
||||
} |
||||
} |
||||
|
||||
/** |
||||
* Update the quota cache with the given entity and quota key/value. If remove is set, the value is ignore and |
||||
* the quota entry is removed for the given key. No validation on quota keys is performed here, it is assumed |
||||
* that the caller has already done this. |
||||
* |
||||
* @param entity A quota entity, either a specific entity or the default entity for the given type(s) |
||||
* @param key The quota key |
||||
* @param value The quota value |
||||
* @param remove True if we should remove the given quota key from the entity's quota cache |
||||
*/ |
||||
def updateQuotaCache(entity: QuotaEntity, key: String, value: Double, remove: Boolean): Unit = inWriteLock(lock) { |
||||
val quotaValues = quotaCache.getOrElseUpdate(entity, mutable.HashMap.empty) |
||||
val removeFromIndex = if (remove) { |
||||
quotaValues.remove(key) |
||||
if (quotaValues.isEmpty) { |
||||
quotaCache.remove(entity) |
||||
true |
||||
} else { |
||||
false |
||||
} |
||||
} else { |
||||
quotaValues.put(key, value) |
||||
false |
||||
} |
||||
|
||||
// Update the appropriate indexes with the entity |
||||
val updateCacheIndexPartial: (QuotaCacheIndex, CacheIndexKey) => Unit = updateCacheIndex(entity, removeFromIndex) |
||||
entity match { |
||||
case UserEntity(user) => |
||||
updateCacheIndexPartial(userEntityIndex, SpecificUser(user)) |
||||
case DefaultUserEntity => |
||||
updateCacheIndexPartial(userEntityIndex, DefaultUser) |
||||
|
||||
case ClientIdEntity(clientId) => |
||||
updateCacheIndexPartial(clientIdEntityIndex, SpecificClientId(clientId)) |
||||
case DefaultClientIdEntity => |
||||
updateCacheIndexPartial(clientIdEntityIndex, DefaultClientId) |
||||
|
||||
case ExplicitUserExplicitClientIdEntity(user, clientId) => |
||||
updateCacheIndexPartial(userEntityIndex, SpecificUser(user)) |
||||
updateCacheIndexPartial(clientIdEntityIndex, SpecificClientId(clientId)) |
||||
|
||||
case ExplicitUserDefaultClientIdEntity(user) => |
||||
updateCacheIndexPartial(userEntityIndex, SpecificUser(user)) |
||||
updateCacheIndexPartial(clientIdEntityIndex, DefaultClientId) |
||||
|
||||
case DefaultUserExplicitClientIdEntity(clientId) => |
||||
updateCacheIndexPartial(userEntityIndex, DefaultUser) |
||||
updateCacheIndexPartial(clientIdEntityIndex, SpecificClientId(clientId)) |
||||
|
||||
case DefaultUserDefaultClientIdEntity => |
||||
updateCacheIndexPartial(userEntityIndex, DefaultUser) |
||||
updateCacheIndexPartial(clientIdEntityIndex, DefaultClientId) |
||||
|
||||
case IpEntity(ip) => |
||||
updateCacheIndexPartial(ipEntityIndex, SpecificIp(ip)) |
||||
case DefaultIpEntity => |
||||
updateCacheIndexPartial(ipEntityIndex, DefaultIp) |
||||
} |
||||
} |
||||
|
||||
override def toString = s"ClientQuotaCache($quotaCache)" |
||||
} |
@ -0,0 +1,354 @@
@@ -0,0 +1,354 @@
|
||||
/* |
||||
* 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 kafka.controller.StateChangeLogger |
||||
import kafka.server.MetadataCache |
||||
import kafka.utils.Logging |
||||
import org.apache.kafka.common.internals.Topic |
||||
import org.apache.kafka.common.message.MetadataResponseData.{MetadataResponsePartition, MetadataResponseTopic} |
||||
import org.apache.kafka.common.{Cluster, Node, PartitionInfo, TopicPartition, Uuid} |
||||
import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataPartitionState |
||||
import org.apache.kafka.common.network.ListenerName |
||||
import org.apache.kafka.common.protocol.Errors |
||||
import org.apache.kafka.common.requests.MetadataResponse |
||||
import org.apache.kafka.image.MetadataImage |
||||
import java.util |
||||
import java.util.{Collections, Properties} |
||||
import java.util.concurrent.ThreadLocalRandom |
||||
|
||||
import kafka.admin.BrokerMetadata |
||||
import org.apache.kafka.common.config.ConfigResource |
||||
import org.apache.kafka.common.message.{DescribeClientQuotasRequestData, DescribeClientQuotasResponseData} |
||||
import org.apache.kafka.metadata.{PartitionRegistration, Replicas} |
||||
|
||||
import scala.collection.{Seq, Set, mutable} |
||||
import scala.jdk.CollectionConverters._ |
||||
import scala.compat.java8.OptionConverters._ |
||||
|
||||
|
||||
class KRaftMetadataCache(val brokerId: Int) extends MetadataCache with Logging with ConfigRepository { |
||||
this.logIdent = s"[MetadataCache brokerId=$brokerId] " |
||||
|
||||
// This is the cache state. Every MetadataImage instance is immutable, and updates |
||||
// replace this value with a completely new one. This means reads (which are not under |
||||
// any lock) need to grab the value of this variable once, and retain that read copy for |
||||
// the duration of their operation. Multiple reads of this value risk getting different |
||||
// image values. |
||||
@volatile private var _currentImage: MetadataImage = MetadataImage.EMPTY |
||||
|
||||
private val stateChangeLogger = new StateChangeLogger(brokerId, inControllerContext = false, None) |
||||
|
||||
// This method is the main hotspot when it comes to the performance of metadata requests, |
||||
// we should be careful about adding additional logic here. |
||||
// filterUnavailableEndpoints exists to support v0 MetadataResponses |
||||
private def maybeFilterAliveReplicas(image: MetadataImage, |
||||
brokers: Array[Int], |
||||
listenerName: ListenerName, |
||||
filterUnavailableEndpoints: Boolean): java.util.List[Integer] = { |
||||
if (!filterUnavailableEndpoints) { |
||||
Replicas.toList(brokers) |
||||
} else { |
||||
val res = new util.ArrayList[Integer](brokers.length) |
||||
for (brokerId <- brokers) { |
||||
Option(image.cluster().broker(brokerId)).foreach { b => |
||||
if (!b.fenced() && b.listeners().containsKey(listenerName.value())) { |
||||
res.add(brokerId) |
||||
} |
||||
} |
||||
} |
||||
res |
||||
} |
||||
} |
||||
|
||||
def currentImage(): MetadataImage = _currentImage |
||||
|
||||
// errorUnavailableEndpoints exists to support v0 MetadataResponses |
||||
// If errorUnavailableListeners=true, return LISTENER_NOT_FOUND if listener is missing on the broker. |
||||
// Otherwise, return LEADER_NOT_AVAILABLE for broker unavailable and missing listener (Metadata response v5 and below). |
||||
private def getPartitionMetadata(image: MetadataImage, topicName: String, listenerName: ListenerName, errorUnavailableEndpoints: Boolean, |
||||
errorUnavailableListeners: Boolean): Option[Iterator[MetadataResponsePartition]] = { |
||||
Option(image.topics().getTopic(topicName)) match { |
||||
case None => None |
||||
case Some(topic) => Some(topic.partitions().entrySet().asScala.map { entry => |
||||
val partitionId = entry.getKey |
||||
val partition = entry.getValue |
||||
val filteredReplicas = maybeFilterAliveReplicas(image, partition.replicas, |
||||
listenerName, errorUnavailableEndpoints) |
||||
val filteredIsr = maybeFilterAliveReplicas(image, partition.isr, listenerName, |
||||
errorUnavailableEndpoints) |
||||
val offlineReplicas = getOfflineReplicas(image, partition, listenerName) |
||||
val maybeLeader = getAliveEndpoint(image, partition.leader, listenerName) |
||||
maybeLeader match { |
||||
case None => |
||||
val error = if (!image.cluster().brokers.containsKey(partition.leader)) { |
||||
debug(s"Error while fetching metadata for ${topicName}-${partitionId}: leader not available") |
||||
Errors.LEADER_NOT_AVAILABLE |
||||
} else { |
||||
debug(s"Error while fetching metadata for ${topicName}-${partitionId}: listener $listenerName " + |
||||
s"not found on leader ${partition.leader}") |
||||
if (errorUnavailableListeners) Errors.LISTENER_NOT_FOUND else Errors.LEADER_NOT_AVAILABLE |
||||
} |
||||
new MetadataResponsePartition() |
||||
.setErrorCode(error.code) |
||||
.setPartitionIndex(partitionId) |
||||
.setLeaderId(MetadataResponse.NO_LEADER_ID) |
||||
.setLeaderEpoch(partition.leaderEpoch) |
||||
.setReplicaNodes(filteredReplicas) |
||||
.setIsrNodes(filteredIsr) |
||||
.setOfflineReplicas(offlineReplicas) |
||||
case Some(leader) => |
||||
val error = if (filteredReplicas.size < partition.replicas.size) { |
||||
debug(s"Error while fetching metadata for ${topicName}-${partitionId}: replica information not available for " + |
||||
s"following brokers ${partition.replicas.filterNot(filteredReplicas.contains).mkString(",")}") |
||||
Errors.REPLICA_NOT_AVAILABLE |
||||
} else if (filteredIsr.size < partition.isr.size) { |
||||
debug(s"Error while fetching metadata for ${topicName}-${partitionId}: in sync replica information not available for " + |
||||
s"following brokers ${partition.isr.filterNot(filteredIsr.contains).mkString(",")}") |
||||
Errors.REPLICA_NOT_AVAILABLE |
||||
} else { |
||||
Errors.NONE |
||||
} |
||||
|
||||
new MetadataResponsePartition() |
||||
.setErrorCode(error.code) |
||||
.setPartitionIndex(partitionId) |
||||
.setLeaderId(leader.id()) |
||||
.setLeaderEpoch(partition.leaderEpoch) |
||||
.setReplicaNodes(filteredReplicas) |
||||
.setIsrNodes(filteredIsr) |
||||
.setOfflineReplicas(offlineReplicas) |
||||
} |
||||
}.iterator) |
||||
} |
||||
} |
||||
|
||||
private def getOfflineReplicas(image: MetadataImage, |
||||
partition: PartitionRegistration, |
||||
listenerName: ListenerName): util.List[Integer] = { |
||||
// TODO: in order to really implement this correctly, we would need JBOD support. |
||||
// That would require us to track which replicas were offline on a per-replica basis. |
||||
// See KAFKA-13005. |
||||
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 (broker.fenced() || !broker.listeners().containsKey(listenerName.value())) { |
||||
offlineReplicas.add(brokerId) |
||||
} |
||||
} |
||||
} |
||||
offlineReplicas |
||||
} |
||||
|
||||
/** |
||||
* 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. |
||||
* |
||||
* @return None if broker is not alive or if the broker does not have a listener named `listenerName`. |
||||
*/ |
||||
private def getAliveEndpoint(image: MetadataImage, id: Int, listenerName: ListenerName): Option[Node] = { |
||||
Option(image.cluster().broker(id)).flatMap(_.node(listenerName.value()).asScala) |
||||
} |
||||
|
||||
// errorUnavailableEndpoints exists to support v0 MetadataResponses |
||||
override def getTopicMetadata(topics: Set[String], |
||||
listenerName: ListenerName, |
||||
errorUnavailableEndpoints: Boolean = false, |
||||
errorUnavailableListeners: Boolean = false): Seq[MetadataResponseTopic] = { |
||||
val image = _currentImage |
||||
topics.toSeq.flatMap { topic => |
||||
getPartitionMetadata(image, topic, listenerName, errorUnavailableEndpoints, errorUnavailableListeners).map { partitionMetadata => |
||||
new MetadataResponseTopic() |
||||
.setErrorCode(Errors.NONE.code) |
||||
.setName(topic) |
||||
.setTopicId(Option(image.topics().getTopic(topic).id()).getOrElse(Uuid.ZERO_UUID)) |
||||
.setIsInternal(Topic.isInternal(topic)) |
||||
.setPartitions(partitionMetadata.toBuffer.asJava) |
||||
} |
||||
} |
||||
} |
||||
|
||||
override def getAllTopics(): Set[String] = _currentImage.topics().topicsByName().keySet().asScala |
||||
|
||||
override def getTopicPartitions(topicName: String): Set[TopicPartition] = { |
||||
Option(_currentImage.topics().getTopic(topicName)) match { |
||||
case None => Set.empty |
||||
case Some(topic) => topic.partitions().keySet().asScala.map(new TopicPartition(topicName, _)) |
||||
} |
||||
} |
||||
|
||||
override def hasAliveBroker(brokerId: Int): Boolean = { |
||||
Option(_currentImage.cluster().broker(brokerId)).count(!_.fenced()) == 1 |
||||
} |
||||
|
||||
override def getAliveBrokers(): Iterable[BrokerMetadata] = getAliveBrokers(_currentImage) |
||||
|
||||
private def getAliveBrokers(image: MetadataImage): Iterable[BrokerMetadata] = { |
||||
image.cluster().brokers().values().asScala.filter(!_.fenced()). |
||||
map(b => BrokerMetadata(b.id, b.rack.asScala)) |
||||
} |
||||
|
||||
override def getAliveBrokerNode(brokerId: Int, listenerName: ListenerName): Option[Node] = { |
||||
Option(_currentImage.cluster().broker(brokerId)). |
||||
flatMap(_.node(listenerName.value()).asScala) |
||||
} |
||||
|
||||
override def getAliveBrokerNodes(listenerName: ListenerName): Seq[Node] = { |
||||
_currentImage.cluster().brokers().values().asScala.filter(!_.fenced()). |
||||
flatMap(_.node(listenerName.value()).asScala).toSeq |
||||
} |
||||
|
||||
override def getPartitionInfo(topicName: String, partitionId: Int): Option[UpdateMetadataPartitionState] = { |
||||
Option(_currentImage.topics().getTopic(topicName)). |
||||
flatMap(topic => Some(topic.partitions().get(partitionId))). |
||||
flatMap(partition => Some(new UpdateMetadataPartitionState(). |
||||
setTopicName(topicName). |
||||
setPartitionIndex(partitionId). |
||||
setControllerEpoch(-1). // Controller epoch is not stored in the cache. |
||||
setLeader(partition.leader). |
||||
setLeaderEpoch(partition.leaderEpoch). |
||||
setIsr(Replicas.toList(partition.isr)). |
||||
setZkVersion(partition.partitionEpoch))) |
||||
} |
||||
|
||||
override def numPartitions(topicName: String): Option[Int] = { |
||||
Option(_currentImage.topics().getTopic(topicName)). |
||||
map(topic => topic.partitions().size()) |
||||
} |
||||
|
||||
// if the leader is not known, return None; |
||||
// if the leader is known and corresponding node is available, return Some(node) |
||||
// if the leader is known but corresponding node with the listener name is not available, return Some(NO_NODE) |
||||
override def getPartitionLeaderEndpoint(topicName: String, partitionId: Int, listenerName: ListenerName): Option[Node] = { |
||||
val image = _currentImage |
||||
Option(image.topics().getTopic(topicName)) match { |
||||
case None => None |
||||
case Some(topic) => Option(topic.partitions().get(partitionId)) match { |
||||
case None => None |
||||
case Some(partition) => Option(image.cluster().broker(partition.leader)) match { |
||||
case None => Some(Node.noNode) |
||||
case Some(broker) => Some(broker.node(listenerName.value()).orElse(Node.noNode())) |
||||
} |
||||
} |
||||
} |
||||
} |
||||
|
||||
override def getPartitionReplicaEndpoints(tp: TopicPartition, listenerName: ListenerName): Map[Int, Node] = { |
||||
val image = _currentImage |
||||
val result = new mutable.HashMap[Int, Node]() |
||||
Option(image.topics().getTopic(tp.topic())).foreach { topic => |
||||
topic.partitions().values().forEach { case partition => |
||||
partition.replicas.map { case replicaId => |
||||
result.put(replicaId, Option(image.cluster().broker(replicaId)) match { |
||||
case None => Node.noNode() |
||||
case Some(broker) => broker.node(listenerName.value()).asScala.getOrElse(Node.noNode()) |
||||
}) |
||||
} |
||||
} |
||||
} |
||||
result.toMap |
||||
} |
||||
|
||||
override def getControllerId: Option[Int] = getRandomAliveBroker(_currentImage) |
||||
|
||||
/** |
||||
* Choose a random broker node to report as the controller. We do this because we want |
||||
* the client to send requests destined for the controller to a random broker. |
||||
* Clients do not have direct access to the controller in the KRaft world, as explained |
||||
* in KIP-590. |
||||
*/ |
||||
private def getRandomAliveBroker(image: MetadataImage): Option[Int] = { |
||||
val aliveBrokers = getAliveBrokers(image).toList |
||||
if (aliveBrokers.size == 0) { |
||||
None |
||||
} else { |
||||
Some(aliveBrokers(ThreadLocalRandom.current().nextInt(aliveBrokers.size)).id) |
||||
} |
||||
} |
||||
|
||||
override def getClusterMetadata(clusterId: String, listenerName: ListenerName): Cluster = { |
||||
val image = _currentImage |
||||
val nodes = new util.HashMap[Integer, Node] |
||||
image.cluster().brokers().values().forEach { broker => |
||||
if (!broker.fenced()) { |
||||
broker.node(listenerName.value()).asScala.foreach { node => |
||||
nodes.put(broker.id(), node) |
||||
} |
||||
} |
||||
} |
||||
|
||||
def node(id: Int): Node = { |
||||
Option(nodes.get(id)).getOrElse(Node.noNode()) |
||||
} |
||||
|
||||
val partitionInfos = new util.ArrayList[PartitionInfo] |
||||
val internalTopics = new util.HashSet[String] |
||||
|
||||
image.topics().topicsByName().values().forEach { topic => |
||||
topic.partitions().entrySet().forEach { entry => |
||||
val partitionId = entry.getKey() |
||||
val partition = entry.getValue() |
||||
partitionInfos.add(new PartitionInfo(topic.name(), |
||||
partitionId, |
||||
node(partition.leader), |
||||
partition.replicas.map(replica => node(replica)), |
||||
partition.isr.map(replica => node(replica)), |
||||
getOfflineReplicas(image, partition, listenerName).asScala. |
||||
map(replica => node(replica)).toArray)) |
||||
if (Topic.isInternal(topic.name())) { |
||||
internalTopics.add(topic.name()) |
||||
} |
||||
} |
||||
} |
||||
val controllerNode = node(getRandomAliveBroker(image).getOrElse(-1)) |
||||
// 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, nodes.values(), |
||||
partitionInfos, Collections.emptySet(), internalTopics, controllerNode) |
||||
} |
||||
|
||||
def stateChangeTraceEnabled(): Boolean = { |
||||
stateChangeLogger.isTraceEnabled |
||||
} |
||||
|
||||
def logStateChangeTrace(str: String): Unit = { |
||||
stateChangeLogger.trace(str) |
||||
} |
||||
|
||||
override def contains(topicName: String): Boolean = |
||||
_currentImage.topics().topicsByName().containsKey(topicName) |
||||
|
||||
override def contains(tp: TopicPartition): Boolean = { |
||||
Option(_currentImage.topics().getTopic(tp.topic())) match { |
||||
case None => false |
||||
case Some(topic) => topic.partitions().containsKey(tp.partition()) |
||||
} |
||||
} |
||||
|
||||
def setImage(newImage: MetadataImage): Unit = _currentImage = newImage |
||||
|
||||
override def config(configResource: ConfigResource): Properties = |
||||
_currentImage.configs().configProperties(configResource) |
||||
|
||||
def describeClientQuotas(request: DescribeClientQuotasRequestData): DescribeClientQuotasResponseData = { |
||||
_currentImage.clientQuotas().describe(request) |
||||
} |
||||
} |
@ -1,150 +0,0 @@
@@ -1,150 +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 |
||||
import java.util.Collections |
||||
import java.util.concurrent.ThreadLocalRandom |
||||
import kafka.cluster.{Broker, BrokerEndPoint} |
||||
import kafka.common.BrokerEndPointNotAvailableException |
||||
import org.apache.kafka.common.Node |
||||
import org.apache.kafka.common.metadata.RegisterBrokerRecord |
||||
import org.apache.kafka.common.network.ListenerName |
||||
import org.slf4j.Logger |
||||
|
||||
import scala.jdk.CollectionConverters._ |
||||
|
||||
object MetadataBroker { |
||||
def apply(record: RegisterBrokerRecord): MetadataBroker = { |
||||
new MetadataBroker(record.brokerId, record.rack, |
||||
record.endPoints().asScala.map { endPoint => |
||||
endPoint.name() -> |
||||
new Node(record.brokerId, endPoint.host, endPoint.port, record.rack) |
||||
}.toMap, |
||||
fenced = true) |
||||
} |
||||
|
||||
def apply(broker: Broker): MetadataBroker = { |
||||
new MetadataBroker(broker.id, broker.rack.orNull, |
||||
broker.endPoints.map { endpoint => |
||||
endpoint.listenerName.value -> new Node(broker.id, endpoint.host, endpoint.port, broker.rack.orNull) |
||||
}.toMap, |
||||
fenced = false) |
||||
} |
||||
} |
||||
|
||||
case class MetadataBroker(id: Int, |
||||
rack: String, |
||||
endpoints: collection.Map[String, Node], |
||||
fenced: Boolean) { |
||||
def brokerEndPoint(listenerName: ListenerName): BrokerEndPoint = { |
||||
endpoints.get(listenerName.value) match { |
||||
case None => throw new BrokerEndPointNotAvailableException( |
||||
s"End point with listener name ${listenerName.value} not found for broker $id") |
||||
case Some(node) => new BrokerEndPoint(node.id, node.host, node.port) |
||||
} |
||||
} |
||||
|
||||
def node(listenerName: ListenerName): Node = { |
||||
endpoints.getOrElse(listenerName.value, throw new BrokerEndPointNotAvailableException( |
||||
s"End point with listener name ${listenerName.value} not found for broker $id")) |
||||
} |
||||
} |
||||
|
||||
class MetadataBrokersBuilder(log: Logger, prevBrokers: MetadataBrokers) { |
||||
private var newBrokerMap = prevBrokers.cloneBrokerMap() |
||||
|
||||
def add(broker: MetadataBroker): Unit = { |
||||
newBrokerMap.put(broker.id, broker) |
||||
} |
||||
|
||||
def changeFencing(id: Int, fenced: Boolean): Unit = { |
||||
val broker = newBrokerMap.get(id) |
||||
if (broker == null) { |
||||
throw new RuntimeException(s"Unknown broker id ${id}") |
||||
} |
||||
val newBroker = new MetadataBroker(broker.id, broker.rack, broker.endpoints, fenced) |
||||
newBrokerMap.put(id, newBroker) |
||||
} |
||||
|
||||
def remove(id: Int): Unit = { |
||||
newBrokerMap.remove(id) |
||||
} |
||||
|
||||
def get(brokerId: Int): Option[MetadataBroker] = Option(newBrokerMap.get(brokerId)) |
||||
|
||||
def build(): MetadataBrokers = { |
||||
val result = MetadataBrokers(log, newBrokerMap) |
||||
newBrokerMap = Collections.unmodifiableMap(newBrokerMap) |
||||
result |
||||
} |
||||
} |
||||
|
||||
object MetadataBrokers { |
||||
def apply(log: Logger, |
||||
brokerMap: util.Map[Integer, MetadataBroker]): MetadataBrokers = { |
||||
var listenersIdenticalAcrossBrokers = true |
||||
var prevListeners: collection.Set[String] = null |
||||
val _aliveBrokers = new util.ArrayList[MetadataBroker](brokerMap.size()) |
||||
brokerMap.values().iterator().asScala.foreach { broker => |
||||
if (!broker.fenced) { |
||||
if (prevListeners == null) { |
||||
prevListeners = broker.endpoints.keySet |
||||
} else if (!prevListeners.equals(broker.endpoints.keySet)) { |
||||
listenersIdenticalAcrossBrokers = false |
||||
} |
||||
_aliveBrokers.add(broker) |
||||
} |
||||
} |
||||
if (!listenersIdenticalAcrossBrokers) { |
||||
log.error("Listeners are not identical across alive brokers. " + |
||||
_aliveBrokers.asScala.map( |
||||
broker => s"${broker.id}: ${broker.endpoints.keySet.mkString(", ")}")) |
||||
} |
||||
new MetadataBrokers(_aliveBrokers, brokerMap) |
||||
} |
||||
} |
||||
|
||||
case class MetadataBrokers(private val aliveBrokersList: util.List[MetadataBroker], |
||||
private val brokerMap: util.Map[Integer, MetadataBroker]) { |
||||
def size(): Int = brokerMap.size() |
||||
|
||||
def iterator(): Iterator[MetadataBroker] = brokerMap.values().iterator().asScala |
||||
|
||||
def cloneBrokerMap(): util.Map[Integer, MetadataBroker] = { |
||||
val result = new util.HashMap[Integer, MetadataBroker] |
||||
result.putAll(brokerMap) |
||||
result |
||||
} |
||||
|
||||
def aliveBroker(id: Int): Option[MetadataBroker] = { |
||||
get(id).filter(!_.fenced) |
||||
} |
||||
|
||||
def randomAliveBrokerId(): Option[Int] = { |
||||
if (aliveBrokersList.isEmpty) { |
||||
None |
||||
} else { |
||||
Some(aliveBrokersList.get(ThreadLocalRandom.current().nextInt(aliveBrokersList.size())).id) |
||||
} |
||||
} |
||||
|
||||
def aliveBrokers(): collection.Seq[MetadataBroker] = aliveBrokersList.asScala |
||||
|
||||
def get(id: Int): Option[MetadataBroker] = Option(brokerMap.get(id)) |
||||
} |
@ -1,137 +0,0 @@
@@ -1,137 +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 |
||||
import java.util.Collections |
||||
|
||||
import org.apache.kafka.common.{TopicPartition, Uuid} |
||||
import org.slf4j.Logger |
||||
|
||||
case class MetadataImageBuilder(brokerId: Int, |
||||
log: Logger, |
||||
prevImage: MetadataImage) { |
||||
private var _partitionsBuilder: MetadataPartitionsBuilder = null |
||||
private var _controllerId = prevImage.controllerId |
||||
private var _brokersBuilder: MetadataBrokersBuilder = null |
||||
|
||||
def partitionsBuilder(): MetadataPartitionsBuilder = { |
||||
if (_partitionsBuilder == null) { |
||||
_partitionsBuilder = new MetadataPartitionsBuilder(brokerId, prevImage.partitions) |
||||
} |
||||
_partitionsBuilder |
||||
} |
||||
|
||||
def hasPartitionChanges: Boolean = _partitionsBuilder != null |
||||
|
||||
def topicIdToName(topicId: Uuid): Option[String] = { |
||||
if (_partitionsBuilder != null) { |
||||
_partitionsBuilder.topicIdToName(topicId) |
||||
} else { |
||||
prevImage.topicIdToName(topicId) |
||||
} |
||||
} |
||||
|
||||
def topicNameToId(topicName: String): Option[Uuid] = { |
||||
if (_partitionsBuilder != null) { |
||||
_partitionsBuilder.topicNameToId(topicName) |
||||
} else { |
||||
prevImage.topicNameToId(topicName) |
||||
} |
||||
} |
||||
|
||||
def controllerId(controllerId: Option[Int]): Unit = { |
||||
_controllerId = controllerId |
||||
} |
||||
|
||||
def brokersBuilder(): MetadataBrokersBuilder = { |
||||
if (_brokersBuilder == null) { |
||||
_brokersBuilder = new MetadataBrokersBuilder(log, prevImage.brokers) |
||||
} |
||||
_brokersBuilder |
||||
} |
||||
|
||||
def broker(brokerId: Int): Option[MetadataBroker] = { |
||||
if (_brokersBuilder == null) { |
||||
prevImage.brokers.get(brokerId) |
||||
} else { |
||||
_brokersBuilder.get(brokerId) |
||||
} |
||||
} |
||||
|
||||
def partition(topicName: String, partitionId: Int): Option[MetadataPartition] = { |
||||
if (_partitionsBuilder == null) { |
||||
prevImage.partitions.topicPartition(topicName, partitionId) |
||||
} else { |
||||
_partitionsBuilder.get(topicName, partitionId) |
||||
} |
||||
} |
||||
|
||||
def hasChanges: Boolean = { |
||||
_partitionsBuilder != null || |
||||
!_controllerId.equals(prevImage.controllerId) || |
||||
_brokersBuilder != null |
||||
} |
||||
|
||||
def build(): MetadataImage = { |
||||
val nextPartitions = if (_partitionsBuilder == null) { |
||||
prevImage.partitions |
||||
} else { |
||||
_partitionsBuilder.build() |
||||
} |
||||
MetadataImage(nextPartitions, _controllerId, brokers()) |
||||
} |
||||
|
||||
def brokers(): MetadataBrokers = { |
||||
if (_brokersBuilder == null) { |
||||
prevImage.brokers |
||||
} else { |
||||
_brokersBuilder.build() |
||||
} |
||||
} |
||||
} |
||||
|
||||
case class MetadataImage(partitions: MetadataPartitions, |
||||
controllerId: Option[Int], |
||||
brokers: MetadataBrokers) { |
||||
def this() = { |
||||
this(MetadataPartitions(Collections.emptyMap(), Collections.emptyMap()), |
||||
None, |
||||
new MetadataBrokers(Collections.emptyList(), new util.HashMap[Integer, MetadataBroker]())) |
||||
} |
||||
|
||||
def contains(partition: TopicPartition): Boolean = |
||||
partitions.topicPartition(partition.topic(), partition.partition()).isDefined |
||||
|
||||
def contains(topic: String): Boolean = partitions.topicPartitions(topic).hasNext |
||||
|
||||
def aliveBroker(id: Int): Option[MetadataBroker] = brokers.aliveBroker(id) |
||||
|
||||
def numAliveBrokers(): Int = brokers.aliveBrokers().size |
||||
|
||||
def controller(): Option[MetadataBroker] = controllerId.flatMap(id => brokers.aliveBroker(id)) |
||||
|
||||
def topicIdToName(uuid: Uuid): Option[String] = { |
||||
partitions.topicIdToName(uuid) |
||||
} |
||||
|
||||
def topicNameToId(name: String): Option[Uuid] = { |
||||
partitions.topicNameToId(name) |
||||
} |
||||
} |
||||
|
@ -1,355 +0,0 @@
@@ -1,355 +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 |
||||
import java.util.Collections |
||||
|
||||
import org.apache.kafka.common.message.LeaderAndIsrRequestData |
||||
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState |
||||
import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataPartitionState |
||||
import org.apache.kafka.common.metadata.{PartitionChangeRecord, PartitionRecord} |
||||
import org.apache.kafka.common.{TopicPartition, Uuid} |
||||
|
||||
import scala.jdk.CollectionConverters._ |
||||
|
||||
|
||||
object MetadataPartition { |
||||
val NO_LEADER_CHANGE = -2 |
||||
|
||||
def apply(name: String, record: PartitionRecord): MetadataPartition = { |
||||
MetadataPartition(name, |
||||
record.partitionId(), |
||||
record.leader(), |
||||
record.leaderEpoch(), |
||||
record.replicas(), |
||||
record.isr(), |
||||
record.partitionEpoch(), |
||||
Collections.emptyList(), // TODO KAFKA-12285 handle offline replicas |
||||
Collections.emptyList(), |
||||
Collections.emptyList()) |
||||
} |
||||
|
||||
def apply(prevPartition: Option[MetadataPartition], |
||||
partition: UpdateMetadataPartitionState): MetadataPartition = { |
||||
new MetadataPartition(partition.topicName(), |
||||
partition.partitionIndex(), |
||||
partition.leader(), |
||||
partition.leaderEpoch(), |
||||
partition.replicas(), |
||||
partition.isr(), |
||||
partition.zkVersion(), |
||||
partition.offlineReplicas(), |
||||
prevPartition.flatMap(p => Some(p.addingReplicas)).getOrElse(Collections.emptyList()), |
||||
prevPartition.flatMap(p => Some(p.removingReplicas)).getOrElse(Collections.emptyList()) |
||||
) |
||||
} |
||||
} |
||||
|
||||
case class MetadataPartition(topicName: String, |
||||
partitionIndex: Int, |
||||
leaderId: Int, |
||||
leaderEpoch: Int, |
||||
replicas: util.List[Integer], |
||||
isr: util.List[Integer], |
||||
partitionEpoch: Int, |
||||
offlineReplicas: util.List[Integer], |
||||
addingReplicas: util.List[Integer], |
||||
removingReplicas: util.List[Integer]) { |
||||
def toTopicPartition: TopicPartition = new TopicPartition(topicName, partitionIndex) |
||||
|
||||
def toLeaderAndIsrPartitionState(isNew: Boolean): LeaderAndIsrRequestData.LeaderAndIsrPartitionState = { |
||||
new LeaderAndIsrPartitionState().setTopicName(topicName). |
||||
setPartitionIndex(partitionIndex). |
||||
setLeader(leaderId). |
||||
setLeaderEpoch(leaderEpoch). |
||||
setReplicas(replicas). |
||||
setIsr(isr). |
||||
setAddingReplicas(addingReplicas). |
||||
setRemovingReplicas(removingReplicas). |
||||
setIsNew(isNew). |
||||
setZkVersion(partitionEpoch) |
||||
} |
||||
|
||||
def isReplicaFor(brokerId: Int): Boolean = replicas.contains(Integer.valueOf(brokerId)) |
||||
|
||||
def merge(record: PartitionChangeRecord): MetadataPartition = { |
||||
val (newLeader, newLeaderEpoch) = if (record.leader() == MetadataPartition.NO_LEADER_CHANGE) { |
||||
(leaderId, leaderEpoch) |
||||
} else { |
||||
(record.leader(), leaderEpoch + 1) |
||||
} |
||||
val newIsr = if (record.isr() == null) { |
||||
isr |
||||
} else { |
||||
record.isr() |
||||
} |
||||
MetadataPartition(topicName, |
||||
partitionIndex, |
||||
newLeader, |
||||
newLeaderEpoch, |
||||
replicas, |
||||
newIsr, |
||||
partitionEpoch + 1, |
||||
offlineReplicas, |
||||
addingReplicas, |
||||
removingReplicas) |
||||
} |
||||
} |
||||
|
||||
class MetadataPartitionsBuilder(val brokerId: Int, |
||||
val prevPartitions: MetadataPartitions) { |
||||
private var newNameMap = prevPartitions.copyNameMap() |
||||
private var newIdMap = prevPartitions.copyIdMap() |
||||
private var newReverseIdMap = prevPartitions.copyReverseIdMap() |
||||
private val changed = Collections.newSetFromMap[Any](new util.IdentityHashMap()) |
||||
private val _localChanged = new util.HashSet[MetadataPartition] |
||||
private val _localRemoved = new util.HashSet[MetadataPartition] |
||||
|
||||
def topicIdToName(id: Uuid): Option[String] = Option(newIdMap.get(id)) |
||||
|
||||
def topicNameToId(name: String): Option[Uuid] = Option(newReverseIdMap.get(name)) |
||||
|
||||
def removeTopicById(id: Uuid): Iterable[MetadataPartition] = { |
||||
val name = Option(newIdMap.remove(id)).getOrElse { |
||||
throw new RuntimeException(s"Unable to locate topic with ID $id") |
||||
} |
||||
|
||||
newReverseIdMap.remove(name) |
||||
|
||||
val prevPartitionMap = newNameMap.remove(name) |
||||
if (prevPartitionMap == null) { |
||||
Seq.empty |
||||
} else { |
||||
changed.remove(prevPartitionMap) |
||||
|
||||
val removedPartitions = prevPartitionMap.values |
||||
if (prevImageHasTopicId(id)) { |
||||
removedPartitions.forEach { partition => |
||||
if (partition.isReplicaFor(brokerId)) { |
||||
_localRemoved.add(partition) |
||||
} |
||||
} |
||||
} else { |
||||
removedPartitions.forEach { partition => |
||||
if (partition.isReplicaFor(brokerId)) { |
||||
_localChanged.remove(partition) |
||||
} |
||||
} |
||||
} |
||||
removedPartitions.asScala |
||||
} |
||||
} |
||||
|
||||
def handleChange(record: PartitionChangeRecord): Unit = { |
||||
topicIdToName(record.topicId) match { |
||||
case None => throw new RuntimeException(s"Unable to locate topic with ID ${record.topicId()}") |
||||
case Some(name) => Option(newNameMap.get(name)) match { |
||||
case None => throw new RuntimeException(s"Unable to locate topic with name $name") |
||||
case Some(partitionMap) => Option(partitionMap.get(record.partitionId())) match { |
||||
case None => throw new RuntimeException(s"Unable to locate $name-${record.partitionId}") |
||||
case Some(partition) => set(partition.merge(record)) |
||||
} |
||||
} |
||||
} |
||||
} |
||||
|
||||
def addUuidMapping(name: String, id: Uuid): Unit = { |
||||
newIdMap.put(id, name) |
||||
newReverseIdMap.put(name, id) |
||||
} |
||||
|
||||
def removeUuidMapping(id: Uuid): Unit = { |
||||
val topicName = newIdMap.remove(id) |
||||
if (topicName != null) { |
||||
newReverseIdMap.remove(topicName) |
||||
} |
||||
} |
||||
|
||||
def get(topicName: String, partitionId: Int): Option[MetadataPartition] = { |
||||
Option(newNameMap.get(topicName)).flatMap(m => Option(m.get(partitionId))) |
||||
} |
||||
|
||||
def set(partition: MetadataPartition): Unit = { |
||||
val prevPartitionMap = newNameMap.get(partition.topicName) |
||||
val newPartitionMap = if (prevPartitionMap == null) { |
||||
val m = new util.HashMap[Int, MetadataPartition](1) |
||||
changed.add(m) |
||||
m |
||||
} else if (changed.contains(prevPartitionMap)) { |
||||
prevPartitionMap |
||||
} else { |
||||
val m = new util.HashMap[Int, MetadataPartition](prevPartitionMap.size() + 1) |
||||
m.putAll(prevPartitionMap) |
||||
changed.add(m) |
||||
m |
||||
} |
||||
val prevPartition = newPartitionMap.put(partition.partitionIndex, partition) |
||||
if (partition.isReplicaFor(brokerId)) { |
||||
_localChanged.add(partition) |
||||
} else if (prevPartition != null) { |
||||
maybeAddToLocalRemoved(prevPartition) |
||||
} |
||||
newNameMap.put(partition.topicName, newPartitionMap) |
||||
} |
||||
|
||||
private def maybeAddToLocalRemoved(partition: MetadataPartition): Unit = { |
||||
if (partition.isReplicaFor(brokerId)) { |
||||
val currentTopicId = newReverseIdMap.get(partition.topicName) |
||||
val prevImageHasTopic = if (currentTopicId != null) { |
||||
prevImageHasTopicId(currentTopicId) |
||||
} else { |
||||
prevPartitions.allTopicNames().contains(partition.topicName) |
||||
} |
||||
|
||||
if (prevImageHasTopic) { |
||||
_localRemoved.add(partition) |
||||
} |
||||
} |
||||
} |
||||
|
||||
private def prevImageHasTopicId(topicId: Uuid): Boolean = { |
||||
prevPartitions.topicIdToName(topicId).isDefined |
||||
} |
||||
|
||||
def remove(topicName: String, partitionId: Int): Unit = { |
||||
val prevPartitionMap = newNameMap.get(topicName) |
||||
if (prevPartitionMap != null) { |
||||
val removedPartition = if (changed.contains(prevPartitionMap)) { |
||||
Option(prevPartitionMap.remove(partitionId)) |
||||
} else { |
||||
Option(prevPartitionMap.get(partitionId)).map { prevPartition => |
||||
val newPartitionMap = new util.HashMap[Int, MetadataPartition](prevPartitionMap.size() - 1) |
||||
prevPartitionMap.forEach { (prevPartitionId, prevPartition) => |
||||
if (prevPartitionId != partitionId) { |
||||
newPartitionMap.put(prevPartitionId, prevPartition) |
||||
} |
||||
} |
||||
changed.add(newPartitionMap) |
||||
newNameMap.put(topicName, newPartitionMap) |
||||
prevPartition |
||||
} |
||||
} |
||||
removedPartition.foreach(maybeAddToLocalRemoved) |
||||
} |
||||
} |
||||
|
||||
def build(): MetadataPartitions = { |
||||
val result = new MetadataPartitions(newNameMap, newIdMap, newReverseIdMap) |
||||
newNameMap = Collections.unmodifiableMap(newNameMap) |
||||
newIdMap = Collections.unmodifiableMap(newIdMap) |
||||
newReverseIdMap = Collections.unmodifiableMap(newReverseIdMap) |
||||
result |
||||
} |
||||
|
||||
def localChanged(): collection.Set[MetadataPartition] = _localChanged.asScala |
||||
|
||||
def localRemoved(): collection.Set[MetadataPartition] = _localRemoved.asScala |
||||
} |
||||
|
||||
object MetadataPartitions { |
||||
def apply(nameMap: util.Map[String, util.Map[Int, MetadataPartition]], |
||||
idMap: util.Map[Uuid, String]): MetadataPartitions = { |
||||
val reverseMap = idMap.asScala.map(_.swap).toMap.asJava |
||||
new MetadataPartitions(nameMap, idMap, reverseMap) |
||||
} |
||||
} |
||||
|
||||
case class MetadataPartitions(private val nameMap: util.Map[String, util.Map[Int, MetadataPartition]], |
||||
private val idMap: util.Map[Uuid, String], |
||||
private val reverseIdMap: util.Map[String, Uuid]) { |
||||
|
||||
def topicIdToName(uuid: Uuid): Option[String] = Option(idMap.get(uuid)) |
||||
|
||||
def topicNameToId(name: String): Option[Uuid] = Option(reverseIdMap.get(name)) |
||||
|
||||
def copyNameMap(): util.Map[String, util.Map[Int, MetadataPartition]] = { |
||||
new util.HashMap(nameMap) |
||||
} |
||||
|
||||
def copyIdMap(): util.Map[Uuid, String] = { |
||||
new util.HashMap(idMap) |
||||
} |
||||
|
||||
def copyReverseIdMap(): util.Map[String, Uuid] = { |
||||
new util.HashMap(reverseIdMap) |
||||
} |
||||
|
||||
def allPartitions(): Iterator[MetadataPartition] = new AllPartitionsIterator(nameMap).asScala |
||||
|
||||
def allTopicNames(): collection.Set[String] = nameMap.keySet().asScala |
||||
|
||||
def numTopicPartitions(topicName: String): Option[Int] = { |
||||
val partitionMap = nameMap.get(topicName) |
||||
if (partitionMap == null) { |
||||
None |
||||
} else { |
||||
Some(partitionMap.size()) |
||||
} |
||||
} |
||||
|
||||
def topicPartitions(topicName: String): Iterator[MetadataPartition] = { |
||||
val partitionMap = nameMap.get(topicName) |
||||
if (partitionMap == null) { |
||||
Collections.emptyIterator().asScala |
||||
} else { |
||||
partitionMap.values().iterator().asScala |
||||
} |
||||
} |
||||
|
||||
def topicPartition(topicName: String, partitionId: Int): Option[MetadataPartition] = { |
||||
Option(nameMap.get(topicName)).flatMap(m => Option(m.get(partitionId))) |
||||
} |
||||
|
||||
def contains(topicName: String): Boolean = nameMap.containsKey(topicName) |
||||
} |
||||
|
||||
class AllPartitionsIterator(nameMap: util.Map[String, util.Map[Int, MetadataPartition]]) |
||||
extends util.Iterator[MetadataPartition] { |
||||
|
||||
val outerIterator: util.Iterator[util.Map[Int, MetadataPartition]] = nameMap.values().iterator() |
||||
|
||||
var innerIterator: util.Iterator[MetadataPartition] = Collections.emptyIterator() |
||||
|
||||
var _next: MetadataPartition = _ |
||||
|
||||
override def hasNext: Boolean = { |
||||
if (_next != null) { |
||||
true |
||||
} else { |
||||
while (!innerIterator.hasNext) { |
||||
if (!outerIterator.hasNext) { |
||||
return false |
||||
} else { |
||||
innerIterator = outerIterator.next().values().iterator() |
||||
} |
||||
} |
||||
_next = innerIterator.next() |
||||
true |
||||
} |
||||
} |
||||
|
||||
override def next(): MetadataPartition = { |
||||
if (!hasNext()) { |
||||
throw new NoSuchElementException() |
||||
} |
||||
val result = _next |
||||
_next = null |
||||
result |
||||
} |
||||
} |
@ -0,0 +1,36 @@
@@ -0,0 +1,36 @@
|
||||
/** |
||||
* 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 org.apache.kafka.image.{MetadataDelta, MetadataImage} |
||||
|
||||
/** |
||||
* An object which publishes a new metadata image. |
||||
*/ |
||||
trait MetadataPublisher { |
||||
/** |
||||
* Publish a new metadata image. |
||||
* |
||||
* @param newHighestMetadataOffset The highest metadata offset contained within the image. |
||||
* @param delta The delta between the old image and the new one. |
||||
* @param newImage The new image, which is the result of applying the |
||||
* delta to the previous image. |
||||
*/ |
||||
def publish(newHighestMetadataOffset: Long, |
||||
delta: MetadataDelta, |
||||
newImage: MetadataImage): Unit |
||||
} |
@ -1,394 +0,0 @@
@@ -1,394 +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 kafka.api.LeaderAndIsr |
||||
import kafka.controller.StateChangeLogger |
||||
import kafka.server.MetadataCache |
||||
import kafka.utils.CoreUtils.inLock |
||||
import kafka.utils.Logging |
||||
import org.apache.kafka.common.internals.Topic |
||||
import org.apache.kafka.common.message.MetadataResponseData.{MetadataResponsePartition, MetadataResponseTopic} |
||||
import org.apache.kafka.common.{Cluster, Node, PartitionInfo, TopicPartition, Uuid} |
||||
import org.apache.kafka.common.message.UpdateMetadataRequestData.{UpdateMetadataBroker, UpdateMetadataPartitionState} |
||||
import org.apache.kafka.common.network.ListenerName |
||||
import org.apache.kafka.common.protocol.Errors |
||||
import org.apache.kafka.common.requests.{MetadataResponse, UpdateMetadataRequest} |
||||
import java.util |
||||
import java.util.Collections |
||||
import java.util.concurrent.locks.ReentrantLock |
||||
|
||||
import kafka.admin.BrokerMetadata |
||||
|
||||
import scala.collection.{Seq, Set, mutable} |
||||
import scala.jdk.CollectionConverters._ |
||||
|
||||
object RaftMetadataCache { |
||||
def removePartitionInfo(partitionStates: mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataPartitionState]], |
||||
topic: String, partitionId: Int): Boolean = { |
||||
partitionStates.get(topic).exists { infos => |
||||
infos.remove(partitionId) |
||||
if (infos.isEmpty) partitionStates.remove(topic) |
||||
true |
||||
} |
||||
} |
||||
|
||||
def addOrUpdatePartitionInfo(partitionStates: mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataPartitionState]], |
||||
topic: String, |
||||
partitionId: Int, |
||||
stateInfo: UpdateMetadataPartitionState): Unit = { |
||||
val infos = partitionStates.getOrElseUpdate(topic, mutable.LongMap.empty) |
||||
infos(partitionId) = stateInfo |
||||
} |
||||
} |
||||
|
||||
|
||||
class RaftMetadataCache(val brokerId: Int) extends MetadataCache with Logging { |
||||
this.logIdent = s"[MetadataCache brokerId=$brokerId] " |
||||
|
||||
private val lock = new ReentrantLock() |
||||
|
||||
//this is the cache state. every MetadataImage instance is immutable, and updates (performed under a lock) |
||||
//replace the value with a completely new one. this means reads (which are not under any lock) need to grab |
||||
//the value of this var (into a val) ONCE and retain that read copy for the duration of their operation. |
||||
//multiple reads of this value risk getting different snapshots. |
||||
@volatile private var _currentImage: MetadataImage = new MetadataImage() |
||||
|
||||
private val stateChangeLogger = new StateChangeLogger(brokerId, inControllerContext = false, None) |
||||
|
||||
// This method is the main hotspot when it comes to the performance of metadata requests, |
||||
// we should be careful about adding additional logic here. Relatedly, `brokers` is |
||||
// `List[Integer]` instead of `List[Int]` to avoid a collection copy. |
||||
// filterUnavailableEndpoints exists to support v0 MetadataResponses |
||||
private def maybeFilterAliveReplicas(image: MetadataImage, |
||||
brokers: java.util.List[Integer], |
||||
listenerName: ListenerName, |
||||
filterUnavailableEndpoints: Boolean): java.util.List[Integer] = { |
||||
if (!filterUnavailableEndpoints) { |
||||
brokers |
||||
} else { |
||||
val res = new util.ArrayList[Integer](math.min(image.brokers.aliveBrokers().size, brokers.size)) |
||||
for (brokerId <- brokers.asScala) { |
||||
if (hasAliveEndpoint(image, brokerId, listenerName)) |
||||
res.add(brokerId) |
||||
} |
||||
res |
||||
} |
||||
} |
||||
|
||||
def currentImage(): MetadataImage = _currentImage |
||||
|
||||
// errorUnavailableEndpoints exists to support v0 MetadataResponses |
||||
// If errorUnavailableListeners=true, return LISTENER_NOT_FOUND if listener is missing on the broker. |
||||
// Otherwise, return LEADER_NOT_AVAILABLE for broker unavailable and missing listener (Metadata response v5 and below). |
||||
private def getPartitionMetadata(image: MetadataImage, topic: String, listenerName: ListenerName, errorUnavailableEndpoints: Boolean, |
||||
errorUnavailableListeners: Boolean): Option[Iterator[MetadataResponsePartition]] = { |
||||
val partitionsIterator = image.partitions.topicPartitions(topic) |
||||
if (!partitionsIterator.hasNext) { |
||||
None |
||||
} else { |
||||
Some(partitionsIterator.map { partition => |
||||
val filteredReplicas = maybeFilterAliveReplicas(image, partition.replicas, |
||||
listenerName, errorUnavailableEndpoints) |
||||
val filteredIsr = maybeFilterAliveReplicas(image, partition.isr, listenerName, |
||||
errorUnavailableEndpoints) |
||||
val maybeLeader = getAliveEndpoint(image, partition.leaderId, listenerName) |
||||
maybeLeader match { |
||||
case None => |
||||
val error = if (image.aliveBroker(partition.leaderId).isEmpty) { |
||||
debug(s"Error while fetching metadata for ${partition.toTopicPartition}: leader not available") |
||||
Errors.LEADER_NOT_AVAILABLE |
||||
} else { |
||||
debug(s"Error while fetching metadata for ${partition.toTopicPartition}: listener $listenerName " + |
||||
s"not found on leader ${partition.leaderId}") |
||||
if (errorUnavailableListeners) Errors.LISTENER_NOT_FOUND else Errors.LEADER_NOT_AVAILABLE |
||||
} |
||||
|
||||
new MetadataResponsePartition() |
||||
.setErrorCode(error.code) |
||||
.setPartitionIndex(partition.partitionIndex) |
||||
.setLeaderId(MetadataResponse.NO_LEADER_ID) |
||||
.setLeaderEpoch(partition.leaderEpoch) |
||||
.setReplicaNodes(filteredReplicas) |
||||
.setIsrNodes(filteredIsr) |
||||
.setOfflineReplicas(partition.offlineReplicas) |
||||
|
||||
case Some(leader) => |
||||
val error = if (filteredReplicas.size < partition.replicas.size) { |
||||
debug(s"Error while fetching metadata for ${partition.toTopicPartition}: replica information not available for " + |
||||
s"following brokers ${partition.replicas.asScala.filterNot(filteredReplicas.contains).mkString(",")}") |
||||
Errors.REPLICA_NOT_AVAILABLE |
||||
} else if (filteredIsr.size < partition.isr.size) { |
||||
debug(s"Error while fetching metadata for ${partition.toTopicPartition}: in sync replica information not available for " + |
||||
s"following brokers ${partition.isr.asScala.filterNot(filteredIsr.contains).mkString(",")}") |
||||
Errors.REPLICA_NOT_AVAILABLE |
||||
} else { |
||||
Errors.NONE |
||||
} |
||||
|
||||
new MetadataResponsePartition() |
||||
.setErrorCode(error.code) |
||||
.setPartitionIndex(partition.partitionIndex) |
||||
.setLeaderId(leader.id()) |
||||
.setLeaderEpoch(partition.leaderEpoch) |
||||
.setReplicaNodes(filteredReplicas) |
||||
.setIsrNodes(filteredIsr) |
||||
.setOfflineReplicas(partition.offlineReplicas) |
||||
} |
||||
}) |
||||
} |
||||
} |
||||
|
||||
/** |
||||
* Check whether a broker is alive and has a registered listener matching the provided name. |
||||
* This method was added to avoid unnecessary allocations in [[maybeFilterAliveReplicas]], which is |
||||
* a hotspot in metadata handling. |
||||
*/ |
||||
private def hasAliveEndpoint(image: MetadataImage, id: Int, listenerName: ListenerName): Boolean = { |
||||
image.brokers.aliveBroker(id).exists(_.endpoints.contains(listenerName.value())) |
||||
} |
||||
|
||||
/** |
||||
* 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. |
||||
* |
||||
* @return None if broker is not alive or if the broker does not have a listener named `listenerName`. |
||||
*/ |
||||
private def getAliveEndpoint(image: MetadataImage, id: Int, listenerName: ListenerName): Option[Node] = { |
||||
image.brokers.aliveBroker(id).flatMap(_.endpoints.get(listenerName.value())) |
||||
} |
||||
|
||||
// errorUnavailableEndpoints exists to support v0 MetadataResponses |
||||
override def getTopicMetadata(topics: Set[String], |
||||
listenerName: ListenerName, |
||||
errorUnavailableEndpoints: Boolean = false, |
||||
errorUnavailableListeners: Boolean = false): Seq[MetadataResponseTopic] = { |
||||
val image = _currentImage |
||||
topics.toSeq.flatMap { topic => |
||||
getPartitionMetadata(image, topic, listenerName, errorUnavailableEndpoints, errorUnavailableListeners).map { partitionMetadata => |
||||
new MetadataResponseTopic() |
||||
.setErrorCode(Errors.NONE.code) |
||||
.setName(topic) |
||||
.setTopicId(image.topicNameToId(topic).getOrElse(Uuid.ZERO_UUID)) |
||||
.setIsInternal(Topic.isInternal(topic)) |
||||
.setPartitions(partitionMetadata.toBuffer.asJava) |
||||
} |
||||
} |
||||
} |
||||
|
||||
override def getAllTopics(): Set[String] = _currentImage.partitions.allTopicNames() |
||||
|
||||
override def getTopicPartitions(topicName: String): Set[TopicPartition] = |
||||
_currentImage.partitions.topicPartitions(topicName). |
||||
map(p => new TopicPartition(p.topicName, p.partitionIndex)).toSet |
||||
|
||||
override def hasAliveBroker(brokerId: Int): Boolean = { |
||||
_currentImage.brokers.aliveBroker(brokerId).nonEmpty |
||||
} |
||||
|
||||
override def getAliveBrokers(): Iterable[BrokerMetadata] = { |
||||
_currentImage.brokers.aliveBrokers().map(b => BrokerMetadata(b.id, Option(b.rack))) |
||||
} |
||||
|
||||
override def getAliveBrokerNode(brokerId: Int, listenerName: ListenerName): Option[Node] = { |
||||
_currentImage.brokers.aliveBroker(brokerId).flatMap(_.endpoints.get(listenerName.value())) |
||||
} |
||||
|
||||
override def getAliveBrokerNodes(listenerName: ListenerName): Seq[Node] = { |
||||
_currentImage.brokers.aliveBrokers().flatMap(_.endpoints.get(listenerName.value())) |
||||
} |
||||
|
||||
override def getPartitionInfo(topic: String, partitionId: Int): Option[UpdateMetadataPartitionState] = { |
||||
_currentImage.partitions.topicPartition(topic, partitionId).map { partition => |
||||
new UpdateMetadataPartitionState(). |
||||
setTopicName(partition.topicName). |
||||
setPartitionIndex(partition.partitionIndex). |
||||
setControllerEpoch(-1). // Controller epoch is not stored in the cache. |
||||
setLeader(partition.leaderId). |
||||
setLeaderEpoch(partition.leaderEpoch). |
||||
setIsr(partition.isr). |
||||
setZkVersion(-1) // ZK version is not stored in the cache. |
||||
} |
||||
} |
||||
|
||||
override def numPartitions(topic: String): Option[Int] = { |
||||
_currentImage.partitions.numTopicPartitions(topic) |
||||
} |
||||
|
||||
// if the leader is not known, return None; |
||||
// if the leader is known and corresponding node is available, return Some(node) |
||||
// if the leader is known but corresponding node with the listener name is not available, return Some(NO_NODE) |
||||
override def getPartitionLeaderEndpoint(topic: String, partitionId: Int, listenerName: ListenerName): Option[Node] = { |
||||
val image = _currentImage |
||||
image.partitions.topicPartition(topic, partitionId).map { partition => |
||||
image.aliveBroker(partition.leaderId) match { |
||||
case Some(broker) => |
||||
broker.endpoints.getOrElse(listenerName.value(), Node.noNode) |
||||
case None => |
||||
Node.noNode |
||||
} |
||||
} |
||||
} |
||||
|
||||
override def getPartitionReplicaEndpoints(tp: TopicPartition, listenerName: ListenerName): Map[Int, Node] = { |
||||
val image = _currentImage |
||||
image.partitions.topicPartition(tp.topic(), tp.partition()).map { partition => |
||||
partition.replicas.asScala.map(replicaId => replicaId.intValue() -> { |
||||
image.aliveBroker(replicaId) match { |
||||
case Some(broker) => |
||||
broker.endpoints.getOrElse(listenerName.value(), Node.noNode()) |
||||
case None => |
||||
Node.noNode() |
||||
}}).toMap |
||||
.filter(pair => pair match { |
||||
case (_, node) => !node.isEmpty |
||||
}) |
||||
}.getOrElse(Map.empty[Int, Node]) |
||||
} |
||||
|
||||
override def getControllerId: Option[Int] = { |
||||
_currentImage.controllerId |
||||
} |
||||
|
||||
override def getClusterMetadata(clusterId: String, listenerName: ListenerName): Cluster = { |
||||
val image = _currentImage |
||||
val nodes = new util.HashMap[Integer, Node] |
||||
image.brokers.aliveBrokers().foreach { node => |
||||
if (!node.fenced) { |
||||
node.endpoints.get(listenerName.value()).foreach { nodes.put(node.id, _) } |
||||
} |
||||
} |
||||
|
||||
def node(id: Integer): Node = { |
||||
Option(nodes.get(id)).getOrElse(new Node(id, "", -1)) |
||||
} |
||||
|
||||
val partitionInfos = new util.ArrayList[PartitionInfo] |
||||
val internalTopics = new util.HashSet[String] |
||||
|
||||
image.partitions.allPartitions().foreach { partition => |
||||
partitionInfos.add(new PartitionInfo(partition.topicName, |
||||
partition.partitionIndex, node(partition.leaderId), |
||||
partition.replicas.asScala.map(node).toArray, |
||||
partition.isr.asScala.map(node).toArray, |
||||
partition.offlineReplicas.asScala.map(node).toArray)) |
||||
if (Topic.isInternal(partition.topicName)) { |
||||
internalTopics.add(partition.topicName) |
||||
} |
||||
} |
||||
|
||||
new Cluster(clusterId, nodes.values(), |
||||
partitionInfos, Collections.emptySet[String], internalTopics, |
||||
node(Integer.valueOf(image.controllerId.getOrElse(-1)))) |
||||
} |
||||
|
||||
def stateChangeTraceEnabled(): Boolean = { |
||||
stateChangeLogger.isTraceEnabled |
||||
} |
||||
|
||||
def logStateChangeTrace(str: String): Unit = { |
||||
stateChangeLogger.trace(str) |
||||
} |
||||
|
||||
// This method returns the deleted TopicPartitions received from UpdateMetadataRequest |
||||
override def updateMetadata(correlationId: Int, request: UpdateMetadataRequest): Seq[TopicPartition] = { |
||||
inLock(lock) { |
||||
val image = _currentImage |
||||
val builder = MetadataImageBuilder(brokerId, logger.underlying, image) |
||||
|
||||
builder.controllerId(if (request.controllerId() < 0) None else Some(request.controllerId())) |
||||
|
||||
// Compare the new brokers with the existing ones. |
||||
def toMetadataBroker(broker: UpdateMetadataBroker): MetadataBroker = { |
||||
val endpoints = broker.endpoints().asScala.map { endpoint => |
||||
endpoint.listener -> new Node(broker.id(), endpoint.host(), endpoint.port()) |
||||
}.toMap |
||||
MetadataBroker(broker.id(), broker.rack(), endpoints, fenced = false) |
||||
} |
||||
val found = new util.IdentityHashMap[MetadataBroker, Boolean](image.numAliveBrokers()) |
||||
request.liveBrokers().iterator().asScala.foreach { brokerInfo => |
||||
val newBroker = toMetadataBroker(brokerInfo) |
||||
image.brokers.get(brokerInfo.id) match { |
||||
case None => builder.brokersBuilder().add(newBroker) |
||||
case Some(existingBroker) => |
||||
found.put(existingBroker, true) |
||||
if (!existingBroker.equals(newBroker)) { |
||||
builder.brokersBuilder().add(newBroker) |
||||
} |
||||
} |
||||
} |
||||
image.brokers.iterator().foreach { broker => |
||||
if (!found.containsKey(broker)) { |
||||
builder.brokersBuilder().remove(broker.id) |
||||
} |
||||
} |
||||
|
||||
val topicIds = request.topicStates().iterator().asScala.map { topic => |
||||
topic.topicName() -> topic.topicId() |
||||
}.toMap |
||||
|
||||
val traceEnabled = stateChangeLogger.isTraceEnabled |
||||
var numDeleted = 0 |
||||
var numAdded = 0 |
||||
val deleted = mutable.Buffer[TopicPartition]() |
||||
request.partitionStates().iterator().asScala.foreach { partition => |
||||
if (partition.leader() == LeaderAndIsr.LeaderDuringDelete) { |
||||
if (traceEnabled) { |
||||
stateChangeLogger.trace(s"Deleted partition ${partition.topicName()}-${partition.partitionIndex()} " + |
||||
"from metadata cache in response to UpdateMetadata request sent by " + |
||||
s"controller ${request.controllerId} epoch ${request.controllerEpoch} " + |
||||
s"with correlation id $correlationId") |
||||
} |
||||
builder.partitionsBuilder().remove(partition.topicName(), partition.partitionIndex()) |
||||
deleted += new TopicPartition(partition.topicName(), partition.partitionIndex()) |
||||
numDeleted = numDeleted + 1 |
||||
} else { |
||||
val prevPartition = builder.partition(partition.topicName(), partition.partitionIndex()) |
||||
val newPartition = MetadataPartition(prevPartition, partition) |
||||
if (traceEnabled) { |
||||
stateChangeLogger.trace(s"Cached leader info $newPartition in response to " + |
||||
s"UpdateMetadata request sent by controller $request.controllerId epoch " + |
||||
s"$request.controllerEpoch with correlation id $correlationId") |
||||
} |
||||
builder.partitionsBuilder().set(newPartition) |
||||
topicIds.get(newPartition.topicName).foreach { |
||||
topicId => builder.partitionsBuilder().addUuidMapping(newPartition.topicName, topicId) |
||||
} |
||||
numAdded = numAdded + 1 |
||||
} |
||||
} |
||||
stateChangeLogger.info(s"Add ${numAdded} partitions and deleted ${numDeleted} " + |
||||
"partitions to the metadata cache in response to UpdateMetadata request sent by " + |
||||
s"controller ${request.controllerId} epoch ${request.controllerEpoch} with " + |
||||
s"correlation id ${correlationId}") |
||||
|
||||
_currentImage = builder.build() |
||||
deleted |
||||
} |
||||
} |
||||
|
||||
override def contains(topic: String): Boolean = _currentImage.partitions.contains(topic) |
||||
|
||||
override def contains(tp: TopicPartition): Boolean = { |
||||
_currentImage.partitions.topicPartition(tp.topic(), tp.partition()).isDefined |
||||
} |
||||
|
||||
def image(newImage: MetadataImage): Unit = inLock(lock) { |
||||
_currentImage = newImage |
||||
} |
||||
} |
@ -1,90 +0,0 @@
@@ -1,90 +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.Collections |
||||
import kafka.utils.TestUtils |
||||
import org.junit.jupiter.api.Assertions._ |
||||
import org.junit.jupiter.api.{Test, Timeout} |
||||
import org.slf4j.LoggerFactory |
||||
|
||||
import java.util.concurrent.TimeUnit |
||||
import scala.collection.mutable |
||||
|
||||
|
||||
@Timeout(value = 120000, unit = TimeUnit.MILLISECONDS) |
||||
class MetadataBrokersTest { |
||||
|
||||
private val log = LoggerFactory.getLogger(classOf[MetadataBrokersTest]) |
||||
|
||||
val emptyBrokers = new MetadataBrokers(Collections.emptyList(), Collections.emptyMap()) |
||||
|
||||
@Test |
||||
def testBuildBrokers(): Unit = { |
||||
val builder = new MetadataBrokersBuilder(log, emptyBrokers) |
||||
builder.add(TestUtils.createMetadataBroker(0)) |
||||
builder.add(TestUtils.createMetadataBroker(1)) |
||||
builder.add(TestUtils.createMetadataBroker(2)) |
||||
builder.add(TestUtils.createMetadataBroker(3)) |
||||
builder.remove(0) |
||||
val brokers = builder.build() |
||||
val found = new mutable.HashSet[MetadataBroker] |
||||
brokers.iterator().foreach { found += _ } |
||||
val expected = new mutable.HashSet[MetadataBroker] |
||||
expected += TestUtils.createMetadataBroker(1) |
||||
expected += TestUtils.createMetadataBroker(2) |
||||
expected += TestUtils.createMetadataBroker(3) |
||||
assertEquals(expected, found) |
||||
} |
||||
|
||||
@Test |
||||
def testChangeFencing(): Unit = { |
||||
val builder = new MetadataBrokersBuilder(log, emptyBrokers) |
||||
assertEquals(None, builder.get(0)) |
||||
assertThrows(classOf[RuntimeException], () => builder.changeFencing(0, false)) |
||||
builder.add(TestUtils.createMetadataBroker(0, fenced = true)) |
||||
assertTrue(builder.get(0).get.fenced) |
||||
builder.changeFencing(0, false) |
||||
assertFalse(builder.get(0).get.fenced) |
||||
val brokers = builder.build() |
||||
assertTrue(brokers.aliveBroker(0).isDefined) |
||||
} |
||||
|
||||
@Test |
||||
def testAliveBrokers(): Unit = { |
||||
val builder = new MetadataBrokersBuilder(log, emptyBrokers) |
||||
builder.add(TestUtils.createMetadataBroker(0)) |
||||
builder.add(TestUtils.createMetadataBroker(1)) |
||||
builder.add(TestUtils.createMetadataBroker(2)) |
||||
builder.changeFencing(1, true) |
||||
val brokers = builder.build() |
||||
assertEquals(2, brokers.aliveBrokers().size) |
||||
assertTrue(brokers.aliveBrokers().exists(_.id == 0)) |
||||
assertTrue(!brokers.aliveBrokers().exists(_.id == 1)) |
||||
assertTrue(brokers.aliveBrokers().exists(_.id == 2)) |
||||
while (!brokers.randomAliveBrokerId().contains(0)) { } |
||||
while (!brokers.randomAliveBrokerId().contains(2)) { } |
||||
assertEquals(3, brokers.size()) |
||||
assertEquals(Some(TestUtils.createMetadataBroker(0)), brokers.get(0)) |
||||
assertEquals(Some(TestUtils.createMetadataBroker(1, fenced = true)), brokers.get(1)) |
||||
assertEquals(Some(TestUtils.createMetadataBroker(2)), brokers.get(2)) |
||||
assertEquals(None, brokers.get(3)) |
||||
assertEquals(Some(TestUtils.createMetadataBroker(0)), brokers.aliveBroker(0)) |
||||
assertEquals(None, brokers.aliveBroker(1)) |
||||
} |
||||
} |
@ -1,324 +0,0 @@
@@ -1,324 +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.Collections |
||||
import java.util.concurrent.TimeUnit |
||||
|
||||
import org.apache.kafka.common.{TopicPartition, Uuid} |
||||
import org.apache.kafka.common.metadata.PartitionChangeRecord |
||||
import org.junit.jupiter.api.Assertions._ |
||||
import org.junit.jupiter.api.{Test, Timeout} |
||||
|
||||
import scala.collection.mutable |
||||
import scala.jdk.CollectionConverters._ |
||||
|
||||
|
||||
@Timeout(value = 120000, unit = TimeUnit.MILLISECONDS) |
||||
class MetadataPartitionsTest { |
||||
|
||||
private val emptyPartitions = MetadataPartitions(Collections.emptyMap(), Collections.emptyMap()) |
||||
|
||||
private def newPartition(topicName: String, |
||||
partitionIndex: Int, |
||||
replicas: Option[Seq[Int]] = None, |
||||
isr: Option[Seq[Int]] = None, |
||||
numBrokers: Int = 6): MetadataPartition = { |
||||
val effectiveReplicas = asJavaList(replicas.getOrElse { |
||||
val preferredLeaderId = partitionIndex % numBrokers |
||||
List(preferredLeaderId, preferredLeaderId + 1, preferredLeaderId + 2) |
||||
}) |
||||
|
||||
val effectiveIsr = isr match { |
||||
case None => effectiveReplicas |
||||
case Some(s) => s.map(Integer.valueOf).toList.asJava |
||||
} |
||||
new MetadataPartition(topicName, |
||||
partitionIndex, |
||||
effectiveReplicas.asScala.head, |
||||
leaderEpoch = 100, |
||||
effectiveReplicas, |
||||
effectiveIsr, |
||||
partitionEpoch = 200, |
||||
Collections.emptyList(), |
||||
Collections.emptyList(), |
||||
Collections.emptyList()) |
||||
} |
||||
|
||||
@Test |
||||
def testBuildPartitions(): Unit = { |
||||
val builder = new MetadataPartitionsBuilder(0, emptyPartitions) |
||||
assertEquals(None, builder.get("foo", 0)) |
||||
builder.set(newPartition("foo", 0)) |
||||
assertEquals(Some(newPartition("foo", 0)), builder.get("foo", 0)) |
||||
assertEquals(None, builder.get("foo", 1)) |
||||
builder.set(newPartition("foo", 1)) |
||||
builder.set(newPartition("bar", 0)) |
||||
val partitions = builder.build() |
||||
assertEquals(Some(newPartition("foo", 0)), partitions.topicPartition("foo", 0)) |
||||
assertEquals(Some(newPartition("foo", 1)), partitions.topicPartition("foo", 1)) |
||||
assertEquals(None, partitions.topicPartition("foo", 2)) |
||||
assertEquals(Some(newPartition("bar", 0)), partitions.topicPartition("bar", 0)) |
||||
} |
||||
|
||||
@Test |
||||
def testAllPartitionsIterator(): Unit = { |
||||
val builder = new MetadataPartitionsBuilder(0, emptyPartitions) |
||||
val expected = new mutable.HashSet[MetadataPartition]() |
||||
expected += newPartition("foo", 0) |
||||
expected += newPartition("foo", 1) |
||||
expected += newPartition("foo", 2) |
||||
expected += newPartition("bar", 0) |
||||
expected += newPartition("bar", 1) |
||||
expected += newPartition("baz", 0) |
||||
expected.foreach { builder.set } |
||||
val partitions = builder.build() |
||||
val found = new mutable.HashSet[MetadataPartition]() |
||||
partitions.allPartitions().foreach { found += _ } |
||||
assertEquals(expected, found) |
||||
} |
||||
|
||||
@Test |
||||
def testLocalChangedAndRemoved(): Unit = { |
||||
val builder = new MetadataPartitionsBuilder(0, emptyPartitions) |
||||
builder.set(newPartition("foo", 0)) |
||||
assertTrue(newPartition("foo", 0).isReplicaFor(0)) |
||||
assertFalse(newPartition("foo", 0).isReplicaFor(4)) |
||||
builder.set(newPartition("foo", 1)) |
||||
builder.set(newPartition("foo", 2)) |
||||
builder.set(newPartition("bar", 0)) |
||||
val expectedLocalChanged = new mutable.HashSet[MetadataPartition]() |
||||
expectedLocalChanged += newPartition("foo", 0) |
||||
expectedLocalChanged += newPartition("bar", 0) |
||||
assertEquals(expectedLocalChanged, builder.localChanged()) |
||||
assertEquals(Set(), builder.localRemoved()) |
||||
val image = builder.build() |
||||
assertEquals(Some(3), image.numTopicPartitions("foo")) |
||||
assertEquals(None, image.numTopicPartitions("quux")) |
||||
|
||||
val builder2 = new MetadataPartitionsBuilder(1, image) |
||||
builder2.set(newPartition("foo", 0, replicas = Some(Seq(2, 3, 4)))) |
||||
builder2.set(newPartition("foo", 1, isr = Some(Seq(0, 1)))) |
||||
builder2.set(newPartition("bar", 2)) |
||||
builder2.remove("bar", 0) |
||||
builder2.remove("foo", 2) |
||||
val expectedLocalChanged2 = new mutable.HashSet[MetadataPartition]() |
||||
expectedLocalChanged2 += newPartition("foo", 1, isr = Some(Seq(0, 1))) |
||||
assertEquals(expectedLocalChanged2, builder2.localChanged()) |
||||
val expectedLocalRemoved2 = new mutable.HashSet[MetadataPartition]() |
||||
expectedLocalRemoved2 += newPartition("bar", 0) |
||||
expectedLocalRemoved2 += newPartition("foo", 0) |
||||
assertEquals(expectedLocalRemoved2, builder2.localRemoved()) |
||||
} |
||||
|
||||
@Test |
||||
def testAllTopicNames(): Unit = { |
||||
val builder = new MetadataPartitionsBuilder(0, emptyPartitions) |
||||
createTopic("foo", numPartitions = 3, builder) |
||||
createTopic("bar", numPartitions = 2, builder) |
||||
createTopic("baz", numPartitions = 3, builder) |
||||
val image = builder.build() |
||||
val expectedTopicNames = new mutable.HashSet[String]() |
||||
expectedTopicNames += "foo" |
||||
expectedTopicNames += "bar" |
||||
expectedTopicNames += "baz" |
||||
assertEquals(expectedTopicNames, image.allTopicNames()) |
||||
} |
||||
|
||||
@Test |
||||
def testUuidMappings(): Unit = { |
||||
val builder = new MetadataPartitionsBuilder(0, emptyPartitions) |
||||
builder.addUuidMapping("foo", Uuid.fromString("qbUrhSpXTau_836U7T5ktg")) |
||||
builder.addUuidMapping("bar", Uuid.fromString("a1I0JF3yRzWFyOuY3F_vHw")) |
||||
builder.removeUuidMapping(Uuid.fromString("gdMy05W7QWG4ZjWir1DjBw")) |
||||
val image = builder.build() |
||||
assertEquals(Some("foo"), image.topicIdToName(Uuid.fromString("qbUrhSpXTau_836U7T5ktg"))) |
||||
assertEquals(Some("bar"), image.topicIdToName(Uuid.fromString("a1I0JF3yRzWFyOuY3F_vHw"))) |
||||
assertEquals(None, image.topicIdToName(Uuid.fromString("gdMy05W7QWG4ZjWir1DjBw"))) |
||||
} |
||||
|
||||
@Test |
||||
def testMergePartitionChangeRecord(): Unit = { |
||||
val initialMetadata = newPartition( |
||||
topicName = "foo", |
||||
partitionIndex = 0, |
||||
replicas = Some(Seq(1, 2, 3)), |
||||
isr = Some(Seq(1, 2, 3)) |
||||
) |
||||
assertEquals(1, initialMetadata.leaderId) |
||||
|
||||
// If only the ISR changes, then the leader epoch |
||||
// remains the same and the partition epoch is bumped. |
||||
val updatedIsr = initialMetadata.merge(new PartitionChangeRecord() |
||||
.setPartitionId(0) |
||||
.setIsr(asJavaList(Seq(1, 2)))) |
||||
assertEquals(asJavaList(Seq(1, 2)), updatedIsr.isr) |
||||
assertEquals(initialMetadata.leaderEpoch, updatedIsr.leaderEpoch) |
||||
assertEquals(initialMetadata.partitionEpoch + 1, updatedIsr.partitionEpoch) |
||||
assertEquals(initialMetadata.leaderId, updatedIsr.leaderId) |
||||
|
||||
// If the leader changes, then both the leader epoch |
||||
// and the partition epoch should get bumped. |
||||
val updatedLeader = initialMetadata.merge(new PartitionChangeRecord() |
||||
.setPartitionId(0) |
||||
.setLeader(2) |
||||
.setIsr(asJavaList(Seq(2, 3)))) |
||||
assertEquals(asJavaList(Seq(2, 3)), updatedLeader.isr) |
||||
assertEquals(initialMetadata.leaderEpoch + 1, updatedLeader.leaderEpoch) |
||||
assertEquals(initialMetadata.partitionEpoch + 1, updatedLeader.partitionEpoch) |
||||
assertEquals(2, updatedLeader.leaderId) |
||||
} |
||||
|
||||
@Test |
||||
def testTopicCreateAndDelete(): Unit = { |
||||
val topic = "foo" |
||||
val numPartitions = 3 |
||||
val topicPartitions = (0 until numPartitions).map(new TopicPartition(topic, _)).toSet |
||||
val builder = new MetadataPartitionsBuilder(0, emptyPartitions) |
||||
val topicId = createTopic(topic, numPartitions, builder) |
||||
val localTopicPartitions = localChanged(builder) |
||||
|
||||
assertTrue(localTopicPartitions.subsetOf(topicPartitions)) |
||||
assertTrue(localTopicPartitions.nonEmpty) |
||||
assertNotEquals(topicPartitions, localTopicPartitions) |
||||
|
||||
builder.removeTopicById(topicId) |
||||
assertEquals(None, builder.topicIdToName(topicId)) |
||||
assertEquals(None, builder.topicNameToId(topic)) |
||||
assertEquals(Set.empty, filterPartitions(builder, topicPartitions)) |
||||
assertEquals(Set.empty, localRemoved(builder)) |
||||
assertEquals(Set.empty, localChanged(builder)) |
||||
|
||||
val metadata = builder.build() |
||||
assertEquals(Set.empty, metadata.allTopicNames()) |
||||
assertEquals(None, metadata.topicIdToName(topicId)) |
||||
assertEquals(None, metadata.topicNameToId(topic)) |
||||
assertEquals(Set.empty, metadata.topicPartitions(topic).toSet) |
||||
} |
||||
|
||||
@Test |
||||
def testTopicRemoval(): Unit = { |
||||
val brokerId = 0 |
||||
val topic = "foo" |
||||
val numPartitions = 3 |
||||
val topicPartitions = (0 until numPartitions).map(new TopicPartition(topic, _)).toSet |
||||
val createBuilder = new MetadataPartitionsBuilder(brokerId, emptyPartitions) |
||||
val topicId = createTopic(topic, numPartitions, createBuilder) |
||||
val localTopicPartitions = localChanged(createBuilder) |
||||
val createMetadata = createBuilder.build() |
||||
|
||||
assertTrue(localTopicPartitions.subsetOf(topicPartitions)) |
||||
assertTrue(localTopicPartitions.nonEmpty) |
||||
assertNotEquals(topicPartitions, localTopicPartitions) |
||||
|
||||
val deleteBuilder = new MetadataPartitionsBuilder(brokerId = 0, createMetadata) |
||||
deleteBuilder.removeTopicById(topicId) |
||||
assertEquals(None, deleteBuilder.topicIdToName(topicId)) |
||||
assertEquals(None, deleteBuilder.topicNameToId(topic)) |
||||
assertEquals(Set.empty, filterPartitions(deleteBuilder, topicPartitions)) |
||||
assertEquals(localTopicPartitions, localRemoved(deleteBuilder)) |
||||
assertEquals(Set.empty, localChanged(deleteBuilder)) |
||||
|
||||
val deleteMetadata = deleteBuilder.build() |
||||
assertEquals(Set.empty, deleteMetadata.allTopicNames()) |
||||
assertEquals(None, deleteMetadata.topicIdToName(topicId)) |
||||
assertEquals(None, deleteMetadata.topicNameToId(topic)) |
||||
assertEquals(Set.empty, deleteMetadata.topicPartitions(topic).toSet) |
||||
} |
||||
|
||||
@Test |
||||
def testTopicDeleteAndRecreate(): Unit = { |
||||
val topic = "foo" |
||||
val numPartitions = 3 |
||||
val initialBuilder = new MetadataPartitionsBuilder(0, emptyPartitions) |
||||
val initialTopicId = createTopic(topic, numPartitions, initialBuilder) |
||||
val initialLocalTopicPartitions = initialBuilder.localChanged().map(_.toTopicPartition).toSet |
||||
val initialMetadata = initialBuilder.build() |
||||
|
||||
val recreateBuilder = new MetadataPartitionsBuilder(brokerId = 0, initialMetadata) |
||||
recreateBuilder.removeTopicById(initialTopicId) |
||||
assertEquals(initialLocalTopicPartitions, localRemoved(recreateBuilder)) |
||||
|
||||
val recreatedNumPartitions = 10 |
||||
val recreatedTopicId = createTopic(topic, recreatedNumPartitions, recreateBuilder) |
||||
val recreatedTopicPartitions = (0 until recreatedNumPartitions).map(new TopicPartition(topic, _)).toSet |
||||
val recreatedLocalTopicPartitions = localChanged(recreateBuilder) |
||||
|
||||
assertTrue(recreatedLocalTopicPartitions.nonEmpty) |
||||
assertNotEquals(recreatedLocalTopicPartitions, recreatedTopicPartitions) |
||||
assertTrue(recreatedLocalTopicPartitions.subsetOf(recreatedTopicPartitions)) |
||||
assertFalse(recreatedLocalTopicPartitions.subsetOf(initialLocalTopicPartitions)) |
||||
assertEquals(Some(topic), recreateBuilder.topicIdToName(recreatedTopicId)) |
||||
assertEquals(Some(recreatedTopicId), recreateBuilder.topicNameToId(topic)) |
||||
assertEquals(recreatedTopicPartitions, filterPartitions(recreateBuilder, recreatedTopicPartitions)) |
||||
assertEquals(initialLocalTopicPartitions, localRemoved(recreateBuilder)) |
||||
|
||||
val recreatedMetadata = recreateBuilder.build() |
||||
assertEquals(recreatedTopicPartitions, filterPartitions(recreatedMetadata, topic)) |
||||
assertEquals(Some(recreatedTopicId), recreatedMetadata.topicNameToId(topic)) |
||||
assertEquals(Some(topic), recreatedMetadata.topicIdToName(recreatedTopicId)) |
||||
} |
||||
|
||||
private def localRemoved( |
||||
builder: MetadataPartitionsBuilder |
||||
): Set[TopicPartition] = { |
||||
builder.localRemoved().toSet[MetadataPartition].map(_.toTopicPartition) |
||||
} |
||||
|
||||
private def localChanged( |
||||
builder: MetadataPartitionsBuilder |
||||
): Set[TopicPartition] = { |
||||
builder.localChanged().toSet[MetadataPartition].map(_.toTopicPartition) |
||||
} |
||||
|
||||
private def filterPartitions( |
||||
metadata: MetadataPartitions, |
||||
topic: String |
||||
): Set[TopicPartition] = { |
||||
metadata.topicPartitions(topic).map(_.toTopicPartition).toSet |
||||
} |
||||
|
||||
private def filterPartitions( |
||||
builder: MetadataPartitionsBuilder, |
||||
topicPartitions: Set[TopicPartition] |
||||
): Set[TopicPartition] = { |
||||
topicPartitions.filter { topicPartition => |
||||
builder.get(topicPartition.topic, topicPartition.partition).isDefined |
||||
} |
||||
} |
||||
|
||||
private def createTopic( |
||||
topic: String, |
||||
numPartitions: Int, |
||||
builder: MetadataPartitionsBuilder |
||||
): Uuid = { |
||||
val topicId = Uuid.randomUuid() |
||||
builder.addUuidMapping(topic, topicId) |
||||
|
||||
(0 until numPartitions).foreach { partition => |
||||
builder.set(newPartition(topic, partition)) |
||||
} |
||||
|
||||
topicId |
||||
} |
||||
|
||||
private def asJavaList(replicas: Iterable[Int]): java.util.List[Integer] = { |
||||
replicas.map(Int.box).toList.asJava |
||||
} |
||||
|
||||
} |
@ -1,151 +0,0 @@
@@ -1,151 +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 java.util.Collections |
||||
|
||||
import kafka.cluster.Partition |
||||
import kafka.controller.StateChangeLogger |
||||
import kafka.server.checkpoints.OffsetCheckpoints |
||||
import kafka.server.metadata.{MetadataBroker, MetadataBrokers, MetadataPartition} |
||||
import org.apache.kafka.common.message.LeaderAndIsrRequestData |
||||
import org.apache.kafka.common.network.ListenerName |
||||
import org.apache.kafka.common.{Node, TopicPartition, Uuid} |
||||
import org.junit.jupiter.api.Assertions.assertEquals |
||||
import org.junit.jupiter.params.ParameterizedTest |
||||
import org.junit.jupiter.params.provider.ValueSource |
||||
import org.mockito.ArgumentMatchers |
||||
import org.mockito.ArgumentMatchers._ |
||||
import org.mockito.Mockito._ |
||||
|
||||
import scala.jdk.CollectionConverters._ |
||||
|
||||
class RaftReplicaChangeDelegateTest { |
||||
private val listenerName = new ListenerName("PLAINTEXT") |
||||
|
||||
@ParameterizedTest |
||||
@ValueSource(booleans = Array(true, false)) |
||||
def testLeaderAndIsrPropagation(isLeader: Boolean): Unit = { |
||||
val leaderId = 0 |
||||
val topicPartition = new TopicPartition("foo", 5) |
||||
val replicas = Seq(0, 1, 2).map(Int.box).asJava |
||||
val topicId = Uuid.randomUuid() |
||||
val topicIds = (topicName: String) => if (topicName == "foo") Some(topicId) else None |
||||
|
||||
val helper = mockedHelper() |
||||
val partition = mock(classOf[Partition]) |
||||
when(partition.topicPartition).thenReturn(topicPartition) |
||||
when(partition.topic).thenReturn(topicPartition.topic) |
||||
|
||||
val highWatermarkCheckpoints = mock(classOf[OffsetCheckpoints]) |
||||
when(highWatermarkCheckpoints.fetch( |
||||
anyString(), |
||||
ArgumentMatchers.eq(topicPartition) |
||||
)).thenReturn(None) |
||||
|
||||
val metadataPartition = new MetadataPartition( |
||||
topicName = topicPartition.topic, |
||||
partitionIndex = topicPartition.partition, |
||||
leaderId = leaderId, |
||||
leaderEpoch = 27, |
||||
replicas = replicas, |
||||
isr = replicas, |
||||
partitionEpoch = 50, |
||||
offlineReplicas = Collections.emptyList(), |
||||
addingReplicas = Collections.emptyList(), |
||||
removingReplicas = Collections.emptyList() |
||||
) |
||||
|
||||
val expectedLeaderAndIsr = new LeaderAndIsrRequestData.LeaderAndIsrPartitionState() |
||||
.setTopicName(topicPartition.topic) |
||||
.setPartitionIndex(topicPartition.partition) |
||||
.setIsNew(true) |
||||
.setLeader(leaderId) |
||||
.setLeaderEpoch(27) |
||||
.setReplicas(replicas) |
||||
.setIsr(replicas) |
||||
.setAddingReplicas(Collections.emptyList()) |
||||
.setRemovingReplicas(Collections.emptyList()) |
||||
.setZkVersion(50) |
||||
|
||||
val delegate = new RaftReplicaChangeDelegate(helper) |
||||
val updatedPartitions = if (isLeader) { |
||||
when(partition.makeLeader(expectedLeaderAndIsr, highWatermarkCheckpoints, Some(topicId))) |
||||
.thenReturn(true) |
||||
delegate.makeLeaders( |
||||
prevPartitionsAlreadyExisting = Set.empty, |
||||
partitionStates = Map(partition -> metadataPartition), |
||||
highWatermarkCheckpoints, |
||||
metadataOffset = Some(500), |
||||
topicIds |
||||
) |
||||
} else { |
||||
when(partition.makeFollower(expectedLeaderAndIsr, highWatermarkCheckpoints, Some(topicId))) |
||||
.thenReturn(true) |
||||
when(partition.leaderReplicaIdOpt).thenReturn(Some(leaderId)) |
||||
delegate.makeFollowers( |
||||
prevPartitionsAlreadyExisting = Set.empty, |
||||
currentBrokers = aliveBrokers(replicas), |
||||
partitionStates = Map(partition -> metadataPartition), |
||||
highWatermarkCheckpoints, |
||||
metadataOffset = Some(500), |
||||
topicIds |
||||
) |
||||
} |
||||
|
||||
assertEquals(Set(partition), updatedPartitions) |
||||
} |
||||
|
||||
private def aliveBrokers(replicas: java.util.List[Integer]): MetadataBrokers = { |
||||
def mkNode(replicaId: Int): Node = { |
||||
new Node(replicaId, "localhost", 9092 + replicaId, "") |
||||
} |
||||
|
||||
val brokers = replicas.asScala.map { replicaId => |
||||
replicaId -> MetadataBroker( |
||||
id = replicaId, |
||||
rack = "", |
||||
endpoints = Map(listenerName.value -> mkNode(replicaId)), |
||||
fenced = false |
||||
) |
||||
}.toMap |
||||
|
||||
MetadataBrokers(brokers.values.toList.asJava, brokers.asJava) |
||||
} |
||||
|
||||
private def mockedHelper(): RaftReplicaChangeDelegateHelper = { |
||||
val helper = mock(classOf[RaftReplicaChangeDelegateHelper]) |
||||
|
||||
val stateChangeLogger = mock(classOf[StateChangeLogger]) |
||||
when(helper.stateChangeLogger).thenReturn(stateChangeLogger) |
||||
when(stateChangeLogger.isDebugEnabled).thenReturn(false) |
||||
when(stateChangeLogger.isTraceEnabled).thenReturn(false) |
||||
|
||||
val replicaFetcherManager = mock(classOf[ReplicaFetcherManager]) |
||||
when(helper.replicaFetcherManager).thenReturn(replicaFetcherManager) |
||||
|
||||
val replicaAlterLogDirsManager = mock(classOf[ReplicaAlterLogDirsManager]) |
||||
when(helper.replicaAlterLogDirsManager).thenReturn(replicaAlterLogDirsManager) |
||||
|
||||
val config = mock(classOf[KafkaConfig]) |
||||
when(config.interBrokerListenerName).thenReturn(listenerName) |
||||
when(helper.config).thenReturn(config) |
||||
|
||||
helper |
||||
} |
||||
|
||||
} |
@ -1,317 +0,0 @@
@@ -1,317 +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 java.io.File |
||||
import java.util |
||||
import java.util.concurrent.atomic.AtomicBoolean |
||||
|
||||
import kafka.cluster.Partition |
||||
import kafka.server.QuotaFactory.QuotaManagers |
||||
import kafka.server.checkpoints.LazyOffsetCheckpoints |
||||
import kafka.server.metadata.{MetadataBroker, MetadataBrokers, MetadataImage, MetadataImageBuilder, MetadataPartition, MockConfigRepository, RaftMetadataCache} |
||||
import kafka.utils.{MockScheduler, MockTime, TestUtils} |
||||
import org.apache.kafka.common.errors.InconsistentTopicIdException |
||||
import org.apache.kafka.common.{TopicPartition, Uuid} |
||||
import org.apache.kafka.common.metadata.PartitionRecord |
||||
import org.apache.kafka.common.metrics.Metrics |
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, assertTrue} |
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} |
||||
import org.mockito.{ArgumentCaptor, ArgumentMatchers} |
||||
import org.mockito.ArgumentMatchers.any |
||||
import org.mockito.Mockito.{mock, never, verify, when} |
||||
import org.slf4j.Logger |
||||
|
||||
import scala.collection.{Set, mutable} |
||||
|
||||
trait LeadershipChangeHandler { |
||||
def onLeadershipChange(updatedLeaders: Iterable[Partition], updatedFollowers: Iterable[Partition]): Unit |
||||
} |
||||
|
||||
class RaftReplicaManagerTest { |
||||
private var alterIsrManager: AlterIsrManager = _ |
||||
private var config: KafkaConfig = _ |
||||
private val configRepository = new MockConfigRepository() |
||||
private val metrics = new Metrics |
||||
private var quotaManager: QuotaManagers = _ |
||||
private val time = new MockTime |
||||
private var mockDelegate: RaftReplicaChangeDelegate = _ |
||||
private var imageBuilder: MetadataImageBuilder = _ |
||||
private val brokerId0 = 0 |
||||
private val metadataBroker0 = new MetadataBroker(brokerId0, null, Map.empty, false) |
||||
private val brokerId1 = 1 |
||||
private val metadataBroker1 = new MetadataBroker(brokerId1, null, Map.empty, false) |
||||
private val topicName = "topicName" |
||||
private val topicId = Uuid.randomUuid() |
||||
private val partitionId0 = 0 |
||||
private val partitionId1 = 1 |
||||
private val topicPartition0 = new TopicPartition(topicName, partitionId0) |
||||
private val topicPartition1 = new TopicPartition(topicName, partitionId1) |
||||
private val topicPartitionRecord0 = new PartitionRecord() |
||||
.setPartitionId(partitionId0) |
||||
.setTopicId(topicId) |
||||
.setReplicas(util.Arrays.asList(brokerId0, brokerId1)) |
||||
.setLeader(brokerId0) |
||||
.setLeaderEpoch(0) |
||||
private val topicPartitionRecord1 = new PartitionRecord() |
||||
.setPartitionId(partitionId1) |
||||
.setTopicId(topicId) |
||||
.setReplicas(util.Arrays.asList(brokerId0, brokerId1)) |
||||
.setLeader(brokerId1) |
||||
.setLeaderEpoch(0) |
||||
private val offset1 = 1L |
||||
private val metadataPartition0 = MetadataPartition(topicName, topicPartitionRecord0) |
||||
private val metadataPartition1 = MetadataPartition(topicName, topicPartitionRecord1) |
||||
private var onLeadershipChangeHandler: LeadershipChangeHandler = _ |
||||
private var onLeadershipChange: (Iterable[Partition], Iterable[Partition]) => Unit = _ |
||||
private var metadataCache: RaftMetadataCache = _ |
||||
|
||||
@BeforeEach |
||||
def setUp(): Unit = { |
||||
alterIsrManager = mock(classOf[AlterIsrManager]) |
||||
config = KafkaConfig.fromProps({ |
||||
val nodeId = brokerId0 |
||||
val props = TestUtils.createBrokerConfig(nodeId, "") |
||||
props.put(KafkaConfig.ProcessRolesProp, "broker") |
||||
props.put(KafkaConfig.NodeIdProp, nodeId.toString) |
||||
props |
||||
}) |
||||
metadataCache = new RaftMetadataCache(config.brokerId) |
||||
quotaManager = QuotaFactory.instantiate(config, metrics, time, "") |
||||
mockDelegate = mock(classOf[RaftReplicaChangeDelegate]) |
||||
imageBuilder = MetadataImageBuilder(brokerId0, mock(classOf[Logger]), new MetadataImage()) |
||||
onLeadershipChangeHandler = mock(classOf[LeadershipChangeHandler]) |
||||
onLeadershipChange = onLeadershipChangeHandler.onLeadershipChange _ |
||||
} |
||||
|
||||
@AfterEach |
||||
def tearDown(): Unit = { |
||||
TestUtils.clearYammerMetrics() |
||||
Option(quotaManager).foreach(_.shutdown()) |
||||
metrics.close() |
||||
} |
||||
|
||||
def createRaftReplicaManager(): RaftReplicaManager = { |
||||
val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_))) |
||||
new RaftReplicaManager(config, metrics, time, new MockScheduler(time), mockLogMgr, |
||||
new AtomicBoolean(false), quotaManager, new BrokerTopicStats, |
||||
metadataCache, new LogDirFailureChannel(config.logDirs.size), alterIsrManager, |
||||
configRepository, None) |
||||
} |
||||
|
||||
@Test |
||||
def testRejectsZkConfig(): Unit = { |
||||
assertThrows(classOf[IllegalStateException], () => { |
||||
val zkConfig = KafkaConfig.fromProps(TestUtils.createBrokerConfig(1, "")) |
||||
val mockLogMgr = TestUtils.createLogManager(zkConfig.logDirs.map(new File(_))) |
||||
new RaftReplicaManager(zkConfig, metrics, time, new MockScheduler(time), mockLogMgr, |
||||
new AtomicBoolean(false), quotaManager, new BrokerTopicStats, |
||||
metadataCache, new LogDirFailureChannel(config.logDirs.size), alterIsrManager, |
||||
configRepository) |
||||
}) |
||||
} |
||||
|
||||
@Test |
||||
def testDefersChangesImmediatelyThenAppliesChanges(): Unit = { |
||||
val rrm = createRaftReplicaManager() |
||||
rrm.delegate = mockDelegate |
||||
val partition0 = Partition(topicPartition0, time, rrm) |
||||
val partition1 = Partition(topicPartition1, time, rrm) |
||||
|
||||
processTopicPartitionMetadata(rrm) |
||||
// verify changes would have been deferred |
||||
val partitionsNewMapCaptor: ArgumentCaptor[mutable.Map[Partition, Boolean]] = |
||||
ArgumentCaptor.forClass(classOf[mutable.Map[Partition, Boolean]]) |
||||
verify(mockDelegate).makeDeferred(partitionsNewMapCaptor.capture(), ArgumentMatchers.eq(offset1)) |
||||
val partitionsDeferredMap = partitionsNewMapCaptor.getValue |
||||
assertEquals(Map(partition0 -> true, partition1 -> true), partitionsDeferredMap) |
||||
verify(mockDelegate, never()).makeFollowers(any(), any(), any(), any(), any(), any()) |
||||
|
||||
// now mark those topic partitions as being deferred so we can later apply the changes |
||||
rrm.markPartitionDeferred(partition0, isNew = true) |
||||
rrm.markPartitionDeferred(partition1, isNew = true) |
||||
|
||||
// apply the changes |
||||
// define some return values to avoid NPE |
||||
when(mockDelegate.makeLeaders(any(), any(), any(), any(), any())).thenReturn(Set(partition0)) |
||||
when(mockDelegate.makeFollowers(any(), any(), any(), any(), any(), any())).thenReturn(Set(partition1)) |
||||
|
||||
// Simulate creation of logs in makeLeaders/makeFollowers |
||||
partition0.createLogIfNotExists(isNew = false, isFutureReplica = false, |
||||
new LazyOffsetCheckpoints(rrm.highWatermarkCheckpoints), Some(topicId)) |
||||
partition1.createLogIfNotExists(isNew = false, isFutureReplica = false, |
||||
new LazyOffsetCheckpoints(rrm.highWatermarkCheckpoints), Some(topicId)) |
||||
|
||||
rrm.endMetadataChangeDeferral(onLeadershipChange) |
||||
// verify that the deferred changes would have been applied |
||||
|
||||
// leaders... |
||||
val leaderPartitionStates = verifyMakeLeaders(mutable.Set(), None) |
||||
assertEquals(Map(partition0 -> metadataPartition0), leaderPartitionStates) |
||||
|
||||
// followers... |
||||
val followerPartitionStates = verifyMakeFollowers(mutable.Set(), Set(brokerId0, brokerId1), None) |
||||
assertEquals(Map(partition1 -> metadataPartition1), followerPartitionStates) |
||||
|
||||
// leadership change callbacks |
||||
verifyLeadershipChangeCallbacks(List(partition0), List(partition1)) |
||||
|
||||
// partition.metadata file |
||||
verifyPartitionMetadataFile(rrm, List(topicPartition0, topicPartition1)) |
||||
} |
||||
|
||||
@Test |
||||
def testAppliesChangesWhenNotDeferring(): Unit = { |
||||
val rrm = createRaftReplicaManager() |
||||
rrm.delegate = mockDelegate |
||||
val partition0 = Partition(topicPartition0, time, rrm) |
||||
val partition1 = Partition(topicPartition1, time, rrm) |
||||
|
||||
rrm.endMetadataChangeDeferral(onLeadershipChange) |
||||
|
||||
// define some return values to avoid NPE |
||||
when(mockDelegate.makeLeaders(any(), any(), any(), ArgumentMatchers.eq(Some(offset1)), any())).thenReturn(Set(partition0)) |
||||
when(mockDelegate.makeFollowers(any(), any(), any(), any(), ArgumentMatchers.eq(Some(offset1)), any())).thenReturn(Set(partition1)) |
||||
|
||||
// Simulate creation of logs in makeLeaders/makeFollowers |
||||
partition0.createLogIfNotExists(isNew = false, isFutureReplica = false, |
||||
new LazyOffsetCheckpoints(rrm.highWatermarkCheckpoints), Some(topicId)) |
||||
partition1.createLogIfNotExists(isNew = false, isFutureReplica = false, |
||||
new LazyOffsetCheckpoints(rrm.highWatermarkCheckpoints), Some(topicId)) |
||||
|
||||
// process the changes |
||||
processTopicPartitionMetadata(rrm) |
||||
// verify that the changes would have been applied |
||||
|
||||
// leaders... |
||||
val leaderPartitionStates = verifyMakeLeaders(mutable.Set(), Some(offset1)) |
||||
assertEquals(Map(partition0 -> metadataPartition0), leaderPartitionStates) |
||||
|
||||
// followers... |
||||
val followerPartitionStates = verifyMakeFollowers(mutable.Set(), Set(brokerId0, brokerId1), Some(offset1)) |
||||
assertEquals(Map(partition1 -> metadataPartition1), followerPartitionStates) |
||||
|
||||
// leadership change callbacks |
||||
verifyLeadershipChangeCallbacks(List(partition0), List(partition1)) |
||||
|
||||
// partition.metadata file |
||||
verifyPartitionMetadataFile(rrm, List(topicPartition0, topicPartition1)) |
||||
} |
||||
|
||||
@Test |
||||
def testInconsistentTopicIdDefersChanges(): Unit = { |
||||
val rrm = createRaftReplicaManager() |
||||
rrm.delegate = mockDelegate |
||||
val partition0 = rrm.createPartition(topicPartition0) |
||||
val partition1 = rrm.createPartition(topicPartition1) |
||||
|
||||
partition0.createLogIfNotExists(isNew = false, isFutureReplica = false, |
||||
new LazyOffsetCheckpoints(rrm.highWatermarkCheckpoints), Some(Uuid.randomUuid())) |
||||
partition1.createLogIfNotExists(isNew = false, isFutureReplica = false, |
||||
new LazyOffsetCheckpoints(rrm.highWatermarkCheckpoints), Some(Uuid.randomUuid())) |
||||
|
||||
try { |
||||
processTopicPartitionMetadata(rrm) |
||||
} catch { |
||||
case e: Throwable => assertTrue(e.isInstanceOf[InconsistentTopicIdException]) |
||||
} |
||||
} |
||||
|
||||
@Test |
||||
def testInconsistentTopicIdWhenNotDeferring(): Unit = { |
||||
val rrm = createRaftReplicaManager() |
||||
rrm.delegate = mockDelegate |
||||
val partition0 = rrm.createPartition(topicPartition0) |
||||
val partition1 = rrm.createPartition(topicPartition1) |
||||
|
||||
partition0.createLogIfNotExists(isNew = false, isFutureReplica = false, |
||||
new LazyOffsetCheckpoints(rrm.highWatermarkCheckpoints), Some(Uuid.randomUuid())) |
||||
partition1.createLogIfNotExists(isNew = false, isFutureReplica = false, |
||||
new LazyOffsetCheckpoints(rrm.highWatermarkCheckpoints), Some(Uuid.randomUuid())) |
||||
|
||||
rrm.endMetadataChangeDeferral(onLeadershipChange) |
||||
|
||||
// define some return values to avoid NPE |
||||
when(mockDelegate.makeLeaders(any(), any(), any(), ArgumentMatchers.eq(Some(offset1)), any())).thenReturn(Set(partition0)) |
||||
when(mockDelegate.makeFollowers(any(), any(), any(), any(), ArgumentMatchers.eq(Some(offset1)), any())).thenReturn(Set(partition1)) |
||||
|
||||
try { |
||||
processTopicPartitionMetadata(rrm) |
||||
} catch { |
||||
case e: Throwable => assertTrue(e.isInstanceOf[InconsistentTopicIdException]) |
||||
} |
||||
} |
||||
|
||||
private def verifyMakeLeaders(expectedPrevPartitionsAlreadyExisting: Set[MetadataPartition], |
||||
expectedMetadataOffset: Option[Long]): mutable.Map[Partition, MetadataPartition] = { |
||||
val leaderPartitionStatesCaptor: ArgumentCaptor[mutable.Map[Partition, MetadataPartition]] = |
||||
ArgumentCaptor.forClass(classOf[mutable.Map[Partition, MetadataPartition]]) |
||||
verify(mockDelegate).makeLeaders(ArgumentMatchers.eq(expectedPrevPartitionsAlreadyExisting), |
||||
leaderPartitionStatesCaptor.capture(), any(), ArgumentMatchers.eq(expectedMetadataOffset), any()) |
||||
leaderPartitionStatesCaptor.getValue |
||||
} |
||||
|
||||
private def verifyMakeFollowers(expectedPrevPartitionsAlreadyExisting: Set[MetadataPartition], |
||||
expectedBrokers: Set[Int], |
||||
expectedMetadataOffset: Option[Long]): mutable.Map[Partition, MetadataPartition] = { |
||||
val followerPartitionStatesCaptor: ArgumentCaptor[mutable.Map[Partition, MetadataPartition]] = |
||||
ArgumentCaptor.forClass(classOf[mutable.Map[Partition, MetadataPartition]]) |
||||
val brokersCaptor: ArgumentCaptor[MetadataBrokers] = ArgumentCaptor.forClass(classOf[MetadataBrokers]) |
||||
verify(mockDelegate).makeFollowers(ArgumentMatchers.eq(expectedPrevPartitionsAlreadyExisting), brokersCaptor.capture(), |
||||
followerPartitionStatesCaptor.capture(), any(), ArgumentMatchers.eq(expectedMetadataOffset), any()) |
||||
val brokers = brokersCaptor.getValue |
||||
assertEquals(expectedBrokers.size, brokers.size()) |
||||
expectedBrokers.foreach(brokerId => assertTrue(brokers.aliveBroker(brokerId).isDefined)) |
||||
followerPartitionStatesCaptor.getValue |
||||
} |
||||
|
||||
private def verifyLeadershipChangeCallbacks(expectedUpdatedLeaders: List[Partition], expectedUpdatedFollowers: List[Partition]): Unit = { |
||||
val updatedLeadersCaptor: ArgumentCaptor[Iterable[Partition]] = ArgumentCaptor.forClass(classOf[Iterable[Partition]]) |
||||
val updatedFollowersCaptor: ArgumentCaptor[Iterable[Partition]] = ArgumentCaptor.forClass(classOf[Iterable[Partition]]) |
||||
verify(onLeadershipChangeHandler).onLeadershipChange(updatedLeadersCaptor.capture(), updatedFollowersCaptor.capture()) |
||||
assertEquals(expectedUpdatedLeaders, updatedLeadersCaptor.getValue.toList) |
||||
assertEquals(expectedUpdatedFollowers, updatedFollowersCaptor.getValue.toList) |
||||
} |
||||
|
||||
private def verifyPartitionMetadataFile(rrm: RaftReplicaManager, topicPartitions: List[TopicPartition]) = { |
||||
topicPartitions.foreach ( topicPartition => { |
||||
val log = rrm.getLog(topicPartition).get |
||||
assertTrue(log.partitionMetadataFile.exists()) |
||||
val partitionMetadata = log.partitionMetadataFile.read() |
||||
|
||||
// Current version of PartitionMetadataFile is 0. |
||||
assertEquals(0, partitionMetadata.version) |
||||
assertEquals(topicId, partitionMetadata.topicId) |
||||
}) |
||||
} |
||||
|
||||
private def processTopicPartitionMetadata(raftReplicaManager: RaftReplicaManager): Unit = { |
||||
// create brokers |
||||
imageBuilder.brokersBuilder().add(metadataBroker0) |
||||
imageBuilder.brokersBuilder().add(metadataBroker1) |
||||
// create topic |
||||
imageBuilder.partitionsBuilder().addUuidMapping(topicName, topicId) |
||||
// create deferred partitions |
||||
imageBuilder.partitionsBuilder().set(metadataPartition0) |
||||
imageBuilder.partitionsBuilder().set(metadataPartition1) |
||||
// apply the changes to metadata cache |
||||
metadataCache.image(imageBuilder.build()) |
||||
// apply the changes to replica manager |
||||
raftReplicaManager.handleMetadataRecords(imageBuilder, offset1, onLeadershipChange) |
||||
} |
||||
} |
@ -1,78 +0,0 @@
@@ -1,78 +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 org.junit.jupiter.api.Assertions.assertEquals |
||||
import org.junit.jupiter.api.Test |
||||
|
||||
class CachedConfigRepositoryTest { |
||||
@Test |
||||
def testEmptyRepository(): Unit = { |
||||
val repository = new CachedConfigRepository() |
||||
assertEquals(new Properties(), repository.brokerConfig(0)) |
||||
assertEquals(new Properties(), repository.topicConfig("foo")) |
||||
} |
||||
|
||||
@Test |
||||
def testSetBrokerConfig(): Unit = { |
||||
val repository = new CachedConfigRepository() |
||||
val brokerId0 = 0 |
||||
repository.setBrokerConfig(brokerId0, "foo", null) |
||||
assertEquals(new Properties(), repository.brokerConfig(0)) |
||||
|
||||
val brokerId1 = 1 |
||||
repository.setBrokerConfig(brokerId1, "foo", "bar") |
||||
val brokerProperties = new Properties() |
||||
brokerProperties.put("foo", "bar") |
||||
assertEquals(brokerProperties, repository.brokerConfig(brokerId1)) |
||||
|
||||
val brokerProperties2 = new Properties() |
||||
brokerProperties2.put("foo", "bar") |
||||
brokerProperties2.put("foo2", "baz") |
||||
repository.setBrokerConfig(brokerId1, "foo2", "baz") // add another prop |
||||
assertEquals(brokerProperties2, repository.brokerConfig(brokerId1)) // should get both props |
||||
|
||||
repository.setBrokerConfig(brokerId1, "foo2", null) |
||||
assertEquals(brokerProperties, repository.brokerConfig(brokerId1)) |
||||
} |
||||
|
||||
@Test |
||||
def testSetTopicConfig(): Unit = { |
||||
val repository = new CachedConfigRepository() |
||||
val topic0 = "topic0" |
||||
repository.setTopicConfig(topic0, "foo", null) |
||||
assertEquals(new Properties(), repository.brokerConfig(0)) |
||||
|
||||
val topic1 = "topic1" |
||||
repository.setTopicConfig(topic1, "foo", "bar") |
||||
val topicProperties = new Properties() |
||||
topicProperties.put("foo", "bar") |
||||
assertEquals(topicProperties, repository.topicConfig(topic1)) |
||||
|
||||
val topicProperties2 = new Properties() |
||||
topicProperties2.put("foo", "bar") |
||||
topicProperties2.put("foo2", "baz") |
||||
repository.setTopicConfig(topic1, "foo2", "baz") // add another prop |
||||
assertEquals(topicProperties2, repository.topicConfig(topic1)) // should get both props |
||||
|
||||
repository.setTopicConfig(topic1, "foo2", null) |
||||
assertEquals(topicProperties, repository.topicConfig(topic1)) |
||||
} |
||||
} |
@ -1,452 +0,0 @@
@@ -1,452 +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 kafka.network.ConnectionQuotas |
||||
import kafka.server.QuotaFactory.QuotaManagers |
||||
import kafka.server.{ConfigEntityName, KafkaConfig, QuotaFactory} |
||||
import kafka.utils.{MockTime, TestUtils} |
||||
import org.apache.kafka.common.config.internals.QuotaConfigs |
||||
import org.apache.kafka.common.errors.{InvalidRequestException, UnsupportedVersionException} |
||||
import org.apache.kafka.common.metadata.ClientQuotaRecord |
||||
import org.apache.kafka.common.metrics.{Metrics, Quota} |
||||
import org.apache.kafka.common.quota.{ClientQuotaEntity, ClientQuotaFilterComponent} |
||||
import org.junit.jupiter.api.Assertions._ |
||||
import org.junit.jupiter.api.{BeforeEach, Test} |
||||
import org.mockito.ArgumentMatchers.{any, eq => _eq} |
||||
import org.mockito.Mockito._ |
||||
|
||||
import java.net.InetAddress |
||||
import java.util.Properties |
||||
import scala.collection.mutable |
||||
import scala.jdk.CollectionConverters._ |
||||
|
||||
class ClientQuotaMetadataManagerTest { |
||||
|
||||
var manager: ClientQuotaMetadataManager = _ |
||||
var cache: ClientQuotaCache = _ |
||||
|
||||
@BeforeEach |
||||
def setup(): Unit = { |
||||
val configs = TestUtils.createBrokerConfigs(1, TestUtils.MockZkConnect) |
||||
.map(KafkaConfig.fromProps(_, new Properties())) |
||||
|
||||
val time = new MockTime |
||||
val metrics = new Metrics |
||||
val quotaManagers = QuotaFactory.instantiate(configs.head, metrics, time, "quota-metadata-processor-test") |
||||
val spiedQuotaManagers = QuotaManagers( |
||||
fetch = spy(quotaManagers.fetch), |
||||
produce = spy(quotaManagers.produce), |
||||
request = spy(quotaManagers.request), |
||||
controllerMutation = spy(quotaManagers.controllerMutation), |
||||
leader = quotaManagers.leader, |
||||
follower = quotaManagers.follower, |
||||
alterLogDirs = quotaManagers.alterLogDirs, |
||||
clientQuotaCallback = quotaManagers.clientQuotaCallback |
||||
) |
||||
val connectionQuotas = mock(classOf[ConnectionQuotas]) |
||||
cache = new ClientQuotaCache() |
||||
manager = new ClientQuotaMetadataManager(spiedQuotaManagers, connectionQuotas, cache) |
||||
} |
||||
|
||||
@Test |
||||
def testDescribeStrictMatch(): Unit = { |
||||
setupAndVerify(manager, { case (entity, _) => |
||||
val components = mutable.ListBuffer[ClientQuotaFilterComponent]() |
||||
entityToFilter(entity, components) |
||||
val results = cache.describeClientQuotas(components.toSeq, strict=true) |
||||
assertEquals(1, results.size, s"Should only match one quota for ${entity}") |
||||
}) |
||||
|
||||
val nonMatching = List( |
||||
userClientEntity("user-1", "client-id-2"), |
||||
userClientEntity("user-3", "client-id-1"), |
||||
userClientEntity("user-2", null), |
||||
userEntity("user-4"), |
||||
userClientEntity(null, "client-id-2"), |
||||
clientEntity("client-id-1"), |
||||
clientEntity("client-id-3") |
||||
) |
||||
|
||||
nonMatching.foreach( entity => { |
||||
val components = mutable.ListBuffer[ClientQuotaFilterComponent]() |
||||
entityToFilter(entity, components) |
||||
val results = cache.describeClientQuotas(components.toSeq, strict=true) |
||||
assertEquals(0, results.size) |
||||
}) |
||||
} |
||||
|
||||
@Test |
||||
def testDescribeNonStrictMatch(): Unit = { |
||||
setupAndVerify(manager, { case (_, _) => }) |
||||
|
||||
// Match open-ended existing user. |
||||
val components = mutable.ListBuffer[ClientQuotaFilterComponent]() |
||||
entityToFilter(userEntity("user-1"), components) |
||||
var results = cache.describeClientQuotasInternal(components.toSeq, strict=false) |
||||
assertEquals(3, results.size) |
||||
assertEquals(3, results.keySet.count(quotaEntity => quotaEntity match { |
||||
case UserEntity(user) => user.equals("user-1") |
||||
case ExplicitUserDefaultClientIdEntity(user) => user.equals("user-1") |
||||
case ExplicitUserExplicitClientIdEntity(user, _) => user.equals("user-1") |
||||
case _ => false |
||||
})) |
||||
|
||||
results = cache.describeClientQuotasInternal(components.toSeq, strict=true) |
||||
assertEquals(1, results.size) |
||||
|
||||
// Match open-ended non-existent user. |
||||
components.clear() |
||||
entityToFilter(userEntity("unknown"), components) |
||||
results = cache.describeClientQuotasInternal(components.toSeq, strict=false) |
||||
assertEquals(0, results.size) |
||||
|
||||
// Match open-ended existing client ID. |
||||
components.clear() |
||||
entityToFilter(clientEntity("client-id-2"), components) |
||||
results = cache.describeClientQuotasInternal(components.toSeq, strict=false) |
||||
assertEquals(2, results.size) |
||||
assertEquals(2, results.keySet.count(quotaEntity => quotaEntity match { |
||||
case ClientIdEntity(clientId) => clientId.equals("client-id-2") |
||||
case DefaultUserExplicitClientIdEntity(clientId) => clientId.equals("client-id-2") |
||||
case ExplicitUserExplicitClientIdEntity(_, clientId) => clientId.equals("client-id-2") |
||||
case _ => false |
||||
})) |
||||
|
||||
// Match open-ended default user. |
||||
results = cache.describeClientQuotasInternal( |
||||
Seq(ClientQuotaFilterComponent.ofDefaultEntity(ClientQuotaEntity.USER)), strict=false) |
||||
assertEquals(3, results.size) |
||||
assertEquals(3, results.keySet.count(quotaEntity => quotaEntity match { |
||||
case DefaultUserEntity | DefaultUserExplicitClientIdEntity(_) | DefaultUserDefaultClientIdEntity => true |
||||
case _ => false |
||||
})) |
||||
|
||||
// Match open-ended default client. |
||||
results = cache.describeClientQuotasInternal( |
||||
Seq(ClientQuotaFilterComponent.ofDefaultEntity(ClientQuotaEntity.CLIENT_ID)), strict=false) |
||||
assertEquals(3, results.size) |
||||
assertEquals(3, results.keySet.count(quotaEntity => quotaEntity match { |
||||
case DefaultClientIdEntity | ExplicitUserDefaultClientIdEntity(_) | DefaultUserDefaultClientIdEntity => true |
||||
case _ => false |
||||
})) |
||||
} |
||||
|
||||
@Test |
||||
def testDescribeFilterOnTypes(): Unit = { |
||||
setupAndVerify(manager, { case (_, _) => }) |
||||
|
||||
var results = cache.describeClientQuotasInternal( |
||||
Seq(ClientQuotaFilterComponent.ofEntityType(ClientQuotaEntity.USER)), strict=false) |
||||
assertEquals(11, results.size) |
||||
assertEquals(11, results.keySet.count(quotaEntity => quotaEntity match { |
||||
case UserEntity(_) | DefaultUserEntity | ExplicitUserExplicitClientIdEntity(_, _) | ExplicitUserDefaultClientIdEntity(_) | |
||||
DefaultUserExplicitClientIdEntity(_) | DefaultUserDefaultClientIdEntity => true |
||||
case _ => false |
||||
})) |
||||
|
||||
results = cache.describeClientQuotasInternal( |
||||
Seq(ClientQuotaFilterComponent.ofEntityType(ClientQuotaEntity.CLIENT_ID)), strict=false) |
||||
assertEquals(8, results.size) |
||||
assertEquals(8, results.keySet.count(quotaEntity => quotaEntity match { |
||||
case ClientIdEntity(_) | DefaultClientIdEntity | ExplicitUserExplicitClientIdEntity(_, _) | ExplicitUserDefaultClientIdEntity(_) | |
||||
DefaultUserExplicitClientIdEntity(_) | DefaultUserDefaultClientIdEntity => true |
||||
case _ => false |
||||
})) |
||||
|
||||
results = cache.describeClientQuotasInternal( |
||||
Seq( |
||||
ClientQuotaFilterComponent.ofEntityType(ClientQuotaEntity.USER), |
||||
ClientQuotaFilterComponent.ofEntityType(ClientQuotaEntity.CLIENT_ID) |
||||
), strict=true) |
||||
assertEquals(7, results.size) |
||||
assertEquals(7, results.keySet.count(quotaEntity => quotaEntity match { |
||||
case ExplicitUserExplicitClientIdEntity(_, _) | ExplicitUserDefaultClientIdEntity(_) | |
||||
DefaultUserExplicitClientIdEntity(_) | DefaultUserDefaultClientIdEntity => true |
||||
case _ => false |
||||
})) |
||||
} |
||||
|
||||
@Test |
||||
def testEntityWithDefaultName(): Unit = { |
||||
addQuotaRecord(manager, clientEntity(ConfigEntityName.Default), (QuotaConfigs.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG, 20000.0)) |
||||
addQuotaRecord(manager, clientEntity(null), (QuotaConfigs.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG, 30000.0)) |
||||
|
||||
val components = mutable.ListBuffer[ClientQuotaFilterComponent]() |
||||
entityToFilter(clientEntity(ConfigEntityName.Default), components) |
||||
var results = cache.describeClientQuotas(components.toSeq, strict=true) |
||||
assertEquals(1, results.size) |
||||
|
||||
components.clear() |
||||
entityToFilter(clientEntity(null), components) |
||||
results = cache.describeClientQuotas(components.toSeq, strict=true) |
||||
assertEquals(1, results.size) |
||||
} |
||||
|
||||
@Test |
||||
def testQuotaRemoval(): Unit = { |
||||
val entity = userClientEntity("user", "client-id") |
||||
addQuotaRecord(manager, entity, (QuotaConfigs.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG, 10000.0)) |
||||
addQuotaRecord(manager, entity, (QuotaConfigs.CONSUMER_BYTE_RATE_OVERRIDE_CONFIG, 20000.0)) |
||||
var quotas = describeEntity(entity) |
||||
assertEquals(2, quotas.size) |
||||
assertEquals(10000.0, quotas(QuotaConfigs.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG), 1e-6) |
||||
|
||||
addQuotaRecord(manager, entity, (QuotaConfigs.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG, 10001.0)) |
||||
quotas = describeEntity(entity) |
||||
assertEquals(2, quotas.size) |
||||
assertEquals(10001.0, quotas(QuotaConfigs.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG), 1e-6) |
||||
|
||||
addQuotaRemovalRecord(manager, entity, QuotaConfigs.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG) |
||||
quotas = describeEntity(entity) |
||||
assertEquals(1, quotas.size) |
||||
assertFalse(quotas.contains(QuotaConfigs.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG)) |
||||
|
||||
addQuotaRemovalRecord(manager, entity, QuotaConfigs.CONSUMER_BYTE_RATE_OVERRIDE_CONFIG) |
||||
quotas = describeEntity(entity) |
||||
assertEquals(0, quotas.size) |
||||
|
||||
// Removing non-existent quota should not do anything |
||||
addQuotaRemovalRecord(manager, entity, QuotaConfigs.CONSUMER_BYTE_RATE_OVERRIDE_CONFIG) |
||||
quotas = describeEntity(entity) |
||||
assertEquals(0, quotas.size) |
||||
} |
||||
|
||||
@Test |
||||
def testDescribeClientQuotasInvalidFilterCombination(): Unit = { |
||||
val ipFilterComponent = ClientQuotaFilterComponent.ofEntityType(ClientQuotaEntity.IP) |
||||
val userFilterComponent = ClientQuotaFilterComponent.ofEntityType(ClientQuotaEntity.USER) |
||||
val clientIdFilterComponent = ClientQuotaFilterComponent.ofEntityType(ClientQuotaEntity.CLIENT_ID) |
||||
val expectedExceptionMessage = "Invalid entity filter component combination" |
||||
assertThrows(classOf[InvalidRequestException], |
||||
() => cache.describeClientQuotas(Seq(ipFilterComponent, userFilterComponent), strict=false), |
||||
() => expectedExceptionMessage) |
||||
assertThrows(classOf[InvalidRequestException], |
||||
() => cache.describeClientQuotas(Seq(ipFilterComponent, clientIdFilterComponent), strict=false), |
||||
() => expectedExceptionMessage) |
||||
assertThrows(classOf[InvalidRequestException], |
||||
() => cache.describeClientQuotas(Seq(ipFilterComponent, ipFilterComponent), strict = false), |
||||
() => expectedExceptionMessage) |
||||
assertThrows(classOf[InvalidRequestException], |
||||
() => cache.describeClientQuotas(Seq(userFilterComponent, userFilterComponent), strict=false), |
||||
() => expectedExceptionMessage) |
||||
} |
||||
|
||||
@Test |
||||
def testDescribeEmptyFilter(): Unit = { |
||||
var results = cache.describeClientQuotas(Seq.empty, strict=false) |
||||
assertEquals(0, results.size) |
||||
|
||||
results = cache.describeClientQuotas(Seq.empty, strict=true) |
||||
assertEquals(0, results.size) |
||||
} |
||||
|
||||
@Test |
||||
def testDescribeUnsupportedEntityType(): Unit = { |
||||
assertThrows(classOf[UnsupportedVersionException], |
||||
() => cache.describeClientQuotas(Seq(ClientQuotaFilterComponent.ofEntityType("other")), strict=false)) |
||||
} |
||||
|
||||
@Test |
||||
def testDescribeMissingEntityType(): Unit = { |
||||
assertThrows(classOf[InvalidRequestException], |
||||
() => cache.describeClientQuotas(Seq(ClientQuotaFilterComponent.ofEntity("", "name")), strict = false)) |
||||
} |
||||
|
||||
@Test |
||||
def testQuotaManagers(): Unit = { |
||||
val entity = userClientEntity("user", "client") |
||||
addQuotaRecord(manager, entity, (QuotaConfigs.CONSUMER_BYTE_RATE_OVERRIDE_CONFIG, 100.0)) |
||||
verify(manager.quotaManagers.fetch, times(1)).updateQuota( |
||||
_eq(Some("user")), |
||||
_eq(Some("client")), |
||||
_eq(Some("client")), |
||||
any(classOf[Option[Quota]]) |
||||
) |
||||
|
||||
addQuotaRecord(manager, entity, (QuotaConfigs.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG, 100.0)) |
||||
verify(manager.quotaManagers.produce, times(1)).updateQuota( |
||||
_eq(Some("user")), |
||||
_eq(Some("client")), |
||||
_eq(Some("client")), |
||||
any(classOf[Option[Quota]]) |
||||
) |
||||
|
||||
addQuotaRecord(manager, entity, (QuotaConfigs.REQUEST_PERCENTAGE_OVERRIDE_CONFIG, 100.0)) |
||||
verify(manager.quotaManagers.request, times(1)).updateQuota( |
||||
_eq(Some("user")), |
||||
_eq(Some("client")), |
||||
_eq(Some("client")), |
||||
any(classOf[Option[Quota]]) |
||||
) |
||||
|
||||
addQuotaRecord(manager, entity, (QuotaConfigs.CONTROLLER_MUTATION_RATE_OVERRIDE_CONFIG, 100.0)) |
||||
verify(manager.quotaManagers.controllerMutation, times(1)).updateQuota( |
||||
_eq(Some("user")), |
||||
_eq(Some("client")), |
||||
_eq(Some("client")), |
||||
any(classOf[Option[Quota]]) |
||||
) |
||||
|
||||
addQuotaRemovalRecord(manager, entity, QuotaConfigs.CONTROLLER_MUTATION_RATE_OVERRIDE_CONFIG) |
||||
verify(manager.quotaManagers.controllerMutation, times(1)).updateQuota( |
||||
_eq(Some("user")), |
||||
_eq(Some("client")), |
||||
_eq(Some("client")), |
||||
_eq(None) |
||||
) |
||||
} |
||||
|
||||
@Test |
||||
def testIpQuota(): Unit = { |
||||
val defaultIp = ipEntity(null) |
||||
val knownIp = ipEntity("1.2.3.4") |
||||
|
||||
addQuotaRecord(manager, defaultIp, (QuotaConfigs.IP_CONNECTION_RATE_OVERRIDE_CONFIG, 100.0)) |
||||
addQuotaRecord(manager, knownIp, (QuotaConfigs.IP_CONNECTION_RATE_OVERRIDE_CONFIG, 99.0)) |
||||
|
||||
verify(manager.connectionQuotas, times(2)).updateIpConnectionRateQuota( |
||||
any(classOf[Option[InetAddress]]), |
||||
any(classOf[Option[Int]]) |
||||
) |
||||
|
||||
var quotas = describeEntity(defaultIp) |
||||
assertEquals(1, quotas.size) |
||||
|
||||
quotas = describeEntity(knownIp) |
||||
assertEquals(1, quotas.size) |
||||
|
||||
val results = cache.describeClientQuotas(Seq(ClientQuotaFilterComponent.ofEntityType(ClientQuotaEntity.IP)), strict=false) |
||||
assertEquals(2, results.size) |
||||
|
||||
reset(manager.connectionQuotas) |
||||
addQuotaRecord(manager, knownIp, (QuotaConfigs.IP_CONNECTION_RATE_OVERRIDE_CONFIG, 98.0)) |
||||
verify(manager.connectionQuotas, times(1)).updateIpConnectionRateQuota( |
||||
any(classOf[Option[InetAddress]]), |
||||
_eq(Some(98)) |
||||
) |
||||
|
||||
reset(manager.connectionQuotas) |
||||
addQuotaRemovalRecord(manager, knownIp, QuotaConfigs.IP_CONNECTION_RATE_OVERRIDE_CONFIG) |
||||
verify(manager.connectionQuotas, times(1)).updateIpConnectionRateQuota( |
||||
any(classOf[Option[InetAddress]]), |
||||
_eq(None) |
||||
) |
||||
} |
||||
|
||||
@Test |
||||
def testIpQuotaUnknownKey(): Unit = { |
||||
val defaultIp = ipEntity(null) |
||||
addQuotaRecord(manager, defaultIp, ("not-an-ip-quota-key", 100.0)) |
||||
verify(manager.connectionQuotas, times(0)).updateIpConnectionRateQuota( |
||||
any(classOf[Option[InetAddress]]), |
||||
_eq(Some(100)) |
||||
) |
||||
|
||||
assertEquals(0, describeEntity(defaultIp).size) |
||||
} |
||||
|
||||
@Test |
||||
def testUserQuotaUnknownKey(): Unit = { |
||||
val defaultUser = userEntity(null) |
||||
addQuotaRecord(manager, defaultUser, ("not-a-user-quota-key", 100.0)) |
||||
assertEquals(0, describeEntity(defaultUser).size) |
||||
} |
||||
|
||||
def setupAndVerify(manager: ClientQuotaMetadataManager, |
||||
verifier: (List[ClientQuotaRecord.EntityData], (String, Double)) => Unit ): Unit = { |
||||
val toVerify = List( |
||||
(userClientEntity("user-1", "client-id-1"), 50.50), |
||||
(userClientEntity("user-2", "client-id-1"), 51.51), |
||||
(userClientEntity("user-3", "client-id-2"), 52.52), |
||||
(userClientEntity(null, "client-id-1"), 53.53), |
||||
(userClientEntity("user-1", null), 54.54), |
||||
(userClientEntity("user-3", null), 55.55), |
||||
(userEntity("user-1"), 56.56), |
||||
(userEntity("user-2"), 57.57), |
||||
(userEntity("user-3"), 58.58), |
||||
(userEntity(null), 59.59), |
||||
(clientEntity("client-id-2"), 60.60), |
||||
(userClientEntity(null, null), 61.61) |
||||
) |
||||
|
||||
toVerify.foreach { |
||||
case (entity, value) => addQuotaRecord(manager, entity, (QuotaConfigs.REQUEST_PERCENTAGE_OVERRIDE_CONFIG, value)) |
||||
} |
||||
|
||||
toVerify.foreach { |
||||
case (entity, value) => verifier.apply(entity, (QuotaConfigs.REQUEST_PERCENTAGE_OVERRIDE_CONFIG, value)) |
||||
} |
||||
} |
||||
|
||||
def describeEntity(entity: List[ClientQuotaRecord.EntityData]): Map[String, Double] = { |
||||
val components = mutable.ListBuffer[ClientQuotaFilterComponent]() |
||||
entityToFilter(entity, components) |
||||
val results = cache.describeClientQuotas(components.toSeq, strict=true) |
||||
if (results.isEmpty) { |
||||
Map() |
||||
} else if (results.size == 1) { |
||||
results.head._2 |
||||
} else { |
||||
throw new AssertionError("Matched more than one entity with strict=true describe filter") |
||||
} |
||||
} |
||||
|
||||
def addQuotaRecord(manager: ClientQuotaMetadataManager, entity: List[ClientQuotaRecord.EntityData], quota: (String, Double)): Unit = { |
||||
manager.handleQuotaRecord(new ClientQuotaRecord() |
||||
.setEntity(entity.asJava) |
||||
.setKey(quota._1) |
||||
.setValue(quota._2)) |
||||
} |
||||
|
||||
def addQuotaRemovalRecord(manager: ClientQuotaMetadataManager, entity: List[ClientQuotaRecord.EntityData], quota: String): Unit = { |
||||
manager.handleQuotaRecord(new ClientQuotaRecord() |
||||
.setEntity(entity.asJava) |
||||
.setKey(quota) |
||||
.setRemove(true)) |
||||
} |
||||
|
||||
def entityToFilter(entity: List[ClientQuotaRecord.EntityData], components: mutable.ListBuffer[ClientQuotaFilterComponent]): Unit = { |
||||
entity.foreach(entityData => { |
||||
if (entityData.entityName() == null) { |
||||
components.append(ClientQuotaFilterComponent.ofDefaultEntity(entityData.entityType())) |
||||
} else { |
||||
components.append(ClientQuotaFilterComponent.ofEntity(entityData.entityType(), entityData.entityName())) |
||||
} |
||||
}) |
||||
} |
||||
|
||||
def clientEntity(clientId: String): List[ClientQuotaRecord.EntityData] = { |
||||
List(new ClientQuotaRecord.EntityData().setEntityType(ClientQuotaEntity.CLIENT_ID).setEntityName(clientId)) |
||||
} |
||||
|
||||
def userEntity(user: String): List[ClientQuotaRecord.EntityData] = { |
||||
List(new ClientQuotaRecord.EntityData().setEntityType(ClientQuotaEntity.USER).setEntityName(user)) |
||||
} |
||||
|
||||
def userClientEntity(user: String, clientId: String): List[ClientQuotaRecord.EntityData] = { |
||||
List( |
||||
new ClientQuotaRecord.EntityData().setEntityType(ClientQuotaEntity.USER).setEntityName(user), |
||||
new ClientQuotaRecord.EntityData().setEntityType(ClientQuotaEntity.CLIENT_ID).setEntityName(clientId) |
||||
) |
||||
} |
||||
|
||||
def ipEntity(ip: String): List[ClientQuotaRecord.EntityData] = { |
||||
List(new ClientQuotaRecord.EntityData().setEntityType(ClientQuotaEntity.IP).setEntityName(ip)) |
||||
} |
||||
} |
Loading…
Reference in new issue