背景
最近想了解一下AutoMQ,其号称能大幅降低成本、提升弹性。
但要想真正理解AutoMQ的创新点,还得先吃透Kafka的核心设计。
所以,我决定从Kafka入手,搞明白它的底层机制,再看AutoMQ是如何在此基础上优化的。
本章分析Kafka如何创建Topic,涉及以下方面:
1)Controller选举:Broker中的Controller角色负责Topic管理;
2)Topic创建:
2-1)Controller如何分配partition副本给不同Broker节点;
2-2)初始ISR列表和Leader如何确定;
2-3)Topic管理涉及哪些ZNode;
名词解释:
1)Broker:Kafka集群中的单个服务器节点,负责接收、存储和转发消息。每个Broker通过唯一ID 标识(如 broker.id=0);
2)Controller:Kafka集群中的一个特殊Broker,负责元数据管理、Topic管理、副本管理等等;
3)Partition:分区,Topic的物理分片,每个分区是一个有序的消息队列;
4)Replica:副本,一个分区至少有一个副本,超出的副本保证数据高可用,副本id就是brokerId;
5)Partition Leader:一个Partition下的一个特殊Replica,负责处理该分区的所有读写请求;
6)ISR:In-Sync Replica,与Leader副本保持同步的副本集合(包括Leader自身);
7)Assignment:一个Topic的分区和副本分配情况,比如分区0的副本id为111和222,即分区0的数据在brokerId=111和222两个节点上各有一份;
注:本文基于Kafka2.6.3,使用ZooKeeper管理元数据,还未引入KRaft,代码容易理解。
一、Controller选举
1-1、概述
Kafka集群中的Controller是一个特殊的broker,负责管理整个集群的元数据和协调各种关键操作,其中就包含了Topic管理。
Controller选举涉及两个重要的ZNode。
controller临时节点,data部分的brokerId即为当前集群的controller。

controller_epoch持久节点,data部分是当前controller的任期。
如果controller下发指令时,其他broker校验epoch不通过,则不会遵循这个controller的指令。

Broker启动,Controller选举简要流程如下,后续当controller下线会重复触发选举流程。

