Kafka4源码(三)Share Group共享组

0 阅读38分钟

前言

Share Group(也称 Kafka Queues)共享组,是Kafka4.0中由KIP-932引入的新消费模型。

共享与传统消费组的区别:

  1. 分区独占:共享:一个分区可以被多个消费组成员共同消费;传统:一对一独占;
  2. 消费组成员数量上限:共享:默认最多200,最大支持1000;传统:有效成员≤分区数;
  3. 消息处理粒度:共享:单条消息ack;传统:分区维度offset批量ack;
  4. 顺序保证:共享:仅在单次拉取的批次内有序,整体无顺序保证;传统:分区内有序;

Kafka4.x的共享组类似于RocketMQ5.x的POP消费,但是功能和实现方式上还是有一些区别。

本文基于Kafka 4.2.0,分析Kafka共享组的实现原理。

一、共享组使用案例

使用KafkaShareConsumer创建消费者,使用方式和普通消费者类似。

share.acknowledgement.mode:默认implicit-隐式提交,每次poll会将上次poll返回的消息标记为ACCEPT提交;可选explicit-显式提交,每次poll必须将上次poll返回的消息通过acknowledge api显式标记。

Properties properties = new Properties();
// share.acknowledgement.mode = implicit-默认,隐式提交  explicit-显示提交
properties.put(ConsumerConfig.SHARE_ACKNOWLEDGEMENT_MODE_CONFIG, "explicit");
properties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
properties.put(ConsumerConfig.GROUP_ID_CONFIG, "group001");
properties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer");
properties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer");
ShareConsumer<String, String> consumer = new KafkaShareConsumer<>(properties);
consumer.subscribe(Collections.singleton("TopicA"));
while (running) {
    ConsumerRecords<String, String> records = consumer.poll(Duration.ofMillis(100));
    for (ConsumerRecord<String, String> record : records) {
        try {
            System.out.println(Thread.currentThread() + "===" + record.partition() + "===" + record.offset());
            // consumer.acknowledge(record, AcknowledgeType.RENEW);
            consumer.acknowledge(record, AcknowledgeType.ACCEPT);
        } catch (Exception e) {
            e.printStackTrace();
            // consumer.acknowledge(record, AcknowledgeType.RELEASE);
            consumer.acknowledge(record, AcknowledgeType.REJECT);
        }
    }
}
consumer.close();

二、共享组消费者概览

普通KafkaConsumer,在4.x中有两种消费组协议,配置项group.protocol

  1. classic:ClassicKafkaConsumer,普通消费组默认协议,老版本协议(以前分析过,包括JoinGroup/SyncGroup)。整个消费者实例只用一个线程(应用+IO)+一个心跳线程;
  2. consumer:AsyncKafkaConsumer,可选,未来新协议(KIP-848)将部分客户端消费组逻辑迁移到Broker服务端。并且客户端使用新的线程模型,应用线程(poll api)和IO线程(与Broker通讯)分离,Consumer+threading+refactor+design

对于新出的KafkaShareConsumer,仅支持consumer新协议

2-1、应用线程

ShareConsumerImpl.poll:应用线程循环

  1. handleCompletedAcknowledgements,如果用户注册KafkaShareConsumer.setAcknowledgementCommitCallback回调,当收到ack响应时触发;
  2. acknowledgeBatchIfImplicitAcknowledgement,如果隐式提交,设置上次poll的记录为ACCEPT;
  3. ensureInFlightAcknowledgedIfExplicitAcknowledgement,如果显式提交,校验上次poll的记录都必须调用acknowledge api
  4. 发送SharePollEvent给IO线程;
  5. pollForFetches:应用线程从ShareFetchBuffer拉取数据;
public synchronized ConsumerRecords<K, V> poll(final Duration timeout) {
    Timer timer = time.timer(timeout);
    acquireAndEnsureOpen();
    try {
        // 处理IO线程返回的ErrorEvent,忽略
        processBackgroundEvents();
        // 触发用户KafkaShareConsumer.setAcknowledgementCommitCallback回调,
        // 用于处理ack响应,忽略
        handleCompletedAcknowledgements();
        // share.acknowledgement.mode = implicit-默认隐式提交
        // 设置上次所有poll的记录的ack=ACCEPT
        acknowledgeBatchIfImplicitAcknowledgement();
        // share.acknowledgement.mode = explicit-显示提交
        // 用户必须将上次poll的数据都consumer#acknowledge,否则异常
        ensureInFlightAcknowledgedIfExplicitAcknowledgement();
        shouldSendShareFetchEvent = true;
        do {
            // 应用线程提交SharePollEvent,重置poll超时计时
            applicationEventHandler.add(new SharePollEvent(timer.currentTimeMs()));
            wakeupTrigger.maybeTriggerWakeup();
            // 从内存ShareFetchBuffer拉取数据
            final ShareFetch<K, V> fetch = pollForFetches(timer);
            if (!fetch.isEmpty()) {
                currentFetch = fetch;
                handleCompletedAcknowledgements();
                return new ConsumerRecords<>(fetch.records(), Map.of());
            }
            processBackgroundEvents();
            metadata.maybeThrowAnyException();
        } while (timer.notExpired());
        handleCompletedAcknowledgements();
        return ConsumerRecords.empty();
    } catch (ShareFetchException e) {
        currentFetch = (ShareFetch<K, V>) e.shareFetch();
        throw e.cause();
    } finally {
        release();
    }
}

2-2、IO线程

ConsumerNetworkThread:IO线程。

public class ConsumerNetworkThread extends KafkaThread implements Closeable {
    // 应用线程事件队列
    private final BlockingQueue<ApplicationEvent> applicationEventQueue;
    // 处理应用线程事件
    private ApplicationEventProcessor applicationEventProcessor;
    // 通讯客户端,与服务端通讯
    private NetworkClientDelegate networkClientDelegate;
    // n个请求处理器
    private RequestManagers requestManagers;
}
public RequestManagers(...) {
List<RequestManager> list = new ArrayList<>();
// 1. FindCoordinatorRequest 发现协调者
coordinatorRequestManager.ifPresent(list::add);
// 2. ShareGroupHeartbeatRequest 发送心跳 获取分区分配
shareHeartbeatRequestManager.ifPresent(list::add);
// 3. 维护当前消费组成员状态(不发送数据,处理心跳响应)、更新分区分配
shareMembershipManager.ifPresent(list::add);
// 4. 发送ShareFetchRequest和ShareAcknowledgeRequest
list.add(shareConsumeRequestManager);
entries = Collections.unmodifiableList(list);
}

ConsumerNetworkThread.runOnce:IO线程循环:

  1. 处理应用线程发来的事件,比如:SharePollEvent需要重置poll超时计时(max.poll.interval.ms=5分钟),ShareFetchEvent需要拉消息;
  2. 循环n个RequestManager构造请求,加入请求队列;
  3. networkClientDelegate.poll发送请求并接收响应,比如收到ShareFetch响应,将消息写入ShareFetchBuffer供应用线程消费;
void runOnce() {
    // 1. 处理应用事件
    processApplicationEvents();
    // 2. 调用所有RequestManager,将需要发发送的请求加入networkClientDelegate
    for (RequestManager rm : requestManagers.entries()) {
        // 构建请求
        NetworkClientDelegate.PollResult pollResult = rm.poll(currentTimeMs);
        // 将请求加入通讯客户端的队列
        long timeoutMs = networkClientDelegate.addAll(pollResult);
    }
    // 3. 发送请求接收响应...
    networkClientDelegate.poll(pollWaitTimeMs, currentTimeMs);
    // ...
}
// 处理应用线程发来的事件
public void process(ApplicationEvent event) {
switch (event.type()) {
        // ...
    case SHARE_POLL:
        process((SharePollEvent) event);
        return;
}
}

CoordinatorRequestManager.poll:RequestManager以FindCoordinatorRequest构建请求为例,构建时会注册响应回调,networkClientDelegate.poll如果收到响应会直接触发回调。

public NetworkClientDelegate.PollResult poll(final long currentTimeMs) {
    if (coordinatorRequestState.canSendRequest(currentTimeMs)) {
        NetworkClientDelegate.UnsentRequest request = makeFindCoordinatorRequest(currentTimeMs);
        return new NetworkClientDelegate.PollResult(request);
    }
}
NetworkClientDelegate.UnsentRequest makeFindCoordinatorRequest(final long currentTimeMs) {
    // 构建请求
    NetworkClientDelegate.UnsentRequest unsentRequest = new NetworkClientDelegate.UnsentRequest(
        new FindCoordinatorRequest.Builder(data),
        Optional.empty()
    );
    // 响应回调
    return unsentRequest.whenComplete((clientResponse, throwable) -> {

    });
}

三、协调者概览

简单介绍高版本的消费组协调者写操作的实现方式。

逻辑和低版本差不多:写日志+应用到内存,重启以后重放日志+应用到内存。

在共享组场景下,有两套状态:

  1. GroupCoordinatorService(GroupCoordinatorShard)管理consumer_offset对应的状态机,后文简称消费组状态,管理消费组成员和分区分配;(如果是传统消费组,这里还维护消费进度)
  2. ShareCoordinatorService(ShareCoordinatorShard)管理sharegroupstate对应的状态机,后文简称共享组状态,管理共享组消费进度
public class GroupCoordinatorService implements GroupCoordinator {
    private final CoordinatorRuntime
    <GroupCoordinatorShard, CoordinatorRecord> runtime;
}
public class ShareCoordinatorService implements ShareCoordinator {
    private final CoordinatorRuntime
    <ShareCoordinatorShard, CoordinatorRecord> runtime;
}

GroupCoordinatorService.shareGroupHeartbeat:写操作调用CoordinatorRuntime.scheduleWriteOperation传入4个参数。

// consumer_offset状态机
private final CoordinatorRuntime<GroupCoordinatorShard, CoordinatorRecord> runtime;

// ....
return runtime.scheduleWriteOperation(
    "share-group-heartbeat",
    // TopicPartition
    topicPartitionFor(request.groupId()),
    // 超时时间
    Duration.ofMillis(config.offsetCommitTimeoutMs()),
    // CoordinatorWriteOperation
    coordinator -> coordinator.shareGroupHeartbeat(context, request)
).thenCompose(result -> {
});

CoordinatorRuntime.scheduleWriteOperation:写操作封装为CoordinatorWriteEvent,放入accumulator队列。

// CoordinatorRuntime
private final CoordinatorEventProcessor processor;
public <T> CompletableFuture<T> scheduleWriteOperation(
    String name, TopicPartition tp,
    Duration timeout, CoordinatorWriteOperation<S, T, U> op) {
    throwIfNotRunning();
    CoordinatorWriteEvent<T> event = new CoordinatorWriteEvent<>(name, tp, timeout, op);
    enqueueLast(event);
    return event.future;
}
private void enqueueLast(CoordinatorEvent event) {
    processor.enqueueLast(event);
}
// MultiThreadedEventProcessor
private final EventAccumulator<TopicPartition, CoordinatorEvent> accumulator;
public void enqueueLast(CoordinatorEvent event) throws RejectedExecutionException {
    accumulator.addLast(event);
}

MultiThreadedEventProcessor多线程消费accumulator队列。

  1. consumer_offset使用group.coordinator.threads=4个线程;
  2. share_group_state使用share.coordinator.threads=1个线程;
private final EventAccumulator<TopicPartition, CoordinatorEvent> accumulator;
private final List<EventProcessorThread> threads;
public MultiThreadedEventProcessor(int numThreads, ...) {
    this.accumulator = eventAccumulator;
    this.threads = IntStream.range(0, numThreads).mapToObj(threadId ->
       new EventProcessorThread(
           threadPrefix + threadId
       )
      ).collect(Collectors.toList());
    this.threads.forEach(EventProcessorThread::start);
}
private class EventProcessorThread extends Thread {
    private void handleEvents() {
        while (!shuttingDown) {
            if (event != null) {
                try {
                    event.run();
                } catch (Throwable t) {
                    event.complete(t);
                } finally {
                    accumulator.done(event);
                }
            }
        }
    }
    @Override
    public void run() {
        handleEvents();
    }
}

CoordinatorWriteEvent

  1. withActiveContextOrThrow,执行写操作,会持有分区级别锁;
  2. CoordinatorWriteOperation生成日志记录;
  3. CoordinatorShard.replay应用日志记录到内存;
  4. maybeFlushCurrentBatch写日志;
  5. 响应客户端;
public void run() {
    try {
        withActiveContextOrThrow(tp, context -> {
            // CoordinatorWriteOperation生成log记录
            result = op.generateRecordsAndResult(context.coordinator.coordinator());
            // 写log并应用到内存
            context.append(
                producerId,
                producerEpoch,
                verificationGuard,
                result.records(),
                result.replayRecords(),
                result.isAtomic(),
                this
            );
            // 超时响应
            if (!future.isDone()) {
                operationTimeout = new OperationTimeout(tp, this, writeTimeout.toMillis());
                timer.add(operationTimeout);
                deferredEventQueuedTimestamp = time.milliseconds();
            }
        });
    } catch (Throwable t) {
        complete(t);
    }
}
// CoordinatorRuntime.CoordinatorContext#append
private void append(...) {
// 1. CoordinatorShard应用到内存
coordinator.replay(
    currentBatch.nextOffset, // 该消息批次的起始offset
    producerId,
    producerEpoch,
    recordToReplay
);
// 2. 写日志
maybeFlushCurrentBatch(currentTimeMs);
}
}
private void withActiveContextOrThrow(
    TopicPartition tp,
    Consumer<CoordinatorContext> func
) throws NotCoordinatorException, CoordinatorLoadInProgressException {
    // 上下文会在该分区成为leader时创建
    CoordinatorContext context = contextOrThrow(tp);
    try {
        context.lock.lock();
        if (context.state == CoordinatorState.ACTIVE) {
            func.accept(context);
        }
        // ...
    } finally {
        context.lock.unlock();
    }
}

