/** * Aborts the ongoing transaction. Any unflushed produce messages will be aborted when this call is made. * This call will throw an exception immediately if any prior {@link #send(ProducerRecord)} calls failed with a * {@link ProducerFencedException} or an instance of {@link org.apache.kafka.common.errors.AuthorizationException}. * * @throws IllegalStateException if no transactional.id has been configured or 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.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 */ public void abortTransaction() throws ProducerFencedException { throwIfNoTransactionManager(); TransactionalRequestResult result = transactionManager.beginAbort(); sender.wakeup(); result.await(); }
private void assertFatalError(Class<? extends RuntimeException> cause) { assertTrue(transactionManager.hasError()); try { transactionManager.beginAbort(); fail("Should have raised " + cause.getSimpleName()); } catch (KafkaException e) { assertTrue(cause.isAssignableFrom(e.getCause().getClass())); assertTrue(transactionManager.hasError()); } // Transaction abort cannot clear fatal error state try { transactionManager.beginAbort(); fail("Should have raised " + cause.getSimpleName()); } catch (KafkaException e) { assertTrue(cause.isAssignableFrom(e.getCause().getClass())); assertTrue(transactionManager.hasError()); } }
private void assertAbortableError(Class<? extends RuntimeException> cause) { try { transactionManager.beginCommit(); fail("Should have raised " + cause.getSimpleName()); } catch (KafkaException e) { assertTrue(cause.isAssignableFrom(e.getCause().getClass())); assertTrue(transactionManager.hasError()); } assertTrue(transactionManager.hasError()); transactionManager.beginAbort(); assertFalse(transactionManager.hasError()); }
@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 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 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 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 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. }
transactionManager.beginAbort(); sender.run(time.milliseconds()); assertTrue(responseFuture.isDone());
@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()); }
TransactionalRequestResult abortResult = transactionManager.beginAbort(); prepareEndTxnResponse(Errors.NONE, TransactionResult.ABORT, pid, epoch);
@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()); }
transactionManager.beginAbort(); sender.run(time.milliseconds());
transactionManager.beginAbort(); sender.run(time.milliseconds());
@Test public void testHasOngoingTransactionSuccessfulAbort() { long pid = 13131L; short epoch = 1; TopicPartition partition = new TopicPartition("foo", 0); assertFalse(transactionManager.hasOngoingTransaction()); doInitTransactions(pid, epoch); assertFalse(transactionManager.hasOngoingTransaction()); transactionManager.beginTransaction(); assertTrue(transactionManager.hasOngoingTransaction()); transactionManager.maybeAddPartitionToTransaction(partition); assertTrue(transactionManager.hasOngoingTransaction()); prepareAddPartitionsToTxn(partition, Errors.NONE); sender.run(time.milliseconds()); transactionManager.beginAbort(); assertTrue(transactionManager.hasOngoingTransaction()); prepareEndTxnResponse(Errors.NONE, TransactionResult.ABORT, pid, epoch); sender.run(time.milliseconds()); assertFalse(transactionManager.hasOngoingTransaction()); }
@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. }
assertTrue(transactionManager.transactionContainsPartition(tp0)); TransactionalRequestResult abortResult = transactionManager.beginAbort();
@Test public void testHasOngoingTransactionAbortableError() { long pid = 13131L; short epoch = 1; TopicPartition partition = new TopicPartition("foo", 0); assertFalse(transactionManager.hasOngoingTransaction()); doInitTransactions(pid, epoch); assertFalse(transactionManager.hasOngoingTransaction()); transactionManager.beginTransaction(); assertTrue(transactionManager.hasOngoingTransaction()); transactionManager.maybeAddPartitionToTransaction(partition); assertTrue(transactionManager.hasOngoingTransaction()); prepareAddPartitionsToTxn(partition, Errors.NONE); sender.run(time.milliseconds()); transactionManager.transitionToAbortableError(new KafkaException()); assertTrue(transactionManager.hasOngoingTransaction()); transactionManager.beginAbort(); assertTrue(transactionManager.hasOngoingTransaction()); prepareEndTxnResponse(Errors.NONE, TransactionResult.ABORT, pid, epoch); sender.run(time.milliseconds()); assertFalse(transactionManager.hasOngoingTransaction()); }