chia7712 commented on code in PR #19699: URL: https://github.com/apache/kafka/pull/19699#discussion_r2145226103
########## transaction-coordinator/src/main/java/org/apache/kafka/coordinator/transaction/TransactionMetadata.java: ########## @@ -0,0 +1,723 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.coordinator.transaction; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.config.LogLevelConfig; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.record.RecordBatch; +import org.apache.kafka.server.common.TransactionVersion; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.slf4j.MarkerFactory; + +import java.util.Collection; +import java.util.HashSet; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.locks.ReentrantLock; +import java.util.function.Supplier; + +public class TransactionMetadata { + private static final Logger LOGGER = LoggerFactory.getLogger(TransactionMetadata.class); + private final String transactionalId; + private long producerId; + private long prevProducerId; + private long nextProducerId; + private short producerEpoch; + private short lastProducerEpoch; + private int txnTimeoutMs; + private TransactionState state; + private Set<TopicPartition> topicPartitions; + private volatile long txnStartTimestamp; + private volatile long txnLastUpdateTimestamp; + private TransactionVersion clientTransactionVersion; + + // pending state is used to indicate the state that this transaction is going to + // transit to, and for blocking future attempts to transit it again if it is not legal; + // initialized as the same as the current state + private Optional<TransactionState> pendingState; + + // Indicates that during a previous attempt to fence a producer, the bumped epoch may not have been + // successfully written to the log. If this is true, we will not bump the epoch again when fencing + private boolean hasFailedEpochFence; + + private final ReentrantLock lock; + + public static boolean isEpochExhausted(short producerEpoch) { + return producerEpoch >= Short.MAX_VALUE - 1; + } + + /** + * @param transactionalId transactional id + * @param producerId producer id + * @param prevProducerId producer id for the last committed transaction with this transactional ID + * @param nextProducerId Latest producer ID sent to the producer for the given transactional ID + * @param producerEpoch current epoch of the producer + * @param lastProducerEpoch last epoch of the producer + * @param txnTimeoutMs timeout to be used to abort long running transactions + * @param state current state of the transaction + * @param topicPartitions current set of partitions that are part of this transaction + * @param txnStartTimestamp time the transaction was started, i.e., when first partition is added + * @param txnLastUpdateTimestamp updated when any operation updates the TransactionMetadata. To be used for expiration + * @param clientTransactionVersion TransactionVersion used by the client when the state was transitioned + */ + public TransactionMetadata(String transactionalId, + long producerId, + long prevProducerId, + long nextProducerId, + short producerEpoch, + short lastProducerEpoch, + int txnTimeoutMs, + TransactionState state, + Set<TopicPartition> topicPartitions, + long txnStartTimestamp, + long txnLastUpdateTimestamp, + TransactionVersion clientTransactionVersion) { + this.transactionalId = transactionalId; + this.producerId = producerId; + this.prevProducerId = prevProducerId; + this.nextProducerId = nextProducerId; + this.producerEpoch = producerEpoch; + this.lastProducerEpoch = lastProducerEpoch; + this.txnTimeoutMs = txnTimeoutMs; + this.state = state; + this.topicPartitions = new HashSet<>(topicPartitions); + this.txnStartTimestamp = txnStartTimestamp; + this.txnLastUpdateTimestamp = txnLastUpdateTimestamp; + this.clientTransactionVersion = clientTransactionVersion; + this.pendingState = Optional.empty(); + this.hasFailedEpochFence = false; + this.lock = new ReentrantLock(); + } + + public <T> T inLock(Supplier<T> function) { + lock.lock(); + try { + return function.get(); + } finally { + lock.unlock(); + } + } + + public void addPartitions(Collection<TopicPartition> partitions) { + topicPartitions.addAll(partitions); + } + + public void removePartition(TopicPartition topicPartition) { + if (state != TransactionState.PREPARE_COMMIT && state != TransactionState.PREPARE_ABORT) + throw new IllegalStateException("Transaction metadata's current state is " + state + ", and its pending state is " + + pendingState + " while trying to remove partitions whose txn marker has been sent, this is not expected"); + + topicPartitions.remove(topicPartition); + } + + // this is visible for test only + public TxnTransitMetadata prepareNoTransit() { + // do not call transitTo as it will set the pending state, a follow-up call to abort the transaction will set its pending state + return new TxnTransitMetadata(producerId, prevProducerId, nextProducerId, producerEpoch, lastProducerEpoch, txnTimeoutMs, + state, Set.copyOf(topicPartitions), txnStartTimestamp, txnLastUpdateTimestamp, clientTransactionVersion); + } + + public TxnTransitMetadata prepareFenceProducerEpoch() { + if (producerEpoch == Short.MAX_VALUE) + throw new IllegalStateException("Cannot fence producer with epoch equal to Short.MaxValue since this would overflow"); + + // If we've already failed to fence an epoch (because the write to the log failed), we don't increase it again. + // This is safe because we never return the epoch to client if we fail to fence the epoch + short bumpedEpoch = hasFailedEpochFence ? producerEpoch : (short) (producerEpoch + 1); + + return prepareTransitionTo( + TransactionState.PREPARE_EPOCH_FENCE, + producerId, + nextProducerId, + bumpedEpoch, + lastProducerEpoch, + txnTimeoutMs, + Set.copyOf(topicPartitions), + txnStartTimestamp, + txnLastUpdateTimestamp, + clientTransactionVersion + ); + } + + public TxnTransitMetadata prepareIncrementProducerEpoch( + int newTxnTimeoutMs, + Optional<Short> expectedProducerEpoch, + long updateTimestamp) { + if (isProducerEpochExhausted()) + throw new IllegalStateException("Cannot allocate any more producer epochs for producerId " + producerId); + + short bumpedEpoch = (short) (producerEpoch + 1); + short produceEpochResult; + short lastProducerEpochResult; + + if (expectedProducerEpoch.isEmpty()) { + // If no expected epoch was provided by the producer, bump the current epoch and set the last epoch to -1 + // In the case of a new producer, producerEpoch will be -1 and bumpedEpoch will be 0 + produceEpochResult = bumpedEpoch; + lastProducerEpochResult = RecordBatch.NO_PRODUCER_EPOCH; + } else { + short expectedEpoch = expectedProducerEpoch.get(); + if (producerEpoch == RecordBatch.NO_PRODUCER_EPOCH || expectedEpoch == producerEpoch) { + // If the expected epoch matches the current epoch, or if there is no current epoch, the producer is attempting + // to continue after an error and no other producer has been initialized. Bump the current and last epochs. + // The no current epoch case means this is a new producer; producerEpoch will be -1 and bumpedEpoch will be 0 + produceEpochResult = bumpedEpoch; + lastProducerEpochResult = producerEpoch; + } else if (expectedEpoch == lastProducerEpoch) { + // If the expected epoch matches the previous epoch, it is a retry of a successful call, so just return the + // current epoch without bumping. There is no danger of this producer being fenced, because a new producer + // calling InitProducerId would have caused the last epoch to be set to -1. + // Note that if the IBP is prior to 2.4.IV1, the lastProducerId and lastProducerEpoch will not be written to + // the transaction log, so a retry that spans a coordinator change will fail. We expect this to be a rare case. + produceEpochResult = producerEpoch; + lastProducerEpochResult = lastProducerEpoch; + } else { + // Otherwise, the producer has a fenced epoch and should receive an PRODUCER_FENCED error + LOGGER.info("Expected producer epoch {} does not match current producer epoch {} or previous producer epoch {}", + expectedEpoch, producerEpoch, lastProducerEpoch); + throw Errors.PRODUCER_FENCED.exception(); + } + } + + return prepareTransitionTo( + TransactionState.EMPTY, + producerId, + nextProducerId, + produceEpochResult, + lastProducerEpochResult, + newTxnTimeoutMs, + Set.of(), + -1L, + updateTimestamp, + clientTransactionVersion + ); + } + + public TxnTransitMetadata prepareProducerIdRotation(long newProducerId, + int newTxnTimeoutMs, + long updateTimestamp, + boolean recordLastEpoch) { + if (hasPendingTransaction()) + throw new IllegalStateException("Cannot rotate producer ids while a transaction is still pending"); + + return prepareTransitionTo( + TransactionState.EMPTY, + newProducerId, + nextProducerId, + (short) 0, + recordLastEpoch ? producerEpoch : RecordBatch.NO_PRODUCER_EPOCH, + newTxnTimeoutMs, + Set.of(), + -1L, + updateTimestamp, + clientTransactionVersion + ); + } + + public TxnTransitMetadata prepareAddPartitions(Set<TopicPartition> addedTopicPartitions, + long updateTimestamp, + TransactionVersion clientTransactionVersion) { + long newTxnStartTimestamp; + if (state == TransactionState.EMPTY || state == TransactionState.COMPLETE_ABORT || state == TransactionState.COMPLETE_COMMIT) { + newTxnStartTimestamp = updateTimestamp; + } else { + newTxnStartTimestamp = txnStartTimestamp; + } + + Set<TopicPartition> newTopicPartitions = new HashSet<>(topicPartitions); + newTopicPartitions.addAll(addedTopicPartitions); + + return prepareTransitionTo( Review Comment: Maybe we can create a small class with all fields having default values. ```java private class TransitionData { TransactionState state = TransactionState.DEAD; long producerId = TransactionMetadata.this.producerId; long nextProducerId = TransactionMetadata.this.nextProducerId; short producerEpoch = TransactionMetadata.this.producerEpoch; short lastProducerEpoch = TransactionMetadata.this.lastProducerEpoch; int txnTimeoutMs = TransactionMetadata.this.txnTimeoutMs; Set<TopicPartition> topicPartitions = Set.of(); long txnStartTimestamp = TransactionMetadata.this.txnStartTimestamp; long txnLastUpdateTimestamp = TransactionMetadata.this.txnLastUpdateTimestamp; TransactionVersion clientTransactionVersion = TransactionMetadata.this.clientTransactionVersion; TransitionData(TransactionState state) { this.state = state; } } ``` ```java var transition = new TransitionData(newState); transition.producerId = updatedProducerId; transition.nextProducerId = RecordBatch.NO_PRODUCER_ID; transition.producerEpoch = updatedProducerEpoch; transition.txnLastUpdateTimestamp = updateTimestamp; return prepareTransitionTo(transition); ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org