四、消费者发现协调者

CoordinatorRequestManager.poll:

  1. 构建FindCoordinatorRequest,注意入参keyType=0(GROUP),即共享组协调者与普通消费组完全一致
  2. 注册处理FindCoordinatorResponse回调,将协调者节点缓存在coordinator;
// 消费组协调者节点
private Node coordinator;
public NetworkClientDelegate.PollResult poll(final long currentTimeMs) {
    // 如果协调者已经发现,跳过
    if (closing || this.coordinator != null)
        return EMPTY;
    // 可以发送
    if (coordinatorRequestState.canSendRequest(currentTimeMs)) {
        NetworkClientDelegate.UnsentRequest request = makeFindCoordinatorRequest(currentTimeMs);
        return new NetworkClientDelegate.PollResult(request);
    }
    // 上次失败了,在backoff时间内,不发送
    return new NetworkClientDelegate.PollResult(coordinatorRequestState.remainingBackoffMs(currentTimeMs));
} 
NetworkClientDelegate.UnsentRequest makeFindCoordinatorRequest(final long currentTimeMs) {
    coordinatorRequestState.onSendAttempt(currentTimeMs);
    FindCoordinatorRequestData data = new FindCoordinatorRequestData()
    // 注意keyType仍然是0-GROUP
    .setKeyType(FindCoordinatorRequest.CoordinatorType.GROUP.id())
    // 消费组id
    .setKey(this.groupId);
    NetworkClientDelegate.UnsentRequest unsentRequest = new NetworkClientDelegate.UnsentRequest(
        new FindCoordinatorRequest.Builder(data),
        // 不指定node,请求bootstrap中任意节点
        Optional.empty()
    );
    return unsentRequest.whenComplete((clientResponse, throwable) -> {
        getAndClearFatalError();
        if (clientResponse != null) {
            FindCoordinatorResponse response = (FindCoordinatorResponse) clientResponse.responseBody();
            onResponse(clientResponse.receivedTimeMs(), response);
        } else {
            onFailedResponse(unsentRequest.handler().completionTimeMs(), throwable);
        }
    });
}
private void onSuccessfulResponse(
    final long currentTimeMs,
    final FindCoordinatorResponseData.Coordinator coordinator
) {
    // 使用非brokerId作为connectionId,客户端与协调者broker使用单独的连接
    int coordinatorConnectionId = Integer.MAX_VALUE - coordinator.nodeId();
    this.coordinator = new Node(
        coordinatorConnectionId,
        coordinator.host(),
        coordinator.port());
}

KafkaApis.getCoordinator:broker计算leader(hash(groupId) % consumer_offsets分区数)作为协调者节点返回 默认consumer_offsets分区数为50。

private def getCoordinator(request: RequestChannel.Request, keyType: Byte, key: String): (Errors, Node) = {
    val (partition, internalTopicName) = CoordinatorType.forId(keyType) match {
        case CoordinatorType.GROUP =>
        // 仍然走这 topic=__consumer_offsets
        (groupCoordinator.partitionFor(key), GROUP_METADATA_TOPIC_NAME)
        case CoordinatorType.TRANSACTION =>
        (txnCoordinator.partitionFor(key), TRANSACTION_STATE_TOPIC_NAME)
        case CoordinatorType.SHARE =>
        // 并不会走这
        (shareCoordinator.partitionFor(SharePartitionKey.getInstance(key)), SHARE_GROUP_STATE_TOPIC_NAME)
    }
}
public int partitionFor(String groupId) {
return Utils.abs(groupId.hashCode()) % numPartitions;
}

五、消费者心跳

classic协议需要通过JoinGroup+SyncGroup完成Rebalance,新协议通过心跳完成Rebalance。

5-1、客户端

AbstractHeartbeatRequestManager.poll:新协议消费者构建心跳请求

  1. pollTimer.isExpired:应用线程超过max.poll.interval.ms=5分钟未执行poll(SharePollEvent),执行LeaveGroup,心跳包中memberEpoch=-1代表LeaveGroup;
  2. 定时发送心跳;
public NetworkClientDelegate.PollResult poll(long currentTimeMs) {
// 没发现协调者,不发心跳
if (coordinatorRequestManager.coordinator().isEmpty() || 
    membershipManager().shouldSkipHeartbeat()) {
    membershipManager().onHeartbeatRequestSkipped();
    maybePropagateCoordinatorFatalErrorEvent();
    return NetworkClientDelegate.PollResult.EMPTY;
}
pollTimer.update(currentTimeMs);
// max.poll.interval.ms = 300000 = 5分钟 自动离组
if (pollTimer.isExpired() && !membershipManager().isLeavingGroup()) {
    // member_epoch = -1
    membershipManager().transitionToSendingLeaveGroup(true);
    // 构建心跳
    NetworkClientDelegate.UnsentRequest leaveHeartbeat = makeHeartbeatRequest(currentTimeMs, true);
    heartbeatRequestState.reset();
    resetHeartbeatState();
    return new NetworkClientDelegate.PollResult(heartbeatRequestState.heartbeatIntervalMs(), Collections.singletonList(leaveHeartbeat));
}
boolean heartbeatNow = shouldSendLeaveHeartbeatNow() ||
(membershipManager().shouldHeartbeatNow() && !heartbeatRequestState.requestInFlight());
// canSendRequest=false 没到心跳时间,直接返回
if (!heartbeatRequestState.canSendRequest(currentTimeMs) && !heartbeatNow) {
    return new NetworkClientDelegate.PollResult(heartbeatRequestState.timeToNextHeartbeatMs(currentTimeMs));
}
// 构建心跳
NetworkClientDelegate.UnsentRequest request = makeHeartbeatRequest(currentTimeMs, false);
return new NetworkClientDelegate.PollResult(heartbeatRequestState.heartbeatIntervalMs(), Collections.singletonList(request));
}

需要注意,在新协议(consumer)下,消费者heartbeat.interval.ms和session.timeout.ms配置不生效,心跳间隔和心跳超时时间由服务端决定。消费者需要通过一次成功的心跳响应,获取后续的心跳间隔。(见KIP-848)

ShareHeartbeatRequestManager.buildHeartbeatRequest:心跳请求

  1. groupId:消费组id;
  2. memberId:uuid,消费者实例id;
  3. memberEpoch:消费者实例版本,刚上线是0,-1代表离组心跳,其他由服务端响应更新;
  4. subscribedTopicNames:订阅的topics,首次进组或订阅变更才发送;
public NetworkClientDelegate.UnsentRequest buildHeartbeatRequest() {
    return new NetworkClientDelegate.UnsentRequest(
        new ShareGroupHeartbeatRequest.Builder(this.heartbeatState.buildRequestData()),
        // 发送给协调者节点
        coordinatorRequestManager.coordinator());
}

public ShareGroupHeartbeatRequestData buildRequestData() {
    ShareGroupHeartbeatRequestData data = new ShareGroupHeartbeatRequestData();
    data.setGroupId(shareMembershipManager.groupId());
    // uuid
    data.setMemberId(shareMembershipManager.memberId());
    // 0-刚上线 -1-LeaveGroup
    data.setMemberEpoch(shareMembershipManager.memberEpoch());
    if (sentFields.rackId == null) {
        data.setRackId(shareMembershipManager.rackId());
        sentFields.rackId = shareMembershipManager.rackId();
    }
    // 首次进组/订阅变更,需要发送订阅的topic
    boolean sendAllFields = shareMembershipManager.state() == MemberState.JOINING;
    TreeSet<String> subscribedTopicNames = new TreeSet<>(this.subscriptions.subscription());
    if (sendAllFields || !subscribedTopicNames.equals(sentFields.subscribedTopicNames)) {
        data.setSubscribedTopicNames(new ArrayList<>(this.subscriptions.subscription()));
        sentFields.subscribedTopicNames = subscribedTopicNames;
    }
    return data;
}

AbstractHeartbeatRequestManager.onResponse:协调者返回心跳响应

  1. 心跳间隔=服务端返回heartbeatIntervalMs;
  2. 消费分区=服务端返回assignment;
private void onResponse(final R response, final long currentTimeMs) {
    if (errorForResponse(response) == Errors.NONE) {
        // 服务端返回heartbeatIntervalMs,更新到内存,作为后续心跳间隔
        heartbeatRequestState.updateHeartbeatIntervalMs(
            heartbeatIntervalForResponse(response));
        heartbeatRequestState.onSuccessfulAttempt(currentTimeMs);
        // share和非share 实现不同
        membershipManager().onHeartbeatSuccess(response);
        return;
    }
    onErrorResponse(response, currentTimeMs);
}

public void onHeartbeatSuccess(ShareGroupHeartbeatResponse response) {
ShareGroupHeartbeatResponseData responseData = response.data();
// ...
// 更新epoch
updateMemberEpoch(responseData.memberEpoch());
// 协调者下发给自己的分区
ShareGroupHeartbeatResponseData.Assignment assignment = responseData.assignment();
if (assignment != null) {
    if (!state.canHandleNewAssignment()) {
        return;
    }
    Map<Uuid, SortedSet<Integer>> newAssignment = new HashMap<>();
    assignment.topicPartitions().forEach(topicPartition -> newAssignment.put(topicPartition.topicId(), new TreeSet<>(topicPartition.partitions())));
    // 处理下发的分区
    processAssignmentReceived(newAssignment);
}
}

AbstractMembershipManager.processAssignmentReceived:处理心跳响应时,消费者发现协调者下发分区(currentTargetAssignment)与当前(currentAssignment)不同,进入RECONCILING调谐状态。

新协议下,消费者通过心跳响应得知分区分配变化,是Rebalance的开始时间。

// 自己正在消费的分区
private LocalAssignment currentAssignment;
// 协调者给自己分配的分区
private LocalAssignment currentTargetAssignment;
protected void processAssignmentReceived(Map<Uuid, SortedSet<Integer>> assignment) {
    // currentTargetAssignment = 协调者分区分区
    replaceTargetAssignmentWithNewAssignment(assignment);
    if (!targetAssignmentReconciled()) {
        // currentAssignment != currentTargetAssignment 进入 RECONCILING 状态
        transitionTo(MemberState.RECONCILING);
    } else {
        // currentAssignment == currentTargetAssignment 进入 STABLE 状态
        if (state == MemberState.RECONCILING 
            || state == MemberState.JOINING) {
            transitionTo(MemberState.STABLE);
        }
    }
}

调谐状态,需要消费者将自身assignment更新为目标assignment,比如:对于普通消费者需要把分区消费进度提交,才能更新assignment。

调谐状态推进由IO线程处理,触发方式:

  1. 应用线程调用poll api发送SharePollEvent;
  2. IO线程循环中,RequestManager(AbstractMembershipManager).poll;
// ApplicationEventProcessor#process
private void process(final SharePollEvent event) {
    // 处理 调谐 状态
    requestManagers.consumerMembershipManager.ifPresent(
        consumerMembershipManager -> consumerMembershipManager.maybeReconcile(true));
    requestManagers.shareHeartbeatRequestManager.ifPresent(hrm -> {
        hrm.membershipManager().onConsumerPoll();
        // 重置poll超时计数器
        hrm.resetPollTimer(event.pollTimeMs());
    });
}
// AbstractMembershipManager#poll
public NetworkClientDelegate.PollResult poll(final long currentTimeMs) {
    maybeReconcile(false);
    return NetworkClientDelegate.PollResult.EMPTY;
}

AbstractMembershipManager.maybeReconcile:处理调谐

  1. 比对目标状态currentTargetAssignment和当前状态currentAssignment;
  2. markPendingRevocationToPauseFetching,对于撤销分区,停止fetch;
  3. 更新currentAssignment=currentTargetAssignment,如果是普通消费者需要先完成消费进度提交,share消费者不用;

客户端调谐完成,通过心跳响应发现自己assignment=服务端返回assignment,进入STABLE状态,结束Rebalance。

