brandboat commented on code in PR #18448: URL: https://github.com/apache/kafka/pull/18448#discussion_r1912486167
########## core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIntegrationTest.scala: ########## @@ -0,0 +1,185 @@ +/** + * 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 kafka.coordinator.transaction + +import kafka.network.SocketServer +import kafka.server.IntegrationTestUtils +import org.apache.kafka.clients.admin.{Admin, TransactionState} +import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, ConsumerRecords, OffsetAndMetadata} +import org.apache.kafka.clients.producer.{Producer, ProducerConfig, ProducerRecord} +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterFeature, ClusterInstance, ClusterTest, ClusterTestDefaults, ClusterTestExtensions, ClusterTests, Type} +import org.apache.kafka.common.message.InitProducerIdRequestData +import org.apache.kafka.common.network.ListenerName +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.record.RecordBatch +import org.apache.kafka.common.requests.{InitProducerIdRequest, InitProducerIdResponse} +import org.apache.kafka.common.test.TestUtils +import org.apache.kafka.coordinator.group.GroupCoordinatorConfig +import org.apache.kafka.coordinator.transaction.TransactionLogConfig +import org.apache.kafka.server.common.{Feature, MetadataVersion} +import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} +import org.junit.jupiter.api.extension.ExtendWith + +import java.time.Duration +import java.util +import java.util.Collections +import java.util.stream.{Collectors, IntStream, StreamSupport} +import scala.concurrent.duration.DurationInt +import scala.jdk.CollectionConverters._ + +@ClusterTestDefaults(types = Array(Type.KRAFT), serverProperties = Array( + new ClusterConfigProperty(key = TransactionLogConfig.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "1"), + new ClusterConfigProperty(key = TransactionLogConfig.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, value = "1"), + new ClusterConfigProperty(key = TransactionLogConfig.TRANSACTIONS_TOPIC_MIN_ISR_CONFIG, value = "1"), + new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "1"), + new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"), +)) +@ExtendWith(value = Array(classOf[ClusterTestExtensions])) +class ProducerIntegrationTest { + + @ClusterTests(Array( + new ClusterTest(metadataVersion = MetadataVersion.IBP_3_3_IV0) + )) + def testUniqueProducerIds(clusterInstance: ClusterInstance): Unit = { + verifyUniqueIds(clusterInstance) + } + + @ClusterTests(Array( + new ClusterTest(features = Array( + new ClusterFeature(feature = Feature.TRANSACTION_VERSION, version = 0))), + new ClusterTest(features = Array( + new ClusterFeature(feature = Feature.TRANSACTION_VERSION, version = 1))), + new ClusterTest(features = Array( + new ClusterFeature(feature = Feature.TRANSACTION_VERSION, version = 2))), + )) + def testTransactionWithAndWithoutSend(cluster: ClusterInstance): Unit = { + val properties = new util.HashMap[String, Object] + properties.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "foobar") + properties.put(ProducerConfig.CLIENT_ID_CONFIG, "test") + properties.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true") + val producer: Producer[Array[Byte], Array[Byte]] = cluster.producer(properties) + try { + producer.initTransactions() + producer.beginTransaction() + producer.send(new ProducerRecord[Array[Byte], Array[Byte]]("test", "key".getBytes, "value".getBytes)) + producer.commitTransaction() + + producer.beginTransaction() + producer.commitTransaction() + } finally if (producer != null) producer.close() + } + + @ClusterTests(Array( + new ClusterTest(features = Array( + new ClusterFeature(feature = Feature.TRANSACTION_VERSION, version = 0))), + new ClusterTest(features = Array( + new ClusterFeature(feature = Feature.TRANSACTION_VERSION, version = 1))), + new ClusterTest(features = Array( + new ClusterFeature(feature = Feature.TRANSACTION_VERSION, version = 2))), + )) + def testTransactionWithSendOffset(cluster: ClusterInstance): Unit = { Review Comment: There is one test, `testFencingOnAddPartitions` in TransactionTest.scala [[0]], which seems to do something similar—creating a scenario where `TransactionCoordinator#handleAddPartitionsToTransaction` raises an error, `Errors.PRODUCER_FENCED` due to a mismatch in the producer epoch [[1]] and cause AddPartitionToTxn request failed. However, in this test, we cannot commit or abort the transaction successfully. Committing the transaction directly fails from the client side since the last error caused by the failed send is not empty[[2]]. In the case of aborting the transaction, since there is a newer producer (producer2), the broker raises the error Errors.PRODUCER_FENCED, causing the abort to fail. Perhaps the producer fenced test case is not what we want? I'm just wondering under what circumstances the `AddPartitionToTxn` would fail and in producer we can abort transaction successfully but commit transaction would fail. [0]: https://github.com/apache/kafka/blob/f79d7dc3f4d8418561f8c89bfa7ad19e529005b8/core/src/test/scala/integration/kafka/api/TransactionsTest.scala#L557 [1]: https://github.com/apache/kafka/blob/3ef10a3f617cfa7dc0bc7cb29dc2e3950765f1dc/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala#L413-L414 [2]: https://github.com/apache/kafka/blob/94eb21cab4e86d0d777553b0a52af77e32c2f20f/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java#L319 -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
