mirror of https://github.com/apache/kafka.git
				
				
				
			KAFKA-12690 Remove deprecated Producer#sendOffsetsToTransaction (#17865)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
		
							parent
							
								
									38aca3a045
								
							
						
					
					
						commit
						7db4d53f18
					
				|  | @ -689,46 +689,6 @@ public class KafkaProducer<K, V> implements Producer<K, V> { | ||||||
|         producerMetrics.recordBeginTxn(time.nanoseconds() - now); |         producerMetrics.recordBeginTxn(time.nanoseconds() - now); | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     /** |  | ||||||
|      * Sends a list of specified offsets to the consumer group coordinator, and also marks |  | ||||||
|      * those offsets as part of the current transaction. These offsets will be considered |  | ||||||
|      * committed only if the transaction is committed successfully. The committed offset should |  | ||||||
|      * be the next message your application will consume, i.e. lastProcessedMessageOffset + 1. |  | ||||||
|      * <p> |  | ||||||
|      * This method should be used when you need to batch consumed and produced messages |  | ||||||
|      * together, typically in a consume-transform-produce pattern. Thus, the specified |  | ||||||
|      * {@code consumerGroupId} should be the same as config parameter {@code group.id} of the used |  | ||||||
|      * {@link KafkaConsumer consumer}. Note, that the consumer should have {@code enable.auto.commit=false} |  | ||||||
|      * and should also not commit offsets manually (via {@link KafkaConsumer#commitSync(Map) sync} or |  | ||||||
|      * {@link KafkaConsumer#commitAsync(Map, OffsetCommitCallback) async} commits). |  | ||||||
|      * |  | ||||||
|      * <p> |  | ||||||
|      * This method is a blocking call that waits until the request has been received and acknowledged by the consumer group |  | ||||||
|      * coordinator; but the offsets are not considered as committed until the transaction itself is successfully committed later (via |  | ||||||
|      * the {@link #commitTransaction()} call). |  | ||||||
|      * |  | ||||||
|      * @throws IllegalStateException if no transactional.id has been configured, no transaction has been started |  | ||||||
|      * @throws ProducerFencedException fatal error indicating another producer with the same transactional.id is active |  | ||||||
|      * @throws org.apache.kafka.common.errors.UnsupportedVersionException fatal error indicating the broker |  | ||||||
|      *         does not support transactions (i.e. if its version is lower than 0.11.0.0) |  | ||||||
|      * @throws org.apache.kafka.common.errors.UnsupportedForMessageFormatException fatal error indicating the message |  | ||||||
|      *         format used for the offsets topic on the broker does not support transactions |  | ||||||
|      * @throws org.apache.kafka.common.errors.AuthorizationException fatal error indicating that the configured |  | ||||||
|      *         transactional.id is not authorized, or the consumer group id is not authorized. |  | ||||||
|      * @throws org.apache.kafka.common.errors.InvalidProducerEpochException if the producer has attempted to produce with an old epoch |  | ||||||
|      *         to the partition leader. See the exception for more details |  | ||||||
|      * @throws TimeoutException if the time taken for sending the offsets has surpassed <code>max.block.ms</code>. |  | ||||||
|      * @throws KafkaException if the producer has encountered a previous fatal or abortable error, or for any |  | ||||||
|      *         other unexpected error |  | ||||||
|      * |  | ||||||
|      * @deprecated Since 3.0.0, please use {@link #sendOffsetsToTransaction(Map, ConsumerGroupMetadata)} instead. |  | ||||||
|      */ |  | ||||||
|     @Deprecated |  | ||||||
|     public void sendOffsetsToTransaction(Map<TopicPartition, OffsetAndMetadata> offsets, |  | ||||||
|                                          String consumerGroupId) throws ProducerFencedException { |  | ||||||
|         sendOffsetsToTransaction(offsets, new ConsumerGroupMetadata(consumerGroupId)); |  | ||||||
|     } |  | ||||||
| 
 |  | ||||||
|     /** |     /** | ||||||
|      * Sends a list of specified offsets to the consumer group coordinator, and also marks |      * Sends a list of specified offsets to the consumer group coordinator, and also marks | ||||||
|      * those offsets as part of the current transaction. These offsets will be considered |      * those offsets as part of the current transaction. These offsets will be considered | ||||||
|  |  | ||||||
|  | @ -202,14 +202,6 @@ public class MockProducer<K, V> implements Producer<K, V> { | ||||||
|         this.sentOffsets = false; |         this.sentOffsets = false; | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     @Deprecated |  | ||||||
|     @Override |  | ||||||
|     public void sendOffsetsToTransaction(Map<TopicPartition, OffsetAndMetadata> offsets, |  | ||||||
|                                          String consumerGroupId) throws ProducerFencedException { |  | ||||||
|         Objects.requireNonNull(consumerGroupId); |  | ||||||
|         sendOffsetsToTransaction(offsets, new ConsumerGroupMetadata(consumerGroupId)); |  | ||||||
|     } |  | ||||||
| 
 |  | ||||||
|     @Override |     @Override | ||||||
|     public void sendOffsetsToTransaction(Map<TopicPartition, OffsetAndMetadata> offsets, |     public void sendOffsetsToTransaction(Map<TopicPartition, OffsetAndMetadata> offsets, | ||||||
|                                          ConsumerGroupMetadata groupMetadata) throws ProducerFencedException { |                                          ConsumerGroupMetadata groupMetadata) throws ProducerFencedException { | ||||||
|  |  | ||||||
|  | @ -49,13 +49,6 @@ public interface Producer<K, V> extends Closeable { | ||||||
|      */ |      */ | ||||||
|     void beginTransaction() throws ProducerFencedException; |     void beginTransaction() throws ProducerFencedException; | ||||||
| 
 | 
 | ||||||
|     /** |  | ||||||
|      * See {@link KafkaProducer#sendOffsetsToTransaction(Map, String)} |  | ||||||
|      */ |  | ||||||
|     @Deprecated |  | ||||||
|     void sendOffsetsToTransaction(Map<TopicPartition, OffsetAndMetadata> offsets, |  | ||||||
|                                   String consumerGroupId) throws ProducerFencedException; |  | ||||||
| 
 |  | ||||||
|     /** |     /** | ||||||
|      * See {@link KafkaProducer#sendOffsetsToTransaction(Map, ConsumerGroupMetadata)} |      * See {@link KafkaProducer#sendOffsetsToTransaction(Map, ConsumerGroupMetadata)} | ||||||
|      */ |      */ | ||||||
|  |  | ||||||
|  | @ -419,15 +419,6 @@ public class MockProducerTest { | ||||||
|         assertEquals(Collections.singletonList(expectedResult), producer.consumerGroupOffsetsHistory()); |         assertEquals(Collections.singletonList(expectedResult), producer.consumerGroupOffsetsHistory()); | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     @Deprecated |  | ||||||
|     @Test |  | ||||||
|     public void shouldThrowOnNullConsumerGroupIdWhenSendOffsetsToTransaction() { |  | ||||||
|         buildMockProducer(true); |  | ||||||
|         producer.initTransactions(); |  | ||||||
|         producer.beginTransaction(); |  | ||||||
|         assertThrows(NullPointerException.class, () -> producer.sendOffsetsToTransaction(Collections.emptyMap(), (String) null)); |  | ||||||
|     } |  | ||||||
| 
 |  | ||||||
|     @Test |     @Test | ||||||
|     public void shouldThrowOnNullConsumerGroupMetadataWhenSendOffsetsToTransaction() { |     public void shouldThrowOnNullConsumerGroupMetadataWhenSendOffsetsToTransaction() { | ||||||
|         buildMockProducer(true); |         buildMockProducer(true); | ||||||
|  | @ -436,16 +427,6 @@ public class MockProducerTest { | ||||||
|         assertThrows(NullPointerException.class, () -> producer.sendOffsetsToTransaction(Collections.emptyMap(), new ConsumerGroupMetadata(null))); |         assertThrows(NullPointerException.class, () -> producer.sendOffsetsToTransaction(Collections.emptyMap(), new ConsumerGroupMetadata(null))); | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     @Deprecated |  | ||||||
|     @Test |  | ||||||
|     public void shouldIgnoreEmptyOffsetsWhenSendOffsetsToTransactionByGroupId() { |  | ||||||
|         buildMockProducer(true); |  | ||||||
|         producer.initTransactions(); |  | ||||||
|         producer.beginTransaction(); |  | ||||||
|         producer.sendOffsetsToTransaction(Collections.emptyMap(), "groupId"); |  | ||||||
|         assertFalse(producer.sentOffsets()); |  | ||||||
|     } |  | ||||||
| 
 |  | ||||||
|     @Test |     @Test | ||||||
|     public void shouldIgnoreEmptyOffsetsWhenSendOffsetsToTransactionByGroupMetadata() { |     public void shouldIgnoreEmptyOffsetsWhenSendOffsetsToTransactionByGroupMetadata() { | ||||||
|         buildMockProducer(true); |         buildMockProducer(true); | ||||||
|  | @ -455,24 +436,6 @@ public class MockProducerTest { | ||||||
|         assertFalse(producer.sentOffsets()); |         assertFalse(producer.sentOffsets()); | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     @Deprecated |  | ||||||
|     @Test |  | ||||||
|     public void shouldAddOffsetsWhenSendOffsetsToTransactionByGroupId() { |  | ||||||
|         buildMockProducer(true); |  | ||||||
|         producer.initTransactions(); |  | ||||||
|         producer.beginTransaction(); |  | ||||||
| 
 |  | ||||||
|         assertFalse(producer.sentOffsets()); |  | ||||||
| 
 |  | ||||||
|         Map<TopicPartition, OffsetAndMetadata> groupCommit = new HashMap<TopicPartition, OffsetAndMetadata>() { |  | ||||||
|             { |  | ||||||
|                 put(new TopicPartition(topic, 0), new OffsetAndMetadata(42L, null)); |  | ||||||
|             } |  | ||||||
|         }; |  | ||||||
|         producer.sendOffsetsToTransaction(groupCommit, "groupId"); |  | ||||||
|         assertTrue(producer.sentOffsets()); |  | ||||||
|     } |  | ||||||
| 
 |  | ||||||
|     @Test |     @Test | ||||||
|     public void shouldAddOffsetsWhenSendOffsetsToTransactionByGroupMetadata() { |     public void shouldAddOffsetsWhenSendOffsetsToTransactionByGroupMetadata() { | ||||||
|         buildMockProducer(true); |         buildMockProducer(true); | ||||||
|  |  | ||||||
|  | @ -32,7 +32,6 @@ import org.junit.jupiter.api.Assertions._ | ||||||
| import org.junit.jupiter.params.ParameterizedTest | import org.junit.jupiter.params.ParameterizedTest | ||||||
| import org.junit.jupiter.params.provider.MethodSource | import org.junit.jupiter.params.provider.MethodSource | ||||||
| 
 | 
 | ||||||
| import scala.annotation.nowarn |  | ||||||
| import scala.jdk.CollectionConverters._ | import scala.jdk.CollectionConverters._ | ||||||
| import scala.collection.mutable | import scala.collection.mutable | ||||||
| 
 | 
 | ||||||
|  | @ -76,14 +75,6 @@ class TransactionsBounceTest extends IntegrationTestHarness { | ||||||
| 
 | 
 | ||||||
|   override protected def brokerCount: Int = 4 |   override protected def brokerCount: Int = 4 | ||||||
| 
 | 
 | ||||||
|   @nowarn("cat=deprecation") |  | ||||||
|   @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) |  | ||||||
|   @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17961")) |  | ||||||
|   def testWithGroupId(quorum: String, groupProtocol: String): Unit = { |  | ||||||
|     testBrokerFailure((producer, groupId, consumer) => |  | ||||||
|       producer.sendOffsetsToTransaction(TestUtils.consumerPositions(consumer).asJava, groupId)) |  | ||||||
|   } |  | ||||||
| 
 |  | ||||||
|   @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) |   @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) | ||||||
|   @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) |   @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) | ||||||
|   def testWithGroupMetadata(quorum: String, groupProtocol: String): Unit = { |   def testWithGroupMetadata(quorum: String, groupProtocol: String): Unit = { | ||||||
|  |  | ||||||
|  | @ -39,7 +39,6 @@ import java.time.Duration | ||||||
| import java.util | import java.util | ||||||
| import java.util.concurrent.TimeUnit | import java.util.concurrent.TimeUnit | ||||||
| import java.util.{Optional, Properties} | import java.util.{Optional, Properties} | ||||||
| import scala.annotation.nowarn |  | ||||||
| import scala.collection.{Seq, mutable} | import scala.collection.{Seq, mutable} | ||||||
| import scala.collection.mutable.{ArrayBuffer, ListBuffer} | import scala.collection.mutable.{ArrayBuffer, ListBuffer} | ||||||
| import scala.concurrent.ExecutionException | import scala.concurrent.ExecutionException | ||||||
|  | @ -301,14 +300,6 @@ class TransactionsTest extends IntegrationTestHarness { | ||||||
|     assertEquals(3L, second.offset) |     assertEquals(3L, second.offset) | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|   @nowarn("cat=deprecation") |  | ||||||
|   @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) |  | ||||||
|   @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17961")) |  | ||||||
|   def testSendOffsetsWithGroupId(quorum: String, groupProtocol: String): Unit = { |  | ||||||
|     sendOffset((producer, groupId, consumer) => |  | ||||||
|       producer.sendOffsetsToTransaction(TestUtils.consumerPositions(consumer).asJava, groupId)) |  | ||||||
|   } |  | ||||||
| 
 |  | ||||||
|   @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) |   @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) | ||||||
|   @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) |   @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) | ||||||
|   def testSendOffsetsWithGroupMetadata(quorum: String, groupProtocol: String): Unit = { |   def testSendOffsetsWithGroupMetadata(quorum: String, groupProtocol: String): Unit = { | ||||||
|  |  | ||||||
		Loading…
	
		Reference in New Issue