// 目标状态
private LocalAssignment currentTargetAssignment;
// 当前状态
private LocalAssignment currentAssignment;
public void maybeReconcile(boolean canCommit) {
if (state != MemberState.RECONCILING) {
    return;
}
if (targetAssignmentReconciled()) {
    return;
}
if (reconciliationInProgress) {
    return;
}
// 目标分区
TopicIdPartitionSet assignedTopicIdPartitions = findResolvableAssignmentAndTriggerMetadataUpdate();
final LocalAssignment resolvedAssignment = new LocalAssignment(currentTargetAssignment.localEpoch, assignedTopicIdPartitions);
if (!currentAssignment.isNone() && resolvedAssignment.partitions.equals(currentAssignment.partitions)) {
    currentAssignment = resolvedAssignment;
    transitionTo(MemberState.ACKNOWLEDGING);
    return;
}
// share autoCommitEnabled = false
if (autoCommitEnabled && !canCommit) return;
markReconciliationInProgress();
// 目标分区 new
SortedSet<TopicPartition> assignedTopicPartitions = assignedTopicIdPartitions.toTopicNamePartitionSet();
// 当前分区 old
SortedSet<TopicPartition> ownedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR);
ownedPartitions.addAll(subscriptions.assignedPartitions());
// 新增分区
SortedSet<TopicPartition> addedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR);
addedPartitions.addAll(assignedTopicPartitions);
addedPartitions.removeAll(ownedPartitions);
// 撤销分区
SortedSet<TopicPartition> revokedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR);
revokedPartitions.addAll(ownedPartitions);
revokedPartitions.removeAll(assignedTopicPartitions);
// 停止分区fetch
markPendingRevocationToPauseFetching(revokedPartitions);
// 非share 且 自动提交 执行一次提交
CompletableFuture<Void> commitResult = signalReconciliationStarted();
commitResult.whenComplete((__, commitReqError) -> {
    // ...
    if (!maybeAbortReconciliation()) {
        // 提交完成,将目标assignment更新到内存
        // SubscriptionState#assignment
        revokeAndAssign(resolvedAssignment, assignedTopicIdPartitions, revokedPartitions, addedPartitions);
    }

});
}

5-2、协调者

GroupCoordinatorService.shareGroupHeartbeat:协调者处理心跳,处理两个状态:

  1. 消费组状态(consumer_offset):消费组成员分区分配情况;
  2. 共享组状态(share_group_state):共享组消费分区消费进度,首次创建,需要初始化;
public CompletableFuture<ShareGroupHeartbeatResponseData> shareGroupHeartbeat(
    AuthorizableRequestContext context,
    ShareGroupHeartbeatRequestData request) {
    // 处理consumer_offset状态
    return runtime.scheduleWriteOperation(
        "share-group-heartbeat",
        // TopicPartition
        topicPartitionFor(request.groupId()),
        // 超时时间
        Duration.ofMillis(config.offsetCommitTimeoutMs()),
        // CoordinatorWriteOperation
        coordinator -> coordinator.shareGroupHeartbeat(context, request)
    ).thenCompose(result -> {
        if (result.getValue().isPresent()) {
            // 返回了InitializeShareGroupStateParameters,初始化share group状态
            timer.add(new TimerTask(0L) {
                @Override
                public void run() {
                    persisterInitialize(result.getValue().get(), result.getKey())
                }
            });
        }
        return CompletableFuture.completedFuture(result.getKey());
    });
}

5-2-1、消费组状态-成员信息&分区分配

关注:1-构造哪些数据记录;2-如何分配分区;3-数据记录应用到内存。

5-2-1-1、数据记录

GroupMetadataManager.shareGroupHeartbeat:处理心跳请求,关注几个关键的consumer_offset数据记录

  1. getOrMaybeCreateShareGroup,消费组不能修改类型,即普通消费组不能调整为共享消费组
  2. hasMemberSubscriptionChanged:成员变更(订阅Topic变更、新成员上线),key=(groupId,memberId),value=ShareGroupMemberMetadataValue(clientId,topics);
  3. updateTargetAssignment:目标分区分配变化,key=(groupId,memberId),value=ShareGroupTargetAssignmentMemberValue(topics,partitionIds);
  4. maybeReconcile:如果目标分区分配变化,记录当前分区分配情况,key=(groupId,memberId), value=ShareGroupCurrentMemberAssignmentValue(memberEpoch,previousMemberEpoch,state,assignment);
  5. 响应客户端心跳间隔=share.heartbeat.interval.ms=5s,心跳超时检测=share.session.timeout.ms=45s;
  6. 如果客户端首次心跳或分区分配变化,响应assignment分区分配
  7. maybeCreateInitializeShareGroupStateRequest:如果有消费组有分区还没在share_group_state初始化,构造记录key=groupId, value=ShareGroupStatePartitionMetadataValue(已经初始化分区、正在初始化分区、正在删除分区),并返回InitializeShareGroupStateParameters异步初始化share group对应分区状态;

这部分是在处理consumer_offset状态,share_group_state是异步RPC处理,需要补偿。补偿依赖于消费组成员心跳触发,具体查看GroupMetadataManager.subscribedTopicsChangeMap方法。如果分区超过group.share.initialize.retry.interval.ms=30s仍然在consumeroffset处于initializing,认为超时,需要重新发起share_group_state对应分区的初始化。

private CoordinatorResult<Map.Entry<ShareGroupHeartbeatResponseData, 
Optional<InitializeShareGroupStateParameters>>, CoordinatorRecord> shareGroupHeartbeat(
    String groupId, String memberId, int memberEpoch,
    String rackId, String clientId,
    String clientHost, List<String> subscribedTopicNames) throws ApiException {
    final long currentTimeMs = time.milliseconds();
    final List<CoordinatorRecord> records = new ArrayList<>();
    boolean createIfNotExists = memberEpoch == 0;
    // new ShareGroup() 已经存在groupId的普通消费组,改为share,报错GroupIdNotFoundException
    final ShareGroup group = getOrMaybeCreateShareGroup(groupId, createIfNotExists);
    // group.share.max.size = 200 一个组只能有200个消费者实例
    throwIfShareGroupIsFull(group, memberId);
    // new ShareGroupMember()
    ShareGroupMember member = getOrMaybeSubscribeShareGroupMember(
        group, memberId,
        memberEpoch, createIfNotExists);
    ShareGroupMember updatedMember = new ShareGroupMember.Builder(member)
    .maybeUpdateRackId(Optional.ofNullable(rackId))
    .maybeUpdateSubscribedTopicNames(Optional.ofNullable(subscribedTopicNames))
    .setClientId(clientId)
    .setClientHost(clientHost)
    .build();
    // consumer订阅topic变化/新成员上线
    // key=groupId+memberId,value=ShareGroupMemberMetadataValue(clientId、topics)
    boolean bumpGroupEpoch = hasMemberSubscriptionChanged(
        groupId, member, updatedMember, records) || initializedAssignmentPending(group);
    int groupEpoch = group.groupEpoch();
    Map<String, SubscriptionCount> subscribedTopicNamesMap = group.subscribedTopicNames();
    SubscriptionType subscriptionType = group.subscriptionType();
    if (bumpGroupEpoch || group.hasMetadataExpired(currentTimeMs)) {
        subscribedTopicNamesMap = group.computeSubscribedTopicNames(member, updatedMember);
        // 对订阅的n个topic元数据计算一个hash,如果分区变化,这里hash会变
        long groupMetadataHash = ModernGroup.computeMetadataHash(
            subscribedTopicNamesMap,
            topicHashCache,
            metadataImage
        );
        int numMembers = group.numMembers();
        if (!group.hasMember(updatedMember.memberId())) {
            numMembers++;
        }
        subscriptionType = ModernGroup.subscriptionType(
            subscribedTopicNamesMap,
            numMembers
        );
        if (groupMetadataHash != group.metadataHash()) {
            bumpGroupEpoch = true;
        }
        if (bumpGroupEpoch) {
            groupEpoch += 1;
            // group变化
            // key=groupId, value=ShareGroupMetadataValue(groupEpoch,groupMetadataHash)
            records.add(newShareGroupEpochRecord(groupId, groupEpoch, groupMetadataHash));
        }
        group.setMetadataRefreshDeadline(currentTimeMs + METADATA_REFRESH_INTERVAL_MS, groupEpoch);
    }
    final int targetAssignmentEpoch;
    final Assignment targetAssignment;
    if (groupEpoch > group.assignmentEpoch()) {
        // group assignment变化【分区分配】
        targetAssignment = updateTargetAssignment(
            group,
            groupEpoch,
            updatedMember,
            subscriptionType,
            records
        );
        targetAssignmentEpoch = groupEpoch;
    } else {
        targetAssignmentEpoch = group.assignmentEpoch();
        targetAssignment = group.targetAssignment(updatedMember.memberId());
    }
    // member assignment变化
    updatedMember = maybeReconcile(
        groupId,
        updatedMember,
        targetAssignmentEpoch,
        targetAssignment,
        bumpGroupEpoch,
        records
    );
    // share.session.timeout.ms=45s,重新开启定时器,处理member心跳超时
    scheduleShareGroupSessionTimeout(groupId, memberId);
    ShareGroupHeartbeatResponseData response = new ShareGroupHeartbeatResponseData()
        .setMemberId(updatedMember.memberId())
        .setMemberEpoch(updatedMember.memberEpoch())
            // share.heartbeat.interval.ms=5s 心跳间隔
        .setHeartbeatIntervalMs(shareGroupHeartbeatIntervalMs(groupId));
    // 返回消费者的assignment
    boolean isFullRequest = subscribedTopicNames != null;
    if (memberEpoch == 0 || isFullRequest || hasAssignedPartitionsChanged(member, updatedMember)) {
        response.setAssignment(ShareGroupHeartbeatResponse.createAssignment(updatedMember.assignedPartitions()));
    }
    return new CoordinatorResult<>(
        records,
        Map.entry(
            response,
            // 判断是否需要初始化 共享组的topic分区状态
            // key=groupId, value=ShareGroupStatePartitionMetadataValue(已经初始化分区、正在初始化分区、正在删除分区)
            // InitializeShareGroupStateParameters = groupId + 需要初始化share_group_state的分区
            maybeCreateInitializeShareGroupStateRequest(groupId, groupEpoch, subscribedTopicNamesMap.keySet(), records)
        )
    );
}
5-2-1-2、分区分配

share group还是需要分区分配,只不过一个分区可以被一个消费组里多个消费者消费。

SimpleAssignor:共享组分区分配,区分订阅关系是否一致(成员订阅相同Topic)。

  1. 优先满足负载均衡Balance,两个成员的分区数量差异不超过1;
  2. 其次满足粘性策略Stickiness,避免分区在两个成员之间转移;
public class SimpleAssignor implements ShareGroupPartitionAssignor {
    public GroupAssignment assign(GroupSpec groupSpec, 
                                  SubscribedTopicDescriber subscribedTopicDescriber) 
    throws PartitionAssignorException {
        if (groupSpec.subscriptionType().equals(HOMOGENEOUS)) {
            // 订阅关系一致
            return new SimpleHomogeneousAssignmentBuilder(groupSpec, subscribedTopicDescriber).build();
        } else {
            // 订阅关系不一致
            return new SimpleHeterogeneousAssignmentBuilder(groupSpec, subscribedTopicDescriber).build();
        }
    }
}

SimpleHomogeneousAssignmentBuilder:简单看一下订阅关系一致的场景。

  1. desiredSharing=ceil(members/partitions),期望一个分区被n个成员共享,当成员数≤分区数,分区不会被共享;
  2. desiredAssignmentCount[i] ,在desiredSharing条件下,成员i具体分配到几个分区;

例子:

  1. 3成员7分区,因为成员数量<分区数量,最终还是分区独占;
  2. 6成员4分区,desiredSharing=ceil(6/4)=2,成员1、4分配2个分区,成员2、3、5、6分配1个分区;
public SimpleHomogeneousAssignmentBuilder(GroupSpec groupSpec, SubscribedTopicDescriber subscribedTopicDescriber) {
    this.subscribedTopicIds = groupSpec.memberSubscription(groupSpec.memberIds().iterator().next()).subscribedTopicIds();
    this.numGroupMembers = groupSpec.memberIds().size();
    this.memberIds = new ArrayList<>(groupSpec.memberIds());
    this.memberIndices = AssignorHelpers.newHashMap(numGroupMembers);
    for (int memberIndex = 0; memberIndex < numGroupMembers; memberIndex++) {
        memberIndices.put(memberIds.get(memberIndex), memberIndex);
    }
    this.targetPartitions = computeTargetPartitions(groupSpec, subscribedTopicIds, subscribedTopicDescriber);
    int numTargetPartitions = targetPartitions.size();
    if (numTargetPartitions == 0) {
        this.desiredSharing = 0;
    } else {
        // 一个分区被desiredSharing个成员共享 
        // desiredSharing = ceil(members/partitions) = (members + partitions - 1) / partitions
        // 即 成员 <= 分区数,分区不会被共享
        this.desiredSharing = (numGroupMembers + numTargetPartitions - 1) / numTargetPartitions;
    }
    // 每个成员消费n个分区
    this.desiredAssignmentCount = new int[numGroupMembers];
    this.oldGroupAssignment = AssignorHelpers.newHashMap(numGroupMembers);
    this.newGroupAssignment = AssignorHelpers.newHashMap(numGroupMembers);
    this.finalAssignmentByPartition = AssignorHelpers.newHashMap(numTargetPartitions);
    this.finalAssignmentByMember = AssignorHelpers.newHashMap(numGroupMembers);
    this.unfilledMembers = AssignorHelpers.newHashSet(numGroupMembers);
    this.overfilledMembers = AssignorHelpers.newHashSet(numGroupMembers);
    groupSpec.memberIds().forEach(memberId -> {
        int memberIndex = memberIndices.get(memberId);
        oldGroupAssignment.put(memberIndex, groupSpec.memberAssignment(memberId).partitions());
    });
    // 假设 3个成员,7个分区
    // desiredSharing=ceil(3/7) = 1.0
    // preciseDesiredAssignmentCount = 1.0 * 7 / 3 = 2.33
    // 假设 6个成员,4个分区
    // desiredSharing=ceil(6/4) = 2.0
    // preciseDesiredAssignmentCount = 2.0 * 4 / 6 = 1.33
        double preciseDesiredAssignmentCount = desiredSharing * numTargetPartitions / (double) numGroupMembers;
        for (int memberIndex = 0; memberIndex < numGroupMembers; memberIndex++) {
            desiredAssignmentCount[memberIndex] =
                (int) Math.ceil(preciseDesiredAssignmentCount * (double) (memberIndex + 1)) -
                    (int) Math.ceil(preciseDesiredAssignmentCount * (double) memberIndex);
        }
    }