1-2、broker启动
KafkaServer#startup:broker启动有众多步骤,本次讨论两个,broker注册和controller启动。
scala
def startup(): Unit = {
val brokerInfo = createBrokerInfo
val brokerEpoch = zkClient.registerBroker(brokerInfo)
kafkaController = new KafkaController(config, zkClient, time, metrics, brokerInfo, brokerEpoch, tokenManager, threadNamePrefix)
kafkaController.startup()
// ...
}
KafkaZkClient#registerBroker:将当前broker信息注册到zk的/brokers/{brokerId}临时节点,这能让集群中其他broker发现当前节点并与其通讯。数据如:
注:endpoints取自advertised.listeners或listeners配置。
json
{"listener_security_protocol_map":{"PLAINTEXT":"PLAINTEXT"},
"endpoints":["PLAINTEXT://localhost:19092"],
"jmx_port":-1,"port":19092,"host":"localhost","version":4,"timestamp":"xxx"}
scala
def registerBroker(brokerInfo: BrokerInfo): Long = {
val path = brokerInfo.path
val stat = checkedEphemeralCreate(path, brokerInfo.toJsonBytes)
stat.getCzxid
}
KafkaController#startup:controller启动,触发Startup事件,启动ControllerEventManager。
scala
def startup() = {
// ...
// 放入Startup事件=>kafka.controller.KafkaController.processStartup
eventManager.put(Startup)
// 开启controller-event-thread线程
eventManager.start()
}
ControllerEventManager.ControllerEventThread#doWork:
所有事件被放入无界阻塞队列,ControllerEventManager单线程(controller-event-thread线程)处理队列中的事件。
scala
override def doWork(): Unit = {
// 阻塞队列获取事件
val dequeued = queue.take()
dequeued.event match {
case controllerEvent =>
// 处理事件
def process(): Unit = dequeued.process(processor)
rateAndTimeMetrics.get(state) match {
case Some(timer) => timer.time { process() }
case None => process()
}
}
}
}
KafkaController#processStartup:处理Startup事件。
1)KafkaZkClient#registerZNodeChangeHandlerAndCheckExistence:exists注册controller节点监听,如果发生变更由ControllerChangeHandler处理;(忽略)
2)KafkaController#elect:尝试选举controller;
scala
private def processStartup(): Unit = {
// exists /controller注册监听
zkClient.registerZNodeChangeHandlerAndCheckExistence(controllerChangeHandler)
// 尝试选举controller
elect()
}
class ControllerChangeHandler(eventManager: ControllerEventManager) extends ZNodeChangeHandler {
// /controller临时节点被删除,即controller下线,触发Reelect事件
override def handleDeletion(): Unit = eventManager.put(Reelect)
}
// KafkaController#processReelect,处理Reelect事件
private def processReelect(): Unit = {
// 注册/controller监听,如果自己曾经是controller但现在不是,取消controller任务
maybeResign()
// 再次选举controller
elect()
}
1-3、选举
KafkaController#elect:选举controller。
1)查询/controller节点,如果集群中没有controller(activeControllerId=-1),继续选举,否则选举流程结束;
2)KafkaZkClient#registerControllerAndIncrementControllerEpoch,当前broker尝试竞选controller;
3)竞选结果有两种:
3-1)无异常,当前broker成为controller,epoch=controller任期,epochZkVersion=controller_epoch的znode数据版本号,执行onControllerFailover;
3-2)发生ControllerMovedException,其他broker成为controller,执行maybeResign;
scala
private def elect(): Unit = {
// 查询/controller是否存在,不存在返回-1
activeControllerId = zkClient.getControllerId.getOrElse(-1)
// 如果有人已经成为controller直接返回
if (activeControllerId != -1) {
return
}
try {
// 尝试注册为controller,如果失败,抛出ControllerMovedException
val (epoch, epochZkVersion) = zkClient.registerControllerAndIncrementControllerEpoch(config.brokerId)
controllerContext.epoch = epoch
controllerContext.epochZkVersion = epochZkVersion
activeControllerId = config.brokerId
// 如果自己成为controller,注册一些监听,启动一些任务
onControllerFailover()
} catch {
case e: ControllerMovedException =>
// 自己未成为controller,注册/controller监听
// 校验自己是否以前是controller,需要取消一些任务
maybeResign()
}
}
KafkaZkClient#registerControllerAndIncrementControllerEpoch:竞选controller
1)查询/controller_epoch节点,如果不存存在,初始化/controller_epoch,epoch=0,dataVersion=0;
2)controller任期+1;
3)tryCreateControllerZNodeAndIncrementEpoch,通过MultiRequest调用zk,一个是创建controller临时ZNode,二是更新controller_epoch任期为新任期,需要乐观校验controller_epoch的dataVersion与先前版本一致;
4)如果上述所有操作成功,则当前broker成为controller;否则抛出ControllerMovedException异常,是其他broker成为了controller,此时只需要由exists监听/controller做出反应,重新执行elect即可;
scala
def registerControllerAndIncrementControllerEpoch(controllerId: Int): (Int, Int) = {
val timestamp = time.milliseconds()
// 查询/controller_epoch节点,返回当前controller的epoch和/controller_epoch的dataVersion
val (curEpoch, curEpochZkVersion) = getControllerEpoch
.map(e => (e._1, e._2.getVersion))
// 如果/controller_epoch不存在,则创建/controller_epoch节点,初始epoch=0,dataVersion=0
.getOrElse(maybeCreateControllerEpochZNode())
val newControllerEpoch = curEpoch + 1 // controller epoch+1
val expectedControllerEpochZkVersion = curEpochZkVersion
// 如果MultiRequest尝试成为controller失败
def checkControllerAndEpoch(): (Int, Int) = {
// 查询controllerId
val curControllerId = getControllerId.getOrElse(throw new ControllerMovedException())
// 如果controller是自己
// 几乎不可能,意思是controller_epoch发生改变,但是controller仍然是自己
if (controllerId == curControllerId) {
val (epoch, stat) = getControllerEpoch.getOrElse()
if (epoch == newControllerEpoch)
return (newControllerEpoch, stat.getVersion)
}
// 否则,抛出ControllerMovedException异常
throw new ControllerMovedException("Controller moved to another broker. Aborting controller startup procedure")
}
// 尝试成为controller
def tryCreateControllerZNodeAndIncrementEpoch(): (Int, Int) = {
// multi
val response = retryRequestUntilConnected(
MultiRequest(Seq(
// 创建/controller 临时znode data={"version":1,"brokerid":111,"timestamp":"xxx"}
CreateOp(ControllerZNode.path, ControllerZNode.encode(controllerId, timestamp), defaultAcls(ControllerZNode.path), CreateMode.EPHEMERAL),
// 设置/controller_epoch data=controller的epoch,通过期望znode的dataVersion进行验证
SetDataOp(ControllerEpochZNode.path, ControllerEpochZNode.encode(newControllerEpoch), expectedControllerEpochZkVersion)))
)
response.resultCode match {
// 如果/controller存在或/controller_epoch节点version校验不通过,再次校验controller状态
case Code.NODEEXISTS | Code.BADVERSION => checkControllerAndEpoch()
// 其他情况,成功成为controller
case Code.OK =>
val setDataResult = response.zkOpResults(1).rawOpResult.asInstanceOf[SetDataResult]
(newControllerEpoch, setDataResult.getStat.getVersion)
case code => throw KeeperException.create(code)
}
}
tryCreateControllerZNodeAndIncrementEpoch()
}
1-4、成为controller
KafkaController#onControllerFailover:
1)注册znode监听,包括:/brokers/ids集群成员变更、/brokers/topics集群Topic变更等;
2)initializeControllerContext初始化controller上下文;
3)发送元数据更新请求给所有存活broker;
4)其他后续再分析;
scala
private def onControllerFailover(): Unit = {
val childChangeHandlers = Seq(brokerChangeHandler, topicChangeHandler, topicDeletionHandler, logDirEventNotificationHandler,
isrChangeNotificationHandler)
// 子节点监听
childChangeHandlers.foreach(zkClient.registerZNodeChildChangeHandler)
// 初始化controller上下文
initializeControllerContext()
// 发送元数据更新请求给所有存活broker
info("Sending update metadata request")
sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq, Set.empty)
// ...
}
KafkaController#initializeControllerContext:controller从zk中获取集群和topic信息,存储到controllerContext中,与所有broker建立连接(包括自己);
scala
private def initializeControllerContext(): Unit = {
// 查询/brokers/{id},所有broker信息
val curBrokerAndEpochs = zkClient.getAllBrokerAndEpochsInCluster
controllerContext.setLiveBrokers(curBrokerAndEpochs)
// getChildren /brokers/topics,获取所有topic并监听子节点创建
controllerContext.setAllTopics(zkClient.getAllTopicsInCluster(true))
// 对所有/brokers/topics/{topic}注册子节点监听
registerPartitionModificationsHandlers(controllerContext.allTopics.toSeq)
// 查询所有/brokers/topics/{topic},获取topic的partition assignment信息
zkClient.getFullReplicaAssignmentForTopics(controllerContext.allTopics.toSet).foreach {
case (topicPartition, replicaAssignment) =>
controllerContext.updatePartitionFullReplicaAssignment(topicPartition, replicaAssignment)
if (replicaAssignment.isBeingReassigned)
controllerContext.partitionsBeingReassigned.add(topicPartition)
}
controllerContext.partitionLeadershipInfo.clear()
controllerContext.shuttingDownBrokerIds.clear()
// 监听/brokers/ids/{id}
registerBrokerModificationsHandler(controllerContext.liveOrShuttingDownBrokerIds)
// 查询/brokers/topics/{topic}/partitions/{partitionId}/state所有partition信息并缓存
updateLeaderAndIsrCache()
// 与所有broker建立连接
controllerChannelManager.startup()
}
至此controller已经获取到所有集群元数据,并与其他broker建立连接。
AbstractControllerBrokerRequestBatch#sendUpdateMetadataRequests:
controller将controllerContext中的集群信息和partition信息发送给所有存活broker(包括自己)。
scala
private def sendUpdateMetadataRequests(controllerEpoch: Int, stateChangeLog: StateChangeLogger): Unit = {
val liveBrokers = controllerContext.liveOrShuttingDownBrokers.iterator.map { broker =>
// ...
new UpdateMetadataBroker()
.setId(broker.id)
.setEndpoints(endpoints.asJava)
.setRack(broker.rack.orNull)
}.toBuffer
updateMetadataRequestBrokerSet.intersect(controllerContext.liveOrShuttingDownBrokerIds).foreach { broker =>
val brokerEpoch = controllerContext.liveBrokerIdAndEpochs(broker)
val updateMetadataRequestBuilder = new UpdateMetadataRequest.Builder(updateMetadataRequestVersion,
controllerId, controllerEpoch, brokerEpoch, partitionStates.asJava, liveBrokers.asJava)
sendRequest(broker, updateMetadataRequestBuilder, ...)
// ...
}
每个broker维护了一个MetadataCache元数据缓存。
scala
class MetadataCache(brokerId/*自己的brokerId*/: Int) {
private var metadataSnapshot: MetadataSnapshot =
// partition
MetadataSnapshot(partitionStates = mutable.AnyRefMap.empty,
// controllerId
controllerId = None,
// broker
aliveBrokers = mutable.LongMap.empty,
aliveNodes = mutable.LongMap.empty)
}
1-5、竞争controller失败
KafkaController#maybeResign:如果竞争controller失败
1)需要判断自己是否曾经是controller,需要取消监听,这一般是运行过程中发生controller重新选举;
2)如果是刚启动的broker,这里只是再次注册一下监听;
scala
def isActive: Boolean = activeControllerId == config.brokerId/*配置broker.id*/
private def maybeResign(): Unit = {
// 自己曾经是否是controller
val wasActiveBeforeChange = isActive
// exists注册/controller监听
zkClient.registerZNodeChangeHandlerAndCheckExistence(controllerChangeHandler)
// 获取controllerId
activeControllerId = zkClient.getControllerId.getOrElse(-1)
// 自己曾经是controller,但是现在不是了,取消controller所有监听和任务
if (wasActiveBeforeChange && !isActive) {
onControllerResignation()
}
}
private def onControllerResignation(): Unit = {
// 取消zk watch
zkClient.unregisterZNodeChangeHandler(partitionReassignmentHandler.path)
zkClient.unregisterZNodeChildChangeHandler(topicChangeHandler.path)
zkClient.unregisterZNodeChildChangeHandler(topicDeletionHandler.path)
zkClient.unregisterZNodeChildChangeHandler(brokerChangeHandler.path)
controllerContext.resetContext()
// ...
}
二、创建Topic
2-1、概述
当集群处于稳定状态,有controller角色broker,客户端才能创建topic。
创建topic有两种方式:
1)客户端指定分区数量和副本数,由controller自行分配分区副本给不同的broker并下发指令;
2)客户端指定每个分区副本给哪些broker,比如下面0分区的2个副本给brokerId是88和100的broker,controller仅负责下发指令;
java
public static void main(String[] args) throws ExecutionException, InterruptedException {
Properties props = new Properties();
// 随便连接一个节点就行,通过这个节点能拿到集群的元数据
props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
try (AdminClient admin = AdminClient.create(props)) {
// 指定partition数量=3,每个partition的副本数为2
NewTopic topic1 = new NewTopic("topic1", 3, (short) 2);
// 自行分配assignment
// 0分区的副本id为88和100,1分区的副本id为88和89
Map<Integer, List<Integer>> assignments = new HashMap<>();
assignments.put(0, Arrays.asList(88, 100));
assignments.put(1, Arrays.asList(88, 89));
NewTopic topic2 = new NewTopic("topic2", assignments);
CreateTopicsResult result = admin.createTopics(Arrays.asList(topic1, topic2));
KafkaFuture<Void> f = result.all();
f.get();
}
}
注:客户端这边会有两个线程,main是业务线程,kafka-admin-client-thread是io线程,负责发送和接收请求,设置future结果。一般topic管理都有ui页面,这里客户端逻辑仅关注请求响应模型即可。
2-2、client获取元数据
只有controller角色的broker才能接收topic管理请求,所以客户端需要先发现controller。
2-2-1、client发送MetadataRequest
AdminClientRunnable#makeMetadataCall:io线程发现缺失元数据,则发起元数据查询。
客户端发送MetadataRequest给任一broker,不指定topic则获取全量元数据。
java
public class MetadataRequest extends AbstractRequest {
private final MetadataRequestData data;
}
public class MetadataRequestData implements ApiMessage {
private List<MetadataRequestTopic> topics;
}
2-2-2、broker处理MetadataRequest
任一broker都通过MetadataCache 维护了元数据信息(见上面controller选举后元数据通知),可以从内存直接获取并响应MetadataResponse。
注:controller一旦选举出来,client与broker的部分请求就不依赖zk了,即使zk宕机也能正常获取元数据。
KafkaApis#handleTopicMetadataRequest:
scala
def handleTopicMetadataRequest(request: RequestChannel.Request): Unit = {
val metadataRequest = request.body[MetadataRequest]
val requestVersion = request.header.apiVersion
// 存活broker列表
val brokers = metadataCache.getAliveBrokers
// 响应MetadataResponse
sendResponseMaybeThrottle(request, requestThrottleMs =>
MetadataResponse.prepareResponse(
requestThrottleMs,
completeTopicMetadata.asJava,
brokers.flatMap(_.getNode(request.context.listenerName)).asJava,
clusterId,
metadataCache.getControllerId.getOrElse(MetadataResponse.NO_CONTROLLER_ID),
clusterAuthorizedOperations
))
}
2-2-3、client收到MetadataResponse
AdminMetadataManager#update:客户端收到MetadataResponse缓存到AdminMetadataManager#cluster。
元数据包含broker集群信息和topic信息。
java
public class MetadataResponse extends AbstractResponse {
private final MetadataResponseData data;
}
public class MetadataResponseData implements ApiMessage {
private int throttleTimeMs;
// broker列表
private MetadataResponseBrokerCollection brokers;
// 集群id(/cluster/id)
private String clusterId;
// controller角色brokerId
private int controllerId;
// topic列表
private MetadataResponseTopicCollection topics;
private int clusterAuthorizedOperations;
}
broker节点列表包含每个节点的通讯地址。
java
public static class MetadataResponseBrokerCollection extends ImplicitLinkedHashMultiCollection<MetadataResponseBroker> {
}
static public class MetadataResponseBroker implements Message, ImplicitLinkedHashMultiCollection.Element {
private int nodeId;
private String host;
private int port;
private String rack;
}
至此客户端可以发现所有broker节点,并知晓controller角色broker。
2-3、client发送CreateTopicsRequest
client发送CreateTopicsRequest给controller角色broker节点
java
public class CreateTopicsRequest extends AbstractRequest {
private final CreateTopicsRequestData data;
}
public class CreateTopicsRequestData implements ApiMessage {
// topic集合
private CreatableTopicCollection topics;
// request.timeout.ms=30000ms
private int timeoutMs;
private boolean validateOnly;
}
public static class CreatableTopicCollection extends ImplicitLinkedHashMultiCollection<CreatableTopic> {
}
static public class CreatableTopic implements Message, ImplicitLinkedHashMultiCollection.Element {
// topic名
private String name;
// partition数量
private int numPartitions;
// replication副本数量
private short replicationFactor;
// 自定义assignent,指定每个partition的n个副本在哪些brokerId
private CreatableReplicaAssignmentCollection assignments;
private CreateableTopicConfigCollection configs;
}
2-4、controller处理CreateTopicsRequest请求
KafkaApis#handleCreateTopicsRequest
2-4-1、基础校验
如果自己不是controller,返回NOT_CONTROLLER
scala
def handleCreateTopicsRequest(request: RequestChannel.Request): Unit = {
val createTopicsRequest = request.body[CreateTopicsRequest]
val results = new CreatableTopicResultCollection(createTopicsRequest.data.topics.size)
if (!controller.isActive) {
// 自己不是controller,返回NOT_CONTROLLER
createTopicsRequest.data.topics.forEach { topic =>
results.add(new CreatableTopicResult().setName(topic.name).
setErrorCode(Errors.NOT_CONTROLLER.code))
}
sendResponseCallback(results)
}
}
AdminManager#createTopics:如果topic在缓存里已经存在,则返回TopicExistsException。
scala
def createTopics(...): Unit = {
val metadata = toCreate.values.map(topic =>
if (metadataCache.contains(topic.name))
throw new TopicExistsException(s"Topic '${topic.name}' already exists.")
)
}
2-4-2、分配partition副本(assignment)
AdminManager#createTopics:
如果用户未指定assignment,需要controller来分配partition和副本给不同broker;
否则直接使用用户指定的assignment,不会校验brokerId是否存在。
scala
def createTopics(toCreate: Map[String, CreatableTopic],...): Unit = {
// 循环每个topic
val metadata = toCreate.values.map(topic =>
// 默认partition数量=num.partitions=1
val resolvedNumPartitions = if (topic.numPartitions == NO_NUM_PARTITIONS)
defaultNumPartitions else topic.numPartitions
// 默认副本数量=default.replication.factor=1
val resolvedReplicationFactor = if (topic.replicationFactor == NO_REPLICATION_FACTOR)
defaultReplicationFactor else topic.replicationFactor
val assignments = if (topic.assignments().isEmpty) {
// 用户未指定assignment,controller来分配partition和副本给不同broker
AdminUtils.assignReplicasToBrokers(
brokers, resolvedNumPartitions, resolvedReplicationFactor)
} else {
// 如果用户指定了assignment,不会校验指定的brokerId是否存在
val assignments = new mutable.HashMap[Int, Seq[Int]]
topic.assignments.forEach { assignment =>
assignments(assignment.partitionIndex) = assignment.brokerIds.asScala.map(a => a: Int)
}
assignments
})
}
AdminUtils#assignReplicasToBrokers:如果用户指定副本数量大于broker数量,返回InvalidReplicationFactorException。
scala
def assignReplicasToBrokers(brokerMetadatas: Seq[BrokerMetadata],
nPartitions: Int,
replicationFactor: Int,
fixedStartIndex: Int = -1,
startPartitionId: Int = -1): Map[Int, Seq[Int]] = {
// ...
if (replicationFactor > brokerMetadatas.size)
throw new InvalidReplicationFactorException(s"Replication factor: $replicationFactor larger than available brokers: ${brokerMetadatas.size}.")
if (brokerMetadatas.forall(_.rack.isEmpty))
assignReplicasToBrokersRackUnaware(nPartitions, replicationFactor, brokerMetadatas.map(_.id), fixedStartIndex,
startPartitionId)
// ... rack非empty忽略
}
分配策略主要满足两点,具体分配代码见assignReplicasToBrokersRackUnaware。
- 将副本尽量均匀分配在不同broker之间;
- 同一个partition的副本要分布在不同broker之间;
在实现的时候,针对每个partition的第一个副本分配,采取随机brokerId开始RoundRobin,这保证了后面Leader副本均匀分布在不同broker节点上。

