mirror of https://github.com/apache/kafka.git
				
				
				
			KAFKA-19414: Remove 2PC public APIs from 4.1 until release (KIP-939) (#19985)
We are removing some of the previously added public APIs until KIP-939 is ready to use. Reviewers: Justine Olshan <jolshan@confluent.io>
This commit is contained in:
		
							parent
							
								
									aa0d1f5000
								
							
						
					
					
						commit
						c4cac07819
					
				|  | @ -50,13 +50,11 @@ import org.apache.kafka.common.errors.AuthenticationException; | ||||||
| import org.apache.kafka.common.errors.AuthorizationException; | import org.apache.kafka.common.errors.AuthorizationException; | ||||||
| import org.apache.kafka.common.errors.InterruptException; | import org.apache.kafka.common.errors.InterruptException; | ||||||
| import org.apache.kafka.common.errors.InvalidTopicException; | import org.apache.kafka.common.errors.InvalidTopicException; | ||||||
| import org.apache.kafka.common.errors.InvalidTxnStateException; |  | ||||||
| import org.apache.kafka.common.errors.ProducerFencedException; | import org.apache.kafka.common.errors.ProducerFencedException; | ||||||
| import org.apache.kafka.common.errors.RecordTooLargeException; | import org.apache.kafka.common.errors.RecordTooLargeException; | ||||||
| import org.apache.kafka.common.errors.RetriableException; | import org.apache.kafka.common.errors.RetriableException; | ||||||
| import org.apache.kafka.common.errors.SerializationException; | import org.apache.kafka.common.errors.SerializationException; | ||||||
| import org.apache.kafka.common.errors.TimeoutException; | import org.apache.kafka.common.errors.TimeoutException; | ||||||
| import org.apache.kafka.common.errors.UnsupportedVersionException; |  | ||||||
| import org.apache.kafka.common.header.Header; | import org.apache.kafka.common.header.Header; | ||||||
| import org.apache.kafka.common.header.Headers; | import org.apache.kafka.common.header.Headers; | ||||||
| import org.apache.kafka.common.header.internals.RecordHeaders; | import org.apache.kafka.common.header.internals.RecordHeaders; | ||||||
|  | @ -622,13 +620,8 @@ public class KafkaProducer<K, V> implements Producer<K, V> { | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     /** |     /** | ||||||
|      * Initialize the transactional state for this producer, similar to {@link #initTransactions()} but |  | ||||||
|      * with additional capabilities to keep a previously prepared transaction. |  | ||||||
|      * |  | ||||||
|      * Needs to be called before any other methods when the {@code transactional.id} is set in the configuration. |      * Needs to be called before any other methods when the {@code transactional.id} is set in the configuration. | ||||||
|      * |      * This method does the following: | ||||||
|      * When {@code keepPreparedTxn} is {@code false}, this behaves like the standard transactional |  | ||||||
|      * initialization where the method does the following: |  | ||||||
|      * <ol> |      * <ol> | ||||||
|      * <li>Ensures any transactions initiated by previous instances of the producer with the same |      * <li>Ensures any transactions initiated by previous instances of the producer with the same | ||||||
|      *      {@code transactional.id} are completed. If the previous instance had failed with a transaction in |      *      {@code transactional.id} are completed. If the previous instance had failed with a transaction in | ||||||
|  | @ -637,39 +630,26 @@ public class KafkaProducer<K, V> implements Producer<K, V> { | ||||||
|      * <li>Gets the internal producer id and epoch, used in all future transactional |      * <li>Gets the internal producer id and epoch, used in all future transactional | ||||||
|      *      messages issued by the producer.</li> |      *      messages issued by the producer.</li> | ||||||
|      * </ol> |      * </ol> | ||||||
|      * |  | ||||||
|      * <p> |  | ||||||
|      * When {@code keepPreparedTxn} is set to {@code true}, the producer does <em>not</em> automatically abort existing |  | ||||||
|      * transactions. Instead, it enters a recovery mode allowing only finalization of those previously |  | ||||||
|      * prepared transactions. |  | ||||||
|      * This behavior is especially crucial for 2PC scenarios, where transactions should remain intact |  | ||||||
|      * until the external transaction manager decides whether to commit or abort. |  | ||||||
|      * <p> |  | ||||||
|      * |  | ||||||
|      * @param keepPreparedTxn true to retain any in-flight prepared transactions (necessary for 2PC |  | ||||||
|      *                        recovery), false to abort existing transactions and behave like |  | ||||||
|      *                        the standard initTransactions. |  | ||||||
|      * |  | ||||||
|      * Note that this method will raise {@link TimeoutException} if the transactional state cannot |      * Note that this method will raise {@link TimeoutException} if the transactional state cannot | ||||||
|      * be initialized before expiration of {@code max.block.ms}. Additionally, it will raise {@link InterruptException} |      * be initialized before expiration of {@code max.block.ms}. Additionally, it will raise {@link InterruptException} | ||||||
|      * if interrupted. It is safe to retry in either case, but once the transactional state has been successfully |      * if interrupted. It is safe to retry in either case, but once the transactional state has been successfully | ||||||
|      * initialized, this method should no longer be used. |      * initialized, this method should no longer be used. | ||||||
|      * |      * | ||||||
|      * @throws IllegalStateException if no {@code transactional.id} is configured |      * @throws IllegalStateException if no {@code transactional.id} has been configured | ||||||
|      * @throws org.apache.kafka.common.errors.UnsupportedVersionException if the broker does not |      * @throws org.apache.kafka.common.errors.UnsupportedVersionException fatal error indicating the broker | ||||||
|      *         support transactions (i.e. if its version is lower than 0.11.0.0) |      *         does not support transactions (i.e. if its version is lower than 0.11.0.0) | ||||||
|      * @throws org.apache.kafka.common.errors.TransactionalIdAuthorizationException if the configured |      * @throws org.apache.kafka.common.errors.AuthorizationException error indicating that the configured | ||||||
|      *         {@code transactional.id} is unauthorized either for normal transaction writes or 2PC. |      *         transactional.id is not authorized, or the idempotent producer id is unavailable. See the exception for | ||||||
|      * @throws KafkaException if the producer encounters a fatal error or any other unexpected error |      *         more details.  User may retry this function call after fixing the permission. | ||||||
|  |      * @throws KafkaException if the producer has encountered a previous fatal error or for any other unexpected error | ||||||
|      * @throws TimeoutException if the time taken for initialize the transaction has surpassed <code>max.block.ms</code>. |      * @throws TimeoutException if the time taken for initialize the transaction has surpassed <code>max.block.ms</code>. | ||||||
|      * @throws InterruptException if the thread is interrupted while blocked |      * @throws InterruptException if the thread is interrupted while blocked | ||||||
|      */ |      */ | ||||||
|     public void initTransactions(boolean keepPreparedTxn) { |     public void initTransactions() { | ||||||
|         throwIfNoTransactionManager(); |         throwIfNoTransactionManager(); | ||||||
|         throwIfProducerClosed(); |         throwIfProducerClosed(); | ||||||
|         throwIfInPreparedState(); |  | ||||||
|         long now = time.nanoseconds(); |         long now = time.nanoseconds(); | ||||||
|         TransactionalRequestResult result = transactionManager.initializeTransactions(keepPreparedTxn); |         TransactionalRequestResult result = transactionManager.initializeTransactions(false); | ||||||
|         sender.wakeup(); |         sender.wakeup(); | ||||||
|         result.await(maxBlockTimeMs, TimeUnit.MILLISECONDS); |         result.await(maxBlockTimeMs, TimeUnit.MILLISECONDS); | ||||||
|         producerMetrics.recordInit(time.nanoseconds() - now); |         producerMetrics.recordInit(time.nanoseconds() - now); | ||||||
|  | @ -754,7 +734,6 @@ public class KafkaProducer<K, V> implements Producer<K, V> { | ||||||
|         throwIfInvalidGroupMetadata(groupMetadata); |         throwIfInvalidGroupMetadata(groupMetadata); | ||||||
|         throwIfNoTransactionManager(); |         throwIfNoTransactionManager(); | ||||||
|         throwIfProducerClosed(); |         throwIfProducerClosed(); | ||||||
|         throwIfInPreparedState(); |  | ||||||
| 
 | 
 | ||||||
|         if (!offsets.isEmpty()) { |         if (!offsets.isEmpty()) { | ||||||
|             long start = time.nanoseconds(); |             long start = time.nanoseconds(); | ||||||
|  | @ -765,48 +744,6 @@ public class KafkaProducer<K, V> implements Producer<K, V> { | ||||||
|         } |         } | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     /** |  | ||||||
|      * Prepares the current transaction for a two-phase commit. This method will flush all pending messages |  | ||||||
|      * and transition the producer into a mode where only {@link #commitTransaction()}, {@link #abortTransaction()}, |  | ||||||
|      * or completeTransaction(PreparedTxnState) may be called. |  | ||||||
|      * <p> |  | ||||||
|      * This method is used as part of a two-phase commit protocol: |  | ||||||
|      * <ol> |  | ||||||
|      *   <li>Prepare the transaction by calling this method. This returns a {@link PreparedTxnState} if successful.</li> |  | ||||||
|      *   <li>Make any external system changes that need to be atomic with this transaction.</li> |  | ||||||
|      *   <li>Complete the transaction by calling {@link #commitTransaction()}, {@link #abortTransaction()} or |  | ||||||
|      *       completeTransaction(PreparedTxnState).</li> |  | ||||||
|      * </ol> |  | ||||||
|      * |  | ||||||
|      * @return the prepared transaction state to use when completing the transaction |  | ||||||
|      * |  | ||||||
|      * @throws IllegalStateException if no transactional.id has been configured or no transaction has been started yet. |  | ||||||
|      * @throws InvalidTxnStateException if the producer is not in a state where preparing |  | ||||||
|      *         a transaction is possible or 2PC is not enabled. |  | ||||||
|      * @throws ProducerFencedException fatal error indicating another producer with the same transactional.id is active |  | ||||||
|      * @throws UnsupportedVersionException fatal error indicating the broker |  | ||||||
|      *         does not support transactions (i.e. if its version is lower than 0.11.0.0) |  | ||||||
|      * @throws AuthorizationException fatal error indicating that the configured |  | ||||||
|      *         transactional.id is not authorized. See the exception for more details |  | ||||||
|      * @throws KafkaException if the producer has encountered a previous fatal error or for any other unexpected error |  | ||||||
|      * @throws TimeoutException if the time taken for preparing the transaction has surpassed <code>max.block.ms</code> |  | ||||||
|      * @throws InterruptException if the thread is interrupted while blocked |  | ||||||
|      */ |  | ||||||
|     @Override |  | ||||||
|     public PreparedTxnState prepareTransaction() throws ProducerFencedException { |  | ||||||
|         throwIfNoTransactionManager(); |  | ||||||
|         throwIfProducerClosed(); |  | ||||||
|         throwIfInPreparedState(); |  | ||||||
|         if (!transactionManager.is2PCEnabled()) { |  | ||||||
|             throw new InvalidTxnStateException("Cannot prepare a transaction when 2PC is not enabled"); |  | ||||||
|         } |  | ||||||
|         long now = time.nanoseconds(); |  | ||||||
|         flush(); |  | ||||||
|         transactionManager.prepareTransaction(); |  | ||||||
|         producerMetrics.recordPrepareTxn(time.nanoseconds() - now); |  | ||||||
|         return transactionManager.preparedTransactionState(); |  | ||||||
|     } |  | ||||||
| 
 |  | ||||||
|     /** |     /** | ||||||
|      * Commits the ongoing transaction. This method will flush any unsent records before actually committing the transaction. |      * Commits the ongoing transaction. This method will flush any unsent records before actually committing the transaction. | ||||||
|      * <p> |      * <p> | ||||||
|  | @ -884,40 +821,6 @@ public class KafkaProducer<K, V> implements Producer<K, V> { | ||||||
|         producerMetrics.recordAbortTxn(time.nanoseconds() - abortStart); |         producerMetrics.recordAbortTxn(time.nanoseconds() - abortStart); | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     /** |  | ||||||
|      * Completes a prepared transaction by comparing the provided prepared transaction state with the |  | ||||||
|      * current prepared state on the producer. |  | ||||||
|      * If they match, the transaction is committed; otherwise, it is aborted. |  | ||||||
|      *  |  | ||||||
|      * @param preparedTxnState              The prepared transaction state to compare against the current state |  | ||||||
|      * @throws IllegalStateException if no transactional.id has been configured or no transaction has been started |  | ||||||
|      * @throws InvalidTxnStateException if the producer is not in prepared state |  | ||||||
|      * @throws ProducerFencedException fatal error indicating another producer with the same transactional.id is active |  | ||||||
|      * @throws KafkaException if the producer has encountered a previous fatal error or for any other unexpected error |  | ||||||
|      * @throws TimeoutException if the time taken for completing the transaction has surpassed <code>max.block.ms</code> |  | ||||||
|      * @throws InterruptException if the thread is interrupted while blocked |  | ||||||
|      */ |  | ||||||
|     @Override |  | ||||||
|     public void completeTransaction(PreparedTxnState preparedTxnState) throws ProducerFencedException { |  | ||||||
|         throwIfNoTransactionManager(); |  | ||||||
|         throwIfProducerClosed(); |  | ||||||
|          |  | ||||||
|         if (!transactionManager.isPrepared()) { |  | ||||||
|             throw new InvalidTxnStateException("Cannot complete transaction because no transaction has been prepared. " + |  | ||||||
|                 "Call prepareTransaction() first, or make sure initTransaction(true) was called."); |  | ||||||
|         } |  | ||||||
|          |  | ||||||
|         // Get the current prepared transaction state |  | ||||||
|         PreparedTxnState currentPreparedState = transactionManager.preparedTransactionState(); |  | ||||||
|          |  | ||||||
|         // Compare the prepared transaction state token and commit or abort accordingly |  | ||||||
|         if (currentPreparedState.equals(preparedTxnState)) { |  | ||||||
|             commitTransaction(); |  | ||||||
|         } else { |  | ||||||
|             abortTransaction(); |  | ||||||
|         } |  | ||||||
|     } |  | ||||||
| 
 |  | ||||||
|     /** |     /** | ||||||
|      * Asynchronously send a record to a topic. Equivalent to <code>send(record, null)</code>. |      * Asynchronously send a record to a topic. Equivalent to <code>send(record, null)</code>. | ||||||
|      * See {@link #send(ProducerRecord, Callback)} for details. |      * See {@link #send(ProducerRecord, Callback)} for details. | ||||||
|  |  | ||||||
|  | @ -142,7 +142,7 @@ public class MockProducer<K, V> implements Producer<K, V> { | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     @Override |     @Override | ||||||
|     public void initTransactions(boolean keepPreparedTxn) { |     public void initTransactions() { | ||||||
|         verifyNotClosed(); |         verifyNotClosed(); | ||||||
|         verifyNotFenced(); |         verifyNotFenced(); | ||||||
|         if (this.transactionInitialized) { |         if (this.transactionInitialized) { | ||||||
|  | @ -200,18 +200,6 @@ public class MockProducer<K, V> implements Producer<K, V> { | ||||||
|         this.sentOffsets = true; |         this.sentOffsets = true; | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     @Override |  | ||||||
|     public PreparedTxnState prepareTransaction() throws ProducerFencedException { |  | ||||||
|         verifyNotClosed(); |  | ||||||
|         verifyNotFenced(); |  | ||||||
|         verifyTransactionsInitialized(); |  | ||||||
|         verifyTransactionInFlight(); |  | ||||||
|          |  | ||||||
|         // Return a new PreparedTxnState with mock values for producerId and epoch |  | ||||||
|         // Using 1000L and (short)1 as arbitrary values for a valid PreparedTxnState |  | ||||||
|         return new PreparedTxnState(1000L, (short) 1); |  | ||||||
|     } |  | ||||||
| 
 |  | ||||||
|     @Override |     @Override | ||||||
|     public void commitTransaction() throws ProducerFencedException { |     public void commitTransaction() throws ProducerFencedException { | ||||||
|         verifyNotClosed(); |         verifyNotClosed(); | ||||||
|  | @ -257,27 +245,6 @@ public class MockProducer<K, V> implements Producer<K, V> { | ||||||
|         this.transactionInFlight = false; |         this.transactionInFlight = false; | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     @Override |  | ||||||
|     public void completeTransaction(PreparedTxnState preparedTxnState) throws ProducerFencedException { |  | ||||||
|         verifyNotClosed(); |  | ||||||
|         verifyNotFenced(); |  | ||||||
|         verifyTransactionsInitialized(); |  | ||||||
|          |  | ||||||
|         if (!this.transactionInFlight) { |  | ||||||
|             throw new IllegalStateException("There is no prepared transaction to complete."); |  | ||||||
|         } |  | ||||||
| 
 |  | ||||||
|         // For testing purposes, we'll consider a prepared state with producerId=1000L and epoch=1 as valid |  | ||||||
|         // This should match what's returned in prepareTransaction() |  | ||||||
|         PreparedTxnState currentState = new PreparedTxnState(1000L, (short) 1); |  | ||||||
|          |  | ||||||
|         if (currentState.equals(preparedTxnState)) { |  | ||||||
|             commitTransaction(); |  | ||||||
|         } else { |  | ||||||
|             abortTransaction(); |  | ||||||
|         } |  | ||||||
|     } |  | ||||||
| 
 |  | ||||||
|     private synchronized void verifyNotClosed() { |     private synchronized void verifyNotClosed() { | ||||||
|         if (this.closed) { |         if (this.closed) { | ||||||
|             throw new IllegalStateException("MockProducer is already closed."); |             throw new IllegalStateException("MockProducer is already closed."); | ||||||
|  |  | ||||||
|  | @ -42,14 +42,7 @@ public interface Producer<K, V> extends Closeable { | ||||||
|     /** |     /** | ||||||
|      * See {@link KafkaProducer#initTransactions()} |      * See {@link KafkaProducer#initTransactions()} | ||||||
|      */ |      */ | ||||||
|     default void initTransactions() { |     void initTransactions(); | ||||||
|         initTransactions(false); |  | ||||||
|     } |  | ||||||
| 
 |  | ||||||
|     /** |  | ||||||
|      * See {@link KafkaProducer#initTransactions(boolean)} |  | ||||||
|      */ |  | ||||||
|     void initTransactions(boolean keepPreparedTxn); |  | ||||||
| 
 | 
 | ||||||
|     /** |     /** | ||||||
|      * See {@link KafkaProducer#beginTransaction()} |      * See {@link KafkaProducer#beginTransaction()} | ||||||
|  | @ -62,11 +55,6 @@ public interface Producer<K, V> extends Closeable { | ||||||
|     void sendOffsetsToTransaction(Map<TopicPartition, OffsetAndMetadata> offsets, |     void sendOffsetsToTransaction(Map<TopicPartition, OffsetAndMetadata> offsets, | ||||||
|                                   ConsumerGroupMetadata groupMetadata) throws ProducerFencedException; |                                   ConsumerGroupMetadata groupMetadata) throws ProducerFencedException; | ||||||
| 
 | 
 | ||||||
|     /** |  | ||||||
|      * See {@link KafkaProducer#prepareTransaction()} |  | ||||||
|      */ |  | ||||||
|     PreparedTxnState prepareTransaction() throws ProducerFencedException; |  | ||||||
| 
 |  | ||||||
|     /** |     /** | ||||||
|      * See {@link KafkaProducer#commitTransaction()} |      * See {@link KafkaProducer#commitTransaction()} | ||||||
|      */ |      */ | ||||||
|  | @ -77,11 +65,6 @@ public interface Producer<K, V> extends Closeable { | ||||||
|      */ |      */ | ||||||
|     void abortTransaction() throws ProducerFencedException; |     void abortTransaction() throws ProducerFencedException; | ||||||
| 
 | 
 | ||||||
|     /** |  | ||||||
|      * See {@link KafkaProducer#completeTransaction(PreparedTxnState)} |  | ||||||
|      */ |  | ||||||
|     void completeTransaction(PreparedTxnState preparedTxnState) throws ProducerFencedException; |  | ||||||
| 
 |  | ||||||
|     /** |     /** | ||||||
|      * @see KafkaProducer#registerMetricForSubscription(KafkaMetric)  |      * @see KafkaProducer#registerMetricForSubscription(KafkaMetric)  | ||||||
|      */ |      */ | ||||||
|  |  | ||||||
|  | @ -320,9 +320,7 @@ public class TransactionManager { | ||||||
|                     .setTransactionalId(transactionalId) |                     .setTransactionalId(transactionalId) | ||||||
|                     .setTransactionTimeoutMs(transactionTimeoutMs) |                     .setTransactionTimeoutMs(transactionTimeoutMs) | ||||||
|                     .setProducerId(producerIdAndEpoch.producerId) |                     .setProducerId(producerIdAndEpoch.producerId) | ||||||
|                     .setProducerEpoch(producerIdAndEpoch.epoch) |                     .setProducerEpoch(producerIdAndEpoch.epoch); | ||||||
|                     .setEnable2Pc(enable2PC) |  | ||||||
|                     .setKeepPreparedTxn(keepPreparedTxn); |  | ||||||
| 
 | 
 | ||||||
|             InitProducerIdHandler handler = new InitProducerIdHandler(new InitProducerIdRequest.Builder(requestData), |             InitProducerIdHandler handler = new InitProducerIdHandler(new InitProducerIdRequest.Builder(requestData), | ||||||
|                     isEpochBump); |                     isEpochBump); | ||||||
|  |  | ||||||
|  | @ -32,7 +32,6 @@ import org.apache.kafka.clients.producer.internals.ProducerMetadata; | ||||||
| import org.apache.kafka.clients.producer.internals.RecordAccumulator; | import org.apache.kafka.clients.producer.internals.RecordAccumulator; | ||||||
| import org.apache.kafka.clients.producer.internals.Sender; | import org.apache.kafka.clients.producer.internals.Sender; | ||||||
| import org.apache.kafka.clients.producer.internals.TransactionManager; | import org.apache.kafka.clients.producer.internals.TransactionManager; | ||||||
| import org.apache.kafka.clients.producer.internals.TransactionalRequestResult; |  | ||||||
| import org.apache.kafka.common.Cluster; | import org.apache.kafka.common.Cluster; | ||||||
| import org.apache.kafka.common.KafkaException; | import org.apache.kafka.common.KafkaException; | ||||||
| import org.apache.kafka.common.Metric; | import org.apache.kafka.common.Metric; | ||||||
|  | @ -47,7 +46,6 @@ import org.apache.kafka.common.config.SslConfigs; | ||||||
| import org.apache.kafka.common.errors.ClusterAuthorizationException; | import org.apache.kafka.common.errors.ClusterAuthorizationException; | ||||||
| import org.apache.kafka.common.errors.InterruptException; | import org.apache.kafka.common.errors.InterruptException; | ||||||
| import org.apache.kafka.common.errors.InvalidTopicException; | import org.apache.kafka.common.errors.InvalidTopicException; | ||||||
| import org.apache.kafka.common.errors.InvalidTxnStateException; |  | ||||||
| import org.apache.kafka.common.errors.RecordTooLargeException; | import org.apache.kafka.common.errors.RecordTooLargeException; | ||||||
| import org.apache.kafka.common.errors.TimeoutException; | import org.apache.kafka.common.errors.TimeoutException; | ||||||
| import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; | import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; | ||||||
|  | @ -78,7 +76,6 @@ import org.apache.kafka.common.requests.AddOffsetsToTxnResponse; | ||||||
| import org.apache.kafka.common.requests.EndTxnResponse; | import org.apache.kafka.common.requests.EndTxnResponse; | ||||||
| import org.apache.kafka.common.requests.FindCoordinatorRequest; | import org.apache.kafka.common.requests.FindCoordinatorRequest; | ||||||
| import org.apache.kafka.common.requests.FindCoordinatorResponse; | import org.apache.kafka.common.requests.FindCoordinatorResponse; | ||||||
| import org.apache.kafka.common.requests.InitProducerIdRequest; |  | ||||||
| import org.apache.kafka.common.requests.InitProducerIdResponse; | import org.apache.kafka.common.requests.InitProducerIdResponse; | ||||||
| import org.apache.kafka.common.requests.JoinGroupRequest; | import org.apache.kafka.common.requests.JoinGroupRequest; | ||||||
| import org.apache.kafka.common.requests.MetadataResponse; | import org.apache.kafka.common.requests.MetadataResponse; | ||||||
|  | @ -107,7 +104,6 @@ import org.junit.jupiter.api.BeforeEach; | ||||||
| import org.junit.jupiter.api.Test; | import org.junit.jupiter.api.Test; | ||||||
| import org.junit.jupiter.api.TestInfo; | import org.junit.jupiter.api.TestInfo; | ||||||
| import org.junit.jupiter.params.ParameterizedTest; | import org.junit.jupiter.params.ParameterizedTest; | ||||||
| import org.junit.jupiter.params.provider.CsvSource; |  | ||||||
| import org.junit.jupiter.params.provider.ValueSource; | import org.junit.jupiter.params.provider.ValueSource; | ||||||
| import org.mockito.MockedStatic; | import org.mockito.MockedStatic; | ||||||
| import org.mockito.Mockito; | import org.mockito.Mockito; | ||||||
|  | @ -154,7 +150,6 @@ import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||||
| import static org.junit.jupiter.api.Assertions.assertNotEquals; | import static org.junit.jupiter.api.Assertions.assertNotEquals; | ||||||
| import static org.junit.jupiter.api.Assertions.assertNotNull; | import static org.junit.jupiter.api.Assertions.assertNotNull; | ||||||
| import static org.junit.jupiter.api.Assertions.assertNull; | import static org.junit.jupiter.api.Assertions.assertNull; | ||||||
| import static org.junit.jupiter.api.Assertions.assertSame; |  | ||||||
| import static org.junit.jupiter.api.Assertions.assertThrows; | import static org.junit.jupiter.api.Assertions.assertThrows; | ||||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | import static org.junit.jupiter.api.Assertions.assertTrue; | ||||||
| import static org.junit.jupiter.api.Assertions.fail; | import static org.junit.jupiter.api.Assertions.fail; | ||||||
|  | @ -165,7 +160,6 @@ import static org.mockito.ArgumentMatchers.anyString; | ||||||
| import static org.mockito.ArgumentMatchers.eq; | import static org.mockito.ArgumentMatchers.eq; | ||||||
| import static org.mockito.ArgumentMatchers.notNull; | import static org.mockito.ArgumentMatchers.notNull; | ||||||
| import static org.mockito.Mockito.atMostOnce; | import static org.mockito.Mockito.atMostOnce; | ||||||
| import static org.mockito.Mockito.doNothing; |  | ||||||
| import static org.mockito.Mockito.mock; | import static org.mockito.Mockito.mock; | ||||||
| import static org.mockito.Mockito.mockStatic; | import static org.mockito.Mockito.mockStatic; | ||||||
| import static org.mockito.Mockito.never; | import static org.mockito.Mockito.never; | ||||||
|  | @ -1389,294 +1383,6 @@ public class KafkaProducerTest { | ||||||
|         } |         } | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     @ParameterizedTest |  | ||||||
|     @CsvSource({ |  | ||||||
|         "true, false", |  | ||||||
|         "true, true", |  | ||||||
|         "false, true" |  | ||||||
|     }) |  | ||||||
|     public void testInitTransactionsWithKeepPreparedTxnAndTwoPhaseCommit(boolean keepPreparedTxn, boolean enable2PC) { |  | ||||||
|         Map<String, Object> configs = new HashMap<>(); |  | ||||||
|         configs.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "test-txn-id"); |  | ||||||
|         configs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, 10000); |  | ||||||
|         configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9000"); |  | ||||||
|         if (enable2PC) { |  | ||||||
|             configs.put(ProducerConfig.TRANSACTION_TWO_PHASE_COMMIT_ENABLE_CONFIG, true); |  | ||||||
|         } |  | ||||||
| 
 |  | ||||||
|         Time time = new MockTime(1); |  | ||||||
|         MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap("topic", 1)); |  | ||||||
|         ProducerMetadata metadata = newMetadata(0, 0, Long.MAX_VALUE); |  | ||||||
|         MockClient client = new MockClient(time, metadata); |  | ||||||
|         client.updateMetadata(initialUpdateResponse); |  | ||||||
| 
 |  | ||||||
|         // Capture flags from the InitProducerIdRequest |  | ||||||
|         boolean[] requestFlags = new boolean[2]; // [keepPreparedTxn, enable2Pc] |  | ||||||
|          |  | ||||||
|         client.prepareResponse( |  | ||||||
|             request -> request instanceof FindCoordinatorRequest && |  | ||||||
|                 ((FindCoordinatorRequest) request).data().keyType() == FindCoordinatorRequest.CoordinatorType.TRANSACTION.id(), |  | ||||||
|             FindCoordinatorResponse.prepareResponse(Errors.NONE, "test-txn-id", NODE)); |  | ||||||
|              |  | ||||||
|         client.prepareResponse( |  | ||||||
|             request -> { |  | ||||||
|                 if (request instanceof InitProducerIdRequest) { |  | ||||||
|                     InitProducerIdRequest initRequest = (InitProducerIdRequest) request; |  | ||||||
|                     requestFlags[0] = initRequest.data().keepPreparedTxn(); |  | ||||||
|                     requestFlags[1] = initRequest.data().enable2Pc(); |  | ||||||
|                     return true; |  | ||||||
|                 } |  | ||||||
|                 return false; |  | ||||||
|             }, |  | ||||||
|             initProducerIdResponse(1L, (short) 5, Errors.NONE)); |  | ||||||
|              |  | ||||||
|         try (Producer<String, String> producer = kafkaProducer(configs, new StringSerializer(), |  | ||||||
|                 new StringSerializer(), metadata, client, null, time)) { |  | ||||||
|             producer.initTransactions(keepPreparedTxn); |  | ||||||
|              |  | ||||||
|             // Verify request flags match expected values |  | ||||||
|             assertEquals(keepPreparedTxn, requestFlags[0],  |  | ||||||
|                 "keepPreparedTxn flag should match input parameter"); |  | ||||||
|             assertEquals(enable2PC, requestFlags[1],  |  | ||||||
|                 "enable2Pc flag should match producer configuration"); |  | ||||||
|         } |  | ||||||
|     } |  | ||||||
| 
 |  | ||||||
|     @Test |  | ||||||
|     public void testPrepareTransactionSuccess() throws Exception { |  | ||||||
|         StringSerializer serializer = new StringSerializer(); |  | ||||||
|         KafkaProducerTestContext<String> ctx = new KafkaProducerTestContext<>(testInfo, serializer); |  | ||||||
| 
 |  | ||||||
|         when(ctx.transactionManager.isTransactionV2Enabled()).thenReturn(true); |  | ||||||
|         when(ctx.transactionManager.is2PCEnabled()).thenReturn(true); |  | ||||||
|         when(ctx.sender.isRunning()).thenReturn(true); |  | ||||||
| 
 |  | ||||||
|         doNothing().when(ctx.transactionManager).prepareTransaction(); |  | ||||||
| 
 |  | ||||||
|         PreparedTxnState expectedState = mock(PreparedTxnState.class); |  | ||||||
|         when(ctx.transactionManager.preparedTransactionState()).thenReturn(expectedState); |  | ||||||
| 
 |  | ||||||
|         try (KafkaProducer<String, String> producer = ctx.newKafkaProducer()) { |  | ||||||
|             PreparedTxnState returned = producer.prepareTransaction(); |  | ||||||
|             assertSame(expectedState, returned); |  | ||||||
| 
 |  | ||||||
|             verify(ctx.transactionManager).prepareTransaction(); |  | ||||||
|             verify(ctx.accumulator).beginFlush(); |  | ||||||
|             verify(ctx.accumulator).awaitFlushCompletion(); |  | ||||||
|         } |  | ||||||
|     } |  | ||||||
| 
 |  | ||||||
|     @Test |  | ||||||
|     public void testSendNotAllowedInPreparedTransactionState() throws Exception { |  | ||||||
|         StringSerializer serializer = new StringSerializer(); |  | ||||||
|         KafkaProducerTestContext<String> ctx = new KafkaProducerTestContext<>(testInfo, serializer); |  | ||||||
| 
 |  | ||||||
|         String topic = "foo"; |  | ||||||
|         Cluster cluster = TestUtils.singletonCluster(topic, 1); |  | ||||||
| 
 |  | ||||||
|         when(ctx.sender.isRunning()).thenReturn(true); |  | ||||||
|         when(ctx.metadata.fetch()).thenReturn(cluster); |  | ||||||
| 
 |  | ||||||
|         // Mock transaction manager to simulate being in a prepared state |  | ||||||
|         when(ctx.transactionManager.isTransactional()).thenReturn(true); |  | ||||||
|         when(ctx.transactionManager.isPrepared()).thenReturn(true); |  | ||||||
| 
 |  | ||||||
|         // Create record to send |  | ||||||
|         long timestamp = ctx.time.milliseconds(); |  | ||||||
|         ProducerRecord<String, String> record = new ProducerRecord<>(topic, 0, timestamp, "key", "value"); |  | ||||||
| 
 |  | ||||||
|         try (KafkaProducer<String, String> producer = ctx.newKafkaProducer()) { |  | ||||||
|             // Verify that sending a record throws IllegalStateException with the correct message |  | ||||||
|             IllegalStateException exception = assertThrows( |  | ||||||
|                 IllegalStateException.class, |  | ||||||
|                 () -> producer.send(record) |  | ||||||
|             ); |  | ||||||
| 
 |  | ||||||
|             assertTrue(exception.getMessage().contains("Cannot perform operation while the transaction is in a prepared state")); |  | ||||||
| 
 |  | ||||||
|             // Verify transactionManager methods were called |  | ||||||
|             verify(ctx.transactionManager).isTransactional(); |  | ||||||
|             verify(ctx.transactionManager).isPrepared(); |  | ||||||
| 
 |  | ||||||
|             // Verify that no message was actually sent (accumulator was not called) |  | ||||||
|             verify(ctx.accumulator, never()).append( |  | ||||||
|                 eq(topic), |  | ||||||
|                 anyInt(), |  | ||||||
|                 anyLong(), |  | ||||||
|                 any(), |  | ||||||
|                 any(), |  | ||||||
|                 any(), |  | ||||||
|                 any(), |  | ||||||
|                 anyLong(), |  | ||||||
|                 anyLong(), |  | ||||||
|                 any() |  | ||||||
|             ); |  | ||||||
|         } |  | ||||||
|     } |  | ||||||
| 
 |  | ||||||
|     @Test |  | ||||||
|     public void testSendOffsetsNotAllowedInPreparedTransactionState() throws Exception { |  | ||||||
|         StringSerializer serializer = new StringSerializer(); |  | ||||||
|         KafkaProducerTestContext<String> ctx = new KafkaProducerTestContext<>(testInfo, serializer); |  | ||||||
| 
 |  | ||||||
|         String topic = "foo"; |  | ||||||
|         Cluster cluster = TestUtils.singletonCluster(topic, 1); |  | ||||||
| 
 |  | ||||||
|         when(ctx.sender.isRunning()).thenReturn(true); |  | ||||||
|         when(ctx.metadata.fetch()).thenReturn(cluster); |  | ||||||
| 
 |  | ||||||
|         // Mock transaction manager to simulate being in a prepared state |  | ||||||
|         when(ctx.transactionManager.isTransactional()).thenReturn(true); |  | ||||||
|         when(ctx.transactionManager.isPrepared()).thenReturn(true); |  | ||||||
| 
 |  | ||||||
|         // Create consumer group metadata |  | ||||||
|         String groupId = "test-group"; |  | ||||||
|         Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>(); |  | ||||||
|         offsets.put(new TopicPartition(topic, 0), new OffsetAndMetadata(100L)); |  | ||||||
|         ConsumerGroupMetadata groupMetadata = new ConsumerGroupMetadata(groupId); |  | ||||||
| 
 |  | ||||||
|         try (KafkaProducer<String, String> producer = ctx.newKafkaProducer()) { |  | ||||||
|             // Verify that sending offsets throws IllegalStateException with the correct message |  | ||||||
|             IllegalStateException exception = assertThrows( |  | ||||||
|                 IllegalStateException.class, |  | ||||||
|                 () -> producer.sendOffsetsToTransaction(offsets, groupMetadata) |  | ||||||
|             ); |  | ||||||
| 
 |  | ||||||
|             assertTrue(exception.getMessage().contains("Cannot perform operation while the transaction is in a prepared state")); |  | ||||||
| 
 |  | ||||||
|             // Verify transactionManager methods were called |  | ||||||
|             verify(ctx.transactionManager).isTransactional(); |  | ||||||
|             verify(ctx.transactionManager).isPrepared(); |  | ||||||
| 
 |  | ||||||
|             // Verify that no offsets were actually sent |  | ||||||
|             verify(ctx.transactionManager, never()).sendOffsetsToTransaction( |  | ||||||
|                 eq(offsets), |  | ||||||
|                 eq(groupMetadata) |  | ||||||
|             ); |  | ||||||
|         } |  | ||||||
|     } |  | ||||||
| 
 |  | ||||||
|     @Test |  | ||||||
|     public void testBeginTransactionNotAllowedInPreparedTransactionState() throws Exception { |  | ||||||
|         StringSerializer serializer = new StringSerializer(); |  | ||||||
|         KafkaProducerTestContext<String> ctx = new KafkaProducerTestContext<>(testInfo, serializer); |  | ||||||
| 
 |  | ||||||
|         when(ctx.sender.isRunning()).thenReturn(true); |  | ||||||
| 
 |  | ||||||
|         // Mock transaction manager to simulate being in a prepared state |  | ||||||
|         when(ctx.transactionManager.isTransactional()).thenReturn(true); |  | ||||||
|         when(ctx.transactionManager.isPrepared()).thenReturn(true); |  | ||||||
| 
 |  | ||||||
|         try (KafkaProducer<String, String> producer = ctx.newKafkaProducer()) { |  | ||||||
|             // Verify that calling beginTransaction throws IllegalStateException with the correct message |  | ||||||
|             IllegalStateException exception = assertThrows( |  | ||||||
|                 IllegalStateException.class, |  | ||||||
|                 producer::beginTransaction |  | ||||||
|             ); |  | ||||||
| 
 |  | ||||||
|             assertTrue(exception.getMessage().contains("Cannot perform operation while the transaction is in a prepared state")); |  | ||||||
| 
 |  | ||||||
|             // Verify transactionManager methods were called |  | ||||||
|             verify(ctx.transactionManager).isTransactional(); |  | ||||||
|             verify(ctx.transactionManager).isPrepared(); |  | ||||||
|         } |  | ||||||
|     } |  | ||||||
| 
 |  | ||||||
|     @Test |  | ||||||
|     public void testPrepareTransactionFailsWhen2PCDisabled() { |  | ||||||
|         StringSerializer serializer = new StringSerializer(); |  | ||||||
|         KafkaProducerTestContext<String> ctx = new KafkaProducerTestContext<>(testInfo, serializer); |  | ||||||
| 
 |  | ||||||
|         // Disable 2PC |  | ||||||
|         when(ctx.transactionManager.isTransactionV2Enabled()).thenReturn(true); |  | ||||||
|         when(ctx.transactionManager.is2PCEnabled()).thenReturn(false); |  | ||||||
|         when(ctx.sender.isRunning()).thenReturn(true); |  | ||||||
| 
 |  | ||||||
|         try (KafkaProducer<String, String> producer = ctx.newKafkaProducer()) { |  | ||||||
|             assertThrows( |  | ||||||
|                 InvalidTxnStateException.class, |  | ||||||
|                 producer::prepareTransaction, |  | ||||||
|                 "prepareTransaction() should fail if 2PC is disabled" |  | ||||||
|             ); |  | ||||||
|         } |  | ||||||
|     } |  | ||||||
|      |  | ||||||
|     @Test |  | ||||||
|     public void testCompleteTransactionWithMatchingState() throws Exception { |  | ||||||
|         StringSerializer serializer = new StringSerializer(); |  | ||||||
|         KafkaProducerTestContext<String> ctx = new KafkaProducerTestContext<>(testInfo, serializer); |  | ||||||
| 
 |  | ||||||
|         when(ctx.transactionManager.isPrepared()).thenReturn(true); |  | ||||||
|         when(ctx.sender.isRunning()).thenReturn(true); |  | ||||||
|          |  | ||||||
|         // Create prepared states with matching values |  | ||||||
|         long producerId = 12345L; |  | ||||||
|         short epoch = 5; |  | ||||||
|         PreparedTxnState currentState = new PreparedTxnState(producerId, epoch); |  | ||||||
|         PreparedTxnState inputState = new PreparedTxnState(producerId, epoch); |  | ||||||
|          |  | ||||||
|         // Set up the transaction manager to return the prepared state |  | ||||||
|         when(ctx.transactionManager.preparedTransactionState()).thenReturn(currentState); |  | ||||||
|          |  | ||||||
|         // Should trigger commit when states match |  | ||||||
|         TransactionalRequestResult commitResult = mock(TransactionalRequestResult.class); |  | ||||||
|         when(ctx.transactionManager.beginCommit()).thenReturn(commitResult); |  | ||||||
|          |  | ||||||
|         try (KafkaProducer<String, String> producer = ctx.newKafkaProducer()) { |  | ||||||
|             // Call completeTransaction with the matching state |  | ||||||
|             producer.completeTransaction(inputState); |  | ||||||
|              |  | ||||||
|             // Verify methods called in order |  | ||||||
|             verify(ctx.transactionManager).isPrepared(); |  | ||||||
|             verify(ctx.transactionManager).preparedTransactionState(); |  | ||||||
|             verify(ctx.transactionManager).beginCommit(); |  | ||||||
|              |  | ||||||
|             // Verify abort was never called |  | ||||||
|             verify(ctx.transactionManager, never()).beginAbort(); |  | ||||||
|              |  | ||||||
|             // Verify sender was woken up |  | ||||||
|             verify(ctx.sender).wakeup(); |  | ||||||
|         } |  | ||||||
|     } |  | ||||||
|      |  | ||||||
|     @Test |  | ||||||
|     public void testCompleteTransactionWithNonMatchingState() throws Exception { |  | ||||||
|         StringSerializer serializer = new StringSerializer(); |  | ||||||
|         KafkaProducerTestContext<String> ctx = new KafkaProducerTestContext<>(testInfo, serializer); |  | ||||||
| 
 |  | ||||||
|         when(ctx.transactionManager.isPrepared()).thenReturn(true); |  | ||||||
|         when(ctx.sender.isRunning()).thenReturn(true); |  | ||||||
|          |  | ||||||
|         // Create txn prepared states with different values |  | ||||||
|         long producerId = 12345L; |  | ||||||
|         short epoch = 5; |  | ||||||
|         PreparedTxnState currentState = new PreparedTxnState(producerId, epoch); |  | ||||||
|         PreparedTxnState inputState = new PreparedTxnState(producerId + 1, epoch); |  | ||||||
|          |  | ||||||
|         // Set up the transaction manager to return the prepared state |  | ||||||
|         when(ctx.transactionManager.preparedTransactionState()).thenReturn(currentState); |  | ||||||
|          |  | ||||||
|         // Should trigger abort when states don't match |  | ||||||
|         TransactionalRequestResult abortResult = mock(TransactionalRequestResult.class); |  | ||||||
|         when(ctx.transactionManager.beginAbort()).thenReturn(abortResult); |  | ||||||
|          |  | ||||||
|         try (KafkaProducer<String, String> producer = ctx.newKafkaProducer()) { |  | ||||||
|             // Call completeTransaction with the non-matching state |  | ||||||
|             producer.completeTransaction(inputState); |  | ||||||
|              |  | ||||||
|             // Verify methods called in order |  | ||||||
|             verify(ctx.transactionManager).isPrepared(); |  | ||||||
|             verify(ctx.transactionManager).preparedTransactionState(); |  | ||||||
|             verify(ctx.transactionManager).beginAbort(); |  | ||||||
|              |  | ||||||
|             // Verify commit was never called |  | ||||||
|             verify(ctx.transactionManager, never()).beginCommit(); |  | ||||||
|              |  | ||||||
|             // Verify sender was woken up |  | ||||||
|             verify(ctx.sender).wakeup(); |  | ||||||
|         } |  | ||||||
|     } |  | ||||||
|      |  | ||||||
|     @Test |     @Test | ||||||
|     public void testClusterAuthorizationFailure() throws Exception { |     public void testClusterAuthorizationFailure() throws Exception { | ||||||
|         int maxBlockMs = 500; |         int maxBlockMs = 500; | ||||||
|  |  | ||||||
|  | @ -23,7 +23,6 @@ import org.apache.kafka.clients.NodeApiVersions; | ||||||
| import org.apache.kafka.clients.consumer.CommitFailedException; | import org.apache.kafka.clients.consumer.CommitFailedException; | ||||||
| import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; | import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; | ||||||
| import org.apache.kafka.clients.consumer.OffsetAndMetadata; | import org.apache.kafka.clients.consumer.OffsetAndMetadata; | ||||||
| import org.apache.kafka.clients.producer.PreparedTxnState; |  | ||||||
| import org.apache.kafka.clients.producer.RecordMetadata; | import org.apache.kafka.clients.producer.RecordMetadata; | ||||||
| import org.apache.kafka.common.KafkaException; | import org.apache.kafka.common.KafkaException; | ||||||
| import org.apache.kafka.common.Node; | import org.apache.kafka.common.Node; | ||||||
|  | @ -4025,56 +4024,6 @@ public class TransactionManagerTest { | ||||||
|         assertFalse(transactionManager.hasOngoingTransaction()); |         assertFalse(transactionManager.hasOngoingTransaction()); | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     @Test |  | ||||||
|     public void testInitializeTransactionsWithKeepPreparedTxn() { |  | ||||||
|         doInitTransactionsWith2PCEnabled(true); |  | ||||||
|         runUntil(transactionManager::hasProducerId); |  | ||||||
| 
 |  | ||||||
|         // Expect a bumped epoch in the response. |  | ||||||
|         assertTrue(transactionManager.hasProducerId()); |  | ||||||
|         assertFalse(transactionManager.hasOngoingTransaction()); |  | ||||||
|         assertEquals(ongoingProducerId, transactionManager.producerIdAndEpoch().producerId); |  | ||||||
|         assertEquals(bumpedOngoingEpoch, transactionManager.producerIdAndEpoch().epoch); |  | ||||||
|     } |  | ||||||
| 
 |  | ||||||
|     @Test |  | ||||||
|     public void testPrepareTransaction() { |  | ||||||
|         doInitTransactionsWith2PCEnabled(false); |  | ||||||
|         runUntil(transactionManager::hasProducerId); |  | ||||||
| 
 |  | ||||||
|         // Begin a transaction |  | ||||||
|         transactionManager.beginTransaction(); |  | ||||||
|         assertTrue(transactionManager.hasOngoingTransaction()); |  | ||||||
| 
 |  | ||||||
|         // Add a partition to the transaction |  | ||||||
|         transactionManager.maybeAddPartition(tp0); |  | ||||||
| 
 |  | ||||||
|         // Capture the current producer ID and epoch before preparing the response |  | ||||||
|         long producerId = transactionManager.producerIdAndEpoch().producerId; |  | ||||||
|         short epoch = transactionManager.producerIdAndEpoch().epoch; |  | ||||||
| 
 |  | ||||||
|         // Simulate a produce request |  | ||||||
|         try { |  | ||||||
|             // Prepare the response before sending to ensure it's ready |  | ||||||
|             prepareProduceResponse(Errors.NONE, producerId, epoch); |  | ||||||
| 
 |  | ||||||
|             appendToAccumulator(tp0); |  | ||||||
|             // Wait until the request is processed |  | ||||||
|             runUntil(() -> !client.hasPendingResponses()); |  | ||||||
|         } catch (InterruptedException e) { |  | ||||||
|             fail("Unexpected interruption: " + e); |  | ||||||
|         } |  | ||||||
| 
 |  | ||||||
|         transactionManager.prepareTransaction(); |  | ||||||
|         assertTrue(transactionManager.isPrepared()); |  | ||||||
| 
 |  | ||||||
|         PreparedTxnState preparedState = transactionManager.preparedTransactionState(); |  | ||||||
|         // Validate the state contains the correct serialized producer ID and epoch |  | ||||||
|         assertEquals(producerId + ":" + epoch, preparedState.toString()); |  | ||||||
|         assertEquals(producerId, preparedState.producerId()); |  | ||||||
|         assertEquals(epoch, preparedState.epoch()); |  | ||||||
|     } |  | ||||||
| 
 |  | ||||||
|     private void prepareAddPartitionsToTxn(final Map<TopicPartition, Errors> errors) { |     private void prepareAddPartitionsToTxn(final Map<TopicPartition, Errors> errors) { | ||||||
|         AddPartitionsToTxnResult result = AddPartitionsToTxnResponse.resultForTransaction(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID, errors); |         AddPartitionsToTxnResult result = AddPartitionsToTxnResponse.resultForTransaction(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID, errors); | ||||||
|         AddPartitionsToTxnResponseData data = new AddPartitionsToTxnResponseData().setResultsByTopicV3AndBelow(result.topicResults()).setThrottleTimeMs(0); |         AddPartitionsToTxnResponseData data = new AddPartitionsToTxnResponseData().setResultsByTopicV3AndBelow(result.topicResults()).setThrottleTimeMs(0); | ||||||
|  |  | ||||||
		Loading…
	
		Reference in New Issue