SimpleHomogeneousAssignmentBuilder.build:构造同时满足desiredAssignmentCount[i]和desiredSharing的分配方案。

public GroupAssignment build() {
    if (subscribedTopicIds.isEmpty()) {
        return new GroupAssignment(Map.of());
    }
    // 撤销member不再订阅的topic
    revokeUnassignablePartitions();
    // 满足desiredAssignmentCount[i],撤销member拥有过多分区
    revokeOverfilledMembers();
    // 超过desiredSharing的分区,撤销分区有过多member共享
    revokeOversharedPartitions();
    targetPartitions.forEach(topicPartition -> finalAssignmentByPartition.computeIfAbsent(topicPartition, k -> new HashSet<>()));
    // 未达到desiredSharing的分区,按照desiredAssignmentCount[i]分配给剩余member
    assignRemainingPartitions();
    Map<String, MemberAssignment> targetAssignment = AssignorHelpers.newHashMap(numGroupMembers);
    for (int memberIndex = 0; memberIndex < numGroupMembers; memberIndex++) {
        Map<Uuid, Set<Integer>> memberAssignment = newGroupAssignment.get(memberIndex);
        if (memberAssignment == null) {
            targetAssignment.put(memberIds.get(memberIndex), new MemberAssignmentImpl(oldGroupAssignment.get(memberIndex)));
        } else {
            targetAssignment.put(memberIds.get(memberIndex), new MemberAssignmentImpl(memberAssignment));
        }
    }
    return new GroupAssignment(targetAssignment);
}
5-2-1-3、应用到内存

GroupCoordinatorShard.replay:consumer_offset记录应用到内存的统一入口

public void replay(
    long offset, long producerId, 
    short producerEpoch, CoordinatorRecord record) throws RuntimeException {
    ApiMessage key = record.key();
    ApiMessageAndVersion value = record.value();
    CoordinatorRecordType recordType = 
    CoordinatorRecordType.fromId(key.apiKey());
    switch (recordType) {
            // ... 其他数据记录
            // groupId+memberId => member元数据如订阅topics
        case SHARE_GROUP_MEMBER_METADATA:
            groupMetadataManager.replay(
                (ShareGroupMemberMetadataKey) key,
                (ShareGroupMemberMetadataValue) Utils.messageOrNull(value)
            );
            break;
            // groupId+memberId => 目标分区分配
        case SHARE_GROUP_TARGET_ASSIGNMENT_MEMBER:
            groupMetadataManager.replay(
                (ShareGroupTargetAssignmentMemberKey) key,
                (ShareGroupTargetAssignmentMemberValue) Utils.messageOrNull(value)
            );
            break;
            // groupId+memberId => 当前分区分配
        case SHARE_GROUP_CURRENT_MEMBER_ASSIGNMENT:
            groupMetadataManager.replay(
                (ShareGroupCurrentMemberAssignmentKey) key,
                (ShareGroupCurrentMemberAssignmentValue) Utils.messageOrNull(value)
            );
            break;
            // groupId => share_group_state的分区初始化情况
        case SHARE_GROUP_STATE_PARTITION_METADATA:
            groupMetadataManager.replay(
                (ShareGroupStatePartitionMetadataKey) key,
                (ShareGroupStatePartitionMetadataValue) Utils.messageOrNull(value)
            );
            break;
    }
}

GroupMetadataManager:一方面构造数据记录,另一方面管理内存状态。

  1. groups:每个消费组的信息,包括成员、成员订阅Topics、分区分配情况;
  2. shareGroupStatePartitionMetadata:ShareGroupStatePartitionMetadataValue应用到内存后的形态,记录每个分区是否在sharegroupstate正常初始化;
// groupId -> 消费组(普通/共享)
private final TimelineHashMap<String, Group> groups;
// groupId -> share_group_state元数据
private final TimelineHashMap<String, ShareGroupStatePartitionMetadataInfo> shareGroupStatePartitionMetadata;
public record ShareGroupStatePartitionMetadataInfo(
    // 正在初始化
    Map<Uuid, InitMapValue> initializingTopics,
    // 完成初始化
    Map<Uuid, InitMapValue> initializedTopics,
    Set<Uuid> deletingTopics) {
}
public record InitMapValue(
    String name, Set<Integer> partitions,
    long timestamp) { // 创建时间
}

ModernGroup是新协议消费组的顶层实现,存储消费组消费组成员信息。

targetAssignment=ShareGroupTargetAssignmentMemberValue=分区期望的分配情况。

public abstract class ModernGroup<T extends ModernGroupMember> 
implements Group {
    // groupId
    protected final String groupId;
    // group版本
    protected final TimelineInteger groupEpoch;
    // memberId -> member
    protected final TimelineHashMap<String, T> members;
    // memberId -> assignment(partitions)
    protected final TimelineHashMap<String, Assignment> targetAssignment;
}
public class ShareGroup extends ModernGroup<ShareGroupMember> {
    private final TimelineObject<ShareGroupState> state;
}
public enum ShareGroupState {
    EMPTY("Empty"),
    STABLE("Stable"),
    DEAD("Dead"),
}

public class Assignment implements MemberAssignment {
    private final Map<Uuid, Set<Integer>> partitions;
}

ModernGroupMember:存储组内每个成员的状态:

  1. subscribedTopicNames=ShareGroupMemberMetadataValue=订阅Topics;
  2. assignedPartitions=ShareGroupCurrentMemberAssignmentValue=成员当前消费分区;
public abstract class ModernGroupMember {
    protected String memberId;
    protected int memberEpoch;
    protected int previousMemberEpoch;
    protected MemberState state;
    // 订阅topics
    protected Set<String> subscribedTopicNames;
    // topicId -> partitionId
    protected Map<Uuid, Set<Integer>> assignedPartitions;
}
public class ShareGroupMember extends ModernGroupMember {
}

5-2-2、初始化共享组状态-消费进度

5-2-2-1、消费组协调者节点-发送WriteShareGroupStateRequest

上面完成了消费组状态变更,对于共享组的新加入分区,需要完成共享组状态初始化,初始化参数InitializeShareGroupStateParameters包含n个分区。

public class InitializeShareGroupStateParameters {
    private final GroupTopicPartitionData<PartitionStateData> 
    groupTopicPartitionData;
}
public class GroupTopicPartitionData<P extends PartitionInfoData> {
    private final String groupId;
    private final List<TopicData<P>> topicsData;
}
public class TopicData<P extends PartitionInfoData> {
    private final Uuid topicId;
    // n个分区
    private final List<P> partitions;
}
public class PartitionData implements ...{
    // 分区id
    private final int partition;
    // groupEpoch
    private final int stateEpoch;
    // -1
    private final long startOffset;
}

GroupCoordinatorService.persisterInitialize:异步调用共享组状态存储节点完成后,更新shareGroupStatePartitionMetadata中相关分区状态,还需要走一轮写日志+应用到内存。(这部分忽略)

private final Persister persister;
CompletableFuture<ShareGroupHeartbeatResponseData> persisterInitialize(
    InitializeShareGroupStateParameters request,
    ShareGroupHeartbeatResponseData defaultResponse) {
    // 远程调用,初始化消费组部分分区的share_group_state
    return persister.initializeState(request)
    .handle((response, exp) -> {
        if (exp == null) {
            // 无异常,initializing -> initialized
            return handlePersisterInitializeResponse(request.groupTopicPartitionData().groupId(), response, defaultResponse);
        }
        // 有异常,从initializing移除
        GroupTopicPartitionData<PartitionStateData> gtp = request.groupTopicPartitionData();
        Errors error = Errors.forException(exp);
        Map<Uuid, Set<Integer>> topicPartitionMap = new HashMap<>();
        gtp.topicsData().forEach(topicData -> topicPartitionMap.computeIfAbsent(topicData.topicId(), k -> new HashSet<>())
                                 .addAll(topicData.partitions().stream().map(PartitionStateData::partition).collect(Collectors.toSet())));
        return uninitializeShareGroupState(error, gtp.groupId(), topicPartitionMap);
    })
    .thenCompose(resp -> resp);
}

DefaultStatePersister.initializeState:按照分区维度,构造n个InitializeStateHandler,提交到Sender线程。

public CompletableFuture<InitializeShareGroupStateResult> initializeState(InitializeShareGroupStateParameters request) {
    GroupTopicPartitionData<PartitionStateData> gtp = request.groupTopicPartitionData();
    String groupId = gtp.groupId();
    Map<Uuid, Map<Integer, CompletableFuture<InitializeShareGroupStateResponse>>> futureMap = new HashMap<>();
    // 需要初始化的topic-partition,组装为n个InitializeStateHandler
    List<PersisterStateManager.InitializeStateHandler> handlers = new ArrayList<>();
    gtp.topicsData().forEach(topicData -> {
        topicData.partitions().forEach(partitionData -> {
            CompletableFuture<InitializeShareGroupStateResponse> future = futureMap
            .computeIfAbsent(topicData.topicId(), k -> new HashMap<>())
            .computeIfAbsent(partitionData.partition(), k -> new CompletableFuture<>());
            handlers.add(
                stateManager.new InitializeStateHandler(
                    groupId, topicData.topicId(), partitionData.partition(),
                    partitionData.stateEpoch(), partitionData.startOffset(),
                    future, null
                )
            );
        });
    });
    // 提交请求给sender线程,异步处理
    for (PersisterStateManager.PersisterStateManagerHandler handler : handlers) {
        stateManager.enqueue(handler);
    }
    // sender处理完毕,触发shareGroupStatePartitionMetadata更新
    CompletableFuture<Void> combinedFuture = CompletableFuture.allOf(
        handlers.stream()
        .map(PersisterStateManager.InitializeStateHandler::result)
        .toArray(CompletableFuture[]::new));
    return combinedFuture.thenApply(v -> initializeResponsesToResult(futureMap));
}

PersisterStateManager.SendThread.generateRequests:Sender线程

  1. 优先从当前节点元数据缓存获取共享组状态存储节点 = leaderOf(hash(groupId,topic,partitionId) % 50),默认share_group_state也是50分区
  2. 如果元数据缓存没有,走FindCoordinator请求任意节点获取共享组状态存储节点,和获取普通协调者一致,只是keyType=2=SHARE;
  3. 对每个共享组状态存储节点,合并为一个WriteShareGroupStateRequest
public Collection<RequestAndCompletionHandler> generateRequests() {
    List<RequestAndCompletionHandler> requests = new ArrayList<>();
    if (!queue.isEmpty()) {
        PersisterStateManagerHandler handler = queue.peek();
        queue.poll();
        // 1. 从metadataCache中获取 共享组状态存储节点
        if (handler.lookupNeeded()) {
            Node randomNode = randomNode();
            if (randomNode == Node.noNode()) {
                handler.findCoordinatorErrorResponse(Errors.COORDINATOR_NOT_AVAILABLE);
                return List.of();
            }
            // 2. 如果metadataCache中不存在,发起find coordinator RPC
            return List.of(new RequestAndCompletionHandler(
                time.milliseconds(),
                randomNode,
                handler.findShareCoordinatorBuilder(),
                handler
            ));
        }
    }
    // ...
    // 3. 组装WriteShareGroupStateRequest
    requests.add(new RequestAndCompletionHandler(
        time.milliseconds(),
        coordNode,
        // 按照 共享组状态存储节点 合并请求
        combinedRequestPerTypePerGroup,
        response -> {
            handlersPerGroup.forEach(handler1 -> handler1.onComplete(response));
            wakeup();
        }));
    return requests;
}
5-2-2-2、共享组状态存储节点-初始化共享组分区消费进度

ShareCoordinatorService.writeState:共享组状态存储节点,循环每个分区,提交共享组状态初始化,分区即hash(groupId,topic,partitionId)%50。

