**Apache Kafka 从 0.11.0 开始,支持了一个非常大的 feature,就是对事务性的支持,在 Kafka 中关于事务性,是有三种层面上的含义:一是幂等性的支持;二是事务性的支持;三是 Kafka Streams 的 exactly once 的实现,关于 Kafka 事务性系列的文章我们只重点关注前两种层面上的事务性,与 Kafka Streams 相关的内容暂时不做讨论。社区从开始讨论事务性,前后持续近半年时间,相关的设计文档有六十几页(参考 Exactly Once Delivery and Transactional Messaging in Kafka)。事务性这部分的实现也是非常复杂的,之前 Producer 端的代码实现其实是非常简单的,增加事务性的逻辑之后,这部分代码复杂度提高了很多,本篇及后面几篇关于事务性的文章会以 2.0.0 版的代码实现为例,对这部分做了一下分析,计划分为五篇文章:
第一篇:Kafka 幂等性实现; 第二篇:Kafka 事务性实现; 第三篇:Kafka 事务性相关处理请求在 Server 端如何处理及其实现细节; 第四篇:关于 Kafka 事务性实现的一些思考,也会简单介绍一下 RocketMQ 事务性的实现,做一下对比; 第五篇:Flink + Kafka 如何实现 Exactly Once; 这篇是 Kafka 事务性系列的第一篇文章,主要讲述幂等性实现的整体流程,幂等性的实现相对于事务性的实现简单很多,也是事务性实现的基础。
Producer 幂等性 Producer 的幂等性指的是当发送同一条消息时,数据在 Server 端只会被持久化一次,数据不丟不重,但是这里的幂等性是有条件的:
只能保证 Producer 在单个会话内不丟不重,如果 Producer 出现意外挂掉再重启是无法保证的(幂等性情况下,是无法获取之前的状态信息,因此是无法做到跨会话级别的不丢不重); 幂等性不能跨多个 Topic-Partition,只能保证单个 partition 内的幂等性,当涉及多个 Topic-Partition 时,这中间的状态并没有同步。 如果需要跨会话、跨多个 topic-partition 的情况,需要使用 Kafka 的事务性来实现。
幂等性示例 Producer 使用幂等性的示例非常简单,与正常情况下 Producer 使用相比变化不大,只需要把 Producer 的配置 enable.idempotence 设置为 true 即可,如下所示:
1 2 3 4 5 6 7 8 9 10 Properties props = new Properties(); props.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true"); props.put("acks", "all"); // 当 enable.idempotence 为 true,这里默认为 all props.put("bootstrap.servers", "localhost:9092"); props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer"); props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
KafkaProducer producer = new KafkaProducer(props);
producer.send(new ProducerRecord(topic, "test"); Prodcuer 幂等性对外保留的接口非常简单,其底层的实现对上层应用做了很好的封装,应用层并不需要去关心具体的实现细节,对用户非常友好。
幂等性要解决的问题 在看 Producer 是如何实现幂等性之前,首先先考虑一个问题:幂等性是来解决什么问题的? 在 0.11.0 之前,Kafka 通过 Producer 端和 Server 端的相关配置可以做到数据不丢,也就是 at least once,但是在一些情况下,可能会导致数据重复,比如:网络请求延迟等导致的重试操作,在发送请求重试时 Server 端并不知道这条请求是否已经处理(没有记录之前的状态信息),所以就会有可能导致数据请求的重复发送,这是 Kafka 自身的机制(异常时请求重试机制)导致的数据重复。
对于大多数应用而言,数据保证不丢是可以满足其需求的,但是对于一些其他的应用场景(比如支付数据等),它们是要求精确计数的,这时候如果上游数据有重复,下游应用只能在消费数据时进行相应的去重操作,应用在去重时,最常用的手段就是根据唯一 id 键做 check 去重。
在这种场景下,因为上游生产导致的数据重复问题,会导致所有有精确计数需求的下游应用都需要做这种复杂的、重复的去重处理。试想一下:如果在发送时,系统就能保证 exactly once,这对下游将是多么大的解脱。这就是幂等性要解决的问题,主要是解决数据重复的问题,正如前面所述,数据重复问题,通用的解决方案就是加唯一 id,然后根据 id 判断数据是否重复,Producer 的幂等性也是这样实现的,这一小节就让我们看下 Kafka 的 Producer 如何保证数据的 exactly once 的。
幂等性的实现原理 在讲述幂等性处理流程之前,先看下 Producer 是如何来保证幂等性的,正如前面所述,幂等性要解决的问题是:Producer 设置 at least once 时,由于异常触发重试机制导致数据重复,幂等性的目的就是为了解决这个数据重复的问题,简单来说就是:
at least once + 幂等 = exactly once
通过在 al least once 的基础上加上 幂等性来做到 exactly once,当然这个层面的 exactly once 是有限制的,比如它会要求单会话内有效或者跨会话使用事务性有效等。这里我们先分析最简单的情况,那就是在单会话内如何做到幂等性,进而保证 exactly once。
要做到幂等性,要解决下面的问题:
系统需要有能力鉴别一条数据到底是不是重复的数据?常用的手段是通过 唯一键/唯一 id 来判断,这时候系统一般是需要缓存已经处理的唯一键记录,这样才能更有效率地判断一条数据是不是重复; 唯一键应该选择什么粒度?对于分布式存储系统来说,肯定不能用全局唯一键(全局是针对集群级别),核心的解决思路依然是 分而治之,数据密集型系统为了实现分布式都是有分区概念的,而分区之间是有相应的隔离,对于 Kafka 而言,这里的解决方案就是在分区的维度上去做,重复数据的判断让 partition 的 leader 去判断处理,前提是 Produce 请求需要把唯一键值告诉 leader; 分区粒度实现唯一键会不会有其他问题?这里需要考虑的问题是当一个 Partition 有来自多个 client 写入的情况,这些 client 之间是很难做到使用同一个唯一键(一个是它们之间很难做到唯一键的实时感知,另一个是这样实现是否有必要)。而如果系统在实现时做到了 client + partition 粒度,这样实现的好处是每个 client 都是完全独立的(它们之间不需要有任何的联系,这是非常大的优点),只是在 Server 端对不同的 client 做好相应的区分即可,当然同一个 client 在处理多个 Topic-Partition 时是完全可以使用同一个 PID 的。 有了上面的分析(都是个人见解,如果有误,欢迎指教),就不难理解 Producer 幂等性的实现原理,Kafka Producer 在实现时有以下两个重要机制:
PID(Producer ID),用来标识每个 producer client; sequence numbers,client 发送的每条消息都会带相应的 sequence number,Server 端就是根据这个值来判断数据是否重复。 下面详细讲述这两个实现机制。
PID 每个 Producer 在初始化时都会被分配一个唯一的 PID,这个 PID 对应用是透明的,完全没有暴露给用户。对于一个给定的 PID,sequence number 将会从0开始自增,每个 Topic-Partition 都会有一个独立的 sequence number。Producer 在发送数据时,将会给每条 msg 标识一个 sequence number,Server 也就是通过这个来验证数据是否重复。这里的 PID 是全局唯一的,Producer 故障后重新启动后会被分配一个新的 PID,这也是幂等性无法做到跨会话的一个原因。
Producer PID 申请 这里看下 PID 在 Server 端是如何分配的?Client 通过向 Server 发送一个 InitProducerIdRequest 请求获取 PID(幂等性时,是选择一台连接数最少的 Broker 发送这个请求),这里看下 Server 端是如何处理这个请求的?KafkaApis 中 handleInitProducerIdRequest() 方法的实现如下:
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22 23 24 25 def handleInitProducerIdRequest(request: RequestChannel.Request): Unit = { val initProducerIdRequest = request.body[InitProducerIdRequest] val transactionalId = initProducerIdRequest.transactionalId
if (transactionalId != null) { //note: 设置 txn.id 时,验证对 txn.id 的权限 if (!authorize(request.session, Write, Resource(TransactionalId, transactionalId, LITERAL))) { sendErrorResponseMaybeThrottle(request, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception) return } } else if (!authorize(request.session, IdempotentWrite, Resource.ClusterResource)) { //note: 没有设置 txn.id 时,验证对集群是否有幂等性权限 sendErrorResponseMaybeThrottle(request, Errors.CLUSTER_AUTHORIZATION_FAILED.exception) return }
def sendResponseCallback(result: InitProducerIdResult): Unit = {
def createResponse(requestThrottleMs: Int): AbstractResponse = {
val responseBody = new InitProducerIdResponse(requestThrottleMs, result.error, result.producerId, result.producerEpoch)
trace(s"Completed result from client ${request.header.clientId}.")
responseBody
}
sendResponseMaybeThrottle(request, createResponse)
}
//note: 生成相应的了 pid,返回给 producer
txnCoordinator.handleInitProducerId(transactionalId, initProducerIdRequest.transactionTimeoutMs, sendResponseCallback)
}
这里实际上是调用了 TransactionCoordinator (Broker 在启动 server 服务时都会初始化这个实例)的 handleInitProducerId() 方法做了相应的处理,其实现如下(这里只关注幂等性的处理):
1 2 3 4 5 6 7 8 9 10 11 12 def handleInitProducerId(transactionalId: String, transactionTimeoutMs: Int, responseCallback: InitProducerIdCallback): Unit = {
if (transactionalId == null) { //note: 只设置幂等性时,直接分配 pid 并返回 // if the transactional id is null, then always blindly accept the request // and return a new producerId from the producerId manager val producerId = producerIdManager.generateProducerId() responseCallback(InitProducerIdResult(producerId, producerEpoch = 0, Errors.NONE)) } ... } Server 在给一个 client 初始化 PID 时,实际上是通过 ProducerIdManager 的 generateProducerId() 方法产生一个 PID。
Server PID 管理 如前面所述,在幂等性的情况下,直接通过 ProducerIdManager 的 generateProducerId() 方法产生一个 PID,其中 ProducerIdManager 是在 TransactionCoordinator 对象初始化时初始化的,这个对象主要是用来管理 PID 信息:
在本地的 PID 端用完了或者处于新建状态时,申请 PID 段(默认情况下,每次申请 1000 个 PID); TransactionCoordinator 对象通过 generateProducerId() 方法获取下一个可以使用的 PID; PID 端申请是向 ZooKeeper 申请,zk 中有一个 /latest_producer_id_block 节点,每个 Broker 向 zk 申请一个 PID 段后,都会把自己申请的 PID 段信息写入到这个节点,这样当其他 Broker 再申请 PID 段时,会首先读写这个节点的信息,然后根据 block_end 选择一个 PID 段,最后再把信息写会到 zk 的这个节点,这个节点信息格式如下所示:
1 {"version":1,"broker":35,"block_start":"4000","block_end":"4999"} ProducerIdManager 向 zk 申请 PID 段的方法如下:
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 30 31 32 33 34 35 36 37 38 private def getNewProducerIdBlock(): Unit = { var zkWriteComplete = false while (!zkWriteComplete) { //note: 直到从 zk 拿取到分配的 PID 段 // refresh current producerId block from zookeeper again val (dataOpt, zkVersion) = zkClient.getDataAndVersion(ProducerIdBlockZNode.path)
// generate the new producerId block
currentProducerIdBlock = dataOpt match {
case Some(data) =>
//note: 从 zk 获取当前最新的 pid 信息,如果后面更新失败,这里也会重新从 zk 获取
val currProducerIdBlock = ProducerIdManager.parseProducerIdBlockData(data)
debug(s"Read current producerId block $currProducerIdBlock, Zk path version $zkVersion")
if (currProducerIdBlock.blockEndId > Long.MaxValue - ProducerIdManager.PidBlockSize) {//note: 不足以分配1000个 PID
// we have exhausted all producerIds (wow!), treat it as a fatal error
//note: 当 PID 分配超过限制时,直接报错了(每秒分配1个,够用2百亿年了)
fatal(s"Exhausted all producerIds as the next block's end producerId is will has exceeded long type limit (current block end producerId is ${currProducerIdBlock.blockEndId})")
throw new KafkaException("Have exhausted all producerIds.")
}
ProducerIdBlock(brokerId, currProducerIdBlock.blockEndId + 1L, currProducerIdBlock.blockEndId + ProducerIdManager.PidBlockSize)
case None => //note: 该节点还不存在,第一次初始化
debug(s"There is no producerId block yet (Zk path version $zkVersion), creating the first block")
ProducerIdBlock(brokerId, 0L, ProducerIdManager.PidBlockSize - 1)
}
val newProducerIdBlockData = ProducerIdManager.generateProducerIdBlockJson(currentProducerIdBlock)
// try to write the new producerId block into zookeeper
//note: 将新的 pid 信息写入到 zk,如果写入失败(写入之前会比对 zkVersion,如果这个有变动,证明这期间有别的 Broker 在操作,那么写入失败),重新申请
val (succeeded, version) = zkClient.conditionalUpdatePath(ProducerIdBlockZNode.path,
newProducerIdBlockData, zkVersion, Some(checkProducerIdBlockZkData))
zkWriteComplete = succeeded
if (zkWriteComplete)
info(s"Acquired new producerId block $currentProducerIdBlock by writing to Zk with path version $version")
} } ProducerIdManager 申请 PID 段的流程如下:
先从 zk 的 /latest_producer_id_block 节点读取最新已经分配的 PID 段信息; 如果该节点不存在,直接从 0 开始分配,选择 0~1000 的 PID 段(ProducerIdManager 的 PidBlockSize 默认为 1000,即是每次申请的 PID 段大小); 如果该节点存在,读取其中数据,根据 block_end 选择 这个 PID 段(如果 PID 段超过 Long 类型的最大值,这里会直接返回一个异常); 在选择了相应的 PID 段后,将这个 PID 段信息写回到 zk 的这个节点中,如果写入成功,那么 PID 段就证明申请成功,如果写入失败(写入时会判断当前节点的 zkVersion 是否与步骤1获取的 zkVersion 相同,如果相同,那么可以成功写入,否则写入就会失败,证明这个节点被修改过),证明此时可能其他的 Broker 已经更新了这个节点(当前的 PID 段可能已经被其他 Broker 申请),那么从步骤 1 重新开始,直到写入成功。 明白了 ProducerIdManager 如何申请 PID 段之后,再看 generateProducerId() 这个方法就简单很多了,这个方法在每次调用时,都会更新 nextProducerId 值(下一次可以使用 PID 值),如下所示:
1 2 3 4 5 6 7 8 9 10 11 12 13 14 def generateProducerId(): Long = { this synchronized { // grab a new block of producerIds if this block has been exhausted if (nextProducerId > currentProducerIdBlock.blockEndId) { //note: 如果分配的 pid 用完了,重新再向 zk 申请一批 getNewProducerIdBlock() nextProducerId = currentProducerIdBlock.blockStartId + 1 } else { nextProducerId += 1 }
nextProducerId - 1 //note: 返回当前分配的 pid
} } 这里就是 Producer PID 如何申请(事务性情况下 PID 的申请会复杂一些,下篇文章再讲述)以及 Server 端如何管理 PID 的。
sequence numbers 再有了 PID 之后,在 PID + Topic-Partition 级别上添加一个 sequence numbers 信息,就可以实现 Producer 的幂等性了。ProducerBatch 也提供了一个 setProducerState() 方法,它可以给一个 batch 添加一些 meta 信息(pid、baseSequence、isTransactional),这些信息是会伴随着 ProduceRequest 发到 Server 端,Server 端也正是通过这些 meta 来做相应的判断,如下所示:
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 // ProducerBatch public void setProducerState(ProducerIdAndEpoch producerIdAndEpoch, int baseSequence, boolean isTransactional) { recordsBuilder.setProducerState(producerIdAndEpoch.producerId, producerIdAndEpoch.epoch, baseSequence, isTransactional); }
// MemoryRecordsBuilder public void setProducerState(long producerId, short producerEpoch, int baseSequence, boolean isTransactional) { if (isClosed()) { // Sequence numbers are assigned when the batch is closed while the accumulator is being drained. // If the resulting ProduceRequest to the partition leader failed for a retriable error, the batch will // be re queued. In this case, we should not attempt to set the state again, since changing the producerId and sequence // once a batch has been sent to the broker risks introducing duplicates. throw new IllegalStateException("Trying to set producer state of an already closed batch. This indicates a bug on the client."); } this.producerId = producerId; this.producerEpoch = producerEpoch; this.baseSequence = baseSequence; this.isTransactional = isTransactional; } 幂等性实现整体流程 在前面讲述完 Kafka 幂等性的两个实现机制(PID+sequence numbers)之后,这里详细讲述一下,幂等性时其整体的处理流程,主要讲述幂等性相关的内容,其他的部分会简单介绍(可以参考前面【Kafka 源码分析系列文章】了解 Producer 端处理流程以及 Server 端关于 ProduceRequest 请求的处理流程),其流程如下图所示:
**正常情况下,对于一个事务而言,其状态状态流程应该是 Empty –> Ongoing –> PrepareCommit –> CompleteCommit –> Empty 或者是 Empty –> Ongoing –> PrepareAbort –> CompleteAbort –> Empty。
Client 端事务状态管理 Client 的事务状态信息主要记录本地事务的状态,当然跟其他的系统类似,本地的状态信息与 Server 端的状态信息并不完全一致(状态的设置,就像 GroupCoodinator 会维护一个 Group 的状态,每个 Consumer 也会维护本地的 Consumer 对象的状态一样)。Client 端的事务状态信息主要用于 Client 端的事务状态处理,其主要有以下几种:
UNINITIALIZED:Transactional Producer 初始化时的状态,此时还没有事务处理; INITIALIZING:Transactional Producer 调用 initTransactions() 方法初始化事务相关的内容,比如发送 InitProducerIdRequest 请求; READY:对于新建的事务,Transactional Producer 收到来自 TransactionCoordinator 的 InitProducerIdResponse 后,其状态会置为 READY(对于已有的事务而言,是当前事务完成后 Client 的状态会转移为 READY); IN_TRANSACTION:Transactional Producer 调用 beginTransaction() 方法,开始一个事务,标志着一个事务开始初始化; COMMITTING_TRANSACTION:Transactional Producer 调用 commitTransaction() 方法时,会先更新本地的状态信息; ABORTING_TRANSACTION:Transactional Producer 调用 abortTransaction() 方法时,会先更新本地的状态信息; ABORTABLE_ERROR:在一个事务操作中,如果有数据发送失败,本地状态会转移到这个状态,之后再自动 abort 事务; FATAL_ERROR:转移到这个状态之后,再进行状态转移时,会抛出异常; Client 端状态如下图:
**事务性的整体流程 有了前面对 Kafka 事务性关键实现的讲述之后,这里详细讲述一个事务操作的处理流程,当然这里只是重点讲述事务性相关的内容,官方版的流程图可参考Kafka Exactly-Once Data Flow,这里我做了一些改动,其流程图如下:
这个流程是以 consume-process-produce 场景为例(主要是 kafka streams 的场景),图中红虚框及 4.3a 部分是关于 consumer 的操作,去掉这部分的话,就是只考虑写入情况的场景。这种只考虑写入场景的事务操作目前在业内应用也是非常广泛的,比如 Flink + Kafka 端到端的 Exactly-Once 实现就是这种场景,下面来详细讲述一下整个流程。
- Finding a TransactionCoordinator 对于事务性的处理,第一步首先需要做的就是找到这个事务 txn.id 对应的 TransactionCoordinator,Transaction Producer 会向 Broker (随机选择一台 broker,一般选择本地连接最少的这台 broker)发送 FindCoordinatorRequest 请求,获取其 TransactionCoordinator。
怎么找到对应的 TransactionCoordinator 呢?这个前面已经讲过了,主要是通过下面的方法获取 __transaction_state 的 Partition,该 Partition 对应的 leader 就是这个 txn.id 对应的 TransactionCoordinator。
1 def partitionFor(transactionalId: String): Int = Utils.abs(transactionalId.hashCode) % transactionTopicPartitionCount 2. Getting a PID PID 这里就不再介绍了,不了解的可以看前面那篇文章(Producer ID)。
Transaction Producer 在 initializeTransactions() 方法中会向 TransactionCoordinator 发送 InitPidRequest 请求获取其分配的 PID,有了 PID,事务写入时可以保证幂等性,PID 如何分配可以参考 PID 分配,但是 TransactionCoordinator 在给事务 Producer 分配 PID 会做一些判断,主要的内容是:
如果这个 txn.id 之前没有相应的事务状态(new txn.id),那么会初始化其事务 meta 信息 TransactionMetadata(会给其分配一个 PID,初始的 epoch 为-1),如果有事务状态,获取之前的状态; 校验其 TransactionMetadata 的状态信息(参考下面代码中 prepareInitProduceIdTransit() 方法): 如果前面还有状态转移正在进行,直接返回 CONCURRENT_TRANSACTIONS 异常; 如果此时的状态为 PrepareAbort 或 PrepareCommit,返回 CONCURRENT_TRANSACTIONS 异常; 如果之前的状态为 CompleteAbort、CompleteCommit 或 Empty,那么先将状态转移为 Empty,然后更新一下 epoch 值; 如果之前的状态为 Ongoing,状态会转移成 PrepareEpochFence,然后再 abort 当前的事务,并向 client 返回 CONCURRENT_TRANSACTIONS 异常; 如果状态为 Dead 或 PrepareEpochFence,直接抛出相应的 FATAL 异常; 将 txn.id 与相应的 TransactionMetadata 持久化到事务日志中,对于 new txn.id,这个持久化的数据主要时 txn.id 与 pid 关系信息,如图中的 3a 所示。 1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 30 31 32 33 34 35 36 37 38 39 40 41 42 //note: producer 启用事务性的情况下,检测此时事务的状态信息 private def prepareInitProduceIdTransit(transactionalId: String, transactionTimeoutMs: Int, coordinatorEpoch: Int, txnMetadata: TransactionMetadata): ApiResult[(Int, TxnTransitMetadata)] = { if (txnMetadata.pendingTransitionInProgress) { // return a retriable exception to let the client backoff and retry Left(Errors.CONCURRENT_TRANSACTIONS) } else { // caller should have synchronized on txnMetadata already txnMetadata.state match { case PrepareAbort | PrepareCommit => // reply to client and let it backoff and retry Left(Errors.CONCURRENT_TRANSACTIONS)
case CompleteAbort | CompleteCommit | Empty => //note: 此时需要将状态转移到 Empty(此时状态并没有转移,只是在 PendingState 记录了将要转移的状态)
val transitMetadata = if (txnMetadata.isProducerEpochExhausted) {
val newProducerId = producerIdManager.generateProducerId()
txnMetadata.prepareProducerIdRotation(newProducerId, transactionTimeoutMs, time.milliseconds())
} else { //note: 增加 producer 的 epoch 值
txnMetadata.prepareIncrementProducerEpoch(transactionTimeoutMs, time.milliseconds())
}
Right(coordinatorEpoch, transitMetadata)
case Ongoing => //note: abort 当前的事务,并返回一个 CONCURRENT_TRANSACTIONS 异常,强制 client 去重试
// indicate to abort the current ongoing txn first. Note that this epoch is never returned to the
// user. We will abort the ongoing transaction and return CONCURRENT_TRANSACTIONS to the client.
// This forces the client to retry, which will ensure that the epoch is bumped a second time. In
// particular, if fencing the current producer exhausts the available epochs for the current producerId,
// then when the client retries, we will generate a new producerId.
Right(coordinatorEpoch, txnMetadata.prepareFenceProducerEpoch())
case Dead | PrepareEpochFence => //note: 返回错误
val errorMsg = s"Found transactionalId $transactionalId with state ${txnMetadata.state}. " +
s"This is illegal as we should never have transitioned to this state."
fatal(errorMsg)
throw new IllegalStateException(errorMsg)
}
} } 3. Starting a Transaction 前面两步都是 Transaction Producer 调用 initTransactions() 部分,到这里,Producer 可以调用 beginTransaction() 开始一个事务操作,其实现方法如下面所示:
1 2 3 4 5 6 7 8 9 10 11 12 13 14 //KafkaProducer //note: 应该在一个事务操作之前进行调用 public void beginTransaction() throws ProducerFencedException { throwIfNoTransactionManager(); transactionManager.beginTransaction(); }
// TransactionManager //note: 在一个事务开始之前进行调用,这里实际上只是转换了状态(只在 producer 本地记录了状态的开始) public synchronized void beginTransaction() { ensureTransactional(); maybeFailWithError(); transitionTo(State.IN_TRANSACTION); } 这里只是将本地事务状态转移成 IN_TRANSACTION,并没有与 Server 端进行交互,所以在流程图中没有体现出来(TransactionManager 初始化时,其状态为 UNINITIALIZED,Producer 调用 initializeTransactions() 方法,其状态转移成 INITIALIZING)。
- Consume-Porcess-Produce Loop 在这个阶段,Transaction Producer 会做相应的处理,主要包括:从 consumer 拉取数据、对数据做相应的处理、通过 Producer 写入到下游系统中(对于只有写入场景,忽略前面那一步即可),下面有一个示例(start 和 end 中间的部分),是一个典型的 consume-process-produce 场景:
1 2 3 4 5 6 7 8 9 10 11 12 while (true) { ConsumerRecords records = consumer.poll(Long.MAX_VALUE); producer.beginTransaction(); //start for (ConsumerRecord record : records){ producer.send(producerRecord(“outputTopic1”, record)); producer.send(producerRecord(“outputTopic2”, record)); } producer.sendOffsetsToTransaction(currentOffsets(consumer), group); //end producer.commitTransaction(); } 下面来结合前面的流程图来讲述一下这部分的实现。
4.1. AddPartitionsToTxnRequest Producer 在调用 send() 方法时,Producer 会将这个对应的 Topic—Partition 添加到 TransactionManager 的记录中,如下所示:
1 2 3 //note: 如何开启了幂等性或事务性,需要做一些处理 if (transactionManager != null && transactionManager.isTransactional()) transactionManager.maybeAddPartitionToTransaction(tp); 如果这个 Topic-Partition 之前不存在,那么就添加到 newPartitionsInTransaction 集合中,如下所示:
1 2 3 4 5 6 7 8 9 10 11 //note: 将 tp 添加到 newPartitionsInTransaction 中,记录当前进行事务操作的 tp public synchronized void maybeAddPartitionToTransaction(TopicPartition topicPartition) { failIfNotReadyForSend();
//note: 如果 partition 已经添加到 partitionsInTransaction、pendingPartitionsInTransaction、newPartitionsInTransaction中
if (isPartitionAdded(topicPartition) || isPartitionPendingAdd(topicPartition))
return;
log.debug("Begin adding new partition {} to transaction", topicPartition);
newPartitionsInTransaction.add(topicPartition);
} Producer 端的 Sender 线程会将这个信息通过 AddPartitionsToTxnRequest 请求发送给 TransactionCoordinator,也就是图中的 4.1 过程,TransactionCoordinator 会将这个 Topic-Partition 列表更新到 txn.id 对应的 TransactionMetadata 中,并且会持久化到事务日志中,也就是图中的 4.1 a 部分,这里持久化的数据主要是 txn.id 与其涉及到的 Topic-Partition 信息。
4.2. ProduceRequest 这一步与正常 Producer 写入基本上一样,就是相应的 Leader 在持久化数据时会在头信息中标识这条数据是不是来自事务 Producer 的写入(主要是数据协议有变动,Server 处理并不需要做额外的处理)。
4.3. AddOffsetsToTxnRequest Producer 在调用 sendOffsetsToTransaction() 方法时,第一步会首先向 TransactionCoordinator 发送相应的 AddOffsetsToTxnRequest 请求,如下所示:
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 30 //class KafkaProcducer //note: 当你需要 batch 的消费-处理-写入消息,这个方法需要被使用 //note: 发送指定的 offset 给 group coordinator,用来标记这些 offset 是作为当前事务的一部分,只有这次事务成功时 //note: 这些 offset 才会被认为 commit 了 public void sendOffsetsToTransaction(Map<TopicPartition, OffsetAndMetadata> offsets, String consumerGroupId) throws ProducerFencedException { throwIfNoTransactionManager(); TransactionalRequestResult result = transactionManager.sendOffsetsToTransaction(offsets, consumerGroupId); sender.wakeup(); result.await(); }
// class TransactionManager //note: 发送 AddOffsetsToTxRequest public synchronized TransactionalRequestResult sendOffsetsToTransaction(Map<TopicPartition, OffsetAndMetadata> offsets, String consumerGroupId) { ensureTransactional(); maybeFailWithError(); if (currentState != State.IN_TRANSACTION) throw new KafkaException("Cannot send offsets to transaction either because the producer is not in an " + "active transaction");
log.debug("Begin adding offsets {} for consumer group {} to transaction", offsets, consumerGroupId);
AddOffsetsToTxnRequest.Builder builder = new AddOffsetsToTxnRequest.Builder(transactionalId,
producerIdAndEpoch.producerId, producerIdAndEpoch.epoch, consumerGroupId);
AddOffsetsToTxnHandler handler = new AddOffsetsToTxnHandler(builder, offsets);
enqueueRequest(handler);
return handler.result;
} TransactionCoordinator 在收到这个请求时,处理方法与 4.1 中的一样,把这个 group.id 对应的 __consumer_offsets 的 Partition (与写入涉及的 Topic-Partition 一样)保存到事务对应的 meta 中,之后会持久化相应的事务日志,如图中 4.3a 所示。
4.4. TxnOffsetsCommitRequest Producer 在收到 TransactionCoordinator 关于 AddOffsetsToTxnRequest 请求的结果后,后再次发送 TxnOffsetsCommitRequest 请求给对应的 GroupCoordinator,AddOffsetsToTxnHandler 的 handleResponse() 的实现如下:
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22 23 24 25 26 27 @Override public void handleResponse(AbstractResponse response) { AddOffsetsToTxnResponse addOffsetsToTxnResponse = (AddOffsetsToTxnResponse) response; Errors error = addOffsetsToTxnResponse.error();
if (error == Errors.NONE) {
log.debug("Successfully added partition for consumer group {} to transaction", builder.consumerGroupId());
// note the result is not completed until the TxnOffsetCommit returns
//note: AddOffsetsToTnxRequest 之后,还会再发送 TxnOffsetCommitRequest
pendingRequests.add(txnOffsetCommitHandler(result, offsets, builder.consumerGroupId()));
transactionStarted = true;
} else if (error == Errors.COORDINATOR_NOT_AVAILABLE || error == Errors.NOT_COORDINATOR) {
lookupCoordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId);
reenqueue();
} else if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS || error == Errors.CONCURRENT_TRANSACTIONS) {
reenqueue();
} else if (error == Errors.INVALID_PRODUCER_EPOCH) {
fatalError(error.exception());
} else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) {
fatalError(error.exception());
} else if (error == Errors.GROUP_AUTHORIZATION_FAILED) {
abortableError(new GroupAuthorizationException(builder.consumerGroupId()));
} else {
fatalError(new KafkaException("Unexpected error in AddOffsetsToTxnResponse: " + error.message()));
}
} GroupCoordinator 在收到相应的请求后,会将 offset 信息持久化到 consumer offsets log 中(包含对应的 PID 信息),但是不会更新到缓存中,除非这个事务 commit 了,这样的话就可以保证这个 offset 信息对 consumer 是不可见的(没有更新到缓存中的数据是不可见的,通过接口是获取的,这是 GroupCoordinator 本身来保证的)。
5.Committing or Aborting a Transaction 在一个事务操作处理完成之后,Producer 需要调用 commitTransaction() 或者 abortTransaction() 方法来 commit 或者 abort 这个事务操作。
5.1. EndTxnRequest 无论是 Commit 还是 Abort,对于 Producer 而言,都是向 TransactionCoordinator 发送 EndTxnRequest 请求,这个请求的内容里会标识是 commit 操作还是 abort 操作,Producer 的 commitTransaction() 方法实现如下所示:
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 //class KafkaProducer //note: commit 正在进行的事务操作,这个方法在真正发送 commit 之后将会 flush 所有未发送的数据 //note: 如果在发送中遇到任何一个不能修复的错误,这个方法抛出异常,事务也不会被提交,所有 send 必须成功,这个事务才能 commit 成功 public void commitTransaction() throws ProducerFencedException { throwIfNoTransactionManager(); TransactionalRequestResult result = transactionManager.beginCommit(); sender.wakeup(); result.await(); }
// class TransactionManager //note: 开始 commit,转移本地本地保存的状态以及发送相应的请求 public synchronized TransactionalRequestResult beginCommit() { ensureTransactional(); maybeFailWithError(); transitionTo(State.COMMITTING_TRANSACTION); return beginCompletingTransaction(TransactionResult.COMMIT); } Producer 的 abortTransaction() 方法实现如下:
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 //class KafkaProducer //note: 取消正在进行事务,任何没有 flush 的数据都会被丢弃 public void abortTransaction() throws ProducerFencedException { throwIfNoTransactionManager(); TransactionalRequestResult result = transactionManager.beginAbort(); sender.wakeup(); result.await(); }
// class TransactionManager public synchronized TransactionalRequestResult beginAbort() { ensureTransactional(); if (currentState != State.ABORTABLE_ERROR) maybeFailWithError(); transitionTo(State.ABORTING_TRANSACTION);
// We're aborting the transaction, so there should be no need to add new partitions
newPartitionsInTransaction.clear();
return beginCompletingTransaction(TransactionResult.ABORT);
} 它们最终都是调用了 TransactionManager 的 beginCompletingTransaction() 方法,这个方法会向其 待发送请求列表 中添加 EndTxnRequest 请求,其实现如下:
1 2 3 4 5 6 7 8 9 10 //note: 发送 EndTxnRequest 请求,添加到 pending 队列中 private TransactionalRequestResult beginCompletingTransaction(TransactionResult transactionResult) { if (!newPartitionsInTransaction.isEmpty()) enqueueRequest(addPartitionsToTransactionHandler()); EndTxnRequest.Builder builder = new EndTxnRequest.Builder(transactionalId, producerIdAndEpoch.producerId, producerIdAndEpoch.epoch, transactionResult); EndTxnHandler handler = new EndTxnHandler(builder); enqueueRequest(handler); return handler.result; } TransactionCoordinator 在收到 EndTxnRequest 请求后,会做以下处理:
更新事务的 meta 信息,状态转移成 PREPARE_COMMIT 或 PREPARE_ABORT,并将事务状态信息持久化到事务日志中; 根据事务 meta 信息,向其涉及到的所有 Topic-Partition 的 leader 发送 Transaction Marker 信息(也就是 WriteTxnMarkerRquest 请求,见下面的 5.2 分析); 最后将事务状态更新为 COMMIT 或者 ABORT,并将事务的 meta 持久化到事务日志中,也就是 5.3 步骤。 5.2. WriteTxnMarkerRquest WriteTxnMarkerRquest 是 TransactionCoordinator 收到 Producer 的 EndTxnRequest 请求后向其他 Broker 发送的请求,主要是告诉它们事务已经完成。不论是普通的 Topic-Partition 还是 __consumer_offsets,在收到这个请求后,都会把事务结果(Transaction Marker 的格数据式见前面)持久化到对应的日志文件中,这样下游 Consumer 在消费这个数据时,就知道这个事务是 commit 还是 abort。
5.3. Writing the Final Commit or Abort Message 当这个事务涉及到所有 Topic-Partition 都已经把这个 marker 信息持久化到日志文件之后,TransactionCoordinator 会将这个事务的状态置为 COMMIT 或 ABORT,并持久化到事务日志文件中,到这里,这个事务操作就算真正完成了,TransactionCoordinator 缓存的很多关于这个事务的数据可以被清除了。
小思考 在上面讲述完 Kafka 事务性处理之后,我们来思考一下以下这些问题,上面的流程可能会出现下面这些问题或者很多人可能会有下面的疑问:
txn.id 是否可以被多 Producer 使用,如果有多个 Producer 使用了这个 txn.id 会出现什么问题? TransactionCoordinator Fencing 和 Producer Fencing 分别是什么,它们是用来解决什么问题的? 对于事务的数据,Consumer 端是如何消费的,一个事务可能会 commit,也可能会 abort,这个在 Consumer 端是如何体现的? 对于一个 Topic,如果既有事务数据写入又有其他 topic 数据写入,消费时,其顺序性时怎么保证的? 如果 txn.id 长期不使用,server 端怎么处理? PID Snapshot 是做什么的?是用来解决什么问题? 下面,来详细分析一下上面提到的这些问题。
如果多个 Producer 使用同一个 txn.id 会出现什么情况? 对于这个情况,我们这里直接做了一个相应的实验,两个 Producer 示例都使用了同一个 txn.id(为 test-transactional-matt),Producer 1 先启动,然后过一会再启动 Producer 2,这时候会发现一个现象,那就是 Producer 1 进程会抛出异常退出进程,其异常信息为:
1 2 3 4 5 6 org.apache.kafka.common.KafkaException: Cannot execute transactional method because we are in an error state at org.apache.kafka.clients.producer.internals.TransactionManager.maybeFailWithError(TransactionManager.java:784) at org.apache.kafka.clients.producer.internals.TransactionManager.beginTransaction(TransactionManager.java:215) at org.apache.kafka.clients.producer.KafkaProducer.beginTransaction(KafkaProducer.java:606) at com.matt.test.kafka.producer.ProducerTransactionExample.main(ProducerTransactionExample.java:68) Caused by: org.apache.kafka.common.errors.ProducerFencedException: Producer attempted an operation with an old epoch. Either there is a newer producer with the same transactionalId, or the producer's transaction has been expired by the broker. 这里抛出了 ProducerFencedException 异常,如果打开相应的 Debug 日志,在 Producer 1 的日志文件会看到下面的日志信息
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 [2018-11-03 12:48:52,495] DEBUG [Producer clientId=ProducerTransactionExample, transactionalId=test-transactional-matt] Transition from state COMMITTING_TRANSACTION to error state FATAL_ERROR (org.apache.kafka.clients.producer.internals.TransactionManager) org.apache.kafka.common.errors.ProducerFencedException: Producer attempted an operation with an old epoch. Either there is a newer producer with the same transactionalId, or the producer's transaction has been expired by the broker. [2018-11-03 12:48:52,498] ERROR [Producer clientId=ProducerTransactionExample, transactionalId=test-transactional-matt] Aborting producer batches due to fatal error (org.apache.kafka.clients.producer.internals.Sender) org.apache.kafka.common.errors.ProducerFencedException: Producer attempted an operation with an old epoch. Either there is a newer producer with the same transactionalId, or the producer's transaction has been expired by the broker. [2018-11-03 12:48:52,599] INFO [Producer clientId=ProducerTransactionExample, transactionalId=test-transactional-matt] Closing the Kafka producer with timeoutMillis = 9223372036854775807 ms. (org.apache.kafka.clients.producer.KafkaProducer) [2018-11-03 12:48:52,599] DEBUG [Producer clientId=ProducerTransactionExample, transactionalId=test-transactional-matt] Beginning shutdown of Kafka producer I/O thread, sending remaining records. (org.apache.kafka.clients.producer.internals.Sender) [2018-11-03 12:48:52,601] DEBUG Removed sensor with name connections-closed: (org.apache.kafka.common.metrics.Metrics) [2018-11-03 12:48:52,601] DEBUG Removed sensor with name connections-created: (org.apache.kafka.common.metrics.Metrics) [2018-11-03 12:48:52,602] DEBUG Removed sensor with name successful-authentication: (org.apache.kafka.common.metrics.Metrics) [2018-11-03 12:48:52,602] DEBUG Removed sensor with name failed-authentication: (org.apache.kafka.common.metrics.Metrics) [2018-11-03 12:48:52,602] DEBUG Removed sensor with name bytes-sent-received: (org.apache.kafka.common.metrics.Metrics) [2018-11-03 12:48:52,603] DEBUG Removed sensor with name bytes-sent: (org.apache.kafka.common.metrics.Metrics) [2018-11-03 12:48:52,603] DEBUG Removed sensor with name bytes-received: (org.apache.kafka.common.metrics.Metrics) [2018-11-03 12:48:52,604] DEBUG Removed sensor with name select-time: (org.apache.kafka.common.metrics.Metrics) [2018-11-03 12:48:52,604] DEBUG Removed sensor with name io-time: (org.apache.kafka.common.metrics.Metrics) [2018-11-03 12:48:52,604] DEBUG Removed sensor with name node--1.bytes-sent (org.apache.kafka.common.metrics.Metrics) [2018-11-03 12:48:52,605] DEBUG Removed sensor with name node--1.bytes-received (org.apache.kafka.common.metrics.Metrics) [2018-11-03 12:48:52,605] DEBUG Removed sensor with name node--1.latency (org.apache.kafka.common.metrics.Metrics) [2018-11-03 12:48:52,605] DEBUG Removed sensor with name node-33.bytes-sent (org.apache.kafka.common.metrics.Metrics) [2018-11-03 12:48:52,606] DEBUG Removed sensor with name node-33.bytes-received (org.apache.kafka.common.metrics.Metrics) [2018-11-03 12:48:52,606] DEBUG Removed sensor with name node-33.latency (org.apache.kafka.common.metrics.Metrics) [2018-11-03 12:48:52,606] DEBUG Removed sensor with name node-35.bytes-sent (org.apache.kafka.common.metrics.Metrics) [2018-11-03 12:48:52,606] DEBUG Removed sensor with name node-35.bytes-received (org.apache.kafka.common.metrics.Metrics) [2018-11-03 12:48:52,606] DEBUG Removed sensor with name node-35.latency (org.apache.kafka.common.metrics.Metrics) [2018-11-03 12:48:52,607] DEBUG [Producer clientId=ProducerTransactionExample, transactionalId=test-transactional-matt] Shutdown of Kafka producer I/O thread has completed. (org.apache.kafka.clients.producer.internals.Sender) [2018-11-03 12:48:52,607] DEBUG [Producer clientId=ProducerTransactionExample, transactionalId=test-transactional-matt] Kafka producer has been closed (org.apache.kafka.clients.producer.KafkaProducer) [2018-11-03 12:48:52,808] ERROR Forcing producer close! (com.matt.test.kafka.producer.ProducerTransactionExample) [2018-11-03 12:48:52,808] INFO [Producer clientId=ProducerTransactionExample, transactionalId=test-transactional-matt] Closing the Kafka producer with timeoutMillis = 9223372036854775807 ms. (org.apache.kafka.clients.producer.KafkaProducer) [2018-11-03 12:48:52,808] DEBUG [Producer clientId=ProducerTransactionExample, transactionalId=test-transactional-matt] Kafka producer has been closed (org.apache.kafka.clients.producer.KafkaProducer) Producer 1 本地事务状态从 COMMITTING_TRANSACTION 变成了 FATAL_ERROR 状态,导致 Producer 进程直接退出了,出现这个异常的原因,就是抛出的 ProducerFencedException 异常,简单来说 Producer 1 被 Fencing 了(这是 Producer Fencing 的情况)。因此,这个问题的答案就很清除了,如果多个 Producer 共用一个 txn.id,那么最后启动的 Producer 会成功运行,会它之前启动的 Producer 都 Fencing 掉(至于为什么会 Fencing 下一小节会做分析)。
Fencing 关于 Fencing 这个机制,在分布式系统还是很常见的,我第一个见到这个机制是在 HDFS 中,可以参考我之前总结的一篇文章 HDFS NN 脑裂问题,Fencing 机制解决的主要也是这种类型的问题 —— 脑裂问题,简单来说就是,本来系统这个组件在某个时刻应该只有一个处于 active 状态的,但是在实际生产环境中,特别是切换期间,可能会同时出现两个组件处于 active 状态,这就是脑裂问题,在 Kafka 的事务场景下,用到 Fencing 机制有两个地方:
TransactionCoordinator Fencing; Producer Fencing; TransactionCoordinator Fencing TransactionCoordinator 在遇到上 long FGC 时,可能会导致 脑裂 问题,FGC 时会 stop-the-world,这时候可能会与 zk 连接超时导致临时节点消失进而触发 leader 选举,如果 __transaction_state 发生了 leader 选举,TransactionCoordinator 就会切换,如果此时旧的 TransactionCoordinator FGC 完成,在还没来得及同步到最细 meta 之前,会有一个短暂的时刻,对于一个 txn.id 而言就是这个时刻可能出现了两个 TransactionCoordinator。
相应的解决方案就是 TransactionCoordinator Fencing,这里 Fencing 策略不像离线场景 HDFS 这种直接 Kill 旧的 NN 进程或者强制切换状态这么暴力,而是通过 CoordinatorEpoch 来判断,每个 TransactionCoordinator 都有其 CoordinatorEpoch 值,这个值就是对应 __transaction_state Partition 的 Epoch 值(每当 leader 切换一次,该值就会自增1)。
明白了 TransactionCoordinator 脑裂问题发生情况及解决方案之后,来分析下,Fencing 机制会在哪里发挥作用?仔细想想,是可以推断出来的,只可能是 TransactionCoordinator 向别人发请求时影响才会比较严重(特别是乱发 admin 命令)。有了 CoordinatorEpoch 之后,其他 Server 在收到请求时做相应的判断,如果发现 CoordinatorEpoch 值比缓存的最新的值小,那么 Fencing 就生效,拒绝这个请求,也就是 TransactionCoordinator 发送 WriteTxnMarkerRequest 时可能会触发这一机制。
Producer Fencing Producer Fencing 与前面的类似,如果对于相同 PID 和 txn.id 的 Producer,Server 端会记录最新的 Epoch 值,拒绝来自 zombie Producer (Epoch 值小的 Producer)的请求。前面第一个问题的情况,Producer 2 在启动时,会向 TransactionCoordinator 发送 InitPIDRequest 请求,此时 TransactionCoordinator 已经有了这个 txn.id 对应的 meta,会返回之前分配的 PID,并把 Epoch 自增 1 返回,这样 Producer 2 就被认为是最新的 Producer,而 Producer 1 就会被认为是 zombie Producer,因此,TransactionCoordinator 在处理 Producer 1 的事务请求时,会返回相应的异常信息。
Consumer 端如何消费事务数据 在讲述这个问题之前,需要先介绍一下事务场景下,Consumer 的消费策略,Consumer 有一个 isolation.level 配置,这个是配置对于事务性数据的消费策略,有以下两种可选配置:
read_committed: only consume non-transactional messages or transactional messages that are already committed, in offset ordering. read_uncommitted: consume all available messages in offset ordering. This is the default value. 简单来说就是,read_committed 只会读取 commit 的数据,而 abort 的数据不会向 consumer 显现,对于 read_uncommitted 这种模式,consumer 可以读取到所有数据(control msg 会过滤掉),这种模式与普通的消费机制基本没有区别,就是做了一个 check,过滤掉 control msg(也就是 marker 数据),这部分的难点在于 read_committed 机制的实现。
Last Stable Offset(LSO) 在事务机制的实现中,Kafka 又设置了一个新的 offset 概念,那就是 Last Stable Offset,简称 LSO(其他的 Offset 概念可参考 Kafka Offset 那些事),先看下 LSO 的定义:
The LSO is defined as the latest offset such that the status of all transactional messages at lower offsets have been determined (i.e. committed or aborted).
对于一个 Partition 而言,offset 小于 LSO 的数据,全都是已经确定的数据,这个主要是对于事务操作而言,在这个 offset 之前的事务操作都是已经完成的事务(已经 commit 或 abort),如果这个 Partition 没有涉及到事务数据,那么 LSO 就是其 HW(水位)。
Server 处理 read_committed 类型的 Fetch 请求 如果 Consumer 的消费策略设置的是 read_committed,其在向 Server 发送 Fetch 请求时,Server 端只会返回 LSO 之前的数据,在 LSO 之后的数据不会返回。
这种机制有没有什么问题呢?我现在能想到的就是如果有一个 long transaction,比如其 first offset 是 1000,另外有几个已经完成的小事务操作,比如:txn1(offset:1100~1200)、txn2(offset:1400~1500),假设此时的 LSO 是 1000,也就是说这个 long transaction 还没有完成,那么已经完成的 txn1、txn2 也会对 consumer 不可见(假设都是 commit 操作),此时受 long transaction 的影响可能会导致数据有延迟。
那么我们再来想一下,如果不设计 LSO,又会有什么问题呢?可能分两种情况:
允许读未完成的事务:那么 Consumer 可以直接读取到 Partition 的 HW 位置,对于未完成的事务,因为设置的是 read_committed 机制,所以不能对用户可见,需要在 Consumer 端做缓存,这个缓存应该设置多大?(不限制肯定会出现 OOM 的情况,当然也可以现在 client 端持久化到硬盘,这样的设计太过于复杂,还需要考虑 client 端 IO、磁盘故障等风险),明显这种设计方案是不可行的; 如果不允许读未完成的事务:相当于还是在 Server 端处理,与前面的区别是,这里需要先把示例中的 txn1、txn2 的数据发送给 Consumer,这样的设计会带来什么问题呢? 假设这个 long transaction commit 了,其 end offset 是 2000,这时候有两种方案:第一种是把 1000-2000 的数据全部读出来(可能是磁盘读),把这个 long transaction 的数据过滤出来返回给 Consumer;第二种是随机读,只读这个 long transaction 的数据,无论哪种都有多触发一次磁盘读的风险,可能影响影响 Server 端的性能; Server 端需要维护每个 consumer group 有哪些事务读了、哪些事务没读的 meta 信息,因为 consumer 是随机可能挂掉,需要接上次消费的,这样实现就复杂很多了; 还有一个问题是,消费的顺序性无法保证,两次消费其读取到的数据顺序可能是不同的(两次消费启动时间不一样); 从这些分析来看,个人认为 LSO 机制还是一种相当来说 实现起来比较简单、而且不影响原来 server 端性能、还能保证顺序性的一种设计方案,它不一定是最好的,但也不会差太多。在实际的生产场景中,尽量避免 long transaction 这种操作,而且 long transaction可能也会容易触发事务超时。
Consumer 如何过滤 abort 的事务数据 Consumer 在拉取到相应的数据之后,后面该怎么处理呢?它拉取到的这批数据并不能保证都是完整的事务数据,很有可能是拉取到一个事务的部分数据(marker 数据还没有拉取到),这时候应该怎么办?难道 Consumer 先把这部分数据缓存下来,等后面的 marker 数据到来时再确认数据应该不应该丢弃?(还是又 OOM 的风险)有没有更好的实现方案?
Kafka 的设计总是不会让我们失望,这部分做的优化也是非常高明,Broker 会追踪每个 Partition 涉及到的 abort transactions,Partition 的每个 log segment 都会有一个单独只写的文件(append-only file)来存储 abort transaction 信息,因为 abort transaction 并不是很多,所以这个开销是可以可以接受的,之所以要持久化到磁盘,主要是为了故障后快速恢复,要不然 Broker 需要把这个 Partition 的所有数据都读一遍,才能直到哪些事务是 abort 的,这样的话,开销太大(如果这个 Partition 没有事务操作,就不会生成这个文件)。这个持久化的文件是以 .txnindex 做后缀,前面依然是这个 log segment 的 offset 信息,存储的数据格式如下:
1 2 3 4 5 6 TransactionEntry => Version => int16 PID => int64 FirstOffset => int64 LastOffset => int64 LastStableOffset => int64 有了这个设计,Consumer 在拉取数据时,Broker 会把这批数据涉及到的所有 abort transaction 信息都返回给 Consumer,Server 端会根据拉取的 offset 范围与 abort transaction 的 offset 做对比,返回涉及到的 abort transaction 集合,其实现如下:
1 2 3 4 5 6 7 8 9 10 11 def collectAbortedTxns(fetchOffset: Long, upperBoundOffset: Long): TxnIndexSearchResult = { val abortedTransactions = ListBuffer.empty[AbortedTxn] for ((abortedTxn, _) <- iterator()) { if (abortedTxn.lastOffset >= fetchOffset && abortedTxn.firstOffset < upperBoundOffset) abortedTransactions += abortedTxn //note: 这个 abort 的事务有在在这个范围内,就返回
if (abortedTxn.lastStableOffset >= upperBoundOffset)
return TxnIndexSearchResult(abortedTransactions.toList, isComplete = true)
} TxnIndexSearchResult(abortedTransactions.toList, isComplete = false) } Consumer 在拿到这些数据之后,会进行相应的过滤,大概的判断逻辑如下(Server 端返回的 abort transaction 列表就保存在 abortedTransactions 集合中,abortedProducerIds 最开始时是为空的):
如果这个数据是 control msg(也即是 marker 数据),是 ABORT 的话,那么与这个事务相关的 PID 信息从 abortedProducerIds 集合删掉,是 COMMIT 的话,就忽略(每个这个 PID 对应的 marker 数据收到之后,就从 abortedProducerIds 中清除这个 PID 信息); 如果这个数据是正常的数据,把它的 PID 和 offset 信息与 abortedTransactions 队列(有序队列,头部 transaction 的 first offset 最小)第一个 transaction 做比较,如果 PID 相同,并且 offset 大于等于这个 transaction 的 first offset,就将这个 PID 信息添加到 abortedProducerIds 集合中,同时从 abortedTransactions 队列中删除这个 transaction,最后再丢掉这个 batch(它是 abort transaction 的数据); 检查这个 batch 的 PID 是否在 abortedProducerIds 集合中,在的话,就丢弃,不在的话就返回上层应用。 这部分的实现确实有些绕(有兴趣的可以慢慢咀嚼一下),它严重依赖了 Kafka 提供的下面两种保证:
Consumer 拉取到的数据,在处理时,其 offset 是严格有序的; 同一个 txn.id(PID 相同)在某一个时刻最多只能有一个事务正在进行; 这部分代码实现如下:
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 30 31 32 33 34 35 36 37 38 39 40 41 42 43 44 45 46 47 48 49 50 51 52 53 54 55 56 57 58 59 60 private Record nextFetchedRecord() { while (true) { if (records == null || !records.hasNext()) { //note: records 为空(数据全部丢掉了),records 没有数据(是 control msg) maybeCloseRecordStream();
if (!batches.hasNext()) {
// Message format v2 preserves the last offset in a batch even if the last record is removed
// through compaction. By using the next offset computed from the last offset in the batch,
// we ensure that the offset of the next fetch will point to the next batch, which avoids
// unnecessary re-fetching of the same batch (in the worst case, the consumer could get stuck
// fetching the same batch repeatedly).
if (currentBatch != null)
nextFetchOffset = currentBatch.nextOffset();
drain();
return null;
}
currentBatch = batches.next();
maybeEnsureValid(currentBatch);
if (isolationLevel == IsolationLevel.READ_COMMITTED && currentBatch.hasProducerId()) {
//note: 需要做相应的判断
// remove from the aborted transaction queue all aborted transactions which have begun
// before the current batch's last offset and add the associated producerIds to the
// aborted producer set
//note: 如果这个 batch 的 offset 已经大于等于 abortedTransactions 中第一事务的 first offset
//note: 那就证明下个 abort transaction 的数据已经开始到来,将 PID 添加到 abortedProducerIds 中
consumeAbortedTransactionsUpTo(currentBatch.lastOffset());
long producerId = currentBatch.producerId();
if (containsAbortMarker(currentBatch)) {
abortedProducerIds.remove(producerId); //note: 这个 PID(当前事务)涉及到的数据已经处理完
} else if (isBatchAborted(currentBatch)) { //note: 丢弃这个数据
log.debug("Skipping aborted record batch from partition {} with producerId {} and " +
"offsets {} to {}",
partition, producerId, currentBatch.baseOffset(), currentBatch.lastOffset());
nextFetchOffset = currentBatch.nextOffset();
continue;
}
}
records = currentBatch.streamingIterator(decompressionBufferSupplier);
} else {
Record record = records.next();
// skip any records out of range
if (record.offset() >= nextFetchOffset) {
// we only do validation when the message should not be skipped.
maybeEnsureValid(record);
// control records are not returned to the user
if (!currentBatch.isControlBatch()) { //note: 过滤掉 marker 数据
return record;
} else {
// Increment the next fetch offset when we skip a control batch.
nextFetchOffset = record.offset() + 1;
}
}
}
}
} Consumer 消费数据时,其顺序如何保证 有了前面的分析,这个问题就很好回答了,顺序性还是严格按照 offset 的,只不过遇到 abort trsansaction 的数据时就丢弃掉,其他的与普通 Consumer 并没有区别。
如果 txn.id 长期不使用,server 端怎么处理? Producer 在开始一个事务操作时,可以设置其事务超时时间(参数是 transaction.timeout.ms,默认60s),而且 Server 端还有一个最大可允许的事务操作超时时间(参数是 transaction.timeout.ms,默认是15min),Producer 设置超时时间不能超过 Server,否则的话会抛出异常。
上面是关于事务操作的超时设置,而对于 txn.id,我们知道 TransactionCoordinator 会缓存 txn.id 的相关信息,如果没有超时机制,这个 meta 大小是无法预估的,Server 端提供了一个 transaction.id.expiration.ms 参数来配置这个超时时间(默认是7天),如果超过这个时间没有任何事务相关的请求发送过来,那么 TransactionCoordinator 将会使这个 txn.id 过期。
PID Snapshot 是做什么的?用来解决什么问题? 对于每个 Topic-Partition,Broker 都会在内存中维护其 PID 与 sequence number(最后成功写入的 msg 的 sequence number)的对应关系(这个在上面幂等性文章应讲述过,主要是为了不丢补充的实现)。
Broker 重启时,如果想恢复上面的状态信息,那么它读取所有的 log 文件。相比于之下,定期对这个 state 信息做 checkpoint(Snapshot),明显收益是非常大的,此时如果 Broker 重启,只需要读取最近一个 Snapshot 文件,之后的数据再从 log 文件中恢复即可。
这个 PID Snapshot 样式如 00000000000235947656.snapshot,以 .snapshot 作为后缀,其数据格式如下:
1 2 3 4 [matt@XXX-35 app.matt_test_transaction_json_3-2]$ /usr/local/java18/bin/java -Djava.ext.dirs=/XXX/kafka/libs kafka.tools.DumpLogSegments --files 00000000000235947656.snapshot Dumping 00000000000235947656.snapshot producerId: 2000 producerEpoch: 1 coordinatorEpoch: 4 currentTxnFirstOffset: None firstSequence: 95769510 lastSequence: 95769511 lastOffset: 235947654 offsetDelta: 1 timestamp: 1541325156503 producerId: 3000 producerEpoch: 5 coordinatorEpoch: 6 currentTxnFirstOffset: None firstSequence: 91669662 lastSequence: 91669666 lastOffset: 235947651 offsetDelta: 4 timestamp: 1541325156454 在实际的使用中,这个 snapshot 文件一般只会保存最近的两个文件。
中间流程故障如何恢复 对于上面所讲述的一个事务操作流程,实际生产环境中,任何一个地方都有可能出现的失败:
Producer 在发送 beginTransaction() 时,如果出现 timeout 或者错误:Producer 只需要重试即可; Producer 在发送数据时出现错误:Producer 应该 abort 这个事务,如果 Produce 没有 abort(比如设置了重试无限次,并且 batch 超时设置得非常大),TransactionCoordinator 将会在这个事务超时之后 abort 这个事务操作; Producer 发送 commitTransaction() 时出现 timeout 或者错误:Producer 应该重试这个请求; Coordinator Failure:如果 Transaction Coordinator 发生切换(事务 topic leader 切换),Coordinator 可以从日志中恢复。如果发送事务有处于 PREPARE_COMMIT 或 PREPARE_ABORT 状态,那么直接执行 commit 或者 abort 操作,如果是一个正在进行的事务,Coordinator 的失败并不需要 abort 事务,producer 只需要向新的 Coordinator 发送请求即可。