@Test public void shouldFailAbortIfAddOffsetsFailsWithFatalError() { final long pid = 13131L; final short epoch = 1; doInitTransactions(pid, epoch); transactionManager.beginTransaction(); Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>(); offsets.put(tp1, new OffsetAndMetadata(1)); final String consumerGroupId = "myconsumergroup"; transactionManager.sendOffsetsToTransaction(offsets, consumerGroupId); TransactionalRequestResult abortResult = transactionManager.beginAbort(); prepareAddOffsetsToTxnResponse(Errors.UNKNOWN_SERVER_ERROR, consumerGroupId, pid, epoch); sender.run(time.milliseconds()); // Send AddOffsetsToTxnRequest assertFalse(abortResult.isCompleted()); sender.run(time.milliseconds()); assertTrue(abortResult.isCompleted()); assertFalse(abortResult.isSuccessful()); assertTrue(transactionManager.hasFatalError()); }
@Test public void shouldNotSendAbortTxnRequestWhenOnlyAddOffsetsRequestFailed() { final long pid = 13131L; final short epoch = 1; doInitTransactions(pid, epoch); transactionManager.beginTransaction(); Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>(); offsets.put(tp1, new OffsetAndMetadata(1)); final String consumerGroupId = "myconsumergroup"; transactionManager.sendOffsetsToTransaction(offsets, consumerGroupId); TransactionalRequestResult abortResult = transactionManager.beginAbort(); prepareAddOffsetsToTxnResponse(Errors.GROUP_AUTHORIZATION_FAILED, consumerGroupId, pid, epoch); sender.run(time.milliseconds()); // Send AddOffsetsToTxnRequest assertFalse(abortResult.isCompleted()); sender.run(time.milliseconds()); assertTrue(transactionManager.isReady()); assertTrue(abortResult.isCompleted()); assertTrue(abortResult.isSuccessful()); }
@Test public void testCancelUnsentAddPartitionsAndProduceOnAbort() throws InterruptedException { final long pid = 13131L; final short epoch = 1; doInitTransactions(pid, epoch); transactionManager.beginTransaction(); transactionManager.maybeAddPartitionToTransaction(tp0); Future<RecordMetadata> responseFuture = accumulator.append(tp0, time.milliseconds(), "key".getBytes(), "value".getBytes(), Record.EMPTY_HEADERS, null, MAX_BLOCK_TIMEOUT).future; assertFalse(responseFuture.isDone()); TransactionalRequestResult abortResult = transactionManager.beginAbort(); // note since no partitions were added to the transaction, no EndTxn will be sent sender.run(time.milliseconds()); // try to abort assertTrue(abortResult.isCompleted()); assertTrue(abortResult.isSuccessful()); assertTrue(transactionManager.isReady()); // make sure we are ready for a transaction now. try { responseFuture.get(); fail("Expected produce future to raise an exception"); } catch (ExecutionException e) { assertTrue(e.getCause() instanceof KafkaException); } }
@Test public void shouldNotSendAbortTxnRequestWhenOnlyAddPartitionsRequestFailed() { final long pid = 13131L; final short epoch = 1; doInitTransactions(pid, epoch); transactionManager.beginTransaction(); transactionManager.maybeAddPartitionToTransaction(tp0); prepareAddPartitionsToTxnResponse(Errors.TOPIC_AUTHORIZATION_FAILED, tp0, epoch, pid); sender.run(time.milliseconds()); // Send AddPartitionsRequest TransactionalRequestResult abortResult = transactionManager.beginAbort(); assertFalse(abortResult.isCompleted()); sender.run(time.milliseconds()); assertTrue(abortResult.isCompleted()); assertTrue(abortResult.isSuccessful()); }
assertFalse(addOffsetsResult.isCompleted()); // The request should complete only after the TxnOffsetCommit completes. assertFalse(addOffsetsResult.isCompleted()); // We should only be done after both RPCs complete successfully. assertTrue(addOffsetsResult.isCompleted()); assertTrue(addOffsetsResult.isSuccessful());
assertFalse(commitResult.isCompleted()); assertFalse(commitResult.isCompleted()); assertTrue(transactionManager.hasOngoingTransaction()); assertTrue(transactionManager.isCompleting()); assertTrue(commitResult.isCompleted()); assertFalse(transactionManager.hasOngoingTransaction());
@Test public void testAbortResendsAddPartitionErrorIfRetried() throws InterruptedException { final long producerId = 13131L; final short producerEpoch = 1; doInitTransactions(producerId, producerEpoch); transactionManager.beginTransaction(); transactionManager.maybeAddPartitionToTransaction(tp0); prepareAddPartitionsToTxnResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION, tp0, producerEpoch, producerId); Future<RecordMetadata> responseFuture = accumulator.append(tp0, time.milliseconds(), "key".getBytes(), "value".getBytes(), Record.EMPTY_HEADERS, null, MAX_BLOCK_TIMEOUT).future; sender.run(time.milliseconds()); // Send AddPartitions and let it fail assertFalse(responseFuture.isDone()); TransactionalRequestResult abortResult = transactionManager.beginAbort(); // we should resend the AddPartitions prepareAddPartitionsToTxnResponse(Errors.NONE, tp0, producerEpoch, producerId); prepareEndTxnResponse(Errors.NONE, TransactionResult.ABORT, producerId, producerEpoch); sender.run(time.milliseconds()); // Resend AddPartitions sender.run(time.milliseconds()); // Send EndTxn assertTrue(abortResult.isCompleted()); assertTrue(abortResult.isSuccessful()); assertTrue(transactionManager.isReady()); // make sure we are ready for a transaction now. try { responseFuture.get(); fail("Expected produce future to raise an exception"); } catch (ExecutionException e) { assertTrue(e.getCause() instanceof KafkaException); } }
@Test public void testTransactionalIdAuthorizationFailureInAddOffsetsToTxn() { final String consumerGroupId = "consumer"; final long pid = 13131L; final short epoch = 1; final TopicPartition tp = new TopicPartition("foo", 0); doInitTransactions(pid, epoch); transactionManager.beginTransaction(); TransactionalRequestResult sendOffsetsResult = transactionManager.sendOffsetsToTransaction( singletonMap(tp, new OffsetAndMetadata(39L)), consumerGroupId); prepareAddOffsetsToTxnResponse(Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED, consumerGroupId, pid, epoch); sender.run(time.milliseconds()); // AddOffsetsToTxn Handled assertTrue(transactionManager.hasError()); assertTrue(transactionManager.lastError() instanceof TransactionalIdAuthorizationException); assertTrue(sendOffsetsResult.isCompleted()); assertFalse(sendOffsetsResult.isSuccessful()); assertTrue(sendOffsetsResult.error() instanceof TransactionalIdAuthorizationException); assertFatalError(TransactionalIdAuthorizationException.class); }
@Test public void testAllowAbortOnProduceFailure() throws InterruptedException { final long pid = 13131L; final short epoch = 1; doInitTransactions(pid, epoch); transactionManager.beginTransaction(); transactionManager.maybeAddPartitionToTransaction(tp0); Future<RecordMetadata> responseFuture = accumulator.append(tp0, time.milliseconds(), "key".getBytes(), "value".getBytes(), Record.EMPTY_HEADERS, null, MAX_BLOCK_TIMEOUT).future; assertFalse(responseFuture.isDone()); prepareAddPartitionsToTxnResponse(Errors.NONE, tp0, epoch, pid); prepareProduceResponse(Errors.OUT_OF_ORDER_SEQUENCE_NUMBER, pid, epoch); prepareEndTxnResponse(Errors.NONE, TransactionResult.ABORT, pid, epoch); sender.run(time.milliseconds()); // Send AddPartitionsRequest sender.run(time.milliseconds()); // Send Produce Request, returns OutOfOrderSequenceException. TransactionalRequestResult abortResult = transactionManager.beginAbort(); sender.run(time.milliseconds()); // try to abort assertTrue(abortResult.isCompleted()); assertTrue(abortResult.isSuccessful()); assertTrue(transactionManager.isReady()); // make sure we are ready for a transaction now. }
@Test public void testAbortResendsProduceRequestIfRetried() throws Exception { final long producerId = 13131L; final short producerEpoch = 1; doInitTransactions(producerId, producerEpoch); transactionManager.beginTransaction(); transactionManager.maybeAddPartitionToTransaction(tp0); prepareAddPartitionsToTxnResponse(Errors.NONE, tp0, producerEpoch, producerId); prepareProduceResponse(Errors.REQUEST_TIMED_OUT, producerId, producerEpoch); Future<RecordMetadata> responseFuture = accumulator.append(tp0, time.milliseconds(), "key".getBytes(), "value".getBytes(), Record.EMPTY_HEADERS, null, MAX_BLOCK_TIMEOUT).future; sender.run(time.milliseconds()); // Send AddPartitions sender.run(time.milliseconds()); // Send ProduceRequest and let it fail assertFalse(responseFuture.isDone()); TransactionalRequestResult abortResult = transactionManager.beginAbort(); // we should resend the ProduceRequest before aborting prepareProduceResponse(Errors.NONE, producerId, producerEpoch); prepareEndTxnResponse(Errors.NONE, TransactionResult.ABORT, producerId, producerEpoch); sender.run(time.milliseconds()); // Resend ProduceRequest sender.run(time.milliseconds()); // Send EndTxn assertTrue(abortResult.isCompleted()); assertTrue(abortResult.isSuccessful()); assertTrue(transactionManager.isReady()); // make sure we are ready for a transaction now. RecordMetadata recordMetadata = responseFuture.get(); assertEquals(tp0.topic(), recordMetadata.topic()); }
@Test public void testTransactionalIdAuthorizationFailureInTxnOffsetCommit() { final String consumerGroupId = "consumer"; final long pid = 13131L; final short epoch = 1; final TopicPartition tp = new TopicPartition("foo", 0); doInitTransactions(pid, epoch); transactionManager.beginTransaction(); TransactionalRequestResult sendOffsetsResult = transactionManager.sendOffsetsToTransaction( singletonMap(tp, new OffsetAndMetadata(39L)), consumerGroupId); prepareAddOffsetsToTxnResponse(Errors.NONE, consumerGroupId, pid, epoch); sender.run(time.milliseconds()); // AddOffsetsToTxn Handled, TxnOffsetCommit Enqueued sender.run(time.milliseconds()); // FindCoordinator Enqueued prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.GROUP, consumerGroupId); sender.run(time.milliseconds()); // FindCoordinator Returned prepareTxnOffsetCommitResponse(consumerGroupId, pid, epoch, singletonMap(tp, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED)); sender.run(time.milliseconds()); // TxnOffsetCommit Handled assertTrue(transactionManager.hasError()); assertTrue(transactionManager.lastError() instanceof TransactionalIdAuthorizationException); assertTrue(sendOffsetsResult.isCompleted()); assertFalse(sendOffsetsResult.isSuccessful()); assertTrue(sendOffsetsResult.error() instanceof TransactionalIdAuthorizationException); assertFatalError(TransactionalIdAuthorizationException.class); }
@Test public void testUnsupportedForMessageFormatInTxnOffsetCommit() { final String consumerGroupId = "consumer"; final long pid = 13131L; final short epoch = 1; final TopicPartition tp = new TopicPartition("foo", 0); doInitTransactions(pid, epoch); transactionManager.beginTransaction(); TransactionalRequestResult sendOffsetsResult = transactionManager.sendOffsetsToTransaction( singletonMap(tp, new OffsetAndMetadata(39L)), consumerGroupId); prepareAddOffsetsToTxnResponse(Errors.NONE, consumerGroupId, pid, epoch); sender.run(time.milliseconds()); // AddOffsetsToTxn Handled, TxnOffsetCommit Enqueued sender.run(time.milliseconds()); // FindCoordinator Enqueued prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.GROUP, consumerGroupId); sender.run(time.milliseconds()); // FindCoordinator Returned prepareTxnOffsetCommitResponse(consumerGroupId, pid, epoch, singletonMap(tp, Errors.UNSUPPORTED_FOR_MESSAGE_FORMAT)); sender.run(time.milliseconds()); // TxnOffsetCommit Handled assertTrue(transactionManager.hasError()); assertTrue(transactionManager.lastError() instanceof UnsupportedForMessageFormatException); assertTrue(sendOffsetsResult.isCompleted()); assertFalse(sendOffsetsResult.isSuccessful()); assertTrue(sendOffsetsResult.error() instanceof UnsupportedForMessageFormatException); assertFatalError(UnsupportedForMessageFormatException.class); }
@Test public void testLookupCoordinatorOnNotCoordinatorError() { // This is called from the initTransactions method in the producer as the first order of business. // It finds the coordinator and then gets a PID. final long pid = 13131L; final short epoch = 1; TransactionalRequestResult initPidResult = transactionManager.initializeTransactions(); prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId); sender.run(time.milliseconds()); // find coordinator sender.run(time.milliseconds()); assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION)); prepareInitPidResponse(Errors.NOT_COORDINATOR, false, pid, epoch); sender.run(time.milliseconds()); // send pid, get not coordinator. Should resend the FindCoordinator and InitPid requests assertNull(transactionManager.coordinator(CoordinatorType.TRANSACTION)); assertFalse(initPidResult.isCompleted()); assertFalse(transactionManager.hasProducerId()); prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId); sender.run(time.milliseconds()); assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION)); assertFalse(initPidResult.isCompleted()); prepareInitPidResponse(Errors.NONE, false, pid, epoch); sender.run(time.milliseconds()); // get pid and epoch assertTrue(initPidResult.isCompleted()); // The future should only return after the second round of retries succeed. assertTrue(transactionManager.hasProducerId()); assertEquals(pid, transactionManager.producerIdAndEpoch().producerId); assertEquals(epoch, transactionManager.producerIdAndEpoch().epoch); }
@Test public void testLookupCoordinatorOnDisconnectAfterSend() { // This is called from the initTransactions method in the producer as the first order of business. // It finds the coordinator and then gets a PID. final long pid = 13131L; final short epoch = 1; TransactionalRequestResult initPidResult = transactionManager.initializeTransactions(); prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId); sender.run(time.milliseconds()); // find coordinator sender.run(time.milliseconds()); assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION)); prepareInitPidResponse(Errors.NONE, true, pid, epoch); // send pid to coordinator, should get disconnected before receiving the response, and resend the // FindCoordinator and InitPid requests. sender.run(time.milliseconds()); assertNull(transactionManager.coordinator(CoordinatorType.TRANSACTION)); assertFalse(initPidResult.isCompleted()); assertFalse(transactionManager.hasProducerId()); prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId); sender.run(time.milliseconds()); assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION)); assertFalse(initPidResult.isCompleted()); prepareInitPidResponse(Errors.NONE, false, pid, epoch); sender.run(time.milliseconds()); // get pid and epoch assertTrue(initPidResult.isCompleted()); // The future should only return after the second round of retries succeed. assertTrue(transactionManager.hasProducerId()); assertEquals(pid, transactionManager.producerIdAndEpoch().producerId); assertEquals(epoch, transactionManager.producerIdAndEpoch().epoch); }
assertFalse(initPidResult.isCompleted()); assertFalse(transactionManager.hasProducerId()); sender.run(time.milliseconds()); assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION)); assertFalse(initPidResult.isCompleted()); prepareInitPidResponse(Errors.NONE, false, pid, epoch); assertTrue(initPidResult.isCompleted()); // The future should only return after the second round of retries succeed. assertTrue(transactionManager.hasProducerId()); assertEquals(pid, transactionManager.producerIdAndEpoch().producerId);
@Test public void testGroupAuthorizationFailureInTxnOffsetCommit() { final String consumerGroupId = "consumer"; final long pid = 13131L; final short epoch = 1; final TopicPartition tp1 = new TopicPartition("foo", 0); doInitTransactions(pid, epoch); transactionManager.beginTransaction(); TransactionalRequestResult sendOffsetsResult = transactionManager.sendOffsetsToTransaction( singletonMap(tp1, new OffsetAndMetadata(39L)), consumerGroupId); prepareAddOffsetsToTxnResponse(Errors.NONE, consumerGroupId, pid, epoch); sender.run(time.milliseconds()); // AddOffsetsToTxn Handled, TxnOffsetCommit Enqueued sender.run(time.milliseconds()); // FindCoordinator Enqueued prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.GROUP, consumerGroupId); sender.run(time.milliseconds()); // FindCoordinator Returned prepareTxnOffsetCommitResponse(consumerGroupId, pid, epoch, singletonMap(tp1, Errors.GROUP_AUTHORIZATION_FAILED)); sender.run(time.milliseconds()); // TxnOffsetCommit Handled assertTrue(transactionManager.hasError()); assertTrue(transactionManager.lastError() instanceof GroupAuthorizationException); assertTrue(sendOffsetsResult.isCompleted()); assertFalse(sendOffsetsResult.isSuccessful()); assertTrue(sendOffsetsResult.error() instanceof GroupAuthorizationException); assertFalse(transactionManager.hasPendingOffsetCommits()); GroupAuthorizationException exception = (GroupAuthorizationException) sendOffsetsResult.error(); assertEquals(consumerGroupId, exception.groupId()); assertAbortableError(GroupAuthorizationException.class); }
@Test public void testAbortableErrorWhileAbortInProgress() throws InterruptedException { final long pid = 13131L; final short epoch = 1; doInitTransactions(pid, epoch); transactionManager.beginTransaction(); transactionManager.maybeAddPartitionToTransaction(tp0); Future<RecordMetadata> responseFuture = accumulator.append(tp0, time.milliseconds(), "key".getBytes(), "value".getBytes(), Record.EMPTY_HEADERS, null, MAX_BLOCK_TIMEOUT).future; assertFalse(responseFuture.isDone()); prepareAddPartitionsToTxnResponse(Errors.NONE, tp0, epoch, pid); sender.run(time.milliseconds()); // Send AddPartitionsRequest sender.run(time.milliseconds()); // Send Produce Request TransactionalRequestResult abortResult = transactionManager.beginAbort(); assertTrue(transactionManager.isAborting()); assertFalse(transactionManager.hasError()); sendProduceResponse(Errors.OUT_OF_ORDER_SEQUENCE_NUMBER, pid, epoch); prepareEndTxnResponse(Errors.NONE, TransactionResult.ABORT, pid, epoch); sender.run(time.milliseconds()); // receive the produce response // we do not transition to ABORTABLE_ERROR since we were already aborting assertTrue(transactionManager.isAborting()); assertFalse(transactionManager.hasError()); sender.run(time.milliseconds()); // handle the abort assertTrue(abortResult.isCompleted()); assertTrue(abortResult.isSuccessful()); assertTrue(transactionManager.isReady()); // make sure we are ready for a transaction now. }
@Test public void testGroupAuthorizationFailureInFindCoordinator() { final String consumerGroupId = "consumer"; final long pid = 13131L; final short epoch = 1; doInitTransactions(pid, epoch); transactionManager.beginTransaction(); TransactionalRequestResult sendOffsetsResult = transactionManager.sendOffsetsToTransaction( singletonMap(new TopicPartition("foo", 0), new OffsetAndMetadata(39L)), consumerGroupId); prepareAddOffsetsToTxnResponse(Errors.NONE, consumerGroupId, pid, epoch); sender.run(time.milliseconds()); // AddOffsetsToTxn Handled, TxnOffsetCommit Enqueued sender.run(time.milliseconds()); // FindCoordinator Enqueued prepareFindCoordinatorResponse(Errors.GROUP_AUTHORIZATION_FAILED, false, CoordinatorType.GROUP, consumerGroupId); sender.run(time.milliseconds()); // FindCoordinator Failed sender.run(time.milliseconds()); // TxnOffsetCommit Aborted assertTrue(transactionManager.hasError()); assertTrue(transactionManager.lastError() instanceof GroupAuthorizationException); assertTrue(sendOffsetsResult.isCompleted()); assertFalse(sendOffsetsResult.isSuccessful()); assertTrue(sendOffsetsResult.error() instanceof GroupAuthorizationException); GroupAuthorizationException exception = (GroupAuthorizationException) sendOffsetsResult.error(); assertEquals(consumerGroupId, exception.groupId()); assertAbortableError(GroupAuthorizationException.class); }
@Test public void testTransactionalIdAuthorizationFailureInFindCoordinator() { TransactionalRequestResult initPidResult = transactionManager.initializeTransactions(); prepareFindCoordinatorResponse(Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED, false, CoordinatorType.TRANSACTION, transactionalId); sender.run(time.milliseconds()); // find coordinator sender.run(time.milliseconds()); assertTrue(transactionManager.hasError()); assertTrue(transactionManager.lastError() instanceof TransactionalIdAuthorizationException); sender.run(time.milliseconds()); // one more run to fail the InitProducerId future assertTrue(initPidResult.isCompleted()); assertFalse(initPidResult.isSuccessful()); assertTrue(initPidResult.error() instanceof TransactionalIdAuthorizationException); assertFatalError(TransactionalIdAuthorizationException.class); }
@Test public void testTransactionalIdAuthorizationFailureInInitProducerId() { final long pid = 13131L; TransactionalRequestResult initPidResult = transactionManager.initializeTransactions(); prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId); sender.run(time.milliseconds()); // find coordinator sender.run(time.milliseconds()); assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION)); prepareInitPidResponse(Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED, false, pid, RecordBatch.NO_PRODUCER_EPOCH); sender.run(time.milliseconds()); assertTrue(transactionManager.hasError()); assertTrue(initPidResult.isCompleted()); assertFalse(initPidResult.isSuccessful()); assertTrue(initPidResult.error() instanceof TransactionalIdAuthorizationException); assertFatalError(TransactionalIdAuthorizationException.class); }