public CompletableFuture<WriteShareGroupStateResponseData> writeState(RequestContext context, WriteShareGroupStateRequestData request) {
    request.topics().forEach(topicData -> {
        Map<Integer, CompletableFuture<WriteShareGroupStateResponseData>> partitionFut =
        futureMap.computeIfAbsent(topicData.topicId(), k -> new HashMap<>());
        topicData.partitions().forEach(
            partitionData -> {
                CompletableFuture<WriteShareGroupStateResponseData> future = runtime.scheduleWriteOperation(
                    "write-share-group-state",
                    topicPartitionFor(SharePartitionKey.getInstance(groupId, topicData.topicId(), partitionData.partition())),
                    Duration.ofMillis(config.shareCoordinatorWriteTimeoutMs()),
                    coordinator -> coordinator.writeState(new WriteShareGroupStateRequestData()...);
                partitionFut.put(partitionData.partition(), future);
            });
    });
    //..
}

ShareCoordinatorShard.generateShareStateRecord:生成share_group_state数据记录。

// 共享组分区消费进度
private final TimelineHashMap<SharePartitionKey, ShareGroupOffset> shareStateMap; 
private CoordinatorRecord generateShareStateRecord(
    WriteShareGroupStateRequestData.PartitionData partitionData,
    SharePartitionKey key,
    boolean updateLeaderEpoch) {
    long timestamp = time.milliseconds();
    ShareGroupOffset currentState = shareStateMap.get(key);
    int newLeaderEpoch = currentState.leaderEpoch();
    if (updateLeaderEpoch) {
        newLeaderEpoch = partitionData.leaderEpoch() != -1 ? partitionData.leaderEpoch() : newLeaderEpoch;
    }
    // ...
    // key=groupId,topicId,partition,value=ShareUpdateValue
    return ShareCoordinatorRecordHelpers.newShareUpdateRecord(
        key.groupId(), key.topicId(), partitionData.partition(),
        new ShareGroupOffset.Builder()
        .setSnapshotEpoch(currentState.snapshotEpoch())
        .setStartOffset(partitionData.startOffset())
        .setDeliveryCompleteCount(partitionData.deliveryCompleteCount())
        .setLeaderEpoch(newLeaderEpoch)
        .setStateBatches(mergeBatches(List.of(), partitionData))
        .build());
}

ShareCoordinatorShard.shareStateMap:最终share_group_state记录应用到内存,即共享组的消费进度

// groupId,topicId,partition -> 共享组消费进度
private final TimelineHashMap<SharePartitionKey, ShareGroupOffset> shareStateMap; 
private void handleShareUpdate(ShareUpdateKey key, ShareUpdateValue value) {
    SharePartitionKey mapKey = SharePartitionKey.getInstance(key.groupId(), key.topicId(), key.partition());
    maybeUpdateLeaderEpochMap(mapKey, value.leaderEpoch());

    ShareGroupOffset offsetRecord = ShareGroupOffset.fromRecord(value);
    // merge 合并分区消费进度
    shareStateMap.compute(mapKey, (k, v) -> v == null ? offsetRecord : merge(v, value));
    snapshotUpdateCount.compute(mapKey, (k, v) -> v == null ? 0 : v + 1);
}

ShareGroupOffset:共享组消费分区的消费进度

  1. stateBatches:需要存储状态的n个消息批次,比如offset=[1,10]被超时release,变为AVAILABLE,可以被重新拉取消费,消费次数=deliveryCount;
  2. startOffset:stateBatches的第一个offset,当stateBatches为空就是消费进度;
public class ShareGroupOffset {
    private final int snapshotEpoch;
    private final int stateEpoch;
    private final int leaderEpoch;
    private final long startOffset;
    private final int deliveryCompleteCount;
    private final List<PersisterStateBatch> stateBatches;
    private final long createTimestamp;
    private final long writeTimestamp;
}
public class PersisterStateBatch implements Comparable<PersisterStateBatch> {
    // 批次起始offset
    private final long firstOffset;
    // 批次结束offset
    private final long lastOffset;
    // 发送次数
    private final short deliveryCount;
    // RecordState
    private final byte deliveryState;
}
public enum RecordState {
    AVAILABLE((byte) 0),
    ACQUIRED((byte) 1),
    ACKNOWLEDGED((byte) 2),
    ARCHIVED((byte) 4);
}

六、ShareFetch

6-1、消费者发送Fetch

ShareConsumerImpl.collect:应用线程poll消息,优先从ShareFetchBuffer内存中获取,如果没有,则发送ShareFetchEvent给IO线程。

// 内存消息
private final ShareFetchBuffer fetchBuffer;
private ShareFetch<K, V> collect(Map<TopicIdPartition, NodeAcknowledgements> acknowledgementsMap) {
    Map<TopicIdPartition, NodeAcknowledgements> acksToSend = acknowledgementsMap;
    if (currentFetch.isEmpty() && !currentFetch.hasRenewals()) {
        final ShareFetch<K, V> fetch = fetchCollector.collect(fetchBuffer);
        if (fetch.isEmpty()) {
            // 内存里无可用消息
            // 发送ShareFetchEvent,让IO线程发送ShareFetchRequest
            if (shouldSendShareFetchEvent) {
                applicationEventHandler.add(new ShareFetchEvent(acksToSend));
                shouldSendShareFetchEvent = false;
                applicationEventHandler.wakeupNetworkThread();
            }
        }
        return fetch;
    }
}

ShareConsumeRequestManager.poll:IO线程,针对每个node(分区leader)构建ShareFetchRequest。ShareSessionHandler,之前2.x讲过,KIP-227通过与服务端建立Session机制(cwiki.apache.org/confluence/…),支持增量Fetch,不用每次传入全量分区列表,只需要传入变化分区列表。

public PollResult poll(long currentTimeMs) {
    // 如果用户使用commitSync api,不执行fetch
    PollResult pollResult = processAcknowledgements(currentTimeMs);
    if (pollResult != null) {
        return pollResult;
    }
    // broker - session处理
    Map<Node, ShareSessionHandler> handlerMap = new HashMap<>();
    // 分区
    for (TopicPartition partition : partitionsToFetch()) {
        Optional<Node> leaderOpt = metadata.currentLeader(partition).leader;
        if (leaderOpt.isEmpty()) {
            // leader不存在,跳过
            metadata.requestUpdate(false);
            continue;
        }
        Uuid topicId = topicIds.get(partition.topic());
        if (topicId == null) {
            // topic元数据没有,跳过
            metadata.requestUpdate(false);
            continue;
        }
        Node node = leaderOpt.get();
        if (nodesWithPendingRequests.contains(node.id())) {
            // 节点仍然有fetch请求在处理,跳过
        } else {
            ShareSessionHandler handler = handlerMap.computeIfAbsent(node,
                                                                     k -> sessionHandlers.computeIfAbsent(node.id(), n -> new ShareSessionHandler(logContext, n, memberId)));
            TopicIdPartition tip = new TopicIdPartition(topicId, partition);
            Acknowledgements acknowledgementsToSend = null;
            boolean canSendAcknowledgements = true;
            // piggy back ack
            Map<TopicIdPartition, Acknowledgements> nodeAcksFromFetchMap = fetchAcknowledgementsToSend.get(node.id());
            if (nodeAcksFromFetchMap != null) {
                acknowledgementsToSend = nodeAcksFromFetchMap.remove(tip);
                if (acknowledgementsToSend != null) {
                    if (!maybeAddAcknowledgements(handler, node, tip, acknowledgementsToSend)) {
                        canSendAcknowledgements = false;
                    }
                }
            }
            // 分区加入session
            if (canSendAcknowledgements) {
                handler.addPartitionToFetch(tip, acknowledgementsToSend);
            } else {
                handler.addPartitionToFetch(tip, null);
            }
            topicNamesMap.putIfAbsent(new IdAndPartition(tip.topicId(), tip.partition()), tip.topic());
        }
    }
    // ...
    List<UnsentRequest> requests = handlerMap.entrySet().stream().map(entry -> {
        Node target = entry.getKey();
        ShareSessionHandler handler = entry.getValue();
        // ...
        // 构建fetch请求
        ShareFetchRequest.Builder requestBuilder = handler.newShareFetchBuilder(groupId, shareFetchConfig, canSkipIfRequestEmpty);
        nodesWithPendingRequests.add(target.id());
        BiConsumer<ClientResponse, Throwable> responseHandler = (clientResponse, error) -> {
            // fetch响应处理
            if (error != null) {
                handleShareFetchFailure(target, requestBuilder.data(), error);
            } else {
                handleShareFetchSuccess(target, requestBuilder.data(), clientResponse);
            }
        };
        return new UnsentRequest(requestBuilder, Optional.of(target)).whenComplete(responseHandler);
    }).filter(Objects::nonNull).collect(Collectors.toList());
    return new PollResult(requests);
}

ShareFetchRequest.json:ShareFetchRequest请求如下

  1. 相较于普通FetchRequest(普通消费组拉消息请求),不再包含FetchOffset,即消费进度不由客户端传入,由服务端管理
  2. AcknowledgementBatches,针对每个消息批次,支持把ack附着在Fetch请求里,减少网络交互次数;
  3. MaxWaitMs=fetch.max.wait.ms=500ms,没数据,fetch请求在服务端挂起500ms;
  4. MinBytes=fetch.min.bytes=1byte,只要有数据就返回,MaxBytes=fetch.max.bytes=50MB,响应不超过50MB数据;
{
  "apiKey": 78,
  "type": "request",
  "listeners": ["broker"],
  "name": "ShareFetchRequest",
  "validVersions": "1-2",
  "flexibleVersions": "0+",
  "fields": [
    { "name": "GroupId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", "entityType": "groupId",
      "about": "The group identifier." },
    { "name": "MemberId", "type": "string", "versions": "0+", "nullableVersions": "0+",
      "about": "The member ID." },
    { "name": "ShareSessionEpoch", "type": "int32", "versions": "0+",
      "about": "The current share session epoch: 0 to open a share session; -1 to close it; otherwise increments for consecutive requests." },
    { "name": "MaxWaitMs", "type": "int32", "versions": "0+",
      "about": "The maximum time in milliseconds to wait for the response." },
    { "name": "MinBytes", "type": "int32", "versions": "0+",
      "about": "The minimum bytes to accumulate in the response." },
    { "name": "MaxBytes", "type": "int32", "versions": "0+", "default": "0x7fffffff",
      "about": "The maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored." },
    { "name": "MaxRecords", "type": "int32", "versions": "1+",
      "about": "The maximum number of records to fetch. This limit can be exceeded for alignment of batch boundaries." },
    { "name": "BatchSize", "type": "int32", "versions": "1+",
      "about": "The optimal number of records for batches of acquired records and acknowledgements." },
    { "name": "ShareAcquireMode", "type": "int8", "versions": "2+", "default": "0", "ignorable": true,
      "about": "The acquire mode to control the fetch behavior - 0:batch-optimized,1:record-limit." },
    { "name": "IsRenewAck", "type": "bool", "versions": "2+", "default": "false",
      "about": "Whether Renew type acknowledgements present in AcknowledgementBatches." },
    { "name": "Topics", "type": "[]FetchTopic", "versions": "0+",
      "about": "The topics to fetch.", "fields": [
        { "name": "TopicId", "type": "uuid", "versions": "0+", "about": "The unique topic ID.", "mapKey": true },
        { "name": "Partitions", "type": "[]FetchPartition", "versions": "0+",
          "about": "The partitions to fetch.", "fields": [
            { "name": "PartitionIndex", "type": "int32", "versions": "0+", "mapKey": true,
              "about": "The partition index." },
            { "name": "PartitionMaxBytes", "type": "int32", "versions": "0",
              "about": "The maximum bytes to fetch from this partition. 0 when only acknowledgement with no fetching is required. See KIP-74 for cases where this limit may not be honored." },
            { "name": "AcknowledgementBatches", "type": "[]AcknowledgementBatch", "versions": "0+",
              "about": "Record batches to acknowledge.", "fields": [
                { "name": "FirstOffset", "type": "int64", "versions": "0+",
                  "about": "First offset of batch of records to acknowledge." },
                { "name": "LastOffset", "type": "int64", "versions": "0+",
                  "about": "Last offset (inclusive) of batch of records to acknowledge." },
          { "name": "AcknowledgeTypes", "type": "[]int8", "versions": "0+",
            "about": "Array of acknowledge types - 0:Gap,1:Accept,2:Release,3:Reject,4:Renew." }
        ]}
      ]}
    ]}
  ]
}

6-2、分区Leader处理Fetch

KafkaApis.handleShareFetchRequest:Leader处理Fetch,暂时忽略ack。

def handleShareFetchRequest(request: RequestChannel.Request): CompletableFuture[Unit] = {
  val shareFetchRequest = request.body[ShareFetchRequest]
  def isAcknowledgeDataPresentInFetchRequest: Boolean = {
    shareFetchRequest.data.topics.asScala
    .flatMap(t => t.partitions().asScala)
    .exists(partition => partition.acknowledgementBatches != null && !partition.acknowledgementBatches.isEmpty)
  }
  val isAcknowledgeDataPresent = isAcknowledgeDataPresentInFetchRequest
  // 1. 处理ack
  if (isAcknowledgeDataPresent) {
    val erroneous = mutable.Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]()
    val acknowledgementDataFromRequest = getAcknowledgeBatchesFromShareFetchRequest(shareFetchRequest, topicIdNames, erroneous)
    acknowledgeResult = handleAcknowledgements(...)
  }
  // 2. 处理fetch
  val fetchResult: CompletableFuture[Map[TopicIdPartition, ShareFetchResponseData.PartitionData]] =
  handleFetchFromShareFetchRequest(request, shareSessionEpoch, erroneousAndValidPartitionData, sharePartitionManager, authorizedTopics)
  // 3. 组合 ack和fetch 响应
  combineShareFetchAndShareAcknowledgeResponses(fetchResult, acknowledgeResult).handle[Unit] {(result, exception) =>
    if (exception != null) {
      requestHelper.sendMaybeThrottle(request, shareFetchRequest.getErrorResponse(AbstractResponse.DEFAULT_THROTTLE_TIME, exception))
    } else {
      requestChannel.sendResponse(request, result, None)
    }
  }
}