2-4-3、调用zk创建topic
AdminZkClient#createTopicWithAssignment:controller调用zk创建topic相关节点。
scala
def createTopicWithAssignment(topic: String,
config: Properties,
partitionReplicaAssignment: Map[Int, Seq[Int]]): Unit = {
// 会先校验一下topic不存在 exist /brokers/topics/{topic}
validateTopicCreate(topic, partitionReplicaAssignment, config)
// set /config/topics/{topic}
zkClient.setOrCreateEntityConfigs(ConfigType.Topic, topic, config)
// set /brokers/topics/{topic}
// 这里如果节点存在,抛出TopicExistsException
writeTopicPartitionAssignment(topic, partitionReplicaAssignment.map { case (k, v) => k -> ReplicaAssignment(v) },
isUpdate = false)
}
创建/config/topics/{topic}节点。
json
{"version":1,"config":{}}
创建/brokers/topics/{topic}节点,data部分就是assignment,partitions的key为partitionId,value为replicaId集合(n个broker的id)。
json
{"version":2,"partitions":{"0":[111,222],"1":[222,111],
"2":[111,222]},"adding_replicas":{},"removing_replicas":{}}
2-4-2、提交DelayedCreatePartitions延迟任务
AdminManager#createTopics最后一步,提交一个DelayedCreatePartitions延迟任务到ExpirationReaper-{brokerId}-topic线程。
scala
class AdminManager {
// ExpirationReaper-{brokerId}-topic线程
private val topicPurgatory =
DelayedOperationPurgatory[DelayedOperation]("topic", config.brokerId)
def createTopics(...): Unit = {
val delayedCreate = new DelayedCreatePartitions(timeout, metadata, this, responseCallback)
val delayedCreateKeys = toCreate.values.map(topic => new TopicKey(topic.name)).toBuffer
// 创建延迟任务,key=topicName,后续topic逻辑完成可以通过key再拿到延迟任务
topicPurgatory.tryCompleteElseWatch(delayedCreate, delayedCreateKeys)
}
}
DelayedCreatePartitions#onComplete:这个延迟任务逻辑就是响应client。
如果创建topic成功,通过key=topicName,移除延迟任务,并主动触发响应client;
如果创建topic超时(客户端请求参数控制超时时间,默认是request.timeout.ms=30000ms),ExpirationReaper线程自动触发。
scala
override def onComplete(): Unit = {
val results = createMetadata.map { metadata =>
if (metadata.error.isSuccess && missingLeaderCount(metadata.topic, metadata.partitions) > 0)
// topic超时leader副本未创建,则返回topic超时
(metadata.topic, new ApiError(Errors.REQUEST_TIMED_OUT, null))
else
// 其他case,如正常响应或其他业务异常
(metadata.topic, metadata.error)
}.toMap
// 响应客户端
responseCallback(results)
}
2-5、controller监听/brokers/topics子节点变更
在第一部分提到,broker成为controller后会监听很多znode,其中就包含/brokers/topics子节点。
2-5-1、收到/brokers/topics子节点变更通知
KafkaController#processTopicChange:
1)getChildren /brokers/topics获取所有topics并注册监听;
2)计算新增newTopics;
3)对所有新增topic注册/brokers/topics/{topic}监听;
4)将新增topic的assignment(每个partition下是哪些replica)更新到内存controllerContext;
5)执行onNewPartitionCreation,传入新增partition;
scala
private def processTopicChange(): Unit = {
if (!isActive) return // 如果自己已经不是controller了,不处理
// 注册/brokers/topics child类型监听,获取topic列表
val topics = zkClient.getAllTopicsInCluster(true)
val newTopics = topics -- controllerContext.allTopics
val deletedTopics = controllerContext.allTopics.diff(topics)
controllerContext.setAllTopics(topics)
// 注册/brokers/topics/{topic}监听
registerPartitionModificationsHandlers(newTopics.toSeq)
// 循环所有新增topic,调用zk获取topic的assignment信息
val addedPartitionReplicaAssignment = zkClient.getFullReplicaAssignmentForTopics(newTopics)
deletedTopics.foreach(controllerContext.removeTopic)
// 将topic的assignment信息更新到内存
addedPartitionReplicaAssignment.foreach {
case (topicAndPartition, newReplicaAssignment) => controllerContext.updatePartitionFullReplicaAssignment(topicAndPartition, newReplicaAssignment)
}
if (addedPartitionReplicaAssignment.nonEmpty)
onNewPartitionCreation(addedPartitionReplicaAssignment.keySet)
}
KafkaController#onNewPartitionCreation:处理新增partition。
scala
private def onNewPartitionCreation(newPartitions: Set[TopicPartition]): Unit = {
// partition状态NewPartition 更新内存 partition->NewPartition
partitionStateMachine.handleStateChanges(newPartitions.toSeq, NewPartition)
// replica状态NewReplica 更新内存replica->NewReplica
replicaStateMachine.handleStateChanges(controllerContext.replicasForPartition(newPartitions).toSeq, NewReplica)
// partition状态OnlinePartition
// 初始化isr列表、选leader、zk持久化/kafka/brokers/topics/{topic}/partitions及其子节点
// 发送LeaderAndIsrRequest和UpdateMetadataRequest
partitionStateMachine.handleStateChanges(
newPartitions.toSeq,
OnlinePartition,
Some(OfflinePartitionLeaderElectionStrategy(false))
)
// replica状态OnlineReplica
replicaStateMachine.handleStateChanges(controllerContext.replicasForPartition(newPartitions).toSeq, OnlineReplica)
}
broker有partition和replica两个状态机,多处代码都会用到这两个状态变更。
ZkPartitionStateMachine#handleStateChanges:处理partition状态变更。
scala
override def handleStateChanges(
partitions: Seq[TopicPartition],
targetState: PartitionState,
partitionLeaderElectionStrategyOpt: Option[PartitionLeaderElectionStrategy]
): Map[TopicPartition, Either[Throwable, LeaderAndIsr]] = {
if (partitions.nonEmpty) {
try {
// new一个请求批次
controllerBrokerRequestBatch.newBatch()
// 做状态变更,可能加一些需要发送的请求到批次里
val result = doHandleStateChanges(
partitions,
targetState,
partitionLeaderElectionStrategyOpt
)
// 发送这个批次中的请求给brokers
controllerBrokerRequestBatch.sendRequestsToBrokers(controllerContext.epoch)
result
} catch {
case e: ControllerMovedException =>
error(s"Controller moved to another broker when moving some partitions to $targetState state", e)
throw e
case e: Throwable =>
error(s"Error while moving some partitions to $targetState state", e)
partitions.iterator.map(_ -> Left(e)).toMap
}
} else {
Map.empty
}
}
ZkReplicaStateMachine#handleStateChanges:处理replica状态变更。
scala
override def handleStateChanges(replicas: Seq[PartitionAndReplica], targetState: ReplicaState): Unit = {
if (replicas.nonEmpty) {
try {
// 新请求批次
controllerBrokerRequestBatch.newBatch()
// replica变更
replicas.groupBy(_.replica).foreach { case (replicaId, replicas) =>
// replicaId(brokerId) -> 入参PartitionAndReplica集合
doHandleStateChanges(replicaId, replicas, targetState)
}
// 发送这批请求给brokers
controllerBrokerRequestBatch.sendRequestsToBrokers(controllerContext.epoch)
} catch {
case e: ControllerMovedException =>
error(s"Controller moved to another broker when moving some replicas to $targetState state", e)
throw e
case e: Throwable => error(s"Error while moving some replicas to $targetState state", e)
}
}
}
2-5-2、partition状态Online
ZkPartitionStateMachine#initializeLeaderAndIsrForPartitions:初始化每个partition状态并持久化
1)过滤出有存活副本的partition,仅处理这些partition;
2)partition初始isr列表=存活副本列表;
3)partition初始isr列表的第一个副本成为partition leader;
4)将partition的isr列表+leader+leader_epoch(0)+controller_epoch持久化到zk;
5)持久化成功,缓存leaderIsrAndControllerEpoch到controllerContext,构建LeaderAndIsrRequest 和UpdateMetadataRequest请求缓存;
scala
private def initializeLeaderAndIsrForPartitions(partitions: Seq[TopicPartition]): Seq[TopicPartition] = {
val successfulInitializations = mutable.Buffer.empty[TopicPartition]
// partitionId -> replicaId(brokerId)集合
val replicasPerPartition = ...
// partition -> 存活的replicaId(brokerId)集合
val liveReplicasPerPartition = replicasPerPartition.map { case (partition, replicas) =>
val liveReplicasForPartition = replicas.filter(replica => controllerContext.isReplicaOnline(replica, partition))
partition -> liveReplicasForPartition
}
// partitionsWithLiveReplicas=有存活副本(broker)的partition集合
val (partitionsWithoutLiveReplicas, partitionsWithLiveReplicas) = liveReplicasPerPartition.partition { case (_, liveReplicas) => liveReplicas.isEmpty }
// 计算partition状态
val leaderIsrAndControllerEpochs = partitionsWithLiveReplicas.map { case (partition, liveReplicas) =>
// brokerId中第一个成为leader,初始LeaderAndIsr的epoch为0
val leaderAndIsr = LeaderAndIsr(liveReplicas.head, liveReplicas.toList)
// 再拼接上controller的epoch
val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerContext.epoch)
partition -> leaderIsrAndControllerEpoch
}.toMap
// 调用zk创建partition节点
// 所有创建请求通过zk的multi,会先校验/controller_epoch节点等于当前controller的epoch
// 如果校验不通过,抛出ControllerMovedException
val createResponses = try {
zkClient.createTopicPartitionStatesRaw(leaderIsrAndControllerEpochs, controllerContext.epochZkVersion)
} catch {
case e: ControllerMovedException =>
throw e
case e: Exception =>
Seq.empty
}
createResponses.foreach { createResponse =>
val code = createResponse.resultCode
val partition = createResponse.ctx.get.asInstanceOf[TopicPartition]
val leaderIsrAndControllerEpoch = leaderIsrAndControllerEpochs(partition)
if (code == Code.OK) {
// 缓存partition的状态
controllerContext.partitionLeadershipInfo.put(partition, leaderIsrAndControllerEpoch)
// 构建LeaderAndIsrRequest和UpdateMetadataRequest请求并缓存
controllerBrokerRequestBatch.addLeaderAndIsrRequestForBrokers(leaderIsrAndControllerEpoch.leaderAndIsr.isr,
partition, leaderIsrAndControllerEpoch, controllerContext.partitionFullReplicaAssignment(partition), isNew = true)
successfulInitializations += partition
} else {
logFailedStateChange(partition, NewPartition, OnlinePartition, code)
}
}
successfulInitializations
}
KafkaZkClient#createTopicPartitionStatesRaw:持久化partition状态。
所有创建请求通过zk的multi,会校验/controller_epoch节点等于当前controller的epoch,如果校验不通过,抛出ControllerMovedException流程结束。
scala
def createTopicPartitionStatesRaw(leaderIsrAndControllerEpochs: Map[TopicPartition, LeaderIsrAndControllerEpoch], expectedControllerEpochZkVersion: Int): Seq[CreateResponse] = {
// 创建/brokers/topics/{topic}/partitions节点,无数据
createTopicPartitions(leaderIsrAndControllerEpochs.keys.map(_.topic).toSet.toSeq, expectedControllerEpochZkVersion)
// 创建/brokers/topics/{topic}/partitions/{partitionId}节点,无数据
createTopicPartition(leaderIsrAndControllerEpochs.keys.toSeq, expectedControllerEpochZkVersion)
// 创建/brokers/topics/{topic}/partitions/{partitionId}/state节点,数据为LeaderIsrAndControllerEpoch
val createRequests = leaderIsrAndControllerEpochs.map { case (partition, leaderIsrAndControllerEpoch) =>
val path = TopicPartitionStateZNode.path(partition)
val data = TopicPartitionStateZNode.encode(leaderIsrAndControllerEpoch)
CreateRequest(path, data, defaultAcls(path), CreateMode.PERSISTENT, Some(partition))
}
retryRequestsUntilConnected(createRequests.toSeq, expectedControllerEpochZkVersion)
}
partition状态持久化在 /brokers/topics/{topic}/partitions/{partitionId}/state节点,数据为:
json
{"controller_epoch":9,"leader":111,"version":1,"leader_epoch":5,"isr":[111,222]}
2-5-3、发送LeaderAndIsrRequest
AbstractControllerBrokerRequestBatch#addLeaderAndIsrRequestForBrokers:
上面一步,对于持久化zk成功的partition,构建leaderAndIsrRequestMap。
map的key是isr列表中的副本id(brokerId),value是partition→partition状态。
scala
// brokerId(isr列表中的) -> partition -> partition状态
val leaderAndIsrRequestMap = mutable.Map.empty[Int,
mutable.Map[TopicPartition, LeaderAndIsrPartitionState]]
def addLeaderAndIsrRequestForBrokers(brokerIds: Seq[Int],
topicPartition: TopicPartition,
leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch,
replicaAssignment: ReplicaAssignment,
isNew: Boolean): Unit = {
// 构建leaderAndIsrRequestMap
brokerIds.filter(_ >= 0).foreach { brokerId =>
val result = leaderAndIsrRequestMap.getOrElseUpdate(brokerId, mutable.Map.empty)
val alreadyNew = result.get(topicPartition).exists(_.isNew)
val leaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr
result.put(topicPartition, new LeaderAndIsrPartitionState()
.setTopicName(topicPartition.topic)
.setPartitionIndex(topicPartition.partition)
.setControllerEpoch(leaderIsrAndControllerEpoch.controllerEpoch)
.setLeader(leaderAndIsr.leader)
.setLeaderEpoch(leaderAndIsr.leaderEpoch)
.setIsr(leaderAndIsr.isr.map(Integer.valueOf).asJava)
.setZkVersion(leaderAndIsr.zkVersion)
.setReplicas(replicaAssignment.replicas.map(Integer.valueOf).asJava)
.setAddingReplicas(replicaAssignment.addingReplicas.map(Integer.valueOf).asJava)
.setRemovingReplicas(replicaAssignment.removingReplicas.map(Integer.valueOf).asJava)
.setIsNew(isNew || alreadyNew))
}
// 构建UpdateMetadataRequest
addUpdateMetadataRequestForBrokers(controllerContext.liveOrShuttingDownBrokerIds.toSeq, Set(topicPartition))
}
AbstractControllerBrokerRequestBatch#sendLeaderAndIsrRequest:
对于本次的partitions变更,循环所有isr列表中的broker节点,发送LeaderAndIsrRequest请求。
比如本次新增partition=0/1/2,其中partition=0和1的isr列表中有brokerId=1,则仅发送0和1的partition状态给broker1。
scala
private def sendLeaderAndIsrRequest(controllerEpoch: Int, stateChangeLog: StateChangeLogger): Unit = {
val leaderAndIsrRequestVersion: Short = 4
// 循环每个broker,给每个broker发送LeaderAndIsrRequest请求
leaderAndIsrRequestMap.foreach { case (broker/*brokerId*/, leaderAndIsrPartitionStates/*partition,state*/) =>
if (controllerContext.liveOrShuttingDownBrokerIds.contains(broker)) {
// 计算哪些partition对于当前broker变成了leader
val numBecomeLeaders = leaderAndIsrPartitionStates.count { case (topicPartition, state) =>
val isBecomeLeader = broker == state.leader
val typeOfRequest =
if (isBecomeLeader) "become-leader"
else "become-follower"
isBecomeLeader
}
// 找到当前broker分配到的partition的所有leader节点
val leaderIds = leaderAndIsrPartitionStates.map(_._2.leader).toSet
val leaders = controllerContext.liveOrShuttingDownBrokers.filter(b => leaderIds.contains(b.id)).map {
_.node(config.interBrokerListenerName)
}
val brokerEpoch = controllerContext.liveBrokerIdAndEpochs(broker)
// 发送LeaderAndIsrRequest请求给目标broker
val leaderAndIsrRequestBuilder = new LeaderAndIsrRequest.Builder(leaderAndIsrRequestVersion, controllerId,
controllerEpoch, brokerEpoch, leaderAndIsrPartitionStates.values.toBuffer.asJava, leaders.asJava)
sendRequest(broker, leaderAndIsrRequestBuilder, (r: AbstractResponse) => {
val leaderAndIsrResponse = r.asInstanceOf[LeaderAndIsrResponse]
sendEvent(LeaderAndIsrResponseReceived(leaderAndIsrResponse, broker))
})
}
}
leaderAndIsrRequestMap.clear()
}
LeaderAndIsrRequest包含目标broker相关的副本情况,如副本列表、isr列表、leader副本。
java
public class LeaderAndIsrRequest extends AbstractControlRequest {
public static class Builder extends AbstractControlRequest.Builder<LeaderAndIsrRequest> {
private final List<LeaderAndIsrPartitionState> partitionStates;
private final Collection<Node> liveLeaders;
public Builder(short version, int controllerId, int controllerEpoch, long brokerEpoch,
List<LeaderAndIsrPartitionState> partitionStates, Collection<Node> liveLeaders) {
super(ApiKeys.LEADER_AND_ISR, version, controllerId, controllerEpoch, brokerEpoch);
this.partitionStates = partitionStates;
this.liveLeaders = liveLeaders;
}
}
static public class LeaderAndIsrPartitionState implements Message {
private String topicName;
private int partitionIndex;
private int controllerEpoch;
private int leader;
private int leaderEpoch;
private List<Integer> isr;
private int zkVersion;
private List<Integer> replicas;
private List<Integer> addingReplicas;
private List<Integer> removingReplicas;
private boolean isNew;
}
}
2-5-4、发送UpdateMetadataRequest
AbstractControllerBrokerRequestBatch#sendUpdateMetadataRequests:
只要partition状态有变更,controller都会发送UpdateMetadataRequest给所有在线的broker,内容是发生变更的partition的状态(leader、isr列表、整体副本列表)。
scala
private def sendUpdateMetadataRequests(controllerEpoch: Int, stateChangeLog: StateChangeLogger): Unit = {
// 上面zk持久化partition状态后,将需要更新的元数据缓存在这里
val partitionStates = updateMetadataRequestPartitionInfoMap.values.toBuffer
val updateMetadataRequestVersion: Short = 6
// 存活的broker
val liveBrokers = controllerContext.liveOrShuttingDownBrokers.iterator.map { broker => ...
}.toBuffer
// 循环存活broker发送UpdateMetadataRequest
updateMetadataRequestBrokerSet.intersect(controllerContext.liveOrShuttingDownBrokerIds).foreach { broker =>
val brokerEpoch = controllerContext.liveBrokerIdAndEpochs(broker)
val updateMetadataRequestBuilder = new UpdateMetadataRequest.Builder(updateMetadataRequestVersion,
controllerId, controllerEpoch, brokerEpoch, partitionStates.asJava, liveBrokers.asJava)
sendRequest(broker, updateMetadataRequestBuilder, (r: AbstractResponse) => {
val updateMetadataResponse = r.asInstanceOf[UpdateMetadataResponse]
sendEvent(UpdateMetadataResponseReceived(updateMetadataResponse, broker))
})
}
// 清空待发送的UpdateMetadataRequest数据
updateMetadataRequestBrokerSet.clear()
updateMetadataRequestPartitionInfoMap.clear()
}
2-6、broker处理LeaderAndIsrRequest
kafka.server.ReplicaManager#becomeLeaderOrFollower:isr列表中的broker处理LeaderAndIsrRequest请求。
scala
def becomeLeaderOrFollower(correlationId: Int,
leaderAndIsrRequest: LeaderAndIsrRequest,
onLeadershipChange: (Iterable[Partition], Iterable[Partition]) => Unit): LeaderAndIsrResponse = {
replicaStateChangeLock synchronized {
}
}
1)校验下发配置的controller的epoch大于等于当前内存的controller的epoch;
scala
if (leaderAndIsrRequest.controllerEpoch < controllerEpoch) {
leaderAndIsrRequest.getErrorResponse(0, Errors.STALE_CONTROLLER_EPOCH.exception)
}
2)内存allPartitions(map)保存partition;
scala
// 2. 内存创建partition
requestPartitionStates.foreach { partitionState =>
val topicPartition = new TopicPartition(partitionState.topicName, partitionState.partitionIndex)
val partitionOpt = getPartition(topicPartition) match {
case HostedPartition.Offline =>
responseMap.put(topicPartition, Errors.KAFKA_STORAGE_ERROR)
None
case HostedPartition.Online(partition) =>
Some(partition)
case HostedPartition.None =>
// 新建partition,内存存储
val partition = Partition(topicPartition, time, this)
allPartitions.putIfNotExists(topicPartition, HostedPartition.Online(partition))
Some(partition)
}
3)循环所有下发的partition,只有partition的leader的epoch大于内存的partition的leader的epoch才能进行后续操作,新增topic时这一定是通过的,一开始内存里的partitionState.leaderEpoch是-1;
scala
// 3. 校验partition leader的epoch,需要大于当前节点看到的leader的epoch,才能做其他操作
val partitionStates = new mutable.HashMap[Partition, LeaderAndIsrPartitionState]()
partitionOpt.foreach { partition =>
val currentLeaderEpoch = partition.getLeaderEpoch
val requestLeaderEpoch = partitionState.leaderEpoch
if (requestLeaderEpoch > currentLeaderEpoch) {
partitionStates.put(partition, partitionState)
} else if (requestLeaderEpoch < currentLeaderEpoch) {
responseMap.put(topicPartition, Errors.STALE_CONTROLLER_EPOCH)
} else {
responseMap.put(topicPartition, Errors.STALE_CONTROLLER_EPOCH)
}
}
4)区分哪些partition成为leader,哪些成为follower,走不同逻辑;
scala
// 4. 根据partition状态,成为leader或follower,会创建log
// 当前broker对于这些partition成为leader
val partitionsToBeLeader = partitionStates.filter { case (_, partitionState) =>
partitionState.leader == localBrokerId
}
// 当前broker对于这些partition成为follower
val partitionsToBeFollower = partitionStates.filter { case (k, _) => !partitionsToBeLeader.contains(k) }
// 成为leader的partition,调用makeLeaders
val partitionsBecomeLeader = if (partitionsToBeLeader.nonEmpty)
makeLeaders(controllerId, controllerEpoch, partitionsToBeLeader, correlationId, responseMap,
highWatermarkCheckpoints)
else
Set.empty[Partition]
// 成为follower的partition,调用makeFollowers
val partitionsBecomeFollower = if (partitionsToBeFollower.nonEmpty)
makeFollowers(controllerId, controllerEpoch, partitionsToBeFollower, correlationId, responseMap,
highWatermarkCheckpoints)
else
Set.empty[Partition]
// ...
无论是成为leader还是follower,最终如果数据log不存在,都会创建log
LogManager#getOrCreateLog:创建log,包括partition目录即{log.dirs}/{topic}-{partition},每个partition目录包含初始的segment(.log/.index/.timeindex文件)。
scala
def getOrCreateLog(topicPartition: TopicPartition, loadConfig: () => LogConfig, isNew: Boolean = false, isFuture: Boolean = false): Log = {
logCreationOrDeletionLock synchronized {
getLog(topicPartition, isFuture).getOrElse {
// ...
// 创建log
val log = Log(
dir = logDir,
config = config,
logStartOffset = 0L,
recoveryPoint = 0L,
maxProducerIdExpirationMs = maxPidExpirationMs,
producerIdExpirationCheckIntervalMs = LogManager.ProducerIdExpirationCheckIntervalMs,
scheduler = scheduler,
time = time,
brokerTopicStats = brokerTopicStats,
logDirFailureChannel = logDirFailureChannel)
// ...
log
}
}
}
2-7、broker处理UpdateMetadataRequest
所有存活broker收到UpdateMetadataRequest。
KafkaApis#handleUpdateMetadataRequest:
1)校验请求中的broker的epoch是否小于当前broker的epoch;
这个broker的epoch就是/brokers/ids/{brokerId}临时znode的czxid,即创建zk事务id,如果controller请求中的broker epoch比这个小,可能是当前broker刚重启过,返回STALEBROKEREPOCH。
scala
def handleUpdateMetadataRequest(request: RequestChannel.Request): Unit = {
val correlationId = request.header.correlationId
val updateMetadataRequest = request.body[UpdateMetadataRequest]
// 请求中的broker的epoch小于当前broker的epoch
if (isBrokerEpochStale(updateMetadataRequest.brokerEpoch)) {
// When the broker restarts very quickly, it is possible for this broker to receive request intended
// for its previous generation so the broker should skip the stale request.
sendResponseExemptThrottle(request,
new UpdateMetadataResponse(new UpdateMetadataResponseData().setErrorCode(Errors.STALE_BROKER_EPOCH.code)))
}
// ...
}
private def isBrokerEpochStale(brokerEpochInRequest: Long): Boolean = {
if (brokerEpochInRequest == AbstractControlRequest.UNKNOWN_BROKER_EPOCH) false
else {
brokerEpochInRequest < controller.brokerEpoch(当前broker的epoch)
}
}
2)更新内存中的MetadataCache.metadataSnapshot
scala
// 更新内存中的MetadataCache.metadataSnapshot
val deletedPartitions = replicaManager.maybeUpdateMetadataCache(correlationId, updateMetadataRequest)
3)如果当前broker节点有hold住partition相关topic的请求,比如创建topic,则触发创建topic响应
scala
// 如果当前broker节点有hold住partition相关topic的请求,比如创建topic,则触发创建topic响应
if (adminManager.hasDelayedTopicOperations) {
updateMetadataRequest.partitionStates.forEach { partitionState =>
adminManager.tryCompleteDelayedTopicOperations(partitionState.topicName)
}
}
// AdminManager
def tryCompleteDelayedTopicOperations(topic: String): Unit = {
val key = TopicKey(topic)
// 这里根据topicName这个key,可以找到之前提交的延迟任务,触发客户端创建topic响应
val completed = topicPurgatory.checkAndComplete(key)
}
总结
Controller选举
Controller角色Broker具备Topic管理能力。
Controller选举通过竞争controller临时znode完成,这个znode保存了controller节点的brokerId。
此外controller_epoch节点保存了controller节点的任期epoch,如果broker节点发现请求中(比如LeaderAndIsrRequest)的controller_epoch比当前元数据中的controller_epoch小,则不会接收这种控制请求。
当Controller选举完成后:
- Controller下发UpdateMetadataRequest,每个broker节点缓存MetadataCache,都能够给客户端提供元数据;
- Controller监听了众多zk的znode,其中包含/brokers/topics子节点监听,为后续创建Topic做铺垫;
Topic创建
Topic创建流程:
1)client,调用任意Broker获取元数据,其中包含controller节点信息;
2)client,调用controller创建Topic;
3)controller,第一步,执行assignment。
分配partition的副本到不同broker上,创建 /brokers/topics/{topic} 节点持久化assignment。
json
{"version":2,"partitions":{"0":[222,111],"1":[111,222]},
"adding_replicas":{},"removing_replicas":{}}
提交一个延迟任务,key=topic,如果客户端超时时间内未完成topic创建,则响应客户端超时。
4)controller,第二步,监听 /brokers/topics子节点变更。
查询/brokers/topics/{topic},将topic的assignment缓存到内存controllerContext;
新增partition,初始化partition leader和isr列表。assignment中的在线broker进入isr列表,isr列表中第一个broker成为leader;
创建 /brokers/topics/{topic}/partitions/{partitionId}/state节点持久化partition状态LeaderAndIsr;
json
{"controller_epoch":60,"leader":222,"version":1,"leader_epoch":16,"isr":[111,222]}
缓存LeaderAndIsr到内存controllerContext;
发送LeaderAndIsrRequest给所有isr列表中的broker,内容就是LeaderAndIsr;
发送UpdateMetadataRequest给所有存活的broker,内容是发生变更的partition的状态(leader、isr列表、整体副本列表);
5)brokers,处理LeaderAndIsrRequest。
brokers收到controller的控制请求,需要校验:1、controller任期大于等于当前broker所见的controller任期;2、partition leader任期大于当前所见leader任期;
根据LeaderAndIsr中partition的leader和自己的brokerId比较,确定自己成为partition的leader还是follower,走不同逻辑,最终都会创建partition目录{log.dirs}/{topic}-{partition},每个partition目录包含初始的segment(.log/.index/.timeindex文件)。
6)brokers,处理UpdateMetadataRequest。
当partition状态变更,每个broker都会收到controller下发的元数据更新请求。
broker更新内存中的元数据缓存-MetadataCache.metadataSnapshot。
检查是否有key=topic的topic创建延迟任务,如果有则响应客户端topic创建成功,至此流程结束。