从源码理解kafka-1.0.1对消费组元数据信息的管理维护
1. 概要
早期版本的kafka中,消费组(group)和消费偏移量(offset)(后续统称GroupMetadata)都是保存在zookeeper上的。然而,zk并不擅长高频读写,将需要频繁提交的offset放在zk上维护不是一个恰当的选择。所以,kafka在0.8.2版本以后,将对于GroupMetadata的维护放在了一个特殊的topic:__consumer_offsets 上面维护。
既然,将groupMetadata放在topic上维护,就涉及到基于topic存储信息的增删查改。
这就引申出了以下几个问题:
1. 如何通过topic实现删改的语义?
2. 如何高效的定位查询特定的group
3. 如果解决分布式情况下,同一个消费组中,并发提交相同topic-partition中的offset的问题?
本文将针对上面几个问题,深入解析新版本的kafka对于消费组元数据信息的维护。
2. 消费组模型
先对基本模型有个宏观的认识,可以更好的理解代码细节。
2.1. __consumer_offsets的存储结构
__consumer_offsets是一个内部topic,和其他的topic消息一样,以key-value的形式,存储根据partition分布在不同broker的磁盘目录。
存储的结构分为两种:
-
offset类型的record(下称offset-record)
指定offset: key: [group,topic,partition] value:[offset, metadata, commit_timestamp, expire_timestamp] 或者value: [offset, metadata, timestamp] (旧版本的,1.0.0已经改用上述版本)
offset-key结构:
字段 group topic partition 含义 消费组Id topic名称 partitionId 示例 platform_intimacy_level user.room.online.heartbeat 1 offset-value结构:
字段 offset metadata commit_timestamp expire_timestamp 含义 偏移量 用于事物的METADATA 提交时间 过期时间 示例 2494848 NO_METADATA 1641287873819 1641374273819
示例:
key:[platform_intimacy_level,user.room.online.heartbeat,1]
value: [OffsetMetadata[2494848,NO_METADATA],CommitTime 1641287873819,ExpirationTime 1641374273819]
-
group类型的record(下称group-record)
用于指定groupid,key:[group],value:[protocol_type, generation, protocol, leader, members] generation
group-key结构:
字段 group 含义 消费组Id 示例 platform_intimacy_level group-value结构:
字段 protocol_type generation protocol leaderId members 含义 协议 代际(每当有消费成员变化的时候,就会+1) 协议 消费组的leader 成员列表 示例 consumer 16424 range consumer-1-7da8bed4-07c7-446d-b2c2-d0d3142a3994 见下面member结构 member的结构:
字段 memberId groupId 含义 成员id(客户端id+uuid) 消费组id 示例 consumer-1-7da8bed4-07c7-446d-b2c2-d0d3142a3994 platform_intimacy_level clientId clientHost consumer-1 /10.246.100.162 rebalanceTimeoutMs sessionTimeoutMs protocolType supportedProtocols 重平衡超时时间 session超时时间(用于心跳检测) 协议类型 支持的协议类型列表 10000 10000 consumer List(range)
示例: key: platform_intimacy_level value: GroupMetadata(groupId=platform_intimacy_level, generation=16424, protocolType=Some(consumer), currentState=Stable, members=Map(consumer-1-7da8bed4-07c7-446d-b2c2-d0d3142a3994 -> MemberMetadata(memberId=consumer-1-7da8bed4-07c7-446d-b2c2-d0d3142a3994, clientId=consumer-1, clientHost=/10.246.100.162, sessionTimeoutMs=10000, rebalanceTimeoutMs=10000, supportedProtocols=List(range), ), consumer-1-46ae9344-4925-4c83-a6aa-1ed9f3c06980 -> MemberMetadata(memberId=consumer-1-46ae9344-4925-4c83-a6aa-1ed9f3c06980, clientId=consumer-1, clientHost=/10.17.8.55, sessionTimeoutMs=10000, rebalanceTimeoutMs=10000, supportedProtocols=List(range), )))
2.2. 消费组的操作模型
__consumer_offsets是一个内部topic,和其他topic一样,每条record都是顺序存储在各个partition中的。那么,它是如何在这种结构下,实现更新和删除操作的呢?
首先,它会用groupid的哈希值,对 __consumer_offsets 的partitions数取模,得到groupid对应的partitionId。每个groupid在 __consumer_offsets 上对应唯一一个partition,计算公式:hash(groupId)%partitionNumOf(“__consumer_offsets”)。这样就保证了每条record的有序性,也保证了groupId在不同broker上的分布(当然,每个group的提交次数不同,并不能保证均衡)。
其次,它的每条record其实就和binlog类似,每一条都代表一个操作事件。在kafka上这个操作相对简化,如果每条record存在value,那么这个value就是它对应的key的插入/更新(upsert)。如果record的value是null,那么等同于对key值的删除操作,如图:
对于group-record,每个groupId就是它的key值,按顺序扫描partition文件,在经历一系列的upsert和delete后,最终得到的结果就是当前group的成员分配结果。
对于offset-record,group+topic+partition的组合是它的key值,按顺序扫描partition文件,在经历一系列的upsert和delete后,最终得到的结果就是当前groupId在topic-partition中的消费偏移量。
每个broker中都会有个GroupMetadataManager对象,用于缓存/操作group信息。一般在broker启动或者Partition的Leader发生改变或者发生ISR的时候,broker会将属于本地的group信息提前加载缓存,保证查询相应的时效性。每次更新group信息也会先落盘到topic,成功后再更新缓存。
3. 源码解读
下面将利用代码和注释的方式,对group信息的维护细节进行解读。
3.1. 消费组的查询
3.1.1. 列出消费组信息
ApiKey:DESCRIBE_GROUPS
handle-method:KafkaApis#handleDescribeGroupRequest()
3.1.1.1. KafkaApis#handleDescribeGroupRequest()
入口函数,主要看GroupCoordinator#handleDescribeGroup() 方法,其他都是封装操作。
def handleDescribeGroupRequest(request: RequestChannel.Request) {
val describeRequest = request.body[DescribeGroupsRequest]
val groups = describeRequest.groupIds.asScala.map { groupId =>
if (!authorize(request.session, Describe, new Resource(Group, groupId))) {
groupId -> DescribeGroupsResponse.GroupMetadata.forError(Errors.GROUP_AUTHORIZATION_FAILED)
} else {
//核心部分:通过groupid获取消费信息
val (error, summary) = groupCoordinator.handleDescribeGroup(groupId)
//序列化消费组成员信息成二进制
val members = summary.members.map { member =>
val metadata = ByteBuffer.wrap(member.metadata)
val assignment = ByteBuffer.wrap(member.assignment)
new DescribeGroupsResponse.GroupMember(member.memberId, member.clientId, member.clientHost, metadata, assignment)
}
//将消费组信息封装Response消息
groupId -> new DescribeGroupsResponse.GroupMetadata(error, summary.state, summary.protocolType,
summary.protocol, members.asJava)
}
}.toMap
//反馈Response给请求者
sendResponseMaybeThrottle(request, requestThrottleMs => new DescribeGroupsResponse(requestThrottleMs, groups.asJava))
}
3.1.1.2. GroupCoordinator#handleDescribeGroup()
该方法对无法获取GroupMetadata的情况做了反馈。
真正获取GroupMetadata需要查看GroupMetadataManager#getGroup() 方法
def handleDescribeGroup(groupId: String): (Errors, GroupSummary) = {
if (!isActive.get) {
//当前broker是否启动
(Errors.COORDINATOR_NOT_AVAILABLE, GroupCoordinator.EmptyGroup)
} else if (!isCoordinatorForGroup(groupId)) {
//当前broker是否是消费组应该分配的Coordinator
(Errors.NOT_COORDINATOR, GroupCoordinator.EmptyGroup)
} else if (isCoordinatorLoadInProgress(groupId)) {
//当消费的topic存在partition的易主操作,就会触发group信息的重新加载,处于重加载状态中时,则触发当前情况
(Errors.COORDINATOR_LOAD_IN_PROGRESS, GroupCoordinator.EmptyGroup)
} else {
//核心
groupManager.getGroup(groupId) match {
case None => (Errors.NONE, GroupCoordinator.DeadGroup)
case Some(group) =>
group.inLock {
(Errors.NONE, group.summary)
}
}
}
}
3.1.1.3. GroupMetadataManager#getGroup()
可以看出最终查询GroupMetadata只是根据groupid,从缓存中获取的。其中Pool类本质只是一个ConcurrentMap对象。这也是获取消费组信息速度快的原因。
private val groupMetadataCache = new Pool[String, GroupMetadata]
/**
* Get the group associated with the given groupId, or null if not found
*/
def getGroup(groupId: String): Option[GroupMetadata] = {
Option(groupMetadataCache.get(groupId))
}
3.1.2. 列出topic-partition的offset信息
ApiKey:OFFSET_FETCH
handle-method:KafkaApis#handleOffsetFetchRequest()
3.1.2.1. GroupMetadataManager#getOffsets()
前面的方法忽略,直接看核心方法
/**
* The most important guarantee that this API provides is that it should never return a stale offset. i.e., it either
* returns the current offset or it begins to sync the cache from the log (and returns an error code).
*/
def getOffsets(groupId: String, topicPartitionsOpt: Option[Seq[TopicPartition]]): Map[TopicPartition, OffsetFetchResponse.PartitionData] = {
trace("Getting offsets of %s for group %s.".format(topicPartitionsOpt.getOrElse("all partitions"), groupId))
// 依旧从缓存里面拿
val group = groupMetadataCache.get(groupId)
if (group == null) {
topicPartitionsOpt.getOrElse(Seq.empty[TopicPartition]).map { topicPartition =>
(topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.NONE))
}.toMap
} else {
group.inLock {
if (group.is(Dead)) {
topicPartitionsOpt.getOrElse(Seq.empty[TopicPartition]).map { topicPartition =>
(topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.NONE))
}.toMap
} else {
topicPartitionsOpt match {
case None =>
// Return offsets for all partitions owned by this consumer group. (this only applies to consumers
// that commit offsets to Kafka.)
group.allOffsets.map { case (topicPartition, offsetAndMetadata) =>
topicPartition -> new OffsetFetchResponse.PartitionData(offsetAndMetadata.offset, offsetAndMetadata.metadata, Errors.NONE)
}
case Some(topicPartitions) =>
topicPartitionsOpt.getOrElse(Seq.empty[TopicPartition]).map { topicPartition =>
// 调用内部的offset拿到topic-partition
val partitionData = group.offset(topicPartition) match {
case None =>
new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.NONE)
case Some(offsetAndMetadata) =>
new OffsetFetchResponse.PartitionData(offsetAndMetadata.offset, offsetAndMetadata.metadata, Errors.NONE)
}
topicPartition -> partitionData
}.toMap
}
}
}
}
}
3.2. 新增和更新消费组信息(提交偏移量)
消费组的新增和偏移量的更新是由客户端的消费者KafkaConsumer通过提交方法commitAsync() 调用ApiKeys.OFFSET_COMMIT的远程方法触发的操作的。
3.2.1. 提交offset
3.2.1.1. 入口api
ApiKey:OFFSET_COMMIT
handle-method:KafkaApis#handleOffsetCommitRequest()
3.2.1.2. KafkaApis#handleOffsetCommitRequest()
该方法主要实现对权限的校验,对不同版本的api做分发以及offset的封装。我们主要关注version-1的处理逻辑。其中针对每一个offset都计算出了一个超时时间,并传入GroupCoordinator#handleCommitOffsets() 方法。
/**
* Handle an offset commit request
*/
def handleOffsetCommitRequest(request: RequestChannel.Request) {
val header = request.header
val offsetCommitRequest = request.body[OffsetCommitRequest]
// reject the request if not authorized to the group
if (!authorize(request.session, Read, new Resource(Group, offsetCommitRequest.groupId))) {
//权限不通过
val error = Errors.GROUP_AUTHORIZATION_FAILED
val results = offsetCommitRequest.offsetData.keySet.asScala.map { topicPartition =>
(topicPartition, error)
}.toMap
sendResponseMaybeThrottle(request, requestThrottleMs => new OffsetCommitResponse(requestThrottleMs, results.asJava))
} else {
//收集没有权限的topic,基本不影响非权限验证的kafka。只需要关注保存了请求的topic信息的authorizedTopicRequestInfo对象即可
val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
val authorizedTopicRequestInfoBldr = immutable.Map.newBuilder[TopicPartition, OffsetCommitRequest.PartitionData]
for ((topicPartition, partitionData) <- offsetCommitRequest.offsetData.asScala) {
if (!authorize(request.session, Read, new Resource(Topic, topicPartition.topic)))
unauthorizedTopicErrors += (topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED)
else if (!metadataCache.contains(topicPartition.topic))
nonExistingTopicErrors += (topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION)
else
authorizedTopicRequestInfoBldr += (topicPartition -> partitionData)
}
val authorizedTopicRequestInfo = authorizedTopicRequestInfoBldr.result()
// the callback for sending an offset commit response
//提交结果的回调Response函数
def sendResponseCallback(commitStatus: immutable.Map[TopicPartition, Errors]) {
val combinedCommitStatus = commitStatus ++ unauthorizedTopicErrors ++ nonExistingTopicErrors
if (isDebugEnabled)
combinedCommitStatus.foreach { case (topicPartition, error) =>
if (error != Errors.NONE) {
debug(s"Offset commit request with correlation id ${header.correlationId} from client ${header.clientId} " +
s"on partition $topicPartition failed due to ${error.exceptionName}")
}
}
sendResponseMaybeThrottle(request, requestThrottleMs =>
new OffsetCommitResponse(requestThrottleMs, combinedCommitStatus.asJava))
}
if (authorizedTopicRequestInfo.isEmpty)
sendResponseCallback(Map.empty)
else if (header.apiVersion == 0) {
// for version 0 always store offsets to ZK
//用于兼容早期Group信息在zk上面维护的版本。
val responseInfo = authorizedTopicRequestInfo.map {
case (topicPartition, partitionData) =>
val topicDirs = new ZKGroupTopicDirs(offsetCommitRequest.groupId, topicPartition.topic)
try {
if (partitionData.metadata != null && partitionData.metadata.length > config.offsetMetadataMaxSize)
(topicPartition, Errors.OFFSET_METADATA_TOO_LARGE)
else {
zkUtils.updatePersistentPath(s"${topicDirs.consumerOffsetDir}/${topicPartition.partition}", partitionData.offset.toString)
(topicPartition, Errors.NONE)
}
} catch {
case e: Throwable => (topicPartition, Errors.forException(e))
}
}
sendResponseCallback(responseInfo)
} else {
// for version 1 and beyond store offsets in offset manager
//新版本的消费组信息,在此过程中处理
// compute the retention time based on the request version:
// if it is v1 or not specified by user, we can use the default retention
//计算group的超时时间,1.0.1的kafka版本中并不支持用户自己定义超时时间,而使用默认时间1天。
val offsetRetention =
if (header.apiVersion <= 1 ||
offsetCommitRequest.retentionTime == OffsetCommitRequest.DEFAULT_RETENTION_TIME)
groupCoordinator.offsetConfig.offsetsRetentionMs
else
offsetCommitRequest.retentionTime
// commit timestamp is always set to now.
// "default" expiration timestamp is now + retention (and retention may be overridden if v2)
// expire timestamp is computed differently for v1 and v2.
// - If v1 and no explicit commit timestamp is provided we use default expiration timestamp.
// - If v1 and explicit commit timestamp is provided we calculate retention from that explicit commit timestamp
// - If v2 we use the default expiration timestamp
//计算offset的超时时间和提交时间
val currentTimestamp = time.milliseconds
val defaultExpireTimestamp = offsetRetention + currentTimestamp
val partitionData = authorizedTopicRequestInfo.mapValues { partitionData =>
val metadata = if (partitionData.metadata == null) OffsetMetadata.NoMetadata else partitionData.metadata
new OffsetAndMetadata(
offsetMetadata = OffsetMetadata(partitionData.offset, metadata),
commitTimestamp = currentTimestamp,
expireTimestamp = {
if (partitionData.timestamp == OffsetCommitRequest.DEFAULT_TIMESTAMP)
defaultExpireTimestamp
else
offsetRetention + partitionData.timestamp
}
)
}
// 提交偏移量
// call coordinator to handle commit offset
groupCoordinator.handleCommitOffsets(
offsetCommitRequest.groupId,
offsetCommitRequest.memberId,
offsetCommitRequest.generationId,
partitionData,
sendResponseCallback)
}
}
}
3.2.1.3. GroupCoordinator#handleCommitOffsets()
该方法主要用于校验groupid的合法,以及在groupid尚未创建的时候,将其添加到缓存中。
而所有对于offset的提交进行落盘的操作,都是在GroupMetadataManager#storeOffsets()中进行的。
GroupCoordinator类
def handleCommitOffsets(groupId: String,
memberId: String,
generationId: Int,
offsetMetadata: immutable.Map[TopicPartition, OffsetAndMetadata],
responseCallback: immutable.Map[TopicPartition, Errors] => Unit) {
// 校验groupid是否有效
validateGroup(groupId) match {
case Some(error) => responseCallback(offsetMetadata.mapValues(_ => error))
case None =>
groupManager.getGroup(groupId) match {
case None =>
//如果缓存中不存在groupid,则创建groupid(新增在这里)。
if (generationId < 0) {
// generationId:用于指定group-member的版本,每当有新的member加入的时候,就会+1。
// the group is not relying on Kafka for group management, so allow the commit
// 在缓存中加入GroupId的信息。
val group = groupManager.addGroup(new GroupMetadata(groupId, initialState = Empty))
// 添加groupid后,再进行offset的提交。
doCommitOffsets(group, memberId, generationId, NO_PRODUCER_ID, NO_PRODUCER_EPOCH,
offsetMetadata, responseCallback)
} else {
// generationId >=0:说明这个请求是从已存在的groupid提交上来的,但是缓存中又不存在,则说明groupid因为超时等原因已经被删除了。
// or this is a request coming from an older generation. either way, reject the commit
responseCallback(offsetMetadata.mapValues(_ => Errors.ILLEGAL_GENERATION))
}
case Some(group) =>
// groupid存在,直接提交offset
doCommitOffsets(group, memberId, generationId, NO_PRODUCER_ID, NO_PRODUCER_EPOCH,
offsetMetadata, responseCallback)
}
}
}
private def doCommitOffsets(group: GroupMetadata,
memberId: String,
generationId: Int,
producerId: Long,
producerEpoch: Short,
offsetMetadata: immutable.Map[TopicPartition, OffsetAndMetadata],
responseCallback: immutable.Map[TopicPartition, Errors] => Unit) {
// 因为上文的所有GroupMetadata都是从GroupMetadataManager的缓存中获取的,所以这里对GroupMetadata就加锁都是同一个对象
group.inLock {
if (group.is(Dead)) {
responseCallback(offsetMetadata.mapValues(_ => Errors.UNKNOWN_MEMBER_ID))
} else if ((generationId < 0 && group.is(Empty)) || (producerId != NO_PRODUCER_ID)) {
// 新增groupid或者使用了事务的操作在这里
// The group is only using Kafka to store offsets.
// Also, for transactional offset commits we don't need to validate group membership and the generation.
groupManager.storeOffsets(group, memberId, offsetMetadata, responseCallback, producerId, producerEpoch)
} else if (group.is(AwaitingSync)) {
responseCallback(offsetMetadata.mapValues(_ => Errors.REBALANCE_IN_PROGRESS))
} else if (!group.has(memberId)) {
responseCallback(offsetMetadata.mapValues(_ => Errors.UNKNOWN_MEMBER_ID))
} else if (generationId != group.generationId) {
// 提交的generationId和当前broker上的generationId不同,说明已经有新的member加入,并且已经重平衡,需要重新平衡以后再提交
responseCallback(offsetMetadata.mapValues(_ => Errors.ILLEGAL_GENERATION))
} else {
// groupid已存在的情况下提交,
val member = group.get(memberId)
// 和新增groupid的区别
// 将member对应的心跳时间设置为当前操作的时间,并将已有的心跳检测器置为完成并移除,重新设置延迟心跳检测的调度。
completeAndScheduleNextHeartbeatExpiration(group, member)
groupManager.storeOffsets(group, memberId, offsetMetadata, responseCallback)
}
}
}
GroupMetadataManager类
/**
* Add a group or get the group associated with the given groupId if it already exists
*/
// 如果缓存中不存在groupid,则在缓存中新增groupid和GroupMetadata的映射
def addGroup(group: GroupMetadata): GroupMetadata = {
val currentGroup = groupMetadataCache.putIfNotExists(group.groupId, group)
if (currentGroup != null) {
currentGroup
} else {
group
}
}
3.2.1.4. GroupMetadataManager#storeOffsets()
我们先关注非事务的offset保存。事务需要单开一篇讲。
/**
* Store offsets by appending it to the replicated log and then inserting to cache
*/
def storeOffsets(group: GroupMetadata,
consumerId: String,
offsetMetadata: immutable.Map[TopicPartition, OffsetAndMetadata],
responseCallback: immutable.Map[TopicPartition, Errors] => Unit,
producerId: Long = RecordBatch.NO_PRODUCER_ID,
producerEpoch: Short = RecordBatch.NO_PRODUCER_EPOCH): Unit = {
// first filter out partitions with offset metadata size exceeding limit
val filteredOffsetMetadata = offsetMetadata.filter { case (_, offsetAndMetadata) =>
validateOffsetMetadataLength(offsetAndMetadata.metadata)
}
group.inLock {
if (!group.hasReceivedConsistentOffsetCommits)
warn(s"group: ${group.groupId} with leader: ${group.leaderOrNull} has received offset commits from consumers as well " +
s"as transactional producers. Mixing both types of offset commits will generally result in surprises and " +
s"should be avoided.")
}
val isTxnOffsetCommit = producerId != RecordBatch.NO_PRODUCER_ID
// construct the message set to append
if (filteredOffsetMetadata.isEmpty) {
// compute the final error codes for the commit response
val commitStatus = offsetMetadata.mapValues(_ => Errors.OFFSET_METADATA_TOO_LARGE)
responseCallback(commitStatus)
None
} else {
// 检查消息groupid对应提交的消息是否属于这个broker
// 如果不是,则返回None。
// 如果是,则返回消息格式的版本和时间戳
getMagic(partitionFor(group.groupId)) match {
case Some(magicValue) =>
// We always use CREATE_TIME, like the producer. The conversion to LOG_APPEND_TIME (if necessary) happens automatically.
// CreateTime : 表示producer创建这条消息的时间
// LogAppendTime : 表示broker写入磁盘的时间
val timestampType = TimestampType.CREATE_TIME
val timestamp = time.milliseconds()
// 生成log-record的key和value
val records = filteredOffsetMetadata.map { case (topicPartition, offsetAndMetadata) =>
// Key:groupId | topicName | partitionNo
val key = GroupMetadataManager.offsetCommitKey(group.groupId, topicPartition)
// value: offset | metadata | commitTimestamp | expireTimestamp
val value = GroupMetadataManager.offsetCommitValue(offsetAndMetadata)
// log-record对象
new SimpleRecord(timestamp, key, value)
}
// 指定topic:__consumer_offsets和根据groupId的哈希值%__consumer_offsets的partition数生成的partitionNo。
val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, partitionFor(group.groupId))
// 预创建堆内存
val buffer = ByteBuffer.allocate(AbstractRecords.estimateSizeInBytes(magicValue, compressionType, records.asJava))
if (isTxnOffsetCommit && magicValue < RecordBatch.MAGIC_VALUE_V2)
throw Errors.UNSUPPORTED_FOR_MESSAGE_FORMAT.exception("Attempting to make a transaction offset commit with an invalid magic: " + magicValue)
// 绑定buffer和record创建器
val builder = MemoryRecords.builder(buffer, magicValue, compressionType, timestampType, 0L, time.milliseconds(),
producerId, producerEpoch, 0, isTxnOffsetCommit, RecordBatch.NO_PARTITION_LEADER_EPOCH)
// 将records批量序列化为byte写入buffer中
records.foreach(builder.append)
val entries = Map(offsetTopicPartition -> builder.build())
// set the callback function to insert offsets into cache after log append completed
// record落盘成功以后,更新缓存中的offset的回调函数
def putCacheCallback(responseStatus: Map[TopicPartition, PartitionResponse]) {
// the append response should only contain the topics partition
// 同一个groupid只能提交到一个partition
if (responseStatus.size != 1 || !responseStatus.contains(offsetTopicPartition))
throw new IllegalStateException("Append status %s should only have one partition %s"
.format(responseStatus, offsetTopicPartition))
// construct the commit response status and insert
// the offset and metadata to cache if the append status has no error
val status = responseStatus(offsetTopicPartition)
val responseError = group.inLock {
if (status.error == Errors.NONE) {
if (!group.is(Dead)) {
filteredOffsetMetadata.foreach { case (topicPartition, offsetAndMetadata) =>
if (isTxnOffsetCommit)
group.onTxnOffsetCommitAppend(producerId, topicPartition, CommitRecordMetadataAndOffset(Some(status.baseOffset), offsetAndMetadata))
else
// 在缓存中更新offset信息
group.onOffsetCommitAppend(topicPartition, CommitRecordMetadataAndOffset(Some(status.baseOffset), offsetAndMetadata))
}
}
Errors.NONE
} else {
if (!group.is(Dead)) {
if (!group.hasPendingOffsetCommitsFromProducer(producerId))
removeProducerGroup(producerId, group.groupId)
filteredOffsetMetadata.foreach { case (topicPartition, offsetAndMetadata) =>
if (isTxnOffsetCommit)
group.failPendingTxnOffsetCommit(producerId, topicPartition)
else
//失败后移除bending中的offset
group.failPendingOffsetWrite(topicPartition, offsetAndMetadata)
}
}
debug(s"Offset commit $filteredOffsetMetadata from group ${group.groupId}, consumer $consumerId " +
s"with generation ${group.generationId} failed when appending to log due to ${status.error.exceptionName}")
// transform the log append error code to the corresponding the commit status error code
status.error match {
case Errors.UNKNOWN_TOPIC_OR_PARTITION
| Errors.NOT_ENOUGH_REPLICAS
| Errors.NOT_ENOUGH_REPLICAS_AFTER_APPEND =>
Errors.COORDINATOR_NOT_AVAILABLE
case Errors.NOT_LEADER_FOR_PARTITION
| Errors.KAFKA_STORAGE_ERROR =>
Errors.NOT_COORDINATOR
case Errors.MESSAGE_TOO_LARGE
| Errors.RECORD_LIST_TOO_LARGE
| Errors.INVALID_FETCH_SIZE =>
Errors.INVALID_COMMIT_OFFSET_SIZE
case other => other
}
}
}
// compute the final error codes for the commit response
val commitStatus = offsetMetadata.map { case (topicPartition, offsetAndMetadata) =>
if (validateOffsetMetadataLength(offsetAndMetadata.metadata))
(topicPartition, responseError)
else
(topicPartition, Errors.OFFSET_METADATA_TOO_LARGE)
}
// finally trigger the callback logic passed from the API layer
responseCallback(commitStatus)
}
if (isTxnOffsetCommit) {
group.inLock {
addProducerGroup(producerId, group.groupId)
group.prepareTxnOffsetCommit(producerId, offsetMetadata)
}
} else {
// 将即将提交的offset放进GroupMetadata的pendingOffsetCommits中。
// 这里可能是为了拓展,因为上层函数本身就加了锁,根本不需要做二次提交。
group.inLock {
group.prepareOffsetCommit(offsetMetadata)
}
}
// 将record落盘,其实只是对通用的落盘方法replicaManager#appendRecords()做了一层封装,固定了一些诸如internalTopicsAllowed和isFromClient的参数
// END:offset的提交到此结束,以后是kafka的消息落盘的流程,也是需要单开一篇文章讲的内容了
appendForGroup(group, entries, putCacheCallback)
case None =>
val commitStatus = offsetMetadata.map { case (topicPartition, _) =>
(topicPartition, Errors.NOT_COORDINATOR)
}
responseCallback(commitStatus)
None
}
}
}
3.2.2. 消费者发送/接收分配结果(SYNC_GROUP)和新member加入(JOIN_GROUP)
新member加入:
ApiKey:JOIN_GROUP
handle-method:KafkaApis#handleJoinGroupRequest()
3.2.2.1. GroupMetadataManager#storeGroup()
消费者发送/接收分配结果:
ApiKey:SYNC_GROUP
handle-method:KafkaApis#handleSyncGroupRequest()
rebalance的过程可以单开一篇文章,直接看核心的GroupMetadataManager#storeGroup()。它其实和storeOffset的思路一样,就是生成对应的key-value的record,然后将record保存到**__consumer_offsets**,落盘成功以后,再刷新缓存。
def storeGroup(group: GroupMetadata,
groupAssignment: Map[String, Array[Byte]],
responseCallback: Errors => Unit): Unit = {
getMagic(partitionFor(group.groupId)) match {
case Some(magicValue) =>
val groupMetadataValueVersion = {
if (interBrokerProtocolVersion < KAFKA_0_10_1_IV0)
0.toShort
else
GroupMetadataManager.CURRENT_GROUP_VALUE_SCHEMA_VERSION
}
// We always use CREATE_TIME, like the producer. The conversion to LOG_APPEND_TIME (if necessary) happens automatically.
val timestampType = TimestampType.CREATE_TIME
val timestamp = time.milliseconds()
// 生成存储消费组格式的key值: groupId
val key = GroupMetadataManager.groupMetadataKey(group.groupId)
// 生成存储消费组格式的value值: protocolType | generationId | protocol | leader | members [member_id,client_id,client_host,session_timeout,rebalance_timeout,subscription,assignment]
val value = GroupMetadataManager.groupMetadataValue(group, groupAssignment, version = groupMetadataValueVersion)
val records = {
val buffer = ByteBuffer.allocate(AbstractRecords.estimateSizeInBytes(magicValue, compressionType,
Seq(new SimpleRecord(timestamp, key, value)).asJava))
val builder = MemoryRecords.builder(buffer, magicValue, compressionType, timestampType, 0L)
builder.append(timestamp, key, value)
builder.build()
}
val groupMetadataPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, partitionFor(group.groupId))
val groupMetadataRecords = Map(groupMetadataPartition -> records)
val generationId = group.generationId
// set the callback function to insert the created group into cache after log append completed
def putCacheCallback(responseStatus: Map[TopicPartition, PartitionResponse]) {
// the append response should only contain the topics partition
if (responseStatus.size != 1 || !responseStatus.contains(groupMetadataPartition))
throw new IllegalStateException("Append status %s should only have one partition %s"
.format(responseStatus, groupMetadataPartition))
// construct the error status in the propagated assignment response in the cache
val status = responseStatus(groupMetadataPartition)
val responseError = if (status.error == Errors.NONE) {
Errors.NONE
} else {
debug(s"Metadata from group ${group.groupId} with generation $generationId failed when appending to log " +
s"due to ${status.error.exceptionName}")
// transform the log append error code to the corresponding the commit status error code
status.error match {
case Errors.UNKNOWN_TOPIC_OR_PARTITION
| Errors.NOT_ENOUGH_REPLICAS
| Errors.NOT_ENOUGH_REPLICAS_AFTER_APPEND =>
Errors.COORDINATOR_NOT_AVAILABLE
case Errors.NOT_LEADER_FOR_PARTITION
| Errors.KAFKA_STORAGE_ERROR =>
Errors.NOT_COORDINATOR
case Errors.REQUEST_TIMED_OUT =>
Errors.REBALANCE_IN_PROGRESS
case Errors.MESSAGE_TOO_LARGE
| Errors.RECORD_LIST_TOO_LARGE
| Errors.INVALID_FETCH_SIZE =>
error(s"Appending metadata message for group ${group.groupId} generation $generationId failed due to " +
s"${status.error.exceptionName}, returning UNKNOWN error code to the client")
Errors.UNKNOWN_SERVER_ERROR
case other =>
error(s"Appending metadata message for group ${group.groupId} generation $generationId failed " +
s"due to unexpected error: ${status.error.exceptionName}")
other
}
}
responseCallback(responseError)
}
appendForGroup(group, groupMetadataRecords, putCacheCallback)
case None =>
responseCallback(Errors.NOT_COORDINATOR)
None
}
}
GroupCoordinator##doSyncGroup()中的回调函数
GroupCoordinator类
groupManager.storeGroup(group, assignment, (error: Errors) => {
group.inLock {
// 分配结果落盘以后的回调函数
// another member may have joined the group while we were awaiting this callback,
// so we must ensure we are still in the AwaitingSync state and the same generation
// when it gets invoked. if we have transitioned to another state, then do nothing
if (group.is(AwaitingSync) && generationId == group.generationId) {
if (error != Errors.NONE) {
resetAndPropagateAssignmentError(group, error)
// 切换成PrepareRebalance,重新平衡
maybePrepareRebalance(group)
} else {
// 分配结果落盘成功,将消费组状态设置为Stable
// 并唤醒其他成员的等待
setAndPropagateAssignment(group, assignment)
group.transitionTo(Stable)
}
}
}
})
3.3. 删除消费组信息
消费组信息的清除有两种情况:1. Kafka的partition发生变动时;2. 定期清理过期的Group过期清除的时候。
清除的重点是在于创建了和group分配或者offset提交相同的key值,但是value=null。
GroupMetadataManager#cleanupGroupMetadata()
// 触发情况有两种:1. Kafka的partition发生变动时;2. Group过期清除的时候
def cleanupGroupMetadata(deletedTopicPartitions: Option[Seq[TopicPartition]]) {
val startMs = time.milliseconds()
var offsetsRemoved = 0
groupMetadataCache.foreach { case (groupId, group) =>
val (removedOffsets, groupIsDead, generation) = group.inLock {
val removedOffsets = deletedTopicPartitions match {
// 先从缓存中将opicPartitions的offset移除
case Some(topicPartitions) => group.removeOffsets(topicPartitions)
case None => group.removeExpiredOffsets(startMs)
}
if (group.is(Empty) && !group.hasOffsets) {
// Empty状态在Rebalance,并进行代际更迭的时候,如果member是空,则设置
// 如果group是Empty状态,且group中不存在消费的offset信息,则将group设置为Dead状态
info(s"Group $groupId transitioned to Dead in generation ${group.generationId}")
group.transitionTo(Dead)
}
(removedOffsets, group.is(Dead), group.generationId)
}
val offsetsPartition = partitionFor(groupId)
val appendPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, offsetsPartition)
getMagic(offsetsPartition) match {
case Some(magicValue) =>
// We always use CREATE_TIME, like the producer. The conversion to LOG_APPEND_TIME (if necessary) happens automatically.
val timestampType = TimestampType.CREATE_TIME
val timestamp = time.milliseconds()
replicaManager.nonOfflinePartition(appendPartition).foreach { partition =>
//针对所有未下线的partition,进行检查
val tombstones = ListBuffer.empty[SimpleRecord]
// 对于被移除的topicPartition,创建墓碑消息,用于标记消费偏移量不再使用
removedOffsets.foreach { case (topicPartition, offsetAndMetadata) =>
trace(s"Removing expired/deleted offset and metadata for $groupId, $topicPartition: $offsetAndMetadata")
// key的格式和storeOffset的对应
val commitKey = GroupMetadataManager.offsetCommitKey(groupId, topicPartition)
// 消息的value=null
tombstones += new SimpleRecord(timestamp, commitKey, null)
}
trace(s"Marked ${removedOffsets.size} offsets in $appendPartition for deletion.")
// We avoid writing the tombstone when the generationId is 0, since this group is only using
// Kafka for offset storage.
// 如果group已经死了,则从缓存中移除
if (groupIsDead && groupMetadataCache.remove(groupId, group) && generation > 0) {
// Append the tombstone messages to the partition. It is okay if the replicas don't receive these (say,
// if we crash or leaders move) since the new leaders will still expire the consumers with heartbeat and
// retry removing this group.
// 创建group对应的墓碑消息,key和storeGroup的消息对应,value=null
val groupMetadataKey = GroupMetadataManager.groupMetadataKey(group.groupId)
tombstones += new SimpleRecord(timestamp, groupMetadataKey, null)
trace(s"Group $groupId removed from the metadata cache and marked for deletion in $appendPartition.")
}
if (tombstones.nonEmpty) {
//墓碑落盘
try {
// do not need to require acks since even if the tombstone is lost,
// it will be appended again in the next purge cycle
val records = MemoryRecords.withRecords(magicValue, 0L, compressionType, timestampType, tombstones: _*)
partition.appendRecordsToLeader(records, isFromClient = false, requiredAcks = 0)
offsetsRemoved += removedOffsets.size
trace(s"Successfully appended ${tombstones.size} tombstones to $appendPartition for expired/deleted " +
s"offsets and/or metadata for group $groupId")
} catch {
case t: Throwable =>
error(s"Failed to append ${tombstones.size} tombstones to $appendPartition for expired/deleted " +
s"offsets and/or metadata for group $groupId.", t)
// ignore and continue
}
}
}
case None =>
info(s"BrokerId $brokerId is no longer a coordinator for the group $groupId. Proceeding cleanup for other alive groups")
}
}
info(s"Removed $offsetsRemoved expired offsets in ${time.milliseconds() - startMs} milliseconds.")
}
3.4. 消费组的加载
消费组的加载发生在:1. 启动的时候;2. Partition的Leader发生改变或者发生ISR的时候
关键函数:GroupMetadataManager#loadGroupsAndOffsets()
比较长,可以略过事物的部分
// topicPartition是上层传进来的__consumer_offsets和group对应的partitionid
private[group] def loadGroupsAndOffsets(topicPartition: TopicPartition, onGroupLoaded: GroupMetadata => Unit) {
// 获取partition的最后一个offset
def highWaterMark = replicaManager.getLogEndOffset(topicPartition).getOrElse(-1L)
replicaManager.getLog(topicPartition) match {
case None =>
warn(s"Attempted to load offsets and group metadata from $topicPartition, but found no log")
case Some(log) =>
var currOffset = log.logStartOffset
lazy val buffer = ByteBuffer.allocate(config.loadBufferSize)
// loop breaks if leader changes at any time during the load, since getHighWatermark is -1
val loadedOffsets = mutable.Map[GroupTopicPartition, CommitRecordMetadataAndOffset]()
val pendingOffsets = mutable.Map[Long, mutable.Map[GroupTopicPartition, CommitRecordMetadataAndOffset]]()
val loadedGroups = mutable.Map[String, GroupMetadata]()
val removedGroups = mutable.Set[String]()
while (currOffset < highWaterMark && !shuttingDown.get()) {
// 读取__consumer_offsets或者缓存中的消息(被读出来过的消息会缓存)
val fetchDataInfo = log.read(currOffset, config.loadBufferSize, maxOffset = None,
minOneMessage = true, isolationLevel = IsolationLevel.READ_UNCOMMITTED)
val memRecords = fetchDataInfo.records match {
// 记录已经被缓存
case records: MemoryRecords => records
// 需要从文件中读取
case fileRecords: FileRecords =>
buffer.clear()
val bufferRead = fileRecords.readInto(buffer, 0)
MemoryRecords.readableRecords(bufferRead)
}
memRecords.batches.asScala.foreach { batch =>
val isTxnOffsetCommit = batch.isTransactional
if (batch.isControlBatch) {
//忽略事物的流程
val record = batch.iterator.next()
val controlRecord = ControlRecordType.parse(record.key)
if (controlRecord == ControlRecordType.COMMIT) {
pendingOffsets.getOrElse(batch.producerId, mutable.Map[GroupTopicPartition, CommitRecordMetadataAndOffset]())
.foreach {
case (groupTopicPartition, commitRecordMetadataAndOffset) =>
if (!loadedOffsets.contains(groupTopicPartition) || loadedOffsets(groupTopicPartition).olderThan(commitRecordMetadataAndOffset))
loadedOffsets.put(groupTopicPartition, commitRecordMetadataAndOffset)
}
}
pendingOffsets.remove(batch.producerId)
} else {
for (record <- batch.asScala) {
require(record.hasKey, "Group metadata/offset entry key should not be null")
// 判断key的类型:1. offset key; 2. group key
GroupMetadataManager.readMessageKey(record.key) match {
case offsetKey: OffsetKey =>
//offset key 的处理流程
if (isTxnOffsetCommit && !pendingOffsets.contains(batch.producerId))
pendingOffsets.put(batch.producerId, mutable.Map[GroupTopicPartition, CommitRecordMetadataAndOffset]())
// load offset
val groupTopicPartition = offsetKey.key
if (!record.hasValue) {
// 记录是墓碑记录
// 如果value是空,则将groupTopicPartition的偏移key从缓存中移除
if (isTxnOffsetCommit)
pendingOffsets(batch.producerId).remove(groupTopicPartition)
else
loadedOffsets.remove(groupTopicPartition)
} else {
// 将bytebuffer形式的value转成java对象
val offsetAndMetadata = GroupMetadataManager.readOffsetMessageValue(record.value)
// 将对应的java对象存入本地缓存
if (isTxnOffsetCommit)
pendingOffsets(batch.producerId).put(groupTopicPartition, CommitRecordMetadataAndOffset(Some(batch.baseOffset), offsetAndMetadata))
else
loadedOffsets.put(groupTopicPartition, CommitRecordMetadataAndOffset(Some(batch.baseOffset), offsetAndMetadata))
}
case groupMetadataKey: GroupMetadataKey =>
// Group key的处理流程
// load group metadata
val groupId = groupMetadataKey.key
// 将bytebuffer形式的value转成java对象
val groupMetadata = GroupMetadataManager.readGroupMessageValue(groupId, record.value)
if (groupMetadata != null) {
// 将groupMetadata的信息更新到本地缓存中
removedGroups.remove(groupId)
loadedGroups.put(groupId, groupMetadata)
} else {
// 墓碑记录,移除消费组,记录被移除的消费组
loadedGroups.remove(groupId)
removedGroups.add(groupId)
}
case unknownKey =>
throw new IllegalStateException(s"Unexpected message key $unknownKey while loading offsets and group metadata")
}
}
}
currOffset = batch.nextOffset
}
}
val (groupOffsets, emptyGroupOffsets) = loadedOffsets
.groupBy(_._1.group)
.mapValues(_.map { case (groupTopicPartition, offset) => (groupTopicPartition.topicPartition, offset) })
.partition { case (group, _) => loadedGroups.contains(group) }
val pendingOffsetsByGroup = mutable.Map[String, mutable.Map[Long, mutable.Map[TopicPartition, CommitRecordMetadataAndOffset]]]()
pendingOffsets.foreach { case (producerId, producerOffsets) =>
producerOffsets.keySet.map(_.group).foreach(addProducerGroup(producerId, _))
producerOffsets
.groupBy(_._1.group)
.mapValues(_.map { case (groupTopicPartition, offset) => (groupTopicPartition.topicPartition, offset) })
.foreach { case (group, offsets) =>
val groupPendingOffsets = pendingOffsetsByGroup.getOrElseUpdate(group, mutable.Map.empty[Long, mutable.Map[TopicPartition, CommitRecordMetadataAndOffset]])
val groupProducerOffsets = groupPendingOffsets.getOrElseUpdate(producerId, mutable.Map.empty[TopicPartition, CommitRecordMetadataAndOffset])
groupProducerOffsets ++= offsets
}
}
val (pendingGroupOffsets, pendingEmptyGroupOffsets) = pendingOffsetsByGroup
.partition { case (group, _) => loadedGroups.contains(group) }
// 从本地记录的loadedGroups中,去更新缓存中的group-offset
loadedGroups.values.foreach { group =>
val offsets = groupOffsets.getOrElse(group.groupId, Map.empty[TopicPartition, CommitRecordMetadataAndOffset])
val pendingOffsets = pendingGroupOffsets.getOrElse(group.groupId, Map.empty[Long, mutable.Map[TopicPartition, CommitRecordMetadataAndOffset]])
debug(s"Loaded group metadata $group with offsets $offsets and pending offsets $pendingOffsets")
//更新缓存
loadGroup(group, offsets, pendingOffsets)
//对所有member创建心跳监听任务
onGroupLoaded(group)
}
// load groups which store offsets in kafka, but which have no active members and thus no group
// metadata stored in the log
(emptyGroupOffsets.keySet ++ pendingEmptyGroupOffsets.keySet).foreach { groupId =>
val group = new GroupMetadata(groupId, initialState = Empty)
val offsets = emptyGroupOffsets.getOrElse(groupId, Map.empty[TopicPartition, CommitRecordMetadataAndOffset])
val pendingOffsets = pendingEmptyGroupOffsets.getOrElse(groupId, Map.empty[Long, mutable.Map[TopicPartition, CommitRecordMetadataAndOffset]])
debug(s"Loaded group metadata $group with offsets $offsets and pending offsets $pendingOffsets")
loadGroup(group, offsets, pendingOffsets)
onGroupLoaded(group)
}
removedGroups.foreach { groupId =>
// if the cache already contains a group which should be removed, raise an error. Note that it
// is possible (however unlikely) for a consumer group to be removed, and then to be used only for
// offset storage (i.e. by "simple" consumers)
if (groupMetadataCache.contains(groupId) && !emptyGroupOffsets.contains(groupId))
throw new IllegalStateException(s"Unexpected unload of active group $groupId while " +
s"loading partition $topicPartition")
}
}
}
4. 总结
最后,可以回答开头的三个问题:
- 如何通过topic实现删改的语义?
答:将同一groupId或者同一group-topic-partition作为key值,利用topic在同一partition的有序性,对同一key值的内容进行upsert。利用value中的不同值,达到更新的效果。或者value=null的时候,达到删除的效果。 - 如何高效的定位查询特定的group?
答:每个broker会提前加载所有属于自身broker管理的group,用ConcurrentHashMap管理group信息,在group信息发生变动的时候,同步更新缓存信息,以达到快速查找group信息的目的。 - 如果解决分布式情况下,同一个消费组中,并发提交相同topic-partition中的offset的问题?
答:(1) 将每个groupId唯一分配给 __consumer_offsets 中的固定partitionId,避免groupId分散在不同的broker中。(2) 在对group操作前,以group为对象进行本地加锁。
4. 总结
最后,可以回答开头的三个问题:
- 如何通过topic实现删改的语义?
答:将同一groupId或者同一group-topic-partition作为key值,利用topic在同一partition的有序性,对同一key值的内容进行upsert。利用value中的不同值,达到更新的效果。或者value=null的时候,达到删除的效果。 - 如何高效的定位查询特定的group?
答:每个broker会提前加载所有属于自身broker管理的group,用ConcurrentHashMap管理group信息,在group信息发生变动的时候,同步更新缓存信息,以达到快速查找group信息的目的。 - 如果解决分布式情况下,同一个消费组中,并发提交相同topic-partition中的offset的问题?
答:(1) 将每个groupId唯一分配给 __consumer_offsets 中的固定partitionId,避免groupId分散在不同的broker中。(2) 在对group操作前,以group为对象进行本地加锁。