6-2-1、初始化分区消费进度

SharePartition:管理某个消费组分区的消费状态。

  1. cachedState:n个正在消费的批次的状态,key=批次起始offset;
  2. startOffsetendOffset:正在消费的消息批次的上下界offset;
private final String groupId;
private final TopicIdPartition topicIdPartition;
private final int leaderEpoch;
// 正在消费批次的起始offset -> 正在消费批次的状态
private final NavigableMap<Long, InFlightBatch> cachedState;
// 是否需要重新计算下一个fetchOffset
private boolean findNextFetchOffset;
// share_group_state存储,rpc调用共享组协调者
private final Persister persister;
// cachedState的起始offset
private long startOffset;
// cachedState的结束offset
private long endOffset;

InFlightBatch是正在消费的消息批次状态。

  1. offsetState:如果消息批次中不同消息被不同方式ack,比如:offset=1 ack release 需要重新消费,offset=[2,3] ack accept 消费成功,offsetState单独跟踪每条记录状态
  2. batchState:如果消息批次被整体处理,比如整体ack release 需要重新消费,跟踪批次整体状态
cachedState: ConcurrentSkipListMap<Long, InFlightBatch>
     
     ├── Key: Long (firstOffset)   批次的起始偏移量
     
     └── Value: InFlightBatch     批次状态
          
          ├── firstOffset: long       批次起始 offset
          ├── lastOffset: long        批次结束 offset
          ├── batchState: InFlightState   批次整体状态
                  
                  ├── state: RecordState      状态 
                  ├── deliveryCount: int      消费次数
                  ├── memberId: String       持有该批次的消费者
                  └── acquisitionLockTimeoutTask: TimerTask  超时释放任务
          
          └── offsetState: NavigableMap<Long, InFlightState>  单独跟踪每条记录状态

SharePartitionManager.processShareFetch:当消费组分区首次加入,Leader分区并不存储share_group_state,需要初始化消费组分区消费进度,存储在内存SharePartition。后续直接根据SharePartition,决定从哪个offset开始fetch。

void processShareFetch(ShareFetch shareFetch) {
  List<DelayedShareFetchKey> delayedShareFetchWatchKeys = new ArrayList<>();
  LinkedHashMap<TopicIdPartition, SharePartition> sharePartitions = new LinkedHashMap<>();
  Set<String> topics = new HashSet<>();
  for (TopicIdPartition topicIdPartition : shareFetch.topicIdPartitions()) {
    topics.add(topicIdPartition.topic());
    SharePartitionKey sharePartitionKey = sharePartitionKey(
      shareFetch.groupId(),
      topicIdPartition
    );
    // 1. 创建内存SharePartition,管理share group的消费情况
    SharePartition sharePartition;
    try {
      sharePartition = getOrCreateSharePartition(sharePartitionKey);
    } catch (Exception e) {
      shareFetch.addErroneous(topicIdPartition, e);
      continue;
    }
    DelayedShareFetchKey delayedShareFetchKey = new DelayedShareFetchGroupKey(shareFetch.groupId(),
                                                                              topicIdPartition.topicId(), topicIdPartition.partition());
    delayedShareFetchWatchKeys.add(delayedShareFetchKey);
    delayedShareFetchWatchKeys.add(new DelayedShareFetchPartitionKey(topicIdPartition.topicId(), topicIdPartition.partition()));
    // 2. 发送ReadShareGroupStateRequest给share group协调者,初始化内存SharePartition
    CompletableFuture<Void> initializationFuture = sharePartition.maybeInitialize();
    //...
    sharePartitions.put(topicIdPartition, sharePartition);
  }
  if (shareFetch.errorInAllPartitions()) {
    shareFetch.maybeComplete(Map.of());
    return;
  }
  // 3. 尝试执行fetch,或挂起
  addDelayedShareFetch(new DelayedShareFetch(shareFetch, replicaManager, fencedSharePartitionHandler(), sharePartitions, shareGroupMetrics, time, remoteFetchMaxWaitMs), 
                     delayedShareFetchWatchKeys);
}

SharePartition.maybeInitialize:消费分区Leader通过leader(hash(group、topic、partition)%50)找share_group_state存储节点,发送ReadShareGroupStateRequest读取分区消费进度,初始化消费进度。如果startOffset=-1,代表无消费进度,默认消费组配置share.auto.offset.reset=latest(可选earliest/by_duration),将分区高水位作为初始消费进度;

传统消费者需要自行通过ListOffsetRequest获取初始fetchOffset。

public CompletableFuture<Void> maybeInitialize() {
    CompletableFuture<Void> future = new CompletableFuture<>();
    persister.readState(new ReadShareGroupStateParameters.Builder()
                        .setGroupTopicPartitionData(new GroupTopicPartitionData.Builder<PartitionIdLeaderEpochData>()
                                            .setGroupId(this.groupId)
                                            .setTopicsData(List.of(new TopicData<>(topicIdPartition.topicId(),
                                                                                   List.of(PartitionFactory.newPartitionIdLeaderEpochData(topicIdPartition.partition(), leaderEpoch)))))
                                            .build())
                        .build()
                       ).whenComplete((result, exception) -> {
        lock.writeLock().lock();
        try {
            TopicData<PartitionAllData> state = result.topicsData().get(0);
            // 如果消费组分区没消费进度,初始化起始offset
            startOffset = startOffsetDuringInitialization(partitionData.startOffset());
            stateEpoch = partitionData.stateEpoch();
            // 正在消费的消息批次状态
            List<PersisterStateBatch> stateBatches = partitionData.stateBatches();
            long gapStartOffset = -1;
            long previousBatchLastOffset = startOffset - 1;
            for (PersisterStateBatch stateBatch : stateBatches) {
                if (gapStartOffset == -1 && stateBatch.firstOffset() > previousBatchLastOffset + 1) {
                    gapStartOffset = previousBatchLastOffset + 1;
                }
                previousBatchLastOffset = stateBatch.lastOffset();
                // 正在消费的消息批次状态
                InFlightBatch inFlightBatch = new InFlightBatch(timer, time, EMPTY_MEMBER_ID, stateBatch.firstOffset(),
                                                                stateBatch.lastOffset(), RecordState.forId(stateBatch.deliveryState()), stateBatch.deliveryCount(),
                                                                null, timeoutHandler, sharePartitionMetrics);
                cachedState.put(stateBatch.firstOffset(), inFlightBatch);
                if (isStateTerminal(RecordState.forId(stateBatch.deliveryState()))) {
                    deliveryCompleteCount.addAndGet((int) (stateBatch.lastOffset() - stateBatch.firstOffset() + 1));
                }
            }
            if (!cachedState.isEmpty()) {
                // 有正在消费的消息批次
                updateFindNextFetchOffset(true);
                endOffset = cachedState.lastEntry().getValue().lastOffset();
                if (gapStartOffset != -1) {
                    persisterReadResultGapWindow = new GapWindow(endOffset, gapStartOffset);
                }
                maybeUpdateCachedStateAndOffsets();
            } else {
                // 无正在消费的消息批次,start=end
                endOffset = startOffset;
            }
            partitionState = SharePartitionState.ACTIVE;
        } catch (Exception e) {
            throwable = e;
        } finally {
            boolean isFailed = throwable != null;
            if (isFailed) {
                partitionState = SharePartitionState.FAILED;
            }
            lock.writeLock().unlock();
            if (isFailed) {
                future.completeExceptionally(throwable);
            } else {
                future.complete(null);
            }
        }
    });
    return future;
}

ShareCoordinatorShard.readStateAndMaybeUpdateLeaderEpoch:share_group_state所在节点,读取内存中的消费进度,返回消费分区Leader。

private final TimelineHashMap<SharePartitionKey, ShareGroupOffset> shareStateMap;  
public CoordinatorResult<ReadShareGroupStateResponseData, CoordinatorRecord> 
readStateAndMaybeUpdateLeaderEpoch(ReadShareGroupStateRequestData request) {
    ReadShareGroupStateRequestData.ReadStateData topicData = request.topics().get(0);
    ReadShareGroupStateRequestData.PartitionData partitionData = topicData.partitions().get(0);
    Uuid topicId = topicData.topicId();
    int partitionId = partitionData.partition();
    int leaderEpoch = partitionData.leaderEpoch();
    SharePartitionKey key = SharePartitionKey.getInstance(request.groupId(), topicId, partitionId);
    ShareGroupOffset offsetValue = shareStateMap.get(key);
    List<ReadShareGroupStateResponseData.StateBatch> stateBatches = (offsetValue.stateBatches() != null && !offsetValue.stateBatches().isEmpty()) ?
    offsetValue.stateBatches().stream()
    .map(
        stateBatch -> new ReadShareGroupStateResponseData.StateBatch()
        .setFirstOffset(stateBatch.firstOffset())
        .setLastOffset(stateBatch.lastOffset())
        .setDeliveryState(stateBatch.deliveryState())
        .setDeliveryCount(stateBatch.deliveryCount())
    ).toList() : List.of();
    ReadShareGroupStateResponseData responseData = ReadShareGroupStateResponse.toResponseData(
        topicId,
        partitionId,
        offsetValue.startOffset(),
        offsetValue.stateEpoch(),
        stateBatches
    );
    // 分区leaderEpoch没变化,直接返回
    if (leaderEpoch == -1 ||
        (leaderEpochMap.get(key) != null && leaderEpochMap.get(key) == leaderEpoch)) {
        return new CoordinatorResult<>(List.of(), responseData);
    }
    // leaderEpoch变更,要写一条记录
    WriteShareGroupStateRequestData.PartitionData writePartitionData = new WriteShareGroupStateRequestData.PartitionData()
    .setPartition(partitionId)
    .setLeaderEpoch(leaderEpoch)
    .setStateBatches(List.of())
    .setStartOffset(responseData.results().get(0).partitions().get(0).startOffset())
    .setDeliveryCompleteCount(offsetValue.deliveryCompleteCount())
    .setStateEpoch(responseData.results().get(0).partitions().get(0).stateEpoch());
    CoordinatorRecord record = generateShareStateRecord(writePartitionData, key, true);
    return new CoordinatorResult<>(List.of(record), responseData);
}

6-2-2、执行Fetch

DelayedShareFetch.tryComplete:

  1. acquirablePartitions,获取分区锁和fetchOffset,获取分区锁失败,则挂起,等待超时或其他线程唤醒;
  2. maybeReadFromLog,针对获取分区锁成功的分区,读消息数据;
  3. forceComplete,读异常或读取字节数满足minBytes(1),释放分区锁,响应客户端;
  4. releasePartitionLocks,不满足3则释放分区锁;
// fetch请求
private final ShareFetch shareFetch;
// SharePartition
private final LinkedHashMap<TopicIdPartition, SharePartition> sharePartitions;
public boolean tryComplete() {
  // 1. 尝试获取分区锁,返回 获取成功的分区 -> fetch offset
  LinkedHashMap<TopicIdPartition, Long> topicPartitionData = acquirablePartitions(sharePartitions);
  try {
    if (!topicPartitionData.isEmpty()) {
      // 2. 读log
      LinkedHashMap<TopicIdPartition, LogReadResult> replicaManagerReadResponse = 
      maybeReadFromLog(topicPartitionData);
      maybeUpdateFetchOffsetMetadata(topicPartitionData, replicaManagerReadResponse);
      if (anyPartitionHasLogReadError(replicaManagerReadResponse) 
              || isMinBytesSatisfied(topicPartitionData, 
                                     partitionMaxBytesStrategy.maxBytes(shareFetch.fetchParams().maxBytes, 
                                                        topicPartitionData.keySet(), topicPartitionData.size()))) {
        // 3. 如果有分区异常,或满足minBytes(1-有数据就返回),响应客户端
        partitionsAcquired = topicPartitionData;
        localPartitionsAlreadyFetched = replicaManagerReadResponse;
        return forceComplete();
      } else {
        // 4. 没数据,要挂起请求,释放分区锁
        releasePartitionLocks(topicPartitionData.keySet());
      }
    }
    return false;
  } catch (Exception e) {
    return forceComplete();
  }
}

DelayedShareFetch.acquirablePartitions:循环消费分区,用fetch请求唯一uuid+CAS获取分区锁,并获取nextFetchOffset。

// 每次fetch请求生成uuid,用于获取锁
private final Uuid fetchId;
LinkedHashMap<TopicIdPartition, Long> acquirablePartitions(
  LinkedHashMap<TopicIdPartition, SharePartition> sharePartitionsForAcquire) {
  LinkedHashMap<TopicIdPartition, Long> topicPartitionData = new LinkedHashMap<>();
  sharePartitionsForAcquire.forEach((topicIdPartition, sharePartition) -> {
    if (sharePartition.maybeAcquireFetchLock(fetchId)) {
      try {
        if (sharePartition.canAcquireRecords()) {
          // 如果能获取数据,等读取数据后才会释放锁
          // 计算分区fetchOffset
          topicPartitionData.put(topicIdPartition, 
                                           sharePartition.nextFetchOffset());
        } else {
          // 释放锁
          sharePartition.releaseFetchLock(fetchId);
        }
      } catch (Exception e) {
        sharePartition.releaseFetchLock(fetchId);
      }
    }
  });
  return topicPartitionData;
}
// SharePartition#maybeAcquireFetchLock
// cas成功,则fetch请求获取分区锁成功
private final AtomicReference<Uuid> fetchLock;
public boolean maybeAcquireFetchLock(Uuid fetchId) {
  boolean acquired = fetchLock.compareAndSet(null, Objects.requireNonNull(fetchId));
  if (acquired) {
    long currentTime = time.hiResClockMs();
  }
  return acquired;
}
// SharePartition#canAcquireRecords
boolean canAcquireRecords() {
  // nextFetchOffset = 下一个fetch的offset
  // endOffset = 曾经fetch的最大offset
  // 如果不相等,应该是有以前的消息被释放了,比如超时
  if (nextFetchOffset() != endOffset() + 1) {
    return true;
  }
  // group.share.partition.max.record.locks = 2000,每个分区最多2000条记录被锁
  return numInFlightRecords() < maxInFlightRecords;
}

SharePartition.nextFetchOffset:nextFetchOffset的计算逻辑如下

  1. 简单场景,如所有批次都被完整正常消费,直接返回endOffset(上次fetch进度);
  2. cachedState有正在消费的消息批次
    1. 有gap,如offset=[0,10]和offset=[20,30]正在消费,nextFetchOffset=11。场景如消费者只ack release了0-10和20-30,share_group_state只存储了这两个区间,[11,20)没持久化,broker重启恢复后,只能从11开始消费;
    2. 无gap,正在消费的批次被完整ack(release)释放,返回第一个被完整释放的批次的起始offset。如offset=[10,20]被ack(release),则从10开始消费;
    3. 无gap,正在消费的批次不是完整处理的,比如针对一个批次[10,12]:ack(offset=10, release),ack(offset=11, accept),ack(offset=12, release),则从10开始消费;
// 是否需要重新计算fetchOffset
private boolean findNextFetchOffset;
// 正在被消费的批次 key=批次起始offset
private final NavigableMap<Long, InFlightBatch> cachedState;
// cachedState中第一条消息offset
private long startOffset;
// 被fetch的最后一条消息offset
private long endOffset;
public long nextFetchOffset() {
    lock.writeLock().lock();
    try {
        // case1,简单场景,简单返回endOffset即可
        if (!findNextFetchOffset) {
            if (cachedState.isEmpty() 
                || startOffset > cachedState.lastEntry().getValue().lastOffset()) {
                return endOffset;
            } else {
                return endOffset + 1;
            }
        }
        if (cachedState.isEmpty() 
            || startOffset > cachedState.lastEntry().getValue().lastOffset()) {
            updateFindNextFetchOffset(false);
            return endOffset;
        }
        // findNextFetchOffset=true,需要做复杂计算
        long nextFetchOffset = -1;
        long gapStartOffset = isPersisterReadGapWindowActive() ? persisterReadResultGapWindow.gapStartOffset() : -1;
        // 循环所有正在消费的批次
        for (Map.Entry<Long, InFlightBatch> entry : cachedState.entrySet()) {
            if (isPersisterReadGapWindowActive()) {
                if (entry.getKey() > gapStartOffset) {
                    // case2 有gap存在,找gap起始offset返回
                    nextFetchOffset = gapStartOffset;
                    break;
                }
                gapStartOffset = Math.max(entry.getValue().lastOffset() + 1, gapStartOffset);
            }
            if (entry.getValue().offsetState() == null) {
                // case3 批次被完整处理 且 批次被release ack,允许重复消费
                if (entry.getValue().batchState() == RecordState.AVAILABLE && !entry.getValue().batchHasOngoingStateTransition()) {
                    nextFetchOffset = entry.getValue().firstOffset();
                    break;
                }
            } else {
                // case4 批次没有被完整处理 比如offset=1正常ack(ACCEPT),offset=2被release(RELEASE),返回offset=2
                for (Map.Entry<Long, InFlightState> offsetState : entry.getValue().offsetState().entrySet()) {
                    if (offsetState.getValue().state() == RecordState.AVAILABLE && !offsetState.getValue().hasOngoingStateTransition()) {
                        nextFetchOffset = offsetState.getKey();
                        break;
                    }
                }
                if (nextFetchOffset != -1) {
                    break;
                }
            }
        }
        if (nextFetchOffset == -1) {
            updateFindNextFetchOffset(false);
            nextFetchOffset = endOffset + 1;
        }
        return nextFetchOffset;
    } finally {
        lock.writeLock().unlock();
    }
}

6-2-3、更新消息消费状态(消息锁)

ShareFetchUtils.processFetchResponse:处理响应,主要需要SharePartition.acquire更新内存分区消费状态。

SharePartition.acquire:结合nextFetchOffset的计算逻辑,从底层Log拉取消息批次后,还需要进行过滤并将被拉取的消息批次记录到cachedState的InFlightBatch中。

  1. 如果cachedState为空,代表没有正在消费的批次,需要创建内存InFlightBatch,否则需要结合InFlightBatch过滤实际返回客户端的数据;
  2. inFlightBatch.offsetState() != null,批次被部分提交,筛选AVAILABLE(被释放)的单条消息记录offset返回,更新状态ACQUIRED(正在消费);
  3. inFlightBatch.offsetState() = null,批次被整体释放,筛选AVAILABLE批次,更新状态ACQUIRED
  4. InFlightBatch处理完毕,没达到maxFetchRecords,创建新的内存InFlightBatch;
  5. 共享组也支持隔离级别,仅读取已提交消息,通过消费组share.isolation.level=read_committed(默认read_uncommitted)配置。普通消费组是在客户端做事务消息过滤的(比如跳过控制批次和Abort的消息批次),共享组在服务端做了事务消息过滤
public ShareAcquiredRecords acquire(
    String memberId,
    ShareAcquireMode shareAcquireMode,
    int batchSize,
    int maxFetchRecords,
    long fetchOffset,
    // fetch到的消息记录,n个批次
    FetchPartitionData fetchPartitionData,
    FetchIsolation isolationLevel) {
    RecordBatch lastBatch = fetchPartitionData.records.lastBatch().orElse(null);

    LastOffsetAndMaxRecords lastOffsetAndMaxRecords = lastOffsetAndMaxRecordsToAcquire(fetchOffset,
                                                                                       maxFetchRecords, lastBatch.lastOffset());
    if (lastOffsetAndMaxRecords.maxRecords() <= 0) {
        return ShareAcquiredRecords.empty();
    }
    int maxRecordsToAcquire = lastOffsetAndMaxRecords.maxRecords();
    final long lastOffsetToAcquire = lastOffsetAndMaxRecords.lastOffset();
    RecordBatch firstBatch = fetchPartitionData.records.batches().iterator().next();
    lock.writeLock().lock();
    try {
        long baseOffset = firstBatch.baseOffset();
        // 拉取offset -> 拉取消息的第一个offset 之间数据标记为ARCHIVED
        maybeArchiveStaleBatches(fetchOffset, baseOffset);
        Map.Entry<Long, InFlightBatch> floorEntry = cachedState.floorEntry(baseOffset);
        if (floorEntry == null) {
            if (baseOffset < startOffset) {
                baseOffset = startOffset;
            }
        } else if (floorEntry.getValue().lastOffset() >= baseOffset) {
            baseOffset = floorEntry.getKey();
        }
        final NavigableMap<Long, InFlightBatch> subMap = cachedState.subMap(
            Math.min(firstBatch.baseOffset(), baseOffset), true, lastOffsetToAcquire, true);
        if (subMap.isEmpty()) {
            // case1, cachedState没有InFlightBatch可以分配,创建新的InFlightBatch
            ShareAcquiredRecords shareAcquiredRecords = acquireNewBatchRecords(memberId, fetchPartitionData.records.batches(), isRecordLimitMode,
                                                                               baseOffset, lastOffsetToAcquire, batchSize, maxRecordsToAcquire);
            return maybeFilterAbortedTransactionalAcquiredRecords(fetchPartitionData, isolationLevel, shareAcquiredRecords);
        }
        List<AcquiredRecords> result = new ArrayList<>();
        int acquiredCount = 0;
        long maybeGapStartOffset = baseOffset;
        for (Map.Entry<Long, InFlightBatch> entry : subMap.entrySet()) {
            if (acquiredCount >= maxRecordsToAcquire) {
                break;
            }
            InFlightBatch inFlightBatch = entry.getValue();
            boolean fullMatch = checkForFullMatch(inFlightBatch, firstBatch.baseOffset(), lastOffsetToAcquire);
            int numRecordsRemaining = maxRecordsToAcquire - acquiredCount;
            boolean recordLimitSubsetMatch = isRecordLimitMode && checkForRecordLimitSubsetMatch(inFlightBatch, maxRecordsToAcquire, acquiredCount);
            boolean throttleRecordsDelivery = shouldThrottleRecordsDelivery(inFlightBatch, firstBatch.baseOffset(), lastOffsetToAcquire);
            if (throttleRecordsDelivery && acquiredCount > 0) {
                maxRecordsToAcquire = 0;
                break;
            }
            // inFlightBatch.offsetState() != null - 批次被部分提交,只能获取其中部分offset记录
            if (!fullMatch || inFlightBatch.offsetState() != null || recordLimitSubsetMatch || throttleRecordsDelivery) {
                // case2 获取InFlightBatch中部分offset数据
                int acquiredSubsetCount = acquireSubsetBatchRecords(memberId, isRecordLimitMode, numRecordsRemaining, firstBatch.baseOffset(), lastOffsetToAcquire, inFlightBatch, result);
                acquiredCount += acquiredSubsetCount;
                if (throttleRecordsDelivery && acquiredSubsetCount > 0) {
                    maxRecordsToAcquire = 0;
                    break;
                }
                continue;
            }

            if (inFlightBatch.batchState() != RecordState.AVAILABLE
                || inFlightBatch.batchHasOngoingStateTransition()) {
                continue;
            }
            // case3 获取整个InFlightBatch,更新状态ACQUIRED被获取
            InFlightState updateResult = inFlightBatch.tryUpdateBatchState(RecordState.ACQUIRED, DeliveryCountOps.INCREASE, maxDeliveryCount, memberId);
            // 提交timer定时任务
            AcquisitionLockTimerTask acquisitionLockTimeoutTask = scheduleAcquisitionLockTimeout(memberId, inFlightBatch.firstOffset(), inFlightBatch.lastOffset());
            inFlightBatch.updateAcquisitionLockTimeout(acquisitionLockTimeoutTask);
            result.add(new AcquiredRecords()
                .setFirstOffset(inFlightBatch.firstOffset())
                .setLastOffset(inFlightBatch.lastOffset())
                .setDeliveryCount((short) inFlightBatch.batchDeliveryCount()));
            acquiredCount += (int) (inFlightBatch.lastOffset() - inFlightBatch.firstOffset() + 1);
        }
        // case4 从InFlightBatch中获取了部分数据,还没达到maxRecordsToAcquire,继续获取新的批次
        if (acquiredCount < maxRecordsToAcquire 
            && subMap.lastEntry().getValue().lastOffset() < lastOffsetToAcquire) {
            int numRecordsRemaining = maxRecordsToAcquire - acquiredCount;
            ShareAcquiredRecords shareAcquiredRecords = acquireNewBatchRecords(memberId, fetchPartitionData.records.batches(), isRecordLimitMode,
                subMap.lastEntry().getValue().lastOffset() + 1,
                lastOffsetToAcquire, batchSize, numRecordsRemaining);
            result.addAll(shareAcquiredRecords.acquiredRecords());
            acquiredCount += shareAcquiredRecords.count();
        }
        if (!result.isEmpty()) {
            maybeUpdatePersisterGapWindowStartOffset(result.get(result.size() - 1).lastOffset() + 1);
            return maybeFilterAbortedTransactionalAcquiredRecords(fetchPartitionData, isolationLevel, new ShareAcquiredRecords(result, acquiredCount));
        }
        return new ShareAcquiredRecords(result, acquiredCount);
    } finally {
        lock.writeLock().unlock();
    }
}

6-2-4、开启Ack超时任务

因为消息批次(或单条消息)有锁定超时时间,默认group.share.record.lock.duration.ms=30000,如果一个消息批次(或单条消息)30s内没有ack,将会变为AVAILABLE,可以被重新Fetch。这个事情在创建或更新InFlightBach时处理。

SharePartition.acquireNewBatchRecords:以创建新InFlightBatch为例。

private ShareAcquiredRecords acquireNewBatchRecords(....) {
    lock.writeLock().lock();
    try {
        long firstAcquiredOffset = firstOffset;
        if (cachedState.isEmpty() && endOffset > firstAcquiredOffset) {
            firstAcquiredOffset = endOffset;
        }
        long lastAcquiredOffset = lastOffset;
        if (maxFetchRecords < lastAcquiredOffset - firstAcquiredOffset + 1) {
            lastAcquiredOffset = lastOffsetFromBatchWithRequestOffset(batches, firstAcquiredOffset + maxFetchRecords - 1);
            if (isPersisterReadGapWindowActive() && lastAcquiredOffset > lastOffset) {
                lastAcquiredOffset = lastOffset;
            }
        }
        // 创建新InFilghtBatch
        List<AcquiredRecords> acquiredRecords = createBatches(memberId, batches, isRecordLimitMode, maxFetchRecords, firstAcquiredOffset, lastAcquiredOffset, batchSize);
        // 更新startOffset和endOffset
        if (cachedState.firstKey() == firstAcquiredOffset)  {
            startOffset = firstAcquiredOffset;
        }
        if (lastAcquiredOffset > endOffset) {
            endOffset = lastAcquiredOffset;
        }
        // 更新gap
        maybeUpdatePersisterGapWindowStartOffset(lastAcquiredOffset + 1);
        return new ShareAcquiredRecords(acquiredRecords, (int) (lastAcquiredOffset - firstAcquiredOffset + 1));
    } finally {
        lock.writeLock().unlock();
    }
}

SharePartition.addBatches:创建新InFlightBatch,并开启AcquisitionLockTimerTask

private void addBatches(String memberId, List<AcquiredRecords> acquiredRecordsList) {
    lock.writeLock().lock();
    try {
        acquiredRecordsList.forEach(acquiredRecords -> {
            // 开启定时任务
            AcquisitionLockTimerTask timerTask = scheduleAcquisitionLockTimeout(
                memberId, acquiredRecords.firstOffset(), acquiredRecords.lastOffset());
            // 缓存新的InFlightBatch
            cachedState.put(acquiredRecords.firstOffset(), new InFlightBatch(...));
        });
    } finally {
        lock.writeLock().unlock();
    }
}
AcquisitionLockTimerTask scheduleAcquisitionLockTimeout(...) {
    int recordLockDurationMs = recordLockDurationMsOrDefault(groupConfigManager, groupId, defaultRecordLockDurationMs);
    return scheduleAcquisitionLockTimeout(memberId, firstOffset, lastOffset, recordLockDurationMs);
}
private final Timer timer;
private AcquisitionLockTimerTask scheduleAcquisitionLockTimeout(...) {
    AcquisitionLockTimerTask acquisitionLockTimerTask = acquisitionLockTimerTask(memberId, firstOffset, lastOffset, delayMs);
    timer.add(acquisitionLockTimerTask);
    return acquisitionLockTimerTask;
}

SharePartition.releaseAcquisitionLockOnTimeout:处理InFlightBatch超时

  1. 更新内存InFlightBatch,offsetState=null批次完整超时,offsetState≠null批次部分offset超时,更新对应批次或offset为AVAILABLE/ARCHIVED,默认group.share.delivery.count.limit=5,消费5次还失败(包括超时和客户端主动ack RELEASE),设置为ARCHIVED(等同于ack REJECT);
  2. writeShareGroupState,InFlightBatch变更,异步持久化share_group_state;
  3. maybeUpdateCachedStateAndOffsets,清理内存cachedState的InFlightBatch,因为第1步可能把部分InFlightBatch ARCHIVED,就可以移除了;
  4. maybeCompleteDelayedShareFetchRequest,第1步可能让部分InFlightBatch释放AVAILABLE,尝试唤起同消费组分区的ShareFetch拉取消息;
private AcquisitionLockTimeoutHandler releaseAcquisitionLockOnTimeout() {
    // 获取消息记录的数据范围=[firstOffset,lastOffset]
    return (memberId, firstOffset, lastOffset, timerTask) -> {
        List<PersisterStateBatch> stateBatches;
        lock.writeLock().lock();
        try {
            Map.Entry<Long, InFlightBatch> floorOffset = cachedState.floorEntry(firstOffset);
            if (floorOffset == null) {
                return;
            }
            stateBatches = new ArrayList<>();
            // 找起始offset在[firstOffset,lastOffset]范围的InFlightBatch
            NavigableMap<Long, InFlightBatch> subMap = cachedState.subMap(floorOffset.getKey(), true, lastOffset, true);
            for (Map.Entry<Long, InFlightBatch> entry : subMap.entrySet()) {
                InFlightBatch inFlightBatch = entry.getValue();
                // ...
                // 1. 更新内存InFlightBatch - AVAILABLE/ARCHIVED
                if (inFlightBatch.offsetState() == null) {
                    releaseAcquisitionLockOnTimeoutForCompleteBatch(inFlightBatch, stateBatches, memberId);
                } else {
                    releaseAcquisitionLockOnTimeoutForPerOffsetBatch(inFlightBatch, stateBatches, memberId, firstOffset, lastOffset);
                }
            }
        } finally {
            lock.writeLock().unlock();
        }
        // 2. 写share_group_state
        if (!stateBatches.isEmpty()) {
            writeShareGroupState(stateBatches).whenComplete((result, exception) -> {
                // 3. 更新cachedState,可能可以移除一些内存里的InFlightBatch,比如ARCHIVED
                maybeUpdateCachedStateAndOffsets();
            });
        }
        // 4. 尝试完成延迟的share fetch请求(group+topic+partition纬度),因为有些记录可能超时释放AVAILABLE
        maybeCompleteDelayedShareFetchRequest(!stateBatches.isEmpty());
    };
}

6-3、消费者接收消息

ShareConsumeRequestManager.handleShareFetchSuccess:broker返回数据构造为ShareCompletedFetch,缓存在ShareFetchBuffer,供应用线程poll拉取。

七、Ack

7-1、消费者发送Ack

客户端通过share.acknowledgement.mode控制ack模式:

  1. implicit:implicit,默认,隐式提交;
  2. explicit:显示提交;

explicit模式下,用户代码需要显示对poll返回的每条消息,手动执行ShareConsumer.acknowledge API。

ShareConsumerImpl.acknowledge:acknowledge api仅在内存中,对于正在消费的记录(offset)进行标记。

AcknowledgeType:ack标记有四种类型

  1. ACCEPT:消息处理成功,消息进入终态;
  2. REJECT:消息处理失败,但无需重试消费,消息进入终态;
  3. RELEASE:消息处理失败,释放消息锁,允许broker再次投递;
  4. RENEW:消息续期,告诉broker不要释放锁,消息仍然被当前消费者独占;

implicit模式,工作方式类似于传统消费者的自动提交,每次consumer.poll会将上次poll的所有消息(currentFetch)标记为ACCEPT。

ShareConsumerImpl.collect:应用线程poll拉取消息,会触发ack。ack有两种请求方式

  1. 需要发送ShareFetch(发送ShareFetchEvent给IO线程)拉消息,ack附加在ShareFetch的AcknowledgementBatches参数,减少网络交互次数;
  2. 不需要发送ShareFetch(发送ShareAcknowledgeAsyncEvent给IO线程),ack需要单独发送ShareAcknowledgeRequest

7-2、分区Leader处理Ack

KafkaApis.handleShareFetchRequest:ShareFetch,先处理ack,再处理fetch。

KafkaApis.handleShareAcknowledgeRequest:ShareAcknowledge,单独ack处理。

SharePartition.acknowledge:无论哪种ack,循环每个groupId的消费分区SharePartition执行

  1. 更新内存InFlightBatch,并构造PersisterBatch包含:
    1. InFlightState updatedState:更新后,某个正在消费的整个消息批次(InFlightBatch.batchState)或单条offset(InFlightBatch.offsetState)的内存状态;
    2. PersisterStateBatch stateBatch:updatedState对应的消息持久化状态;
  2. 发送WriteShareGroupStateRequest(PersisterStateBatch)持久化sharegroupstate状态,如果失败,回滚内存InFlightBatch;

InFlightState:InFlightState代表某个消息批次 或 某个offset消息的状态。

startStateTransition修改内存消息状态,默认group.share.delivery.count.limit=5,ack release只能重新被投递5次。

WriteShareGroupStateRequest发送给leaderOf(hash(groupId,消费topic,消费partitionId) % 50) ,持久化到share_group_state,和5-2-2一致。

7-3、关于renew

对于accept/reject ack,消息不再投递。

对于release ack,释放消息锁,消息需要重新传输,可能给不同消费组成员。

对于renew ack,消费者不会将这些消息从内存中移除,消息不需要重新传输。只要broker能成功对消息锁续期,消费者就可以直接重新消费。

ShareInFlightBatch:消费者侧

  1. 应用线程poll,收集本次需要发送的ack时,将renew消息标记为renewing;
  2. IO线程收到ack响应,发送ShareAcknowledgementEvent给应用线程;
  3. 应用线程poll,首先会处理ShareAcknowledgementEvent,将renewing变为renewed,当从内存拉取消息时,将renewed变为inFlight,重新拉取到消息继续消费;

SharePartition.acknowledgePerOffsetBatchRecords:Broker侧,以单消息renew为例。分区Leader处理renew ack并不会修改消息状态,只会重新开启AcquisitionLockTimerTask锁超时任务。

总结

消费者心跳

共享组使用KIP-848新消费组协议,通过心跳完成Rebalance。

心跳请求

1)groupId;

2)memberId:每次进程启动生成uuid;

3)memberEpoch:刚上线=0,离组下线=-1,其他由协调者控制=assignment epoch;

4)topics:订阅Topics;

心跳响应

1)新协议心跳由服务端控制时间:心跳间隔=share.heartbeat.interval.ms=5s,心跳超时检测=share.session.timeout.ms=45s;

2)分区分配assignment,即当前成员可以消费哪些topic分区,后续fetch请求时使用;

共享组场景

1)协调者只用consumer_offset存储消费组成员、订阅情况、分区分配,内存中提现为ModernGroup和ModernGroupMember;

2)协调者通过leaderOf(hash(groupId+topic+partition) % 50),找到topic=share_group_state的leader节点,存储共享组消费进度,内存中提现为SharePartitionKey(groupId+topic+partition) →ShareGroupOffset(stateBatches+startOffset) ;

3)ShareGroupOffset,因为共享组消息可以单独ack,stateBatches存储需要被管理的消息状态,比如offset=1-100因为消费者release ack释放消息,变为available可以重试,另外还要存储重试次数,ShareGroupOffset只会在消息ack、消息超时release场景下会修改;

4)协调者通过心跳,发现共享组新增消费分区,触发SharePartitionKey到ShareGroupOffset的初始化。这一步包含额外异步RPC,需要通过心跳被动补偿;

分区分配

订阅关系一致,采用SimpleHomogeneousAssignmentBuilder。

1个分区被desiredSharing个成员共享,desiredSharing=ceil(members/partitions),即 成员数 / 分区数 向上取整。比如:成员数≤分区数,分区仍然被独占;成员数=7,分区数=3,那么每个分区会被3个成员共享,有2个成员消费2个分区,有5个成员消费1个分区。

Fetch

消费者侧:

  1. 应用线程,如果ShareFetchBuffer为空,发送ShareFetchEvent给IO线程;
  2. IO线程,收到ShareFetchEvent,发送ShareFetch请求给Broker,不同于传统消费组,ShareFetch请求不包含fetchOffset;
  3. IO线程,收到ShareFetch响应,把消息放入ShareFetchBuffer;
  4. 应用线程,从ShareFetchBuffer获取消息消费;

Broker侧(分区Leader):

  1. 如果消费者新增消费分区,找共享组消费进度存储节点,加载消费进度到内存SharePartition
  2. 如果offset=-1,代表无消费进度,默认消费组配置share.auto.offset.reset=latest,将分区高水位作为初始消费进度;(传统消费者需要自行通过ListOffsetRequest获取初始fetchOffset)
  3. 共享组消费fetchOffset由leader计算得到,获取分区锁拉消息;
  4. 将消息记录到内存InFlightBatch,标记为ACQUIRED,后面fetchOffset计算则不会拉取这些消息;
  5. 针对这些消息,开启超时检测,默认group.share.record.lock.duration.ms=30000,30s没ack则自动释放,标记为AVAILABLE,可以被重新Fetch;(释放需要持久化share_group_state,包括重试次数+1)

Ack

AcknowledgeType:ack标记有四种类型

  1. ACCEPT:消息处理成功,消息进入终态;
  2. REJECT:消息处理失败,但无需重试消费,消息进入终态;
  3. RELEASE:消息处理失败,释放消息锁,允许broker再次投递;
  4. RENEW:消息续期,告诉broker不要释放锁,消息仍然被当前消费者独占;

消费者两种ack模式,share.acknowledgement.mode控制ack模式:

  1. implicit:implicit,默认,隐式提交。工作方式类似于传统消费者的自动提交,每次consumer.poll会将上次poll的所有消息标记为ACCEPT;
  2. explicit:显示提交,对每条poll返回消息,调用ShareConsumer.acknowledge API,设置AcknowledgeType

消费者发送Ack请求有两种方式,优先附加在ShareFetchRequest一起发送,否则单独发送ShareAcknowledgeRequest,减少网络交互次数。此外,共享组Ack是发送给消费分区Leader的,而普通组提交offset是发送给协调者的。

Broker侧:

  1. 更新内存InFlightBatch,比如RELEASE将消息标记为AVAILABLE,RENEW重新开启超时检测;
  2. 非RENEW,消息状态需要通过RPC持久化到share_group_state;
  3. 默认group.share.delivery.count.limit=5,RELEASE只能重新投递5次,超出次数消息状态为ARCHIVED(等同于ack reject);