dajac commented on a change in pull request #11452: URL: https://github.com/apache/kafka/pull/11452#discussion_r750335078
########## File path: clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java ########## @@ -903,6 +903,47 @@ public void testPartitionsForWithNullTopic() { } } + @Test + public void testInitTransactionsResponseAfterTimeout() throws Exception { + int maxBlockMs = 500; + + Map<String, Object> configs = new HashMap<>(); + configs.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "bad-transaction"); + configs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, maxBlockMs); + configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9000"); + + Time time = new MockTime(); + MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap("topic", 1)); + ProducerMetadata metadata = newMetadata(0, Long.MAX_VALUE); + metadata.updateWithCurrentRequestVersion(initialUpdateResponse, false, time.milliseconds()); + + MockClient client = new MockClient(time, metadata); + + ExecutorService executor = Executors.newFixedThreadPool(1); + + Producer<String, String> producer = kafkaProducer(configs, new StringSerializer(), + new StringSerializer(), metadata, client, null, time); + try { + client.prepareResponse( + request -> request instanceof FindCoordinatorRequest && + ((FindCoordinatorRequest) request).data().keyType() == FindCoordinatorRequest.CoordinatorType.TRANSACTION.id(), + FindCoordinatorResponse.prepareResponse(Errors.NONE, "bad-transaction", host1)); + + Future<?> future = executor.submit(producer::initTransactions); + TestUtils.waitForCondition(client::hasInFlightRequests, "blah blah"); Review comment: nit: I suppose that you planned to change the message here :) ########## File path: clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java ########## @@ -903,6 +903,47 @@ public void testPartitionsForWithNullTopic() { } } + @Test + public void testInitTransactionsResponseAfterTimeout() throws Exception { + int maxBlockMs = 500; + + Map<String, Object> configs = new HashMap<>(); + configs.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "bad-transaction"); + configs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, maxBlockMs); + configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9000"); + + Time time = new MockTime(); + MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap("topic", 1)); + ProducerMetadata metadata = newMetadata(0, Long.MAX_VALUE); + metadata.updateWithCurrentRequestVersion(initialUpdateResponse, false, time.milliseconds()); + + MockClient client = new MockClient(time, metadata); + + ExecutorService executor = Executors.newFixedThreadPool(1); + + Producer<String, String> producer = kafkaProducer(configs, new StringSerializer(), + new StringSerializer(), metadata, client, null, time); + try { + client.prepareResponse( + request -> request instanceof FindCoordinatorRequest && + ((FindCoordinatorRequest) request).data().keyType() == FindCoordinatorRequest.CoordinatorType.TRANSACTION.id(), + FindCoordinatorResponse.prepareResponse(Errors.NONE, "bad-transaction", host1)); + + Future<?> future = executor.submit(producer::initTransactions); + TestUtils.waitForCondition(client::hasInFlightRequests, "blah blah"); + + time.sleep(maxBlockMs); + TestUtils.assertFutureThrows(future, TimeoutException.class); + + client.respond(initProducerIdResponse(1L, (short) 5, Errors.NONE)); + + Thread.sleep(1000); + producer.initTransactions(); + } finally { + producer.close(Duration.ZERO); + } + } + Review comment: Should we also add new tests for `commitTransaction` and `abortTransaction` which both seem to suffer from the same bug? ########## File path: clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java ########## @@ -1184,15 +1186,19 @@ private TxnOffsetCommitHandler txnOffsetCommitHandler(TransactionalRequestResult } private TransactionalRequestResult handleCachedTransactionRequestResult( - Supplier<TransactionalRequestResult> transactionalRequestResultSupplier, - State targetState) { + Supplier<TransactionalRequestResult> transactionalRequestResultSupplier, + State transientState + ) { ensureTransactional(); - if (pendingResult != null && currentState == targetState) { + if (pendingResult != null && !pendingResult.isAcked()) { TransactionalRequestResult result = pendingResult; - if (result.isCompleted()) + if (result.isCompleted() && priorState == transientState) { pendingResult = null; - return result; + return result; + } else if (currentState == transientState) { + return result; Review comment: Why don't we set `pendingResult` to `null` when `isCompleted()` is `true` here? I suppose that we assume that the `currentState` which transition to the next state whenever the response is received so we should never have a completed request here. Therefore, the "prior state" check will handle all the cases. Is my reasoning correct? -- 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