Browse Source

KAFKA-12998: Implement broker-side KRaft snapshots (#10931)

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
parent
commit
7bd55f5156
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 1
      build.gradle
  2. 1
      clients/src/main/java/org/apache/kafka/common/requests/DescribeClientQuotasResponse.java
  3. 6
      core/src/main/scala/kafka/server/AbstractFetcherManager.scala
  4. 114
      core/src/main/scala/kafka/server/BrokerServer.scala
  5. 4
      core/src/main/scala/kafka/server/ConfigHandler.scala
  6. 3
      core/src/main/scala/kafka/server/DelayedDeleteRecords.scala
  7. 21
      core/src/main/scala/kafka/server/FinalizedFeatureCache.scala
  8. 18
      core/src/main/scala/kafka/server/KafkaApis.scala
  9. 2
      core/src/main/scala/kafka/server/KafkaServer.scala
  10. 15
      core/src/main/scala/kafka/server/MetadataCache.scala
  11. 18
      core/src/main/scala/kafka/server/MetadataSupport.scala
  12. 252
      core/src/main/scala/kafka/server/RaftReplicaChangeDelegate.scala
  13. 404
      core/src/main/scala/kafka/server/RaftReplicaManager.scala
  14. 265
      core/src/main/scala/kafka/server/ReplicaManager.scala
  15. 332
      core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala
  16. 249
      core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala
  17. 111
      core/src/main/scala/kafka/server/metadata/CachedConfigRepository.scala
  18. 307
      core/src/main/scala/kafka/server/metadata/ClientQuotaCache.scala
  19. 97
      core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala
  20. 354
      core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala
  21. 150
      core/src/main/scala/kafka/server/metadata/MetadataBrokers.scala
  22. 137
      core/src/main/scala/kafka/server/metadata/MetadataImage.scala
  23. 355
      core/src/main/scala/kafka/server/metadata/MetadataPartitions.scala
  24. 36
      core/src/main/scala/kafka/server/metadata/MetadataPublisher.scala
  25. 394
      core/src/main/scala/kafka/server/metadata/RaftMetadataCache.scala
  26. 90
      core/src/test/scala/kafka/server/metadata/MetadataBrokersTest.scala
  27. 324
      core/src/test/scala/kafka/server/metadata/MetadataPartitionsTest.scala
  28. 2
      core/src/test/scala/kafka/server/metadata/MockConfigRepository.scala
  29. 2
      core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
  30. 2
      core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala
  31. 5
      core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala
  32. 3
      core/src/test/scala/unit/kafka/server/IsrExpirationTest.scala
  33. 58
      core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
  34. 113
      core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala
  35. 151
      core/src/test/scala/unit/kafka/server/RaftReplicaChangeDelegateTest.scala
  36. 317
      core/src/test/scala/unit/kafka/server/RaftReplicaManagerTest.scala
  37. 4
      core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
  38. 115
      core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
  39. 8
      core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala
  40. 248
      core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala
  41. 78
      core/src/test/scala/unit/kafka/server/metadata/CachedConfigRepositoryTest.scala
  42. 452
      core/src/test/scala/unit/kafka/server/metadata/ClientQuotaMetadataManagerTest.scala
  43. 23
      core/src/test/scala/unit/kafka/utils/TestUtils.scala
  44. 2
      jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java
  45. 9
      metadata/src/main/java/org/apache/kafka/image/ClientQuotasImage.java
  46. 4
      metadata/src/main/java/org/apache/kafka/image/MetadataDelta.java

1
build.gradle

@ -835,6 +835,7 @@ project(':core') { @@ -835,6 +835,7 @@ project(':core') {
compileOnly libs.log4j
testImplementation project(':clients').sourceSets.test.output
testImplementation project(':metadata').sourceSets.test.output
testImplementation libs.bcpkix
testImplementation libs.mockitoCore
testImplementation libs.easymock

1
clients/src/main/java/org/apache/kafka/common/requests/DescribeClientQuotasResponse.java

@ -115,5 +115,4 @@ public class DescribeClientQuotasResponse extends AbstractResponse { @@ -115,5 +115,4 @@ public class DescribeClientQuotasResponse extends AbstractResponse {
.setErrorMessage(null)
.setEntries(entries));
}
}

6
core/src/main/scala/kafka/server/AbstractFetcherManager.scala

@ -151,6 +151,12 @@ abstract class AbstractFetcherManager[T <: AbstractFetcherThread](val name: Stri @@ -151,6 +151,12 @@ abstract class AbstractFetcherManager[T <: AbstractFetcherThread](val name: Stri
}
}
def addFailedPartition(topicPartition: TopicPartition): Unit = {
lock synchronized {
failedPartitions.add(topicPartition)
}
}
protected def addPartitionsToFetcherThread(fetcherThread: T,
initialOffsetAndEpochs: collection.Map[TopicPartition, InitialFetchState]): Unit = {
fetcherThread.addPartitions(initialOffsetAndEpochs)

114
core/src/main/scala/kafka/server/BrokerServer.scala

@ -31,9 +31,9 @@ import kafka.metrics.KafkaYammerMetrics @@ -31,9 +31,9 @@ import kafka.metrics.KafkaYammerMetrics
import kafka.network.SocketServer
import kafka.raft.RaftManager
import kafka.security.CredentialProvider
import kafka.server.metadata.{BrokerMetadataListener, CachedConfigRepository, ClientQuotaCache, ClientQuotaMetadataManager, RaftMetadataCache}
import kafka.server.metadata.{BrokerMetadataListener, BrokerMetadataPublisher, ClientQuotaMetadataManager, KRaftMetadataCache}
import kafka.utils.{CoreUtils, KafkaScheduler}
import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, TRANSACTION_STATE_TOPIC_NAME}
//import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, TRANSACTION_STATE_TOPIC_NAME}
import org.apache.kafka.common.message.ApiMessageType.ListenerType
import org.apache.kafka.common.message.BrokerRegistrationRequestData.{Listener, ListenerCollection}
import org.apache.kafka.common.metrics.Metrics
@ -88,14 +88,15 @@ class BrokerServer( @@ -88,14 +88,15 @@ class BrokerServer(
var authorizer: Option[Authorizer] = None
var socketServer: SocketServer = null
var dataPlaneRequestHandlerPool: KafkaRequestHandlerPool = null
var controlPlaneRequestHandlerPool: KafkaRequestHandlerPool = null
var logDirFailureChannel: LogDirFailureChannel = null
var logManager: LogManager = null
var tokenManager: DelegationTokenManager = null
var replicaManager: RaftReplicaManager = null
var dynamicConfigHandlers: Map[String, ConfigHandler] = null
var replicaManager: ReplicaManager = null
var credentialProvider: CredentialProvider = null
var tokenCache: DelegationTokenCache = null
@ -114,11 +115,11 @@ class BrokerServer( @@ -114,11 +115,11 @@ class BrokerServer(
var kafkaScheduler: KafkaScheduler = null
var metadataCache: RaftMetadataCache = null
var metadataCache: KRaftMetadataCache = null
var quotaManagers: QuotaFactory.QuotaManagers = null
var quotaCache: ClientQuotaCache = null
var clientQuotaMetadataManager: ClientQuotaMetadataManager = null
private var _brokerTopicStats: BrokerTopicStats = null
@ -128,9 +129,9 @@ class BrokerServer( @@ -128,9 +129,9 @@ class BrokerServer(
val clusterId: String = metaProps.clusterId
val configRepository = new CachedConfigRepository()
var metadataListener: BrokerMetadataListener = null
var brokerMetadataListener: BrokerMetadataListener = null
var metadataPublisher: BrokerMetadataPublisher = null
def kafkaYammerMetrics: kafka.metrics.KafkaYammerMetrics = KafkaYammerMetrics.INSTANCE
@ -166,16 +167,15 @@ class BrokerServer( @@ -166,16 +167,15 @@ class BrokerServer(
_brokerTopicStats = new BrokerTopicStats
quotaManagers = QuotaFactory.instantiate(config, metrics, time, threadNamePrefix.getOrElse(""))
quotaCache = new ClientQuotaCache()
logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size)
// Create log manager, but don't start it because we need to delay any potential unclean shutdown log recovery
// until we catch up on the metadata log and have up-to-date topic and broker configs.
logManager = LogManager(config, initialOfflineDirs, configRepository, kafkaScheduler, time,
logManager = LogManager(config, initialOfflineDirs, metadataCache, kafkaScheduler, time,
brokerTopicStats, logDirFailureChannel, keepPartitionMetadataFile = true)
metadataCache = MetadataCache.raftMetadataCache(config.nodeId)
metadataCache = MetadataCache.kRaftMetadataCache(config.nodeId)
// Enable delegation token cache for all SCRAM mechanisms to simplify dynamic update.
// This keeps the cache up-to-date if new SCRAM mechanisms are enabled dynamically.
tokenCache = new DelegationTokenCache(ScramMechanism.mechanismNames)
@ -210,6 +210,8 @@ class BrokerServer( @@ -210,6 +210,8 @@ class BrokerServer(
socketServer = new SocketServer(config, metrics, time, credentialProvider, apiVersionManager)
socketServer.startup(startProcessingRequests = false)
clientQuotaMetadataManager = new ClientQuotaMetadataManager(quotaManagers, socketServer.connectionQuotas)
val alterIsrChannelManager = BrokerToControllerChannelManager(
controllerNodeProvider,
time,
@ -228,10 +230,10 @@ class BrokerServer( @@ -228,10 +230,10 @@ class BrokerServer(
)
alterIsrManager.start()
this.replicaManager = new RaftReplicaManager(config, metrics, time,
this.replicaManager = new ReplicaManager(config, metrics, time, None,
kafkaScheduler, logManager, isShuttingDown, quotaManagers,
brokerTopicStats, metadataCache, logDirFailureChannel, alterIsrManager,
configRepository, threadNamePrefix)
threadNamePrefix)
/* start token manager */
if (config.tokenAuthEnabled) {
@ -264,19 +266,11 @@ class BrokerServer( @@ -264,19 +266,11 @@ class BrokerServer(
/* Add all reconfigurables for config change notification before starting the metadata listener */
config.dynamicConfig.addReconfigurables(this)
val clientQuotaMetadataManager = new ClientQuotaMetadataManager(
quotaManagers, socketServer.connectionQuotas, quotaCache)
dynamicConfigHandlers = Map[String, ConfigHandler](
ConfigType.Topic -> new TopicConfigHandler(logManager, config, quotaManagers, None),
ConfigType.Broker -> new BrokerConfigHandler(config, quotaManagers))
brokerMetadataListener = new BrokerMetadataListener(
config.nodeId,
time,
metadataCache,
configRepository,
groupCoordinator,
replicaManager,
transactionCoordinator,
threadNamePrefix,
clientQuotaMetadataManager)
metadataListener = new BrokerMetadataListener(config.nodeId, time, threadNamePrefix)
val networkListeners = new ListenerCollection()
config.advertisedListeners.foreach { ep =>
@ -286,13 +280,13 @@ class BrokerServer( @@ -286,13 +280,13 @@ class BrokerServer(
setPort(socketServer.boundPort(ep.listenerName)).
setSecurityProtocol(ep.securityProtocol.id))
}
lifecycleManager.start(() => brokerMetadataListener.highestMetadataOffset(),
lifecycleManager.start(() => metadataListener.highestMetadataOffset(),
BrokerToControllerChannelManager(controllerNodeProvider, time, metrics, config,
"heartbeat", threadNamePrefix, config.brokerSessionTimeoutMs.toLong),
metaProps.clusterId, networkListeners, supportedFeatures)
// Register a listener with the Raft layer to receive metadata event notifications
raftManager.register(brokerMetadataListener)
raftManager.register(metadataListener)
val endpoints = new util.ArrayList[Endpoint](networkListeners.size())
var interBrokerListener: Endpoint = null
@ -331,46 +325,41 @@ class BrokerServer( @@ -331,46 +325,41 @@ class BrokerServer(
new FetchSessionCache(config.maxIncrementalFetchSessionCacheSlots,
KafkaServer.MIN_INCREMENTAL_FETCH_SESSION_EVICTION_MS))
// Start processing requests once we've caught up on the metadata log, recovered logs if necessary,
// and started all services that we previously delayed starting.
val raftSupport = RaftSupport(forwardingManager, metadataCache, quotaCache)
// Create the request processor objects.
val raftSupport = RaftSupport(forwardingManager, metadataCache)
dataPlaneRequestProcessor = new KafkaApis(socketServer.dataPlaneRequestChannel, raftSupport,
replicaManager, groupCoordinator, transactionCoordinator, autoTopicCreationManager,
config.nodeId, config, configRepository, metadataCache, metrics, authorizer, quotaManagers,
config.nodeId, config, metadataCache, metadataCache, metrics, authorizer, quotaManagers,
fetchManager, brokerTopicStats, clusterId, time, tokenManager, apiVersionManager)
dataPlaneRequestHandlerPool = new KafkaRequestHandlerPool(config.nodeId, socketServer.dataPlaneRequestChannel, dataPlaneRequestProcessor, time,
config.numIoThreads, s"${SocketServer.DataPlaneMetricPrefix}RequestHandlerAvgIdlePercent", SocketServer.DataPlaneThreadPrefix)
socketServer.controlPlaneRequestChannelOpt.foreach { controlPlaneRequestChannel =>
controlPlaneRequestProcessor = new KafkaApis(controlPlaneRequestChannel, raftSupport,
replicaManager, groupCoordinator, transactionCoordinator, autoTopicCreationManager,
config.nodeId, config, configRepository, metadataCache, metrics, authorizer, quotaManagers,
fetchManager, brokerTopicStats, clusterId, time, tokenManager, apiVersionManager)
dataPlaneRequestHandlerPool = new KafkaRequestHandlerPool(config.nodeId,
socketServer.dataPlaneRequestChannel, dataPlaneRequestProcessor, time,
config.numIoThreads, s"${SocketServer.DataPlaneMetricPrefix}RequestHandlerAvgIdlePercent",
SocketServer.DataPlaneThreadPrefix)
controlPlaneRequestHandlerPool = new KafkaRequestHandlerPool(config.nodeId, socketServer.controlPlaneRequestChannelOpt.get, controlPlaneRequestProcessor, time,
1, s"${SocketServer.ControlPlaneMetricPrefix}RequestHandlerAvgIdlePercent", SocketServer.ControlPlaneThreadPrefix)
if (socketServer.controlPlaneRequestChannelOpt.isDefined) {
throw new RuntimeException(KafkaConfig.ControlPlaneListenerNameProp + " is not " +
"supported when in KRaft mode.")
}
// Block until we've caught up on the metadata log
// Block until we've caught up with the latest metadata from the controller quorum.
lifecycleManager.initialCatchUpFuture.get()
// Start log manager, which will perform (potentially lengthy) recovery-from-unclean-shutdown if required.
logManager.startup(metadataCache.getAllTopics())
// Start other services that we've delayed starting, in the appropriate order.
replicaManager.startup()
replicaManager.startHighWatermarkCheckPointThread()
groupCoordinator.startup(() => metadataCache.
numPartitions(GROUP_METADATA_TOPIC_NAME).getOrElse(config.offsetsTopicPartitions))
transactionCoordinator.startup(() => metadataCache.
numPartitions(TRANSACTION_STATE_TOPIC_NAME).getOrElse(config.transactionTopicPartitions))
// Apply deferred partition metadata changes after starting replica manager and coordinators
// so that those services are ready and able to process the changes.
replicaManager.endMetadataChangeDeferral(
RequestHandlerHelper.onLeadershipChange(groupCoordinator, transactionCoordinator, _, _))
// Apply the metadata log changes that we've accumulated.
metadataPublisher = new BrokerMetadataPublisher(config, metadataCache,
logManager, replicaManager, groupCoordinator, transactionCoordinator,
clientQuotaMetadataManager, featureCache, dynamicConfigHandlers.toMap)
// Tell the metadata listener to start publishing its output, and wait for the first
// publish operation to complete. This first operation will initialize logManager,
// replicaManager, groupCoordinator, and txnCoordinator. The log manager may perform
// a potentially lengthy recovery-from-unclean-shutdown operation here, if required.
metadataListener.startPublishing(metadataPublisher).get()
// Enable inbound TCP connections.
socketServer.startProcessingRequests(authorizerFutures)
// We're now ready to unfence the broker.
// We're now ready to unfence the broker. This also allows this broker to transition
// from RECOVERY state to RUNNING state, once the controller unfences the broker.
lifecycleManager.setReadyToUnfence()
maybeChangeStatus(STARTING, STARTED)
@ -391,7 +380,9 @@ class BrokerServer( @@ -391,7 +380,9 @@ class BrokerServer(
if (config.controlledShutdownEnable) {
// Shut down the broker metadata listener, so that we don't get added to any
// more ISRs.
brokerMetadataListener.beginShutdown()
if (metadataListener != null) {
metadataListener.beginShutdown()
}
lifecycleManager.beginControlledShutdown()
try {
lifecycleManager.controlledShutdownFuture.get(5L, TimeUnit.MINUTES)
@ -411,17 +402,14 @@ class BrokerServer( @@ -411,17 +402,14 @@ class BrokerServer(
}
if (dataPlaneRequestHandlerPool != null)
CoreUtils.swallow(dataPlaneRequestHandlerPool.shutdown(), this)
if (controlPlaneRequestHandlerPool != null)
CoreUtils.swallow(controlPlaneRequestHandlerPool.shutdown(), this)
if (dataPlaneRequestProcessor != null)
CoreUtils.swallow(dataPlaneRequestProcessor.close(), this)
if (controlPlaneRequestProcessor != null)
CoreUtils.swallow(controlPlaneRequestProcessor.close(), this)
CoreUtils.swallow(authorizer.foreach(_.close()), this)
if (brokerMetadataListener != null) {
CoreUtils.swallow(brokerMetadataListener.close(), this)
if (metadataListener != null) {
CoreUtils.swallow(metadataListener.close(), this)
}
if (transactionCoordinator != null)
CoreUtils.swallow(transactionCoordinator.shutdown(), this)

4
core/src/main/scala/kafka/server/ConfigHandler.scala

@ -48,8 +48,8 @@ trait ConfigHandler { @@ -48,8 +48,8 @@ trait ConfigHandler {
}
/**
* The TopicConfigHandler will process topic config changes in ZK.
* The callback provides the topic name and the full properties set read from ZK
* The TopicConfigHandler will process topic config changes from ZooKeeper or the metadata log.
* The callback provides the topic name and the full properties set.
*/
class TopicConfigHandler(private val logManager: LogManager, kafkaConfig: KafkaConfig,
val quotas: QuotaManagers, kafkaController: Option[KafkaController]) extends ConfigHandler with Logging {

3
core/src/main/scala/kafka/server/DelayedDeleteRecords.scala

@ -84,9 +84,6 @@ class DelayedDeleteRecords(delayMs: Long, @@ -84,9 +84,6 @@ class DelayedDeleteRecords(delayMs: Long,
(false, Errors.NOT_LEADER_OR_FOLLOWER, DeleteRecordsResponse.INVALID_LOW_WATERMARK)
}
case _: HostedPartition.Deferred =>
(false, Errors.UNKNOWN_TOPIC_OR_PARTITION, DeleteRecordsResponse.INVALID_LOW_WATERMARK)
case HostedPartition.Offline =>
(false, Errors.KAFKA_STORAGE_ERROR, DeleteRecordsResponse.INVALID_LOW_WATERMARK)

21
core/src/main/scala/kafka/server/FinalizedFeatureCache.scala

@ -17,12 +17,17 @@ @@ -17,12 +17,17 @@
package kafka.server
import java.util
import java.util.Collections
import kafka.utils.Logging
import org.apache.kafka.common.feature.{Features, FinalizedVersionRange}
import org.apache.kafka.image.FeaturesDelta
import scala.concurrent.TimeoutException
import scala.math.max
import scala.compat.java8.OptionConverters._
// Raised whenever there was an error in updating the FinalizedFeatureCache with features.
class FeatureCacheUpdateException(message: String) extends RuntimeException(message) {
}
@ -127,6 +132,22 @@ class FinalizedFeatureCache(private val brokerFeatures: BrokerFeatures) extends @@ -127,6 +132,22 @@ class FinalizedFeatureCache(private val brokerFeatures: BrokerFeatures) extends
}
}
def update(featuresDelta: FeaturesDelta, highestMetadataOffset: Long): Unit = {
val features = featuresAndEpoch.getOrElse(
FinalizedFeaturesAndEpoch(Features.emptyFinalizedFeatures(), -1))
val newFeatures = new util.HashMap[String, FinalizedVersionRange]()
newFeatures.putAll(features.features.features())
featuresDelta.changes().entrySet().forEach { e =>
e.getValue().asScala match {
case None => newFeatures.remove(e.getKey)
case Some(feature) => newFeatures.put(e.getKey,
new FinalizedVersionRange(feature.min(), feature.max()))
}
}
featuresAndEpoch = Some(FinalizedFeaturesAndEpoch(Features.finalizedFeatures(
Collections.unmodifiableMap(newFeatures)), highestMetadataOffset))
}
/**
* Causes the current thread to wait no more than timeoutMs for the specified condition to be met.
* It is guaranteed that the provided condition will always be invoked only from within a

18
core/src/main/scala/kafka/server/KafkaApis.scala

@ -57,7 +57,6 @@ import org.apache.kafka.common.message._ @@ -57,7 +57,6 @@ import org.apache.kafka.common.message._
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.network.{ListenerName, Send}
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.quota.ClientQuotaEntity
import org.apache.kafka.common.record._
import org.apache.kafka.common.replica.ClientMetadata
import org.apache.kafka.common.replica.ClientMetadata.DefaultClientMetadata
@ -3035,17 +3034,12 @@ class KafkaApis(val requestChannel: RequestChannel, @@ -3035,17 +3034,12 @@ class KafkaApis(val requestChannel: RequestChannel,
new DescribeClientQuotasResponse(new DescribeClientQuotasResponseData()
.setThrottleTimeMs(requestThrottleMs)
.setEntries(entriesData.asJava)))
case RaftSupport(fwdMgr, metadataCache, quotaCache) =>
val result = quotaCache.describeClientQuotas(
describeClientQuotasRequest.filter().components().asScala.toSeq,
describeClientQuotasRequest.filter().strict())
val resultAsJava = new util.HashMap[ClientQuotaEntity, util.Map[String, java.lang.Double]](result.size)
result.foreach { case (entity, quotas) =>
resultAsJava.put(entity, quotas.map { case (key, quota) => key -> Double.box(quota)}.asJava)
}
requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
DescribeClientQuotasResponse.fromQuotaEntities(resultAsJava, requestThrottleMs)
)
case RaftSupport(_, metadataCache) =>
val result = metadataCache.describeClientQuotas(describeClientQuotasRequest.data())
requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => {
result.setThrottleTimeMs(requestThrottleMs)
new DescribeClientQuotasResponse(result)
})
}
}
}

2
core/src/main/scala/kafka/server/KafkaServer.scala

@ -435,7 +435,7 @@ class KafkaServer( @@ -435,7 +435,7 @@ class KafkaServer(
protected def createReplicaManager(isShuttingDown: AtomicBoolean): ReplicaManager = {
new ReplicaManager(config, metrics, time, Some(zkClient), kafkaScheduler, logManager, isShuttingDown, quotaManagers,
brokerTopicStats, metadataCache, logDirFailureChannel, alterIsrManager, configRepository)
brokerTopicStats, metadataCache, logDirFailureChannel, alterIsrManager)
}
private def initZkClient(time: Time): Unit = {

15
core/src/main/scala/kafka/server/MetadataCache.scala

@ -18,12 +18,10 @@ @@ -18,12 +18,10 @@
package kafka.server
import kafka.admin.BrokerMetadata
import kafka.server.metadata.RaftMetadataCache
import kafka.server.metadata.KRaftMetadataCache
import org.apache.kafka.common.{Cluster, Node, TopicPartition}
import org.apache.kafka.common.message.{MetadataResponseData, UpdateMetadataRequestData}
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.requests.UpdateMetadataRequest
trait MetadataCache {
@ -79,13 +77,6 @@ trait MetadataCache { @@ -79,13 +77,6 @@ trait MetadataCache {
def getClusterMetadata(clusterId: String, listenerName: ListenerName): Cluster
/**
* Update the metadata cache with a given UpdateMetadataRequest.
*
* @return The deleted topics from the given UpdateMetadataRequest.
*/
def updateMetadata(correlationId: Int, request: UpdateMetadataRequest): collection.Seq[TopicPartition]
def contains(topic: String): Boolean
def contains(tp: TopicPartition): Boolean
@ -96,7 +87,7 @@ object MetadataCache { @@ -96,7 +87,7 @@ object MetadataCache {
new ZkMetadataCache(brokerId)
}
def raftMetadataCache(brokerId: Int): RaftMetadataCache = {
new RaftMetadataCache(brokerId)
def kRaftMetadataCache(brokerId: Int): KRaftMetadataCache = {
new KRaftMetadataCache(brokerId)
}
}

18
core/src/main/scala/kafka/server/MetadataSupport.scala

@ -19,7 +19,7 @@ package kafka.server @@ -19,7 +19,7 @@ package kafka.server
import kafka.controller.KafkaController
import kafka.network.RequestChannel
import kafka.server.metadata.{ClientQuotaCache, RaftMetadataCache}
import kafka.server.metadata.KRaftMetadataCache
import kafka.zk.{AdminZkClient, KafkaZkClient}
import org.apache.kafka.common.requests.AbstractResponse
@ -91,7 +91,7 @@ case class ZkSupport(adminManager: ZkAdminManager, @@ -91,7 +91,7 @@ case class ZkSupport(adminManager: ZkAdminManager,
override def controllerId: Option[Int] = metadataCache.getControllerId
}
case class RaftSupport(fwdMgr: ForwardingManager, metadataCache: RaftMetadataCache, quotaCache: ClientQuotaCache)
case class RaftSupport(fwdMgr: ForwardingManager, metadataCache: KRaftMetadataCache)
extends MetadataSupport {
override val forwardingManager: Option[ForwardingManager] = Some(fwdMgr)
override def requireZkOrThrow(createException: => Exception): ZkSupport = throw createException
@ -113,13 +113,9 @@ case class RaftSupport(fwdMgr: ForwardingManager, metadataCache: RaftMetadataCac @@ -113,13 +113,9 @@ case class RaftSupport(fwdMgr: ForwardingManager, metadataCache: RaftMetadataCac
}
}
override def controllerId: Option[Int] = {
// We send back a random controller ID when running with a Raft-based metadata quorum.
// Raft-based controllers are not directly accessible to clients; rather, clients can send
// requests destined for the controller to any broker node, and the receiving broker will
// automatically forward the request on the client's behalf to the active Raft-based
// controller as per KIP-590.
metadataCache.currentImage().brokers.randomAliveBrokerId()
}
/**
* Get the broker ID to return from a MetadataResponse. This will be a broker ID, as
* described in KRaftMetadataCache#getControllerId. See that function for more details.
*/
override def controllerId: Option[Int] = metadataCache.getControllerId
}

252
core/src/main/scala/kafka/server/RaftReplicaChangeDelegate.scala

@ -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
}
}

404
core/src/main/scala/kafka/server/RaftReplicaManager.scala

@ -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")
}
}
}

265
core/src/main/scala/kafka/server/ReplicaManager.scala

@ -21,6 +21,7 @@ import java.util.Optional @@ -21,6 +21,7 @@ import java.util.Optional
import java.util.concurrent.TimeUnit
import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.locks.Lock
import com.yammer.metrics.core.Meter
import kafka.api._
import kafka.cluster.{BrokerEndPoint, Partition}
@ -32,7 +33,6 @@ import kafka.server.{FetchMetadata => SFetchMetadata} @@ -32,7 +33,6 @@ import kafka.server.{FetchMetadata => SFetchMetadata}
import kafka.server.HostedPartition.Online
import kafka.server.QuotaFactory.QuotaManagers
import kafka.server.checkpoints.{LazyOffsetCheckpoints, OffsetCheckpointFile, OffsetCheckpoints}
import kafka.server.metadata.ConfigRepository
import kafka.utils._
import kafka.utils.Implicits._
import kafka.zk.KafkaZkClient
@ -59,6 +59,8 @@ import org.apache.kafka.common.requests.FetchRequest.PartitionData @@ -59,6 +59,8 @@ import org.apache.kafka.common.requests.FetchRequest.PartitionData
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.apache.kafka.common.requests._
import org.apache.kafka.common.utils.Time
import org.apache.kafka.image.{MetadataImage, TopicsDelta}
import org.apache.kafka.metadata.PartitionRegistration
import scala.jdk.CollectionConverters._
import scala.collection.{Map, Seq, Set, mutable}
@ -81,6 +83,8 @@ case class LogDeleteRecordsResult(requestedOffset: Long, lowWatermark: Long, exc @@ -81,6 +83,8 @@ case class LogDeleteRecordsResult(requestedOffset: Long, lowWatermark: Long, exc
}
}
case class LocalLeaderInfo(topicId: Uuid, partition: PartitionRegistration)
/**
* Result metadata of a log read operation on the log
* @param info @FetchDataInfo returned by the @Log read
@ -160,12 +164,6 @@ case class FetchPartitionData(error: Errors = Errors.NONE, @@ -160,12 +164,6 @@ case class FetchPartitionData(error: Errors = Errors.NONE,
*/
sealed trait HostedPartition
/**
* Trait to represent a partition that isn't Offline -- i.e. it is either Online or it is Deferred.
*/
sealed trait NonOffline extends HostedPartition {
val partition: Partition
}
object HostedPartition {
/**
* This broker does not have any state for this partition locally.
@ -175,16 +173,7 @@ object HostedPartition { @@ -175,16 +173,7 @@ object HostedPartition {
/**
* This broker hosts the partition and it is online.
*/
final case class Online(partition: Partition) extends NonOffline
/**
* This broker hosted the partition (or will soon host it if it is new) but
* it is deferring metadata changes until it catches up on the Raft-based metadata
* log. This state only applies to brokers that are using a Raft-based metadata
* quorum; it never happens when using ZooKeeper. The isNew value indicates
* if the partition needs to be created when we apply the deferred changes.
*/
final case class Deferred(partition: Partition, isNew: Boolean) extends NonOffline
final case class Online(partition: Partition) extends HostedPartition
/**
* This broker hosts the partition, but it is in an offline log directory.
@ -212,7 +201,6 @@ class ReplicaManager(val config: KafkaConfig, @@ -212,7 +201,6 @@ class ReplicaManager(val config: KafkaConfig,
val delayedDeleteRecordsPurgatory: DelayedOperationPurgatory[DelayedDeleteRecords],
val delayedElectLeaderPurgatory: DelayedOperationPurgatory[DelayedElectLeader],
threadNamePrefix: Option[String],
configRepository: ConfigRepository,
val alterIsrManager: AlterIsrManager) extends Logging with KafkaMetricsGroup {
def this(config: KafkaConfig,
@ -227,7 +215,6 @@ class ReplicaManager(val config: KafkaConfig, @@ -227,7 +215,6 @@ class ReplicaManager(val config: KafkaConfig,
metadataCache: MetadataCache,
logDirFailureChannel: LogDirFailureChannel,
alterIsrManager: AlterIsrManager,
configRepository: ConfigRepository,
threadNamePrefix: Option[String] = None) = {
this(config, metrics, time, zkClient, scheduler, logManager, isShuttingDown,
quotaManagers, brokerTopicStats, metadataCache, logDirFailureChannel,
@ -242,7 +229,7 @@ class ReplicaManager(val config: KafkaConfig, @@ -242,7 +229,7 @@ class ReplicaManager(val config: KafkaConfig,
purgeInterval = config.deleteRecordsPurgatoryPurgeIntervalRequests),
DelayedOperationPurgatory[DelayedElectLeader](
purgatoryName = "ElectLeader", brokerId = config.brokerId),
threadNamePrefix, configRepository, alterIsrManager)
threadNamePrefix, alterIsrManager)
}
/* epoch of the controller that last changed the leader */
@ -332,7 +319,7 @@ class ReplicaManager(val config: KafkaConfig, @@ -332,7 +319,7 @@ class ReplicaManager(val config: KafkaConfig,
private def maybeRemoveTopicMetrics(topic: String): Unit = {
val topicHasNonOfflinePartition = allPartitions.values.exists {
case nonOffline: NonOffline => topic == nonOffline.partition.topic
case online: HostedPartition.Online => topic == online.partition.topic
case HostedPartition.None | HostedPartition.Offline => false
}
if (!topicHasNonOfflinePartition) // nothing online or deferred
@ -410,9 +397,6 @@ class ReplicaManager(val config: KafkaConfig, @@ -410,9 +397,6 @@ class ReplicaManager(val config: KafkaConfig,
responseMap.put(topicPartition, Errors.FENCED_LEADER_EPOCH)
}
case _: HostedPartition.Deferred =>
throw new IllegalStateException("We should never be deferring partition metadata changes and stopping a replica when using ZooKeeper")
case HostedPartition.None =>
// Delete log and corresponding folders in case replica manager doesn't hold them anymore.
// This could happen when topic is being deleted while broker is down and recovers.
@ -461,7 +445,7 @@ class ReplicaManager(val config: KafkaConfig, @@ -461,7 +445,7 @@ class ReplicaManager(val config: KafkaConfig,
partitionsToStop.forKeyValue { (topicPartition, shouldDelete) =>
if (shouldDelete) {
getPartition(topicPartition) match {
case hostedPartition: NonOffline =>
case hostedPartition: HostedPartition.Online =>
if (allPartitions.remove(topicPartition, hostedPartition)) {
maybeRemoveTopicMetrics(topicPartition.topic)
// Logs are not deleted here. They are deleted in a single batch later on.
@ -552,11 +536,6 @@ class ReplicaManager(val config: KafkaConfig, @@ -552,11 +536,6 @@ class ReplicaManager(val config: KafkaConfig,
// the local replica has been deleted.
Left(Errors.NOT_LEADER_OR_FOLLOWER)
case _: HostedPartition.Deferred =>
// The topic exists, but this broker is deferring metadata changes for it, so we return NOT_LEADER_OR_FOLLOWER
// which forces clients to refresh metadata.
Left(Errors.NOT_LEADER_OR_FOLLOWER)
case HostedPartition.None =>
Left(Errors.UNKNOWN_TOPIC_OR_PARTITION)
}
@ -750,9 +729,6 @@ class ReplicaManager(val config: KafkaConfig, @@ -750,9 +729,6 @@ class ReplicaManager(val config: KafkaConfig,
case HostedPartition.Offline =>
throw new KafkaStorageException(s"Partition $topicPartition is offline")
case _: HostedPartition.Deferred =>
throw new IllegalStateException(s"Partition $topicPartition is deferred")
case HostedPartition.None => // Do nothing
}
@ -1302,7 +1278,8 @@ class ReplicaManager(val config: KafkaConfig, @@ -1302,7 +1278,8 @@ class ReplicaManager(val config: KafkaConfig,
stateChangeLogger.warn(stateControllerEpochErrorMessage)
throw new ControllerMovedException(stateChangeLogger.messageWithPrefix(stateControllerEpochErrorMessage))
} else {
val deletedPartitions = metadataCache.updateMetadata(correlationId, updateMetadataRequest)
val zkMetadataCache = metadataCache.asInstanceOf[ZkMetadataCache]
val deletedPartitions = zkMetadataCache.updateMetadata(correlationId, updateMetadataRequest)
controllerEpoch = updateMetadataRequest.controllerEpoch
deletedPartitions
}
@ -1358,9 +1335,6 @@ class ReplicaManager(val config: KafkaConfig, @@ -1358,9 +1335,6 @@ class ReplicaManager(val config: KafkaConfig,
responseMap.put(topicPartition, Errors.KAFKA_STORAGE_ERROR)
None
case _: HostedPartition.Deferred =>
throw new IllegalStateException("We should never be deferring partition metadata changes and becoming a leader or follower when using ZooKeeper")
case HostedPartition.Online(partition) =>
Some(partition)
@ -1964,11 +1938,6 @@ class ReplicaManager(val config: KafkaConfig, @@ -1964,11 +1938,6 @@ class ReplicaManager(val config: KafkaConfig,
.setPartition(offsetForLeaderPartition.partition)
.setErrorCode(Errors.NOT_LEADER_OR_FOLLOWER.code)
case _: HostedPartition.Deferred =>
new EpochEndOffset()
.setPartition(offsetForLeaderPartition.partition)
.setErrorCode(Errors.NOT_LEADER_OR_FOLLOWER.code)
case HostedPartition.None =>
new EpochEndOffset()
.setPartition(offsetForLeaderPartition.partition)
@ -2032,4 +2001,216 @@ class ReplicaManager(val config: KafkaConfig, @@ -2032,4 +2001,216 @@ class ReplicaManager(val config: KafkaConfig,
}
}
private[kafka] def getOrCreatePartition(tp: TopicPartition,
delta: TopicsDelta,
topicId: Uuid): Option[(Partition, Boolean)] = {
getPartition(tp) match {
case HostedPartition.Offline =>
stateChangeLogger.warn(s"Unable to bring up new local leader ${tp} " +
s"with topic id ${topicId} because it resides in an offline log " +
"directory.")
None
case HostedPartition.Online(partition) => {
if (partition.topicId.exists(_ != topicId)) {
// Note: Partition#topicId will be None here if the Log object for this partition
// has not been created.
throw new IllegalStateException(s"Topic ${tp} exists, but its ID is " +
s"${partition.topicId.get}, not ${topicId} as expected")
}
Some(partition, false)
}
case HostedPartition.None =>
if (delta.image().topicsById().containsKey(topicId)) {
stateChangeLogger.error(s"Expected partition ${tp} with topic id " +
s"${topicId} to exist, but it was missing. Creating...")
} else {
stateChangeLogger.info(s"Creating new partition ${tp} with topic id " +
s"${topicId}.")
}
// it's a partition that we don't know about yet, so create it and mark it online
val partition = Partition(tp, time, this)
allPartitions.put(tp, HostedPartition.Online(partition))
Some(partition, true)
}
}
private[kafka] def calculateDeltaChanges(delta: TopicsDelta)
: (mutable.HashMap[TopicPartition, Boolean],
mutable.HashMap[TopicPartition, LocalLeaderInfo],
mutable.HashMap[TopicPartition, LocalLeaderInfo]) = {
val deleted = new mutable.HashMap[TopicPartition, Boolean]()
delta.deletedTopicIds().forEach { topicId =>
val topicImage = delta.image().getTopic(topicId)
topicImage.partitions().keySet().forEach { partitionId =>
deleted.put(new TopicPartition(topicImage.name(), partitionId), true)
}
}
val newLocalLeaders = new mutable.HashMap[TopicPartition, LocalLeaderInfo]()
val newLocalFollowers = new mutable.HashMap[TopicPartition, LocalLeaderInfo]()
delta.changedTopics().values().forEach { topicDelta =>
topicDelta.newLocalLeaders(config.nodeId).forEach { e =>
newLocalLeaders.put(new TopicPartition(topicDelta.name(), e.getKey),
LocalLeaderInfo(topicDelta.id(), e.getValue))
}
topicDelta.newLocalFollowers(config.nodeId).forEach { e =>
newLocalFollowers.put(new TopicPartition(topicDelta.name(), e.getKey),
LocalLeaderInfo(topicDelta.id(), e.getValue))
}
}
(deleted, newLocalLeaders, newLocalFollowers)
}
/**
* Apply a KRaft topic change delta.
*
* @param newImage The new metadata image.
* @param delta The delta to apply.
*/
def applyDelta(newImage: MetadataImage, delta: TopicsDelta): Unit = {
// Before taking the lock, build some hash maps that we will need.
val (deleted, newLocalLeaders, newLocalFollowers) = calculateDeltaChanges(delta)
replicaStateChangeLock.synchronized {
// Handle deleted partitions. We need to do this first because we might subsequently
// create new partitions with the same names as the ones we are deleting here.
if (!deleted.isEmpty) {
stateChangeLogger.info(s"Deleting ${deleted.size} partition(s).")
stopPartitions(deleted).foreach { case (topicPartition, e) =>
if (e.isInstanceOf[KafkaStorageException]) {
stateChangeLogger.error(s"Unable to delete replica ${topicPartition} because " +
"the local replica for the partition is in an offline log directory")
} else {
stateChangeLogger.error(s"Unable to delete replica ${topicPartition} because " +
s"we got an unexpected ${e.getClass.getName} exception: ${e.getMessage}")
}
}
}
// Handle partitions which we are now the leader or follower for.
if (!newLocalLeaders.isEmpty || !newLocalFollowers.isEmpty) {
val lazyOffsetCheckpoints = new LazyOffsetCheckpoints(this.highWatermarkCheckpoints)
val changedPartitions = new mutable.HashSet[Partition]
if (!newLocalLeaders.isEmpty) {
applyLocalLeadersDelta(changedPartitions, delta, lazyOffsetCheckpoints, newLocalLeaders)
}
if (!newLocalFollowers.isEmpty) {
applyLocalFollowersDelta(changedPartitions, newImage, delta, lazyOffsetCheckpoints, newLocalFollowers)
}
maybeAddLogDirFetchers(changedPartitions, lazyOffsetCheckpoints,
name => Option(newImage.topics().getTopic(name)).map(_.id()))
def markPartitionOfflineIfNeeded(tp: TopicPartition): Unit = {
/*
* 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(tp).isEmpty)
markPartitionOffline(tp)
}
newLocalLeaders.keySet.foreach(markPartitionOfflineIfNeeded(_))
newLocalFollowers.keySet.foreach(markPartitionOfflineIfNeeded(_))
replicaFetcherManager.shutdownIdleFetcherThreads()
replicaAlterLogDirsManager.shutdownIdleFetcherThreads()
}
}
}
private def applyLocalLeadersDelta(changedPartitions: mutable.HashSet[Partition],
delta: TopicsDelta,
offsetCheckpoints: OffsetCheckpoints,
newLocalLeaders: mutable.HashMap[TopicPartition, LocalLeaderInfo]): Unit = {
stateChangeLogger.info(s"Transitioning ${newLocalLeaders.size} partition(s) to " +
"local leaders.")
replicaFetcherManager.removeFetcherForPartitions(newLocalLeaders.keySet)
newLocalLeaders.forKeyValue { case (tp, info) =>
getOrCreatePartition(tp, delta, info.topicId).foreach { case (partition, isNew) =>
try {
val state = info.partition.toLeaderAndIsrPartitionState(tp, isNew)
if (!partition.makeLeader(state, offsetCheckpoints, Some(info.topicId))) {
stateChangeLogger.info("Skipped the become-leader state change for " +
s"${tp} with topic id ${info.topicId} because this partition is " +
"already a local leader.")
}
changedPartitions.add(partition)
} catch {
case e: KafkaStorageException =>
stateChangeLogger.info(s"Skipped the become-leader state change for ${tp} " +
s"with topic id ${info.topicId} due to disk error ${e}")
val dirOpt = getLogDir(tp)
error(s"Error while making broker the leader for partition ${tp} in dir " +
s"${dirOpt}", e)
}
}
}
}
private def applyLocalFollowersDelta(changedPartitions: mutable.HashSet[Partition],
newImage: MetadataImage,
delta: TopicsDelta,
offsetCheckpoints: OffsetCheckpoints,
newLocalFollowers: mutable.HashMap[TopicPartition, LocalLeaderInfo]): Unit = {
stateChangeLogger.info(s"Transitioning ${newLocalFollowers.size} partition(s) to " +
"local followers.")
replicaFetcherManager.removeFetcherForPartitions(newLocalFollowers.keySet)
val shuttingDown = isShuttingDown.get()
val partitionsToMakeFollower = new mutable.HashMap[TopicPartition, InitialFetchState]
val newFollowerTopicSet = new mutable.HashSet[String]
newLocalFollowers.forKeyValue { case (tp, info) =>
getOrCreatePartition(tp, delta, info.topicId).foreach { case (partition, isNew) =>
try {
newFollowerTopicSet.add(tp.topic())
completeDelayedFetchOrProduceRequests(tp)
// 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, false, offsetCheckpoints, Some(info.topicId))
if (shuttingDown) {
stateChangeLogger.trace(s"Unable to start fetching ${tp} with topic " +
s"ID ${info.topicId} because the replica manager is shutting down.")
} else {
val listenerName = config.interBrokerListenerName.value()
val leader = info.partition.leader
Option(newImage.cluster().broker(leader)).flatMap(_.node(listenerName).asScala) match {
case None => stateChangeLogger.trace(s"Unable to start fetching ${tp} " +
s"with topic ID ${info.topicId} from leader ${leader} because it is not " +
"alive.")
case Some(node) =>
val leaderEndPoint = new BrokerEndPoint(node.id(), node.host(), node.port())
val log = partition.localLogOrException
val fetchOffset = initialFetchOffset(log)
partitionsToMakeFollower.put(tp,
InitialFetchState(leaderEndPoint, partition.getLeaderEpoch, fetchOffset))
}
}
changedPartitions.add(partition)
} catch {
case e: Throwable => stateChangeLogger.error(s"Unable to start fetching ${tp} " +
s"with topic ID ${info.topicId} due to ${e.getClass.getSimpleName}", e)
replicaFetcherManager.addFailedPartition(tp)
}
}
}
updateLeaderAndFollowerMetrics(newFollowerTopicSet)
replicaFetcherManager.addFetcherForPartitions(partitionsToMakeFollower)
}
def deleteGhostReplicas(topicPartitions: Iterable[TopicPartition]): Unit = {
stopPartitions(topicPartitions.map { tp => tp -> true }.toMap).foreach {
case (topicPartition, e) =>
if (e.isInstanceOf[KafkaStorageException]) {
stateChangeLogger.error(s"Unable to delete ghost replica ${topicPartition} because " +
"the local replica for the partition is in an offline log directory")
} else {
stateChangeLogger.error(s"Unable to delete ghost replica ${topicPartition} because " +
s"we got an unexpected ${e.getClass.getName} exception: ${e.getMessage}")
}
}
}
}

332
core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala

@ -16,40 +16,28 @@ @@ -16,40 +16,28 @@
*/
package kafka.server.metadata
import java.util.concurrent.TimeUnit
import java.util
import java.util.concurrent.{CompletableFuture, TimeUnit}
import java.util.function.Consumer
import kafka.coordinator.group.GroupCoordinator
import kafka.coordinator.transaction.TransactionCoordinator
import kafka.metrics.KafkaMetricsGroup
import kafka.server.{RaftReplicaManager, RequestHandlerHelper, RequestLocal}
import org.apache.kafka.common.config.ConfigResource
import org.apache.kafka.common.metadata.MetadataRecordType._
import org.apache.kafka.common.metadata._
import org.apache.kafka.common.protocol.ApiMessage
import org.apache.kafka.image.{MetadataDelta, MetadataImage}
import org.apache.kafka.common.utils.{LogContext, Time}
import org.apache.kafka.queue.{EventQueue, KafkaEventQueue}
import org.apache.kafka.raft.{Batch, BatchReader, LeaderAndEpoch, RaftClient}
import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.common.ApiMessageAndVersion
import org.apache.kafka.snapshot.SnapshotReader
import scala.compat.java8.OptionConverters._
import scala.jdk.CollectionConverters._
object BrokerMetadataListener{
object BrokerMetadataListener {
val MetadataBatchProcessingTimeUs = "MetadataBatchProcessingTimeUs"
val MetadataBatchSizes = "MetadataBatchSizes"
}
class BrokerMetadataListener(
brokerId: Int,
val brokerId: Int,
time: Time,
metadataCache: RaftMetadataCache,
configRepository: CachedConfigRepository,
groupCoordinator: GroupCoordinator,
replicaManager: RaftReplicaManager,
txnCoordinator: TransactionCoordinator,
threadNamePrefix: Option[String],
clientQuotaManager: ClientQuotaMetadataManager
threadNamePrefix: Option[String]
) extends RaftClient.Listener[ApiMessageAndVersion] with KafkaMetricsGroup {
private val logContext = new LogContext(s"[BrokerMetadataListener id=${brokerId}] ")
private val log = logContext.logger(classOf[BrokerMetadataListener])
@ -70,230 +58,188 @@ class BrokerMetadataListener( @@ -70,230 +58,188 @@ class BrokerMetadataListener(
*/
@volatile private var _highestMetadataOffset = -1L
/**
* The current broker metadata image. Accessed only from the event queue thread.
*/
private var _image = MetadataImage.EMPTY
/**
* The current metadata delta. Accessed only from the event queue thread.
*/
private var _delta = new MetadataDelta(_image)
/**
* The object to use to publish new metadata changes, or None if this listener has not
* been activated yet.
*/
private var _publisher: Option[MetadataPublisher] = None
/**
* The event queue which runs this listener.
*/
val eventQueue = new KafkaEventQueue(time, logContext, threadNamePrefix.getOrElse(""))
/**
* Returns the highest metadata-offset. Thread-safe.
*/
def highestMetadataOffset(): Long = _highestMetadataOffset
/**
* Handle new metadata records.
*/
override def handleCommit(reader: BatchReader[ApiMessageAndVersion]): Unit = {
override def handleCommit(reader: BatchReader[ApiMessageAndVersion]): Unit =
eventQueue.append(new HandleCommitsEvent(reader))
class HandleCommitsEvent(reader: BatchReader[ApiMessageAndVersion])
extends EventQueue.FailureLoggingEvent(log) {
override def run(): Unit = {
val results = try {
val loadResults = loadBatches(_delta, reader)
if (isDebugEnabled) {
debug(s"Loaded new commits: ${loadResults}")
}
loadResults
} finally {
reader.close()
}
maybePublish(results.highestMetadataOffset)
}
}
/**
* Handle metadata snapshots
*/
override def handleSnapshot(reader: SnapshotReader[ApiMessageAndVersion]): Unit = {
// Loading snapshot on the broker is currently not supported.
reader.close();
throw new UnsupportedOperationException(s"Loading snapshot (${reader.snapshotId()}) is not supported")
}
// Visible for testing. It's useful to execute events synchronously in order
// to make tests deterministic. This object is responsible for closing the reader.
private[metadata] def execCommits(batchReader: BatchReader[ApiMessageAndVersion]): Unit = {
new HandleCommitsEvent(batchReader).run()
}
override def handleSnapshot(reader: SnapshotReader[ApiMessageAndVersion]): Unit =
eventQueue.append(new HandleSnapshotEvent(reader))
class HandleCommitsEvent(
reader: BatchReader[ApiMessageAndVersion]
) extends EventQueue.FailureLoggingEvent(log) {
class HandleSnapshotEvent(reader: SnapshotReader[ApiMessageAndVersion])
extends EventQueue.FailureLoggingEvent(log) {
override def run(): Unit = {
try {
while (reader.hasNext()) {
apply(reader.next())
}
val results = try {
info(s"Loading snapshot ${reader.snapshotId().offset}-${reader.snapshotId().epoch}.")
_delta = new MetadataDelta(_image) // Discard any previous deltas.
val loadResults = loadBatches(_delta, reader)
_delta.finishSnapshot()
info(s"Loaded snapshot ${reader.snapshotId().offset}-${reader.snapshotId().epoch}: " +
s"${loadResults}")
loadResults
} finally {
reader.close()
}
maybePublish(results.highestMetadataOffset)
}
}
private def apply(batch: Batch[ApiMessageAndVersion]): Unit = {
val records = batch.records
val lastOffset = batch.lastOffset
case class BatchLoadResults(numBatches: Int,
numRecords: Int,
elapsedUs: Long,
highestMetadataOffset: Long) {
override def toString(): String = {
s"${numBatches} batch(es) with ${numRecords} record(s) ending at offset " +
s"${highestMetadataOffset} in ${elapsedUs} microseconds"
}
}
if (isDebugEnabled) {
debug(s"Metadata batch $lastOffset: handling ${records.size()} record(s).")
}
val imageBuilder =
MetadataImageBuilder(brokerId, log, metadataCache.currentImage())
val startNs = time.nanoseconds()
private def loadBatches(delta: MetadataDelta,
iterator: util.Iterator[Batch[ApiMessageAndVersion]]): BatchLoadResults = {
val startTimeNs = time.nanoseconds()
var numBatches = 0
var numRecords = 0
var newHighestMetadataOffset = _highestMetadataOffset
while (iterator.hasNext()) {
val batch = iterator.next()
var index = 0
metadataBatchSizeHist.update(records.size())
records.iterator().asScala.foreach { record =>
try {
if (isTraceEnabled) {
trace("Metadata batch %d: processing [%d/%d]: %s.".format(lastOffset, index + 1,
records.size(), record.toString))
}
handleMessage(imageBuilder, record.message, lastOffset)
} catch {
case e: Exception => error(s"Unable to handle record $index in batch " +
s"ending at offset $lastOffset", e)
}
index = index + 1
}
if (imageBuilder.hasChanges) {
val newImage = imageBuilder.build()
batch.records().forEach { messageAndVersion =>
newHighestMetadataOffset = batch.lastOffset()
if (isTraceEnabled) {
trace(s"Metadata batch $lastOffset: creating new metadata image ${newImage}")
} else if (isDebugEnabled) {
debug(s"Metadata batch $lastOffset: creating new metadata image")
trace("Metadata batch %d: processing [%d/%d]: %s.".format(batch.lastOffset, index + 1,
batch.records().size(), messageAndVersion.message().toString()))
}
metadataCache.image(newImage)
} else if (isDebugEnabled) {
debug(s"Metadata batch $lastOffset: no new metadata image required.")
delta.replay(messageAndVersion.message())
numRecords += 1
index += 1
}
if (imageBuilder.hasPartitionChanges) {
if (isDebugEnabled) {
debug(s"Metadata batch $lastOffset: applying partition changes")
}
replicaManager.handleMetadataRecords(imageBuilder, lastOffset,
RequestHandlerHelper.onLeadershipChange(groupCoordinator, txnCoordinator, _, _))
} else if (isDebugEnabled) {
debug(s"Metadata batch $lastOffset: no partition changes found.")
}
_highestMetadataOffset = lastOffset
val endNs = time.nanoseconds()
val deltaUs = TimeUnit.MICROSECONDS.convert(endNs - startNs, TimeUnit.NANOSECONDS)
debug(s"Metadata batch $lastOffset: advanced highest metadata offset in ${deltaUs} " +
"microseconds.")
batchProcessingTimeHist.update(deltaUs)
}
}
private def handleMessage(imageBuilder: MetadataImageBuilder,
record: ApiMessage,
lastOffset: Long): Unit = {
val recordType = try {
fromId(record.apiKey())
} catch {
case e: Exception => throw new RuntimeException("Unknown metadata record type " +
s"${record.apiKey()} in batch ending at offset ${lastOffset}.")
}
record match {
case rec: RegisterBrokerRecord => handleRegisterBrokerRecord(imageBuilder, rec)
case rec: UnregisterBrokerRecord => handleUnregisterBrokerRecord(imageBuilder, rec)
case rec: FenceBrokerRecord => handleFenceBrokerRecord(imageBuilder, rec)
case rec: UnfenceBrokerRecord => handleUnfenceBrokerRecord(imageBuilder, rec)
case rec: TopicRecord => handleTopicRecord(imageBuilder, rec)
case rec: PartitionRecord => handlePartitionRecord(imageBuilder, rec)
case rec: PartitionChangeRecord => handlePartitionChangeRecord(imageBuilder, rec)
case rec: RemoveTopicRecord => handleRemoveTopicRecord(imageBuilder, rec)
case rec: ConfigRecord => handleConfigRecord(rec)
case rec: ClientQuotaRecord => handleClientQuotaRecord(imageBuilder, rec)
case rec: ProducerIdsRecord => handleProducerIdRecord(rec)
case _ => throw new RuntimeException(s"Unhandled record $record with type $recordType")
metadataBatchSizeHist.update(batch.records().size())
numBatches = numBatches + 1
}
_highestMetadataOffset = newHighestMetadataOffset
val endTimeNs = time.nanoseconds()
val elapsedUs = TimeUnit.MICROSECONDS.convert(endTimeNs - startTimeNs, TimeUnit.NANOSECONDS)
batchProcessingTimeHist.update(elapsedUs)
BatchLoadResults(numBatches, numRecords, elapsedUs, newHighestMetadataOffset)
}
def handleRegisterBrokerRecord(imageBuilder: MetadataImageBuilder,
record: RegisterBrokerRecord): Unit = {
val broker = MetadataBroker(record)
imageBuilder.brokersBuilder().add(broker)
def startPublishing(publisher: MetadataPublisher): CompletableFuture[Void] = {
val event = new StartPublishingEvent(publisher)
eventQueue.append(event)
event.future
}
def handleUnregisterBrokerRecord(imageBuilder: MetadataImageBuilder,
record: UnregisterBrokerRecord): Unit = {
imageBuilder.brokersBuilder().remove(record.brokerId())
}
def handleTopicRecord(imageBuilder: MetadataImageBuilder,
record: TopicRecord): Unit = {
imageBuilder.partitionsBuilder().addUuidMapping(record.name(), record.topicId())
}
class StartPublishingEvent(publisher: MetadataPublisher)
extends EventQueue.FailureLoggingEvent(log) {
val future = new CompletableFuture[Void]()
def handlePartitionRecord(imageBuilder: MetadataImageBuilder,
record: PartitionRecord): Unit = {
imageBuilder.topicIdToName(record.topicId()) match {
case None => throw new RuntimeException(s"Unable to locate topic with ID ${record.topicId}")
case Some(name) =>
val partition = MetadataPartition(name, record)
imageBuilder.partitionsBuilder().set(partition)
override def run(): Unit = {
_publisher = Some(publisher)
log.info(s"Starting to publish metadata events at offset ${_highestMetadataOffset}.")
try {
maybePublish(_highestMetadataOffset)
future.complete(null)
} catch {
case e: Throwable => future.completeExceptionally(e)
}
}
}
def handleConfigRecord(record: ConfigRecord): Unit = {
val t = ConfigResource.Type.forId(record.resourceType())
if (t == ConfigResource.Type.UNKNOWN) {
throw new RuntimeException("Unable to understand config resource type " +
s"${Integer.valueOf(record.resourceType())}")
private def maybePublish(newHighestMetadataOffset: Long): Unit = {
_publisher match {
case None => // Nothing to do
case Some(publisher) => {
val delta = _delta
_image = _delta.apply()
_delta = new MetadataDelta(_image)
publisher.publish(newHighestMetadataOffset, delta, _image)
}
}
val resource = new ConfigResource(t, record.resourceName())
configRepository.setConfig(resource, record.name(), record.value())
}
def handlePartitionChangeRecord(imageBuilder: MetadataImageBuilder,
record: PartitionChangeRecord): Unit = {
imageBuilder.partitionsBuilder().handleChange(record)
override def handleLeaderChange(leaderAndEpoch: LeaderAndEpoch): Unit = {
// TODO: cache leaderAndEpoch so we can use the epoch in broker-initiated snapshots.
}
def handleFenceBrokerRecord(imageBuilder: MetadataImageBuilder,
record: FenceBrokerRecord): Unit = {
// TODO: add broker epoch to metadata cache, and check it here.
imageBuilder.brokersBuilder().changeFencing(record.id(), fenced = true)
}
def handleUnfenceBrokerRecord(imageBuilder: MetadataImageBuilder,
record: UnfenceBrokerRecord): Unit = {
// TODO: add broker epoch to metadata cache, and check it here.
imageBuilder.brokersBuilder().changeFencing(record.id(), fenced = false)
override def beginShutdown(): Unit = {
eventQueue.beginShutdown("beginShutdown", new ShutdownEvent())
}
def handleRemoveTopicRecord(imageBuilder: MetadataImageBuilder,
record: RemoveTopicRecord): Unit = {
imageBuilder.topicIdToName(record.topicId()) match {
case None =>
throw new RuntimeException(s"Unable to locate topic with ID ${record.topicId}")
case Some(topicName) =>
info(s"Processing deletion of topic $topicName with id ${record.topicId}")
val removedPartitions = imageBuilder.partitionsBuilder().removeTopicById(record.topicId())
groupCoordinator.handleDeletedPartitions(removedPartitions.map(_.toTopicPartition).toSeq, RequestLocal.NoCaching)
configRepository.remove(new ConfigResource(ConfigResource.Type.TOPIC, topicName))
class ShutdownEvent() extends EventQueue.FailureLoggingEvent(log) {
override def run(): Unit = {
removeMetric(BrokerMetadataListener.MetadataBatchProcessingTimeUs)
removeMetric(BrokerMetadataListener.MetadataBatchSizes)
}
}
def handleClientQuotaRecord(imageBuilder: MetadataImageBuilder,
record: ClientQuotaRecord): Unit = {
// TODO add quotas to MetadataImageBuilder
clientQuotaManager.handleQuotaRecord(record)
def close(): Unit = {
beginShutdown()
eventQueue.close()
}
def handleProducerIdRecord(record: ProducerIdsRecord): Unit = {
// This is a no-op since brokers get their producer ID blocks directly from the controller via
// AllocateProducerIds RPC response
// VisibleForTesting
private[kafka] def getImageRecords(): CompletableFuture[util.List[ApiMessageAndVersion]] = {
val future = new CompletableFuture[util.List[ApiMessageAndVersion]]()
eventQueue.append(new GetImageRecordsEvent(future))
future
}
class HandleNewLeaderEvent(leaderAndEpoch: LeaderAndEpoch)
extends EventQueue.FailureLoggingEvent(log) {
override def run(): Unit = {
val imageBuilder =
MetadataImageBuilder(brokerId, log, metadataCache.currentImage())
imageBuilder.controllerId(leaderAndEpoch.leaderId.asScala)
metadataCache.image(imageBuilder.build())
class GetImageRecordsEvent(future: CompletableFuture[util.List[ApiMessageAndVersion]])
extends EventQueue.FailureLoggingEvent(log) with Consumer[util.List[ApiMessageAndVersion]] {
val records = new util.ArrayList[ApiMessageAndVersion]()
override def accept(batch: util.List[ApiMessageAndVersion]): Unit = {
records.addAll(batch)
}
}
override def handleLeaderChange(leader: LeaderAndEpoch): Unit = {
eventQueue.append(new HandleNewLeaderEvent(leader))
}
class ShutdownEvent() extends EventQueue.FailureLoggingEvent(log) {
override def run(): Unit = {
removeMetric(BrokerMetadataListener.MetadataBatchProcessingTimeUs)
removeMetric(BrokerMetadataListener.MetadataBatchSizes)
_image.write(this)
future.complete(records)
}
}
override def beginShutdown(): Unit = {
eventQueue.beginShutdown("beginShutdown", new ShutdownEvent())
}
def close(): Unit = {
beginShutdown()
eventQueue.close()
}
}

249
core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala

@ -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()
}
}

111
core/src/main/scala/kafka/server/metadata/CachedConfigRepository.scala

@ -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)
}
}

307
core/src/main/scala/kafka/server/metadata/ClientQuotaCache.scala

@ -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)"
}

97
core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala

@ -22,13 +22,14 @@ import kafka.server.ConfigEntityName @@ -22,13 +22,14 @@ import kafka.server.ConfigEntityName
import kafka.server.QuotaFactory.QuotaManagers
import kafka.utils.Logging
import org.apache.kafka.common.config.internals.QuotaConfigs
import org.apache.kafka.common.metadata.ClientQuotaRecord
import org.apache.kafka.common.metrics.Quota
import org.apache.kafka.common.quota.ClientQuotaEntity
import org.apache.kafka.common.utils.Sanitizer
import java.net.{InetAddress, UnknownHostException}
import scala.collection.mutable
import org.apache.kafka.image.{ClientQuotaDelta, ClientQuotasDelta}
import scala.compat.java8.OptionConverters._
// A strict hierarchy of entities that we support
@ -48,30 +49,32 @@ case object DefaultUserDefaultClientIdEntity extends QuotaEntity @@ -48,30 +49,32 @@ case object DefaultUserDefaultClientIdEntity extends QuotaEntity
* Process quota metadata records as they appear in the metadata log and update quota managers and cache as necessary
*/
class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManagers,
private[metadata] val connectionQuotas: ConnectionQuotas,
private[metadata] val quotaCache: ClientQuotaCache) extends Logging {
private[metadata] val connectionQuotas: ConnectionQuotas) extends Logging {
def handleQuotaRecord(quotaRecord: ClientQuotaRecord): Unit = {
val entityMap = mutable.Map[String, String]()
quotaRecord.entity().forEach { entityData =>
entityMap.put(entityData.entityType(), entityData.entityName())
def update(quotasDelta: ClientQuotasDelta): Unit = {
quotasDelta.changes().entrySet().forEach { e =>
update(e.getKey, e.getValue)
}
}
if (entityMap.contains(ClientQuotaEntity.IP)) {
private def update(entity: ClientQuotaEntity, quotaDelta: ClientQuotaDelta): Unit = {
if (entity.entries().containsKey(ClientQuotaEntity.IP)) {
// In the IP quota manager, None is used for default entity
val ipEntity = Option(entityMap(ClientQuotaEntity.IP)) match {
val ipEntity = Option(entity.entries().get(ClientQuotaEntity.IP)) match {
case Some(ip) => IpEntity(ip)
case None => DefaultIpEntity
}
handleIpQuota(ipEntity, quotaRecord)
} else if (entityMap.contains(ClientQuotaEntity.USER) || entityMap.contains(ClientQuotaEntity.CLIENT_ID)) {
// Need to handle null values for default entity name, so use "getOrElse" combined with "contains" checks
val userVal = entityMap.getOrElse(ClientQuotaEntity.USER, null)
val clientIdVal = entityMap.getOrElse(ClientQuotaEntity.CLIENT_ID, null)
handleIpQuota(ipEntity, quotaDelta)
} else if (entity.entries().containsKey(ClientQuotaEntity.USER) ||
entity.entries().containsKey(ClientQuotaEntity.CLIENT_ID)) {
// These values may be null, which is why we needed to use containsKey.
val userVal = entity.entries().get(ClientQuotaEntity.USER)
val clientIdVal = entity.entries().get(ClientQuotaEntity.CLIENT_ID)
// In User+Client quota managers, "<default>" is used for default entity, so we need to represent all possible
// combinations of values, defaults, and absent entities
val userClientEntity = if (entityMap.contains(ClientQuotaEntity.USER) && entityMap.contains(ClientQuotaEntity.CLIENT_ID)) {
val userClientEntity = if (entity.entries().containsKey(ClientQuotaEntity.USER) &&
entity.entries().containsKey(ClientQuotaEntity.CLIENT_ID)) {
if (userVal == null && clientIdVal == null) {
DefaultUserDefaultClientIdEntity
} else if (userVal == null) {
@ -81,7 +84,7 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag @@ -81,7 +84,7 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag
} else {
ExplicitUserExplicitClientIdEntity(userVal, clientIdVal)
}
} else if (entityMap.contains(ClientQuotaEntity.USER)) {
} else if (entity.entries().containsKey(ClientQuotaEntity.USER)) {
if (userVal == null) {
DefaultUserEntity
} else {
@ -94,16 +97,15 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag @@ -94,16 +97,15 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag
ClientIdEntity(clientIdVal)
}
}
handleUserClientQuota(
userClientEntity,
quotaRecord
)
quotaDelta.changes().entrySet().forEach { e =>
handleUserClientQuotaChange(userClientEntity, e.getKey(), e.getValue().asScala.map(_.toDouble))
}
} else {
warn(s"Ignoring unsupported quota entity ${quotaRecord.entity()}")
warn(s"Ignoring unsupported quota entity ${entity}.")
}
}
def handleIpQuota(ipEntity: QuotaEntity, quotaRecord: ClientQuotaRecord): Unit = {
def handleIpQuota(ipEntity: QuotaEntity, quotaDelta: ClientQuotaDelta): Unit = {
val inetAddress = ipEntity match {
case IpEntity(ip) =>
try {
@ -115,36 +117,30 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag @@ -115,36 +117,30 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag
case _ => throw new IllegalStateException("Should only handle IP quota entities here")
}
// The connection quota only understands the connection rate limit
if (quotaRecord.key() != QuotaConfigs.IP_CONNECTION_RATE_OVERRIDE_CONFIG) {
warn(s"Ignoring unexpected quota key ${quotaRecord.key()} for entity $ipEntity")
return
}
// Convert the value to an appropriate Option for the quota manager
val newValue = if (quotaRecord.remove()) {
None
} else {
Some(quotaRecord.value).map(_.toInt)
}
try {
connectionQuotas.updateIpConnectionRateQuota(inetAddress, newValue)
} catch {
case t: Throwable => error(s"Failed to update IP quota $ipEntity", t)
quotaDelta.changes().entrySet().forEach { e =>
// The connection quota only understands the connection rate limit
val quotaName = e.getKey()
val quotaValue = e.getValue()
if (!quotaName.equals(QuotaConfigs.IP_CONNECTION_RATE_OVERRIDE_CONFIG)) {
warn(s"Ignoring unexpected quota key ${quotaName} for entity $ipEntity")
} else {
try {
connectionQuotas.updateIpConnectionRateQuota(inetAddress, quotaValue.asScala.map(_.toInt))
} catch {
case t: Throwable => error(s"Failed to update IP quota $ipEntity", t)
}
}
}
// Update the cache
quotaCache.updateQuotaCache(ipEntity, quotaRecord.key, quotaRecord.value, quotaRecord.remove)
}
def handleUserClientQuota(quotaEntity: QuotaEntity, quotaRecord: ClientQuotaRecord): Unit = {
val manager = quotaRecord.key() match {
def handleUserClientQuotaChange(quotaEntity: QuotaEntity, key: String, newValue: Option[Double]): Unit = {
val manager = key match {
case QuotaConfigs.CONSUMER_BYTE_RATE_OVERRIDE_CONFIG => quotaManagers.fetch
case QuotaConfigs.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG => quotaManagers.produce
case QuotaConfigs.REQUEST_PERCENTAGE_OVERRIDE_CONFIG => quotaManagers.request
case QuotaConfigs.CONTROLLER_MUTATION_RATE_OVERRIDE_CONFIG => quotaManagers.controllerMutation
case _ =>
warn(s"Ignoring unexpected quota key ${quotaRecord.key()} for entity $quotaEntity")
warn(s"Ignoring unexpected quota key ${key} for entity $quotaEntity")
return
}
@ -161,12 +157,7 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag @@ -161,12 +157,7 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag
case IpEntity(_) | DefaultIpEntity => throw new IllegalStateException("Should not see IP quota entities here")
}
val quotaValue = if (quotaRecord.remove()) {
None
} else {
Some(new Quota(quotaRecord.value(), true))
}
val quotaValue = newValue.map(new Quota(_, true))
try {
manager.updateQuota(
sanitizedUser = sanitizedUser,
@ -176,7 +167,5 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag @@ -176,7 +167,5 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag
} catch {
case t: Throwable => error(s"Failed to update user-client quota $quotaEntity", t)
}
quotaCache.updateQuotaCache(quotaEntity, quotaRecord.key, quotaRecord.value, quotaRecord.remove)
}
}

354
core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala

@ -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)
}
}

150
core/src/main/scala/kafka/server/metadata/MetadataBrokers.scala

@ -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))
}

137
core/src/main/scala/kafka/server/metadata/MetadataImage.scala

@ -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)
}
}

355
core/src/main/scala/kafka/server/metadata/MetadataPartitions.scala

@ -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
}
}

36
core/src/main/scala/kafka/server/metadata/MetadataPublisher.scala

@ -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
}

394
core/src/main/scala/kafka/server/metadata/RaftMetadataCache.scala

@ -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
}
}

90
core/src/test/scala/kafka/server/metadata/MetadataBrokersTest.scala

@ -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))
}
}

324
core/src/test/scala/kafka/server/metadata/MetadataPartitionsTest.scala

@ -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
}
}

2
core/src/test/scala/kafka/server/metadata/MockConfigRepository.scala

@ -30,7 +30,7 @@ object MockConfigRepository { @@ -30,7 +30,7 @@ object MockConfigRepository {
forTopic(topic, properties)
}
def forTopic(topic:String, properties: Properties): MockConfigRepository = {
def forTopic(topic: String, properties: Properties): MockConfigRepository = {
val repository = new MockConfigRepository()
repository.configs.put(new ConfigResource(TOPIC, topic), properties)
repository

2
core/src/test/scala/unit/kafka/cluster/PartitionTest.scala

@ -160,7 +160,7 @@ class PartitionTest extends AbstractPartitionTest { @@ -160,7 +160,7 @@ class PartitionTest extends AbstractPartitionTest {
@Test
def testMakeLeaderDoesNotUpdateEpochCacheForOldFormats(): Unit = {
val leaderEpoch = 8
configRepository.setTopicConfig(topicPartition.topic,
configRepository.setTopicConfig(topicPartition.topic(),
LogConfig.MessageFormatVersionProp, kafka.api.KAFKA_0_10_2_IV0.shortVersion)
val log = logManager.getOrCreateLog(topicPartition, topicId = None)
log.appendAsLeader(TestUtils.records(List(

2
core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala

@ -157,7 +157,7 @@ object AbstractCoordinatorConcurrencyTest { @@ -157,7 +157,7 @@ object AbstractCoordinatorConcurrencyTest {
}
class TestReplicaManager extends ReplicaManager(
null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, None, null, null) {
null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, None, null) {
@volatile var logs: mutable.Map[TopicPartition, (Log, Long)] = _
var producePurgatory: DelayedOperationPurgatory[DelayedProduce] = _

5
core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala

@ -65,8 +65,7 @@ class HighwatermarkPersistenceTest { @@ -65,8 +65,7 @@ class HighwatermarkPersistenceTest {
// create replica manager
val replicaManager = new ReplicaManager(configs.head, metrics, time, None, scheduler,
logManagers.head, new AtomicBoolean(false), quotaManager,
new BrokerTopicStats, MetadataCache.zkMetadataCache(configs.head.brokerId), logDirFailureChannels.head, alterIsrManager,
configRepository)
new BrokerTopicStats, MetadataCache.zkMetadataCache(configs.head.brokerId), logDirFailureChannels.head, alterIsrManager)
replicaManager.startup()
try {
replicaManager.checkpointHighWatermarks()
@ -115,7 +114,7 @@ class HighwatermarkPersistenceTest { @@ -115,7 +114,7 @@ class HighwatermarkPersistenceTest {
// create replica manager
val replicaManager = new ReplicaManager(configs.head, metrics, time, None,
scheduler, logManagers.head, new AtomicBoolean(false), quotaManager,
new BrokerTopicStats, MetadataCache.zkMetadataCache(configs.head.brokerId), logDirFailureChannels.head, alterIsrManager, configRepository)
new BrokerTopicStats, MetadataCache.zkMetadataCache(configs.head.brokerId), logDirFailureChannels.head, alterIsrManager)
replicaManager.startup()
try {
replicaManager.checkpointHighWatermarks()

3
core/src/test/scala/unit/kafka/server/IsrExpirationTest.scala

@ -23,7 +23,6 @@ import java.util.concurrent.atomic.AtomicBoolean @@ -23,7 +23,6 @@ import java.util.concurrent.atomic.AtomicBoolean
import kafka.cluster.Partition
import kafka.log.{Log, LogManager}
import kafka.server.QuotaFactory.QuotaManagers
import kafka.server.metadata.MockConfigRepository
import kafka.utils.TestUtils.MockAlterIsrManager
import kafka.utils._
import org.apache.kafka.common.TopicPartition
@ -68,7 +67,7 @@ class IsrExpirationTest { @@ -68,7 +67,7 @@ class IsrExpirationTest {
quotaManager = QuotaFactory.instantiate(configs.head, metrics, time, "")
replicaManager = new ReplicaManager(configs.head, metrics, time, None, null, logManager, new AtomicBoolean(false),
quotaManager, new BrokerTopicStats, MetadataCache.zkMetadataCache(configs.head.brokerId),
new LogDirFailureChannel(configs.head.logDirs.size), alterIsrManager, new MockConfigRepository())
new LogDirFailureChannel(configs.head.logDirs.size), alterIsrManager)
}
@AfterEach

58
core/src/test/scala/unit/kafka/server/KafkaApisTest.scala

@ -33,7 +33,7 @@ import kafka.coordinator.transaction.{InitProducerIdResult, TransactionCoordinat @@ -33,7 +33,7 @@ import kafka.coordinator.transaction.{InitProducerIdResult, TransactionCoordinat
import kafka.log.AppendOrigin
import kafka.network.RequestChannel
import kafka.server.QuotaFactory.QuotaManagers
import kafka.server.metadata.{ClientQuotaCache, ConfigRepository, MockConfigRepository, RaftMetadataCache}
import kafka.server.metadata.{ConfigRepository, KRaftMetadataCache, MockConfigRepository}
import kafka.utils.{MockTime, TestUtils}
import kafka.zk.KafkaZkClient
import org.apache.kafka.clients.admin.AlterConfigOp.OpType
@ -109,7 +109,6 @@ class KafkaApisTest { @@ -109,7 +109,6 @@ class KafkaApisTest {
private val replicaQuotaManager: ReplicationQuotaManager = EasyMock.createNiceMock(classOf[ReplicationQuotaManager])
private val quotas = QuotaManagers(clientQuotaManager, clientQuotaManager, clientRequestQuotaManager,
clientControllerQuotaManager, replicaQuotaManager, replicaQuotaManager, replicaQuotaManager, None)
private val quotaCache = new ClientQuotaCache()
private val fetchManager: FetchManager = EasyMock.createNiceMock(classOf[FetchManager])
private val brokerTopicStats = new BrokerTopicStats
private val clusterId = "clusterId"
@ -150,11 +149,10 @@ class KafkaApisTest { @@ -150,11 +149,10 @@ class KafkaApisTest {
val metadataSupport = if (raftSupport) {
// it will be up to the test to replace the default ZkMetadataCache implementation
// with a RaftMetadataCache instance
// with a KRaftMetadataCache instance
metadataCache match {
case raftMetadataCache: RaftMetadataCache =>
RaftSupport(forwardingManager, raftMetadataCache, quotaCache)
case _ => throw new IllegalStateException("Test must set an instance of RaftMetadataCache")
case cache: KRaftMetadataCache => RaftSupport(forwardingManager, cache)
case _ => throw new IllegalStateException("Test must set an instance of KRaftMetadataCache")
}
} else {
metadataCache match {
@ -484,7 +482,7 @@ class KafkaApisTest { @@ -484,7 +482,7 @@ class KafkaApisTest {
def testDescribeQuorumForwardedForKRaftClusters(): Unit = {
val requestData = DescribeQuorumRequest.singletonRequest(KafkaRaftServer.MetadataPartition)
val requestBuilder = new DescribeQuorumRequest.Builder(requestData)
metadataCache = MetadataCache.raftMetadataCache(brokerId)
metadataCache = MetadataCache.kRaftMetadataCache(brokerId)
testForwardableApi(
createKafkaApis(raftSupport = true),
@ -3138,7 +3136,7 @@ class KafkaApisTest { @@ -3138,7 +3136,7 @@ class KafkaApisTest {
)
val updateMetadataRequest = new UpdateMetadataRequest.Builder(ApiKeys.UPDATE_METADATA.latestVersion, 0,
0, 0, Seq.empty[UpdateMetadataPartitionState].asJava, brokers.asJava, Collections.emptyMap()).build()
metadataCache.updateMetadata(correlationId = 0, updateMetadataRequest)
MetadataCacheTest.updateCache(metadataCache, updateMetadataRequest)
val describeClusterRequest = new DescribeClusterRequest.Builder(new DescribeClusterRequestData()
.setIncludeClusterAuthorizedOperations(true)).build()
@ -3192,7 +3190,7 @@ class KafkaApisTest { @@ -3192,7 +3190,7 @@ class KafkaApisTest {
)
val updateMetadataRequest = new UpdateMetadataRequest.Builder(ApiKeys.UPDATE_METADATA.latestVersion, 0,
0, 0, Seq.empty[UpdateMetadataPartitionState].asJava, brokers.asJava, Collections.emptyMap()).build()
metadataCache.updateMetadata(correlationId = 0, updateMetadataRequest)
MetadataCacheTest.updateCache(metadataCache, updateMetadataRequest)
(plaintextListener, anotherListener)
}
@ -3342,7 +3340,7 @@ class KafkaApisTest { @@ -3342,7 +3340,7 @@ class KafkaApisTest {
private def addTopicToMetadataCache(topic: String, numPartitions: Int, numBrokers: Int = 1): Unit = {
val updateMetadataRequest = createBasicMetadataRequest(topic, numPartitions, 0, numBrokers)
metadataCache.updateMetadata(correlationId = 0, updateMetadataRequest)
MetadataCacheTest.updateCache(metadataCache, updateMetadataRequest)
}
private def createMetadataBroker(brokerId: Int,
@ -3891,121 +3889,121 @@ class KafkaApisTest { @@ -3891,121 +3889,121 @@ class KafkaApisTest {
@Test
def testRaftShouldNeverHandleLeaderAndIsrRequest(): Unit = {
metadataCache = MetadataCache.raftMetadataCache(brokerId)
metadataCache = MetadataCache.kRaftMetadataCache(brokerId)
verifyShouldNeverHandle(createKafkaApis(raftSupport = true).handleLeaderAndIsrRequest)
}
@Test
def testRaftShouldNeverHandleStopReplicaRequest(): Unit = {
metadataCache = MetadataCache.raftMetadataCache(brokerId)
metadataCache = MetadataCache.kRaftMetadataCache(brokerId)
verifyShouldNeverHandle(createKafkaApis(raftSupport = true).handleStopReplicaRequest)
}
@Test
def testRaftShouldNeverHandleUpdateMetadataRequest(): Unit = {
metadataCache = MetadataCache.raftMetadataCache(brokerId)
metadataCache = MetadataCache.kRaftMetadataCache(brokerId)
verifyShouldNeverHandle(createKafkaApis(raftSupport = true).handleUpdateMetadataRequest(_, RequestLocal.withThreadConfinedCaching))
}
@Test
def testRaftShouldNeverHandleControlledShutdownRequest(): Unit = {
metadataCache = MetadataCache.raftMetadataCache(brokerId)
metadataCache = MetadataCache.kRaftMetadataCache(brokerId)
verifyShouldNeverHandle(createKafkaApis(raftSupport = true).handleControlledShutdownRequest)
}
@Test
def testRaftShouldNeverHandleAlterIsrRequest(): Unit = {
metadataCache = MetadataCache.raftMetadataCache(brokerId)
metadataCache = MetadataCache.kRaftMetadataCache(brokerId)
verifyShouldNeverHandle(createKafkaApis(raftSupport = true).handleAlterIsrRequest)
}
@Test
def testRaftShouldNeverHandleEnvelope(): Unit = {
metadataCache = MetadataCache.raftMetadataCache(brokerId)
metadataCache = MetadataCache.kRaftMetadataCache(brokerId)
verifyShouldNeverHandle(createKafkaApis(raftSupport = true).handleEnvelope(_, RequestLocal.withThreadConfinedCaching))
}
@Test
def testRaftShouldAlwaysForwardCreateTopicsRequest(): Unit = {
metadataCache = MetadataCache.raftMetadataCache(brokerId)
metadataCache = MetadataCache.kRaftMetadataCache(brokerId)
verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleCreateTopicsRequest)
}
@Test
def testRaftShouldAlwaysForwardCreatePartitionsRequest(): Unit = {
metadataCache = MetadataCache.raftMetadataCache(brokerId)
metadataCache = MetadataCache.kRaftMetadataCache(brokerId)
verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleCreatePartitionsRequest)
}
@Test
def testRaftShouldAlwaysForwardDeleteTopicsRequest(): Unit = {
metadataCache = MetadataCache.raftMetadataCache(brokerId)
metadataCache = MetadataCache.kRaftMetadataCache(brokerId)
verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleDeleteTopicsRequest)
}
@Test
def testRaftShouldAlwaysForwardCreateAcls(): Unit = {
metadataCache = MetadataCache.raftMetadataCache(brokerId)
metadataCache = MetadataCache.kRaftMetadataCache(brokerId)
verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleCreateAcls)
}
@Test
def testRaftShouldAlwaysForwardDeleteAcls(): Unit = {
metadataCache = MetadataCache.raftMetadataCache(brokerId)
metadataCache = MetadataCache.kRaftMetadataCache(brokerId)
verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleDeleteAcls)
}
@Test
def testRaftShouldAlwaysForwardAlterConfigsRequest(): Unit = {
metadataCache = MetadataCache.raftMetadataCache(brokerId)
metadataCache = MetadataCache.kRaftMetadataCache(brokerId)
verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleAlterConfigsRequest)
}
@Test
def testRaftShouldAlwaysForwardAlterPartitionReassignmentsRequest(): Unit = {
metadataCache = MetadataCache.raftMetadataCache(brokerId)
metadataCache = MetadataCache.kRaftMetadataCache(brokerId)
verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleAlterPartitionReassignmentsRequest)
}
@Test
def testRaftShouldAlwaysForwardIncrementalAlterConfigsRequest(): Unit = {
metadataCache = MetadataCache.raftMetadataCache(brokerId)
metadataCache = MetadataCache.kRaftMetadataCache(brokerId)
verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleIncrementalAlterConfigsRequest)
}
@Test
def testRaftShouldAlwaysForwardCreateTokenRequest(): Unit = {
metadataCache = MetadataCache.raftMetadataCache(brokerId)
metadataCache = MetadataCache.kRaftMetadataCache(brokerId)
verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleCreateTokenRequest)
}
@Test
def testRaftShouldAlwaysForwardRenewTokenRequest(): Unit = {
metadataCache = MetadataCache.raftMetadataCache(brokerId)
metadataCache = MetadataCache.kRaftMetadataCache(brokerId)
verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleRenewTokenRequest)
}
@Test
def testRaftShouldAlwaysForwardExpireTokenRequest(): Unit = {
metadataCache = MetadataCache.raftMetadataCache(brokerId)
metadataCache = MetadataCache.kRaftMetadataCache(brokerId)
verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleExpireTokenRequest)
}
@Test
def testRaftShouldAlwaysForwardAlterClientQuotasRequest(): Unit = {
metadataCache = MetadataCache.raftMetadataCache(brokerId)
metadataCache = MetadataCache.kRaftMetadataCache(brokerId)
verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleAlterClientQuotasRequest)
}
@Test
def testRaftShouldAlwaysForwardAlterUserScramCredentialsRequest(): Unit = {
metadataCache = MetadataCache.raftMetadataCache(brokerId)
metadataCache = MetadataCache.kRaftMetadataCache(brokerId)
verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleAlterUserScramCredentialsRequest)
}
@Test
def testRaftShouldAlwaysForwardUpdateFeatures(): Unit = {
metadataCache = MetadataCache.raftMetadataCache(brokerId)
metadataCache = MetadataCache.kRaftMetadataCache(brokerId)
verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleUpdateFeatures)
}
}

113
core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala

@ -17,28 +17,109 @@ @@ -17,28 +17,109 @@
package kafka.server
import org.apache.kafka.common.{Node, TopicPartition, Uuid}
import java.util
import util.Arrays.asList
import org.apache.kafka.common.message.UpdateMetadataRequestData.{UpdateMetadataBroker, UpdateMetadataEndpoint, UpdateMetadataPartitionState}
import org.apache.kafka.common.message.UpdateMetadataRequestData.{UpdateMetadataBroker, UpdateMetadataEndpoint, UpdateMetadataPartitionState, UpdateMetadataTopicState}
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.protocol.{ApiKeys, ApiMessage, Errors}
import org.apache.kafka.common.record.RecordBatch
import org.apache.kafka.common.requests.UpdateMetadataRequest
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.MethodSource
import java.util.Collections
import kafka.api.LeaderAndIsr
import kafka.server.metadata.KRaftMetadataCache
import org.apache.kafka.common.metadata.{PartitionRecord, RegisterBrokerRecord, RemoveTopicRecord, TopicRecord}
import org.apache.kafka.common.metadata.RegisterBrokerRecord.{BrokerEndpoint, BrokerEndpointCollection}
import org.apache.kafka.image.{ClusterImage, MetadataDelta, MetadataImage}
import scala.collection.{Seq, mutable}
import scala.jdk.CollectionConverters._
object MetadataCacheTest {
def cacheProvider(): util.stream.Stream[MetadataCache] = {
util.stream.Stream.of(
def zkCacheProvider(): util.stream.Stream[MetadataCache] =
util.stream.Stream.of[MetadataCache](
MetadataCache.zkMetadataCache(1)
)
def cacheProvider(): util.stream.Stream[MetadataCache] =
util.stream.Stream.of[MetadataCache](
MetadataCache.zkMetadataCache(1),
MetadataCache.raftMetadataCache(1)
MetadataCache.kRaftMetadataCache(1)
)
def updateCache(cache: MetadataCache, request: UpdateMetadataRequest): Unit = {
cache match {
case c: ZkMetadataCache => c.updateMetadata(0, request)
case c: KRaftMetadataCache => {
// UpdateMetadataRequest always contains a full list of brokers, but may contain
// a partial list of partitions. Therefore, base our delta off a partial image that
// contains no brokers, but which contains the previous partitions.
val image = c.currentImage()
val partialImage = new MetadataImage(image.features(), ClusterImage.EMPTY,
image.topics(), image.configs(), image.clientQuotas())
val delta = new MetadataDelta(partialImage)
def toRecord(broker: UpdateMetadataBroker): RegisterBrokerRecord = {
val endpoints = new BrokerEndpointCollection()
broker.endpoints().forEach { e =>
endpoints.add(new BrokerEndpoint().
setName(e.listener()).
setHost(e.host()).
setPort(e.port()).
setSecurityProtocol(e.securityProtocol()))
}
val prevBroker = Option(image.cluster().broker(broker.id()))
// UpdateMetadataRequest doesn't contain all the broker registration fields, so get
// them from the previous registration if available.
val (epoch, incarnationId, fenced) = prevBroker match {
case None => (0L, Uuid.ZERO_UUID, false)
case Some(b) => (b.epoch(), b.incarnationId(), b.fenced())
}
new RegisterBrokerRecord().
setBrokerId(broker.id()).
setBrokerEpoch(epoch).
setIncarnationId(incarnationId).
setEndPoints(endpoints).
setRack(broker.rack()).
setFenced(fenced)
}
request.liveBrokers().iterator().asScala.foreach { brokerInfo =>
delta.replay(toRecord(brokerInfo))
}
def toRecords(topic: UpdateMetadataTopicState): Seq[ApiMessage] = {
val results = new mutable.ArrayBuffer[ApiMessage]()
results += new TopicRecord().setName(topic.topicName()).setTopicId(topic.topicId())
topic.partitionStates().forEach { partition =>
if (partition.leader() == LeaderAndIsr.LeaderDuringDelete) {
results += new RemoveTopicRecord().setTopicId(topic.topicId())
} else {
results += new PartitionRecord().
setPartitionId(partition.partitionIndex()).
setTopicId(topic.topicId()).
setReplicas(partition.replicas()).
setIsr(partition.isr()).
setRemovingReplicas(Collections.emptyList()).
setAddingReplicas(Collections.emptyList()).
setLeader(partition.leader()).
setLeaderEpoch(partition.leaderEpoch()).
setPartitionEpoch(partition.zkVersion())
}
}
results
}
request.topicStates().forEach { topic =>
toRecords(topic).foreach(delta.replay)
}
c.setImage(delta.apply())
}
case _ => throw new RuntimeException("Unsupported cache type")
}
}
}
@ -122,7 +203,7 @@ class MetadataCacheTest { @@ -122,7 +203,7 @@ class MetadataCacheTest {
val version = ApiKeys.UPDATE_METADATA.latestVersion
val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, brokerEpoch,
partitionStates.asJava, brokers.asJava, topicIds).build()
cache.updateMetadata(15, updateMetadataRequest)
MetadataCacheTest.updateCache(cache, updateMetadataRequest)
for (securityProtocol <- Seq(SecurityProtocol.PLAINTEXT, SecurityProtocol.SSL)) {
val listenerName = ListenerName.forSecurityProtocol(securityProtocol)
@ -244,7 +325,7 @@ class MetadataCacheTest { @@ -244,7 +325,7 @@ class MetadataCacheTest {
val version = ApiKeys.UPDATE_METADATA.latestVersion
val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, brokerEpoch,
partitionStates.asJava, brokers.asJava, util.Collections.emptyMap()).build()
cache.updateMetadata(15, updateMetadataRequest)
MetadataCacheTest.updateCache(cache, updateMetadataRequest)
val topicMetadatas = cache.getTopicMetadata(Set(topic), listenerName, errorUnavailableListeners = errorUnavailableListeners)
assertEquals(1, topicMetadatas.size)
@ -300,7 +381,7 @@ class MetadataCacheTest { @@ -300,7 +381,7 @@ class MetadataCacheTest {
val version = ApiKeys.UPDATE_METADATA.latestVersion
val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, brokerEpoch,
partitionStates.asJava, brokers.asJava, util.Collections.emptyMap()).build()
cache.updateMetadata(15, updateMetadataRequest)
MetadataCacheTest.updateCache(cache, updateMetadataRequest)
// Validate errorUnavailableEndpoints = false
val topicMetadatas = cache.getTopicMetadata(Set(topic), listenerName, errorUnavailableEndpoints = false)
@ -373,7 +454,7 @@ class MetadataCacheTest { @@ -373,7 +454,7 @@ class MetadataCacheTest {
val version = ApiKeys.UPDATE_METADATA.latestVersion
val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, brokerEpoch,
partitionStates.asJava, brokers.asJava, util.Collections.emptyMap()).build()
cache.updateMetadata(15, updateMetadataRequest)
MetadataCacheTest.updateCache(cache, updateMetadataRequest)
// Validate errorUnavailableEndpoints = false
val topicMetadatas = cache.getTopicMetadata(Set(topic), listenerName, errorUnavailableEndpoints = false)
@ -438,7 +519,7 @@ class MetadataCacheTest { @@ -438,7 +519,7 @@ class MetadataCacheTest {
val version = ApiKeys.UPDATE_METADATA.latestVersion
val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, 2, controllerEpoch, brokerEpoch, partitionStates.asJava,
brokers.asJava, util.Collections.emptyMap()).build()
cache.updateMetadata(15, updateMetadataRequest)
MetadataCacheTest.updateCache(cache, updateMetadataRequest)
val topicMetadata = cache.getTopicMetadata(Set(topic), ListenerName.forSecurityProtocol(SecurityProtocol.SSL))
assertEquals(1, topicMetadata.size)
@ -480,7 +561,7 @@ class MetadataCacheTest { @@ -480,7 +561,7 @@ class MetadataCacheTest {
val version = ApiKeys.UPDATE_METADATA.latestVersion
val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, 2, controllerEpoch, brokerEpoch, partitionStates.asJava,
brokers.asJava, util.Collections.emptyMap()).build()
cache.updateMetadata(15, updateMetadataRequest)
MetadataCacheTest.updateCache(cache, updateMetadataRequest)
}
val initialBrokerIds = (0 to 2)
@ -491,8 +572,10 @@ class MetadataCacheTest { @@ -491,8 +572,10 @@ class MetadataCacheTest {
assertEquals(initialBrokerIds.toSet, aliveBrokersFromCache.map(_.id).toSet)
}
// This test runs only for the ZK cache, because KRaft mode doesn't support offline
// replicas yet. TODO: implement KAFKA-13005.
@ParameterizedTest
@MethodSource(Array("cacheProvider"))
@MethodSource(Array("zkCacheProvider"))
def testGetClusterMetadataWithOfflineReplicas(cache: MetadataCache): Unit = {
val topic = "topic"
val topicPartition = new TopicPartition(topic, 0)
@ -531,7 +614,7 @@ class MetadataCacheTest { @@ -531,7 +614,7 @@ class MetadataCacheTest {
val version = ApiKeys.UPDATE_METADATA.latestVersion
val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, 2, controllerEpoch, brokerEpoch, partitionStates.asJava,
brokers.asJava, Collections.emptyMap()).build()
cache.updateMetadata(15, updateMetadataRequest)
MetadataCacheTest.updateCache(cache, updateMetadataRequest)
val expectedNode0 = new Node(0, "foo", 9092)
val expectedNode1 = new Node(1, "", -1)

151
core/src/test/scala/unit/kafka/server/RaftReplicaChangeDelegateTest.scala

@ -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
}
}

317
core/src/test/scala/unit/kafka/server/RaftReplicaManagerTest.scala

@ -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)
}
}

4
core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala

@ -30,7 +30,6 @@ import org.apache.kafka.common.requests.FetchRequest.PartitionData @@ -30,7 +30,6 @@ import org.apache.kafka.common.requests.FetchRequest.PartitionData
import org.easymock.EasyMock
import EasyMock._
import kafka.server.QuotaFactory.QuotaManagers
import kafka.server.metadata.MockConfigRepository
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, Test}
@ -199,7 +198,6 @@ class ReplicaManagerQuotasTest { @@ -199,7 +198,6 @@ class ReplicaManagerQuotasTest {
def setUpMocks(fetchInfo: Seq[(TopicPartition, PartitionData)], record: SimpleRecord = this.record,
bothReplicasInSync: Boolean = false): Unit = {
val configRepository = new MockConfigRepository()
val scheduler: KafkaScheduler = createNiceMock(classOf[KafkaScheduler])
//Create log which handles both a regular read and a 0 bytes read
@ -245,7 +243,7 @@ class ReplicaManagerQuotasTest { @@ -245,7 +243,7 @@ class ReplicaManagerQuotasTest {
quotaManager = QuotaFactory.instantiate(configs.head, metrics, time, "")
replicaManager = new ReplicaManager(configs.head, metrics, time, None, scheduler, logManager,
new AtomicBoolean(false), quotaManager,
new BrokerTopicStats, MetadataCache.zkMetadataCache(leaderBrokerId), new LogDirFailureChannel(configs.head.logDirs.size), alterIsrManager, configRepository)
new BrokerTopicStats, MetadataCache.zkMetadataCache(leaderBrokerId), new LogDirFailureChannel(configs.head.logDirs.size), alterIsrManager)
//create the two replicas
for ((p, _) <- fetchInfo) {

115
core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala

@ -23,7 +23,6 @@ import kafka.log._ @@ -23,7 +23,6 @@ import kafka.log._
import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota}
import kafka.server.checkpoints.{LazyOffsetCheckpoints, OffsetCheckpointFile}
import kafka.server.epoch.util.ReplicaFetcherMockBlockingSend
import kafka.server.metadata.MockConfigRepository
import kafka.utils.timer.MockTimer
import kafka.utils.{MockScheduler, MockTime, TestUtils}
import org.apache.kafka.common.message.FetchResponseData
@ -49,11 +48,15 @@ import org.mockito.{ArgumentMatchers, Mockito} @@ -49,11 +48,15 @@ import org.mockito.{ArgumentMatchers, Mockito}
import java.io.File
import java.net.InetAddress
import java.nio.file.Files
import java.util
import java.util.concurrent.atomic.{AtomicBoolean, AtomicReference}
import java.util.concurrent.{CountDownLatch, TimeUnit}
import java.util.{Collections, Optional, Properties}
import org.apache.kafka.common.metadata.{PartitionRecord, RemoveTopicRecord, TopicRecord}
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.image.{TopicImage, TopicsDelta, TopicsImage}
import org.apache.kafka.metadata.PartitionRegistration
import org.mockito.invocation.InvocationOnMock
import org.mockito.stubbing.Answer
@ -66,7 +69,6 @@ class ReplicaManagerTest { @@ -66,7 +69,6 @@ class ReplicaManagerTest {
val time = new MockTime
val scheduler = new MockScheduler(time)
val metrics = new Metrics
val configRepository = new MockConfigRepository()
var alterIsrManager: AlterIsrManager = _
var config: KafkaConfig = _
var quotaManager: QuotaManagers = _
@ -97,7 +99,7 @@ class ReplicaManagerTest { @@ -97,7 +99,7 @@ class ReplicaManagerTest {
val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)))
val rm = new ReplicaManager(config, metrics, time, None, new MockScheduler(time), mockLogMgr,
new AtomicBoolean(false), quotaManager, new BrokerTopicStats,
MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager, configRepository)
MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager)
try {
val partition = rm.createPartition(new TopicPartition(topic, 1))
partition.createLogIfNotExists(isNew = false, isFutureReplica = false,
@ -117,7 +119,7 @@ class ReplicaManagerTest { @@ -117,7 +119,7 @@ class ReplicaManagerTest {
val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)))
val rm = new ReplicaManager(config, metrics, time, None, new MockScheduler(time), mockLogMgr,
new AtomicBoolean(false), quotaManager, new BrokerTopicStats,
MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager, configRepository)
MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager)
try {
val partition = rm.createPartition(new TopicPartition(topic, 1))
partition.createLogIfNotExists(isNew = false, isFutureReplica = false,
@ -134,7 +136,7 @@ class ReplicaManagerTest { @@ -134,7 +136,7 @@ class ReplicaManagerTest {
val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)))
val rm = new ReplicaManager(config, metrics, time, None, new MockScheduler(time), mockLogMgr,
new AtomicBoolean(false), quotaManager, new BrokerTopicStats,
MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager, configRepository, Option(this.getClass.getName))
MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager, Option(this.getClass.getName))
try {
def callback(responseStatus: Map[TopicPartition, PartitionResponse]) = {
assert(responseStatus.values.head.error == Errors.INVALID_REQUIRED_ACKS)
@ -181,7 +183,7 @@ class ReplicaManagerTest { @@ -181,7 +183,7 @@ class ReplicaManagerTest {
mockGetAliveBrokerFunctions(metadataCache, aliveBrokers)
val rm = new ReplicaManager(config, metrics, time, None, new MockScheduler(time), mockLogMgr,
new AtomicBoolean(false), quotaManager, new BrokerTopicStats,
metadataCache, new LogDirFailureChannel(config.logDirs.size), alterIsrManager, configRepository)
metadataCache, new LogDirFailureChannel(config.logDirs.size), alterIsrManager)
try {
val brokerList = Seq[Integer](0, 1).asJava
@ -1599,7 +1601,7 @@ class ReplicaManagerTest { @@ -1599,7 +1601,7 @@ class ReplicaManagerTest {
new AtomicBoolean(false), quotaManager, mockBrokerTopicStats,
metadataCache, mockLogDirFailureChannel, mockProducePurgatory, mockFetchPurgatory,
mockDeleteRecordsPurgatory, mockElectLeaderPurgatory, Option(this.getClass.getName),
configRepository, alterIsrManager) {
alterIsrManager) {
override protected def createReplicaFetcherManager(metrics: Metrics,
time: Time,
@ -1776,7 +1778,7 @@ class ReplicaManagerTest { @@ -1776,7 +1778,7 @@ class ReplicaManagerTest {
new AtomicBoolean(false), quotaManager, new BrokerTopicStats,
metadataCache, new LogDirFailureChannel(config.logDirs.size), mockProducePurgatory, mockFetchPurgatory,
mockDeleteRecordsPurgatory, mockDelayedElectLeaderPurgatory, Option(this.getClass.getName),
configRepository, alterIsrManager)
alterIsrManager)
}
@Test
@ -1985,10 +1987,10 @@ class ReplicaManagerTest { @@ -1985,10 +1987,10 @@ class ReplicaManagerTest {
// each replica manager is for a broker
val rm0 = new ReplicaManager(config0, metrics, time, None, new MockScheduler(time), mockLogMgr0,
new AtomicBoolean(false), quotaManager,
brokerTopicStats1, metadataCache0, new LogDirFailureChannel(config0.logDirs.size), alterIsrManager, configRepository)
brokerTopicStats1, metadataCache0, new LogDirFailureChannel(config0.logDirs.size), alterIsrManager)
val rm1 = new ReplicaManager(config1, metrics, time, None, new MockScheduler(time), mockLogMgr1,
new AtomicBoolean(false), quotaManager,
brokerTopicStats2, metadataCache1, new LogDirFailureChannel(config1.logDirs.size), alterIsrManager, configRepository)
brokerTopicStats2, metadataCache1, new LogDirFailureChannel(config1.logDirs.size), alterIsrManager)
(rm0, rm1)
}
@ -2231,7 +2233,7 @@ class ReplicaManagerTest { @@ -2231,7 +2233,7 @@ class ReplicaManagerTest {
val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)))
new ReplicaManager(config, metrics, time, None, new MockScheduler(time), mockLogMgr,
new AtomicBoolean(false), quotaManager, new BrokerTopicStats,
MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager, configRepository) {
MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager) {
override def getPartitionOrException(topicPartition: TopicPartition): Partition = {
throw Errors.NOT_LEADER_OR_FOLLOWER.exception()
}
@ -2511,7 +2513,98 @@ class ReplicaManagerTest { @@ -2511,7 +2513,98 @@ class ReplicaManagerTest {
} finally {
replicaManager.shutdown(checkpointHW = false)
}
}
val FOO_UUID = Uuid.fromString("fFJBx0OmQG-UqeaT6YaSwA")
val BAR_UUID = Uuid.fromString("vApAP6y7Qx23VOfKBzbOBQ")
val BAZ_UUID = Uuid.fromString("7wVsX2aaTk-bdGcOxLRyVQ")
@Test
def testGetOrCreatePartition(): Unit = {
val brokerId = 0
val replicaManager = setupReplicaManagerWithMockedPurgatories(new MockTimer(time), brokerId)
val foo0 = new TopicPartition("foo", 0)
val emptyDelta = new TopicsDelta(TopicsImage.EMPTY)
val (fooPart, fooNew) = replicaManager.getOrCreatePartition(foo0, emptyDelta, FOO_UUID).get
assertTrue(fooNew)
assertEquals(foo0, fooPart.topicPartition)
val (fooPart2, fooNew2) = replicaManager.getOrCreatePartition(foo0, emptyDelta, FOO_UUID).get
assertFalse(fooNew2)
assertTrue(fooPart eq fooPart2)
val bar1 = new TopicPartition("bar", 1)
replicaManager.markPartitionOffline(bar1)
assertEquals(None, replicaManager.getOrCreatePartition(bar1, emptyDelta, BAR_UUID))
}
val TEST_IMAGE = {
val topicsById = new util.HashMap[Uuid, TopicImage]()
val topicsByName = new util.HashMap[String, TopicImage]()
val fooPartitions = new util.HashMap[Integer, PartitionRegistration]()
fooPartitions.put(0, new PartitionRegistration(Array(1, 2, 3),
Array(1, 2, 3), Array.empty[Int], Array.empty[Int], 1, 100, 200))
fooPartitions.put(1, new PartitionRegistration(Array(4, 5, 6),
Array(4, 5), Array.empty[Int], Array.empty[Int], 5, 300, 400))
val foo = new TopicImage("foo", FOO_UUID, fooPartitions)
val barPartitions = new util.HashMap[Integer, PartitionRegistration]()
barPartitions.put(0, new PartitionRegistration(Array(2, 3, 4),
Array(2, 3, 4), Array.empty[Int], Array.empty[Int], 3, 100, 200))
val bar = new TopicImage("bar", BAR_UUID, barPartitions)
topicsById.put(FOO_UUID, foo)
topicsByName.put("foo", foo)
topicsById.put(BAR_UUID, bar)
topicsByName.put("bar", bar)
new TopicsImage(topicsById, topicsByName)
}
val TEST_DELTA = {
val delta = new TopicsDelta(TEST_IMAGE)
delta.replay(new RemoveTopicRecord().setTopicId(FOO_UUID))
delta.replay(new TopicRecord().setName("baz").setTopicId(BAZ_UUID))
delta.replay(new PartitionRecord().setPartitionId(0).
setTopicId(BAZ_UUID).
setReplicas(util.Arrays.asList(1, 2, 4)).
setIsr(util.Arrays.asList(1, 2, 4)).
setRemovingReplicas(Collections.emptyList()).
setAddingReplicas(Collections.emptyList()).
setLeader(1).
setLeaderEpoch(123).
setPartitionEpoch(456))
delta.replay(new PartitionRecord().setPartitionId(1).
setTopicId(BAZ_UUID).
setReplicas(util.Arrays.asList(2, 4, 1)).
setIsr(util.Arrays.asList(2, 4, 1)).
setRemovingReplicas(Collections.emptyList()).
setAddingReplicas(Collections.emptyList()).
setLeader(2).
setLeaderEpoch(123).
setPartitionEpoch(456))
delta.replay(new PartitionRecord().setPartitionId(2).
setTopicId(BAZ_UUID).
setReplicas(util.Arrays.asList(3, 5, 2)).
setIsr(util.Arrays.asList(3, 5, 2)).
setRemovingReplicas(Collections.emptyList()).
setAddingReplicas(Collections.emptyList()).
setLeader(3).
setLeaderEpoch(456).
setPartitionEpoch(789))
delta
}
@Test
def testCalculateDeltaChanges(): Unit = {
val brokerId = 1
val replicaManager = setupReplicaManagerWithMockedPurgatories(new MockTimer(time), brokerId)
assertEquals((
Map(new TopicPartition("foo", 0) -> true,
new TopicPartition("foo", 1) -> true),
Map(new TopicPartition("baz", 0) -> LocalLeaderInfo(BAZ_UUID,
new PartitionRegistration(Array(1, 2, 4), Array(1, 2, 4),
Array.empty[Int], Array.empty[Int], 1, 123, 456))),
Map(new TopicPartition("baz", 1) -> LocalLeaderInfo(BAZ_UUID,
new PartitionRegistration(Array(2, 4, 1), Array(2, 4, 1),
Array.empty[Int], Array.empty[Int], 2, 123, 456)))),
replicaManager.calculateDeltaChanges(TEST_DELTA))
}
}

8
core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala

@ -22,7 +22,6 @@ import java.util.concurrent.atomic.AtomicBoolean @@ -22,7 +22,6 @@ import java.util.concurrent.atomic.AtomicBoolean
import kafka.log.{Log, LogManager}
import kafka.server.QuotaFactory.QuotaManagers
import kafka.server._
import kafka.server.metadata.MockConfigRepository
import kafka.utils.{MockTime, TestUtils}
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.{OffsetForLeaderPartition, OffsetForLeaderTopic}
@ -42,7 +41,6 @@ class OffsetsForLeaderEpochTest { @@ -42,7 +41,6 @@ class OffsetsForLeaderEpochTest {
private val time = new MockTime
private val metrics = new Metrics
private val alterIsrManager = TestUtils.createAlterIsrManager()
private val configRepository = new MockConfigRepository()
private val tp = new TopicPartition("topic", 1)
private var replicaManager: ReplicaManager = _
private var quotaManager: QuotaManagers = _
@ -69,7 +67,7 @@ class OffsetsForLeaderEpochTest { @@ -69,7 +67,7 @@ class OffsetsForLeaderEpochTest {
// create a replica manager with 1 partition that has 1 replica
replicaManager = new ReplicaManager(config, metrics, time, None, null, logManager, new AtomicBoolean(false),
quotaManager, new BrokerTopicStats,
MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager, configRepository)
MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager)
val partition = replicaManager.createPartition(tp)
partition.setLog(mockLog, isFutureLog = false)
partition.leaderReplicaIdOpt = Some(config.brokerId)
@ -92,7 +90,7 @@ class OffsetsForLeaderEpochTest { @@ -92,7 +90,7 @@ class OffsetsForLeaderEpochTest {
//create a replica manager with 1 partition that has 0 replica
replicaManager = new ReplicaManager(config, metrics, time, None, null, logManager, new AtomicBoolean(false),
quotaManager, new BrokerTopicStats,
MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager, configRepository)
MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager)
replicaManager.createPartition(tp)
//Given
@ -117,7 +115,7 @@ class OffsetsForLeaderEpochTest { @@ -117,7 +115,7 @@ class OffsetsForLeaderEpochTest {
//create a replica manager with 0 partition
replicaManager = new ReplicaManager(config, metrics, time, None, null, logManager, new AtomicBoolean(false),
quotaManager, new BrokerTopicStats,
MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager, configRepository)
MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager)
//Given
val epochRequested: Integer = 5

248
core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala

@ -1,10 +1,10 @@ @@ -1,10 +1,10 @@
/*
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* 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
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
@ -14,208 +14,68 @@ @@ -14,208 +14,68 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.server.metadata
import java.util.Properties
import java.util
import java.util.{Collections, Optional}
import kafka.coordinator.group.GroupCoordinator
import kafka.coordinator.transaction.TransactionCoordinator
import kafka.log.LogConfig
import kafka.server.RaftReplicaManager
import kafka.utils.Implicits._
import org.apache.kafka.common.config.ConfigResource
import org.apache.kafka.common.metadata.{ConfigRecord, PartitionRecord, RemoveTopicRecord, TopicRecord}
import org.apache.kafka.common.protocol.ObjectSerializationCache
import org.apache.kafka.common.utils.MockTime
import org.apache.kafka.common.{TopicPartition, Uuid}
import org.apache.kafka.metadata.MetadataRecordSerde
import org.apache.kafka.raft.Batch
import org.apache.kafka.raft.BatchReader;
import org.apache.kafka.raft.internals.MemoryBatchReader;
import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.junit.jupiter.api.Assertions._
import org.apache.kafka.common.{Endpoint, Uuid}
import org.apache.kafka.common.metadata.RegisterBrokerRecord
import org.apache.kafka.common.utils.Time
import org.apache.kafka.image.{MetadataDelta, MetadataImage}
import org.apache.kafka.metadata.{BrokerRegistration, RecordTestUtils, VersionRange}
import org.apache.kafka.server.common.ApiMessageAndVersion
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.Test
import org.mockito.ArgumentMatchers._
import org.mockito.Mockito._
import org.mockito.{ArgumentCaptor, ArgumentMatchers}
import scala.collection.mutable
import scala.jdk.CollectionConverters._
class BrokerMetadataListenerTest {
private val leaderEpoch = 5
private val brokerId = 1
private val time = new MockTime()
private val configRepository = new CachedConfigRepository
private val metadataCache = new RaftMetadataCache(brokerId)
private val groupCoordinator = mock(classOf[GroupCoordinator])
private val replicaManager = mock(classOf[RaftReplicaManager])
private val txnCoordinator = mock(classOf[TransactionCoordinator])
private val clientQuotaManager = mock(classOf[ClientQuotaMetadataManager])
private var lastMetadataOffset = 0L
private val metadataSerde = new MetadataRecordSerde
private val listener = new BrokerMetadataListener(
brokerId,
time,
metadataCache,
configRepository,
groupCoordinator,
replicaManager,
txnCoordinator,
threadNamePrefix = None,
clientQuotaManager
)
@Test
def testTopicCreationAndDeletion(): Unit = {
val topicId = Uuid.randomUuid()
val topic = "foo"
val numPartitions = 10
val config = Map(
LogConfig.CleanupPolicyProp -> LogConfig.Compact,
LogConfig.MaxCompactionLagMsProp -> "5000"
)
val localPartitions = createAndAssert(topicId, topic, config, numPartitions, numBrokers = 4)
deleteTopic(topicId, topic, numPartitions, localPartitions)
def testCreateAndClose(): Unit = {
val listener = new BrokerMetadataListener(0, Time.SYSTEM, None)
listener.close()
}
@Test
def testEmptyBatchReader(): Unit = {
applyBatch(List.empty);
}
private def deleteTopic(
topicId: Uuid,
topic: String,
numPartitions: Int,
localPartitions: Set[TopicPartition]
): Unit = {
val deleteRecord = new RemoveTopicRecord()
.setTopicId(topicId)
applyBatch(List(new ApiMessageAndVersion(deleteRecord, 0.toShort)))
assertFalse(metadataCache.contains(topic))
assertEquals(new Properties, configRepository.topicConfig(topic))
verify(groupCoordinator).handleDeletedPartitions(ArgumentMatchers.argThat[Seq[TopicPartition]] { partitions =>
partitions.toSet == partitionSet(topic, numPartitions)
}, any())
val deleteImageCapture: ArgumentCaptor[MetadataImageBuilder] =
ArgumentCaptor.forClass(classOf[MetadataImageBuilder])
verify(replicaManager).handleMetadataRecords(
deleteImageCapture.capture(),
ArgumentMatchers.eq(lastMetadataOffset),
any()
)
val deleteImage = deleteImageCapture.getValue
assertTrue(deleteImage.hasPartitionChanges)
val localRemoved = deleteImage.partitionsBuilder().localRemoved()
assertEquals(localPartitions, localRemoved.map(_.toTopicPartition).toSet)
}
private def applyBatch(
records: List[ApiMessageAndVersion]
): Unit = {
val baseOffset = lastMetadataOffset + 1
// For testing purposes if "records" is empty just assume that there
// is one control record in the batch
lastMetadataOffset += Math.max(1, records.size)
val batchReader = if (records.isEmpty) {
MemoryBatchReader.empty[ApiMessageAndVersion](baseOffset, baseOffset, _ => ())
} else {
val objectCache = new ObjectSerializationCache()
MemoryBatchReader.of(
List(
Batch.data(
baseOffset,
leaderEpoch,
baseOffset, // Use the base offset as the append time
records.map(metadataSerde.recordSize(_, objectCache)).sum,
records.asJava
)
).asJava,
(_: BatchReader[ApiMessageAndVersion]) => ()
)
}
listener.execCommits(batchReader)
}
private def createAndAssert(
topicId: Uuid,
topic: String,
topicConfig: Map[String, String],
numPartitions: Int,
numBrokers: Int
): Set[TopicPartition] = {
val records = mutable.ListBuffer.empty[ApiMessageAndVersion]
records += new ApiMessageAndVersion(new TopicRecord()
.setName(topic)
.setTopicId(topicId), 0)
val localTopicPartitions = mutable.Set.empty[TopicPartition]
(0 until numPartitions).map { partitionId =>
val preferredLeaderId = partitionId % numBrokers
val replicas = asJavaList(Seq(
preferredLeaderId,
preferredLeaderId + 1,
preferredLeaderId + 2
))
if (replicas.contains(brokerId)) {
localTopicPartitions.add(new TopicPartition(topic, partitionId))
}
records += new ApiMessageAndVersion(new PartitionRecord()
.setTopicId(topicId)
.setPartitionId(partitionId)
.setLeader(preferredLeaderId)
.setLeaderEpoch(0)
.setPartitionEpoch(0)
.setReplicas(replicas)
.setIsr(replicas), 0)
def testPublish(): Unit = {
val listener = new BrokerMetadataListener(0, Time.SYSTEM, None)
try {
listener.handleCommit(RecordTestUtils.mockBatchReader(100L,
util.Arrays.asList(new ApiMessageAndVersion(new RegisterBrokerRecord().
setBrokerId(0).
setBrokerEpoch(100L).
setFenced(false).
setRack(null).
setIncarnationId(Uuid.fromString("GFBwlTcpQUuLYQ2ig05CSg")), 1))));
val imageRecords = listener.getImageRecords().get()
assertEquals(0, imageRecords.size())
assertEquals(100L, listener.highestMetadataOffset())
listener.handleCommit(RecordTestUtils.mockBatchReader(200L,
util.Arrays.asList(new ApiMessageAndVersion(new RegisterBrokerRecord().
setBrokerId(1).
setBrokerEpoch(200L).
setFenced(true).
setRack(null).
setIncarnationId(Uuid.fromString("QkOQtNKVTYatADcaJ28xDg")), 1))));
listener.startPublishing(new MetadataPublisher {
override def publish(newHighestMetadataOffset: Long,
delta: MetadataDelta,
newImage: MetadataImage): Unit = {
assertEquals(200L, newHighestMetadataOffset)
assertEquals(new BrokerRegistration(0, 100L,
Uuid.fromString("GFBwlTcpQUuLYQ2ig05CSg"), Collections.emptyList[Endpoint](),
Collections.emptyMap[String, VersionRange](), Optional.empty[String](), false),
delta.clusterDelta().broker(0))
assertEquals(new BrokerRegistration(1, 200L,
Uuid.fromString("QkOQtNKVTYatADcaJ28xDg"), Collections.emptyList[Endpoint](),
Collections.emptyMap[String, VersionRange](), Optional.empty[String](), true),
delta.clusterDelta().broker(1))
}
}).get()
} finally {
listener.close()
}
topicConfig.forKeyValue { (key, value) =>
records += new ApiMessageAndVersion(new ConfigRecord()
.setResourceName(topic)
.setResourceType(ConfigResource.Type.TOPIC.id())
.setName(key)
.setValue(value), 0)
}
applyBatch(records.toList)
assertTrue(metadataCache.contains(topic))
assertEquals(Some(numPartitions), metadataCache.numPartitions(topic))
assertEquals(topicConfig, configRepository.topicConfig(topic).asScala)
val imageCapture: ArgumentCaptor[MetadataImageBuilder] =
ArgumentCaptor.forClass(classOf[MetadataImageBuilder])
verify(replicaManager).handleMetadataRecords(
imageCapture.capture(),
ArgumentMatchers.eq(lastMetadataOffset),
any()
)
val createImage = imageCapture.getValue
assertTrue(createImage.hasPartitionChanges)
val localChanged = createImage.partitionsBuilder().localChanged()
assertEquals(localTopicPartitions, localChanged.map(_.toTopicPartition).toSet)
localTopicPartitions.toSet
}
private def partitionSet(topic: String, numPartitions: Int): Set[TopicPartition] = {
(0 until numPartitions).map(new TopicPartition(topic, _)).toSet
}
private def asJavaList(replicas: Iterable[Int]): java.util.List[Integer] = {
replicas.map(Int.box).toList.asJava
}
}

78
core/src/test/scala/unit/kafka/server/metadata/CachedConfigRepositoryTest.scala

@ -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))
}
}

452
core/src/test/scala/unit/kafka/server/metadata/ClientQuotaMetadataManagerTest.scala

@ -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))
}
}

23
core/src/test/scala/unit/kafka/utils/TestUtils.scala

@ -36,7 +36,7 @@ import kafka.log._ @@ -36,7 +36,7 @@ import kafka.log._
import kafka.metrics.KafkaYammerMetrics
import kafka.server._
import kafka.server.checkpoints.OffsetCheckpointFile
import kafka.server.metadata.{ConfigRepository, MetadataBroker, MockConfigRepository}
import kafka.server.metadata.{ConfigRepository, MockConfigRepository}
import kafka.utils.Implicits._
import kafka.zk._
import org.apache.kafka.clients.CommonClientConfigs
@ -46,6 +46,7 @@ import org.apache.kafka.clients.consumer._ @@ -46,6 +46,7 @@ import org.apache.kafka.clients.consumer._
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord}
import org.apache.kafka.common.acl.{AccessControlEntry, AccessControlEntryFilter, AclBinding, AclBindingFilter}
import org.apache.kafka.common.config.ConfigResource
import org.apache.kafka.common.config.ConfigResource.Type.TOPIC
import org.apache.kafka.common.errors.{KafkaStorageException, UnknownTopicOrPartitionException}
import org.apache.kafka.common.header.Header
import org.apache.kafka.common.internals.Topic
@ -59,7 +60,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol @@ -59,7 +60,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySerializer, Deserializer, IntegerSerializer, Serializer}
import org.apache.kafka.common.utils.Utils._
import org.apache.kafka.common.utils.{Time, Utils}
import org.apache.kafka.common.{KafkaFuture, Node, TopicPartition}
import org.apache.kafka.common.{KafkaFuture, TopicPartition}
import org.apache.kafka.server.authorizer.{Authorizer => JAuthorizer}
import org.apache.kafka.test.{TestSslUtils, TestUtils => JTestUtils}
import org.apache.zookeeper.KeeperException.SessionExpiredException
@ -170,20 +171,6 @@ object TestUtils extends Logging { @@ -170,20 +171,6 @@ object TestUtils extends Logging {
def boundPort(server: KafkaServer, securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): Int =
server.boundPort(ListenerName.forSecurityProtocol(securityProtocol))
def createBroker(id: Int, host: String, port: Int, securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): MetadataBroker = {
MetadataBroker(id, null, Map(securityProtocol.name -> new Node(id, host, port)), false)
}
def createMetadataBroker(id: Int,
host: String = "localhost",
port: Int = 9092,
securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT,
rack: Option[String] = None,
fenced: Boolean = false): MetadataBroker = {
MetadataBroker(id, rack.getOrElse(null),
Map(securityProtocol.name -> new Node(id, host, port, rack.getOrElse(null))), fenced)
}
def createBrokerAndEpoch(id: Int, host: String, port: Int, securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT,
epoch: Long = 0): (Broker, Long) = {
(new Broker(id, host, port, ListenerName.forSecurityProtocol(securityProtocol), securityProtocol), epoch)
@ -1805,7 +1792,7 @@ object TestUtils extends Logging { @@ -1805,7 +1792,7 @@ object TestUtils extends Logging {
def assignThrottledPartitionReplicas(adminClient: Admin, allReplicasByPartition: Map[TopicPartition, Seq[Int]]): Unit = {
val throttles = allReplicasByPartition.groupBy(_._1.topic()).map {
case (topic, replicasByPartition) =>
new ConfigResource(ConfigResource.Type.TOPIC, topic) -> Seq(
new ConfigResource(TOPIC, topic) -> Seq(
new AlterConfigOp(new ConfigEntry(LogConfig.LeaderReplicationThrottledReplicasProp, formatReplicaThrottles(replicasByPartition)), AlterConfigOp.OpType.SET),
new AlterConfigOp(new ConfigEntry(LogConfig.FollowerReplicationThrottledReplicasProp, formatReplicaThrottles(replicasByPartition)), AlterConfigOp.OpType.SET)
).asJavaCollection
@ -1816,7 +1803,7 @@ object TestUtils extends Logging { @@ -1816,7 +1803,7 @@ object TestUtils extends Logging {
def removePartitionReplicaThrottles(adminClient: Admin, partitions: Set[TopicPartition]): Unit = {
val throttles = partitions.map {
tp =>
new ConfigResource(ConfigResource.Type.TOPIC, tp.topic()) -> Seq(
new ConfigResource(TOPIC, tp.topic()) -> Seq(
new AlterConfigOp(new ConfigEntry(LogConfig.LeaderReplicationThrottledReplicasProp, ""), AlterConfigOp.OpType.DELETE),
new AlterConfigOp(new ConfigEntry(LogConfig.FollowerReplicationThrottledReplicasProp, ""), AlterConfigOp.OpType.DELETE)
).asJavaCollection

2
jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java

@ -88,7 +88,6 @@ public class CheckpointBench { @@ -88,7 +88,6 @@ public class CheckpointBench {
private LogDirFailureChannel failureChannel;
private LogManager logManager;
private AlterIsrManager alterIsrManager;
private final MockConfigRepository configRepository = new MockConfigRepository();
@SuppressWarnings("deprecation")
@ -131,7 +130,6 @@ public class CheckpointBench { @@ -131,7 +130,6 @@ public class CheckpointBench {
metadataCache,
this.failureChannel,
alterIsrManager,
configRepository,
Option.empty());
replicaManager.startup();

9
metadata/src/main/java/org/apache/kafka/image/ClientQuotasImage.java

@ -18,6 +18,7 @@ @@ -18,6 +18,7 @@
package org.apache.kafka.image;
import org.apache.kafka.common.errors.InvalidRequestException;
import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.message.DescribeClientQuotasRequestData;
import org.apache.kafka.common.message.DescribeClientQuotasResponseData;
import org.apache.kafka.common.message.DescribeClientQuotasResponseData.EntityData;
@ -86,6 +87,11 @@ public final class ClientQuotasImage { @@ -86,6 +87,11 @@ public final class ClientQuotasImage {
throw new InvalidRequestException("Entity type " + component.entityType() +
" cannot appear more than once in the filter.");
}
if (!(component.entityType().equals(IP) || component.entityType().equals(USER) ||
component.entityType().equals(CLIENT_ID))) {
throw new UnsupportedVersionException("Unsupported entity type " +
component.entityType());
}
switch (component.matchType()) {
case MATCH_TYPE_EXACT:
if (component.match() == null) {
@ -141,6 +147,9 @@ public final class ClientQuotasImage { @@ -141,6 +147,9 @@ public final class ClientQuotasImage {
}
}
for (Entry<String, String> entry : exactMatch.entrySet()) {
if (!entity.entries().containsKey(entry.getKey())) {
return false;
}
if (!Objects.equals(entity.entries().get(entry.getKey()), entry.getValue())) {
return false;
}

4
metadata/src/main/java/org/apache/kafka/image/MetadataDelta.java

@ -210,6 +210,10 @@ public final class MetadataDelta { @@ -210,6 +210,10 @@ public final class MetadataDelta {
featuresDelta.replay(record);
}
/**
* Create removal deltas for anything which was in the base image, but which was not
* referenced in the snapshot records we just applied.
*/
public void finishSnapshot() {
if (featuresDelta != null) featuresDelta.finishSnapshot();
if (clusterDelta != null) clusterDelta.finishSnapshot();

Loading…
Cancel
Save