@Test public void testShouldResetProducerStateAfterResolvingSequences() { // Create a TransactionManager without a transactionalId to test // shouldResetProducerStateAfterResolvingSequences. TransactionManager manager = new TransactionManager(logContext, null, transactionTimeoutMs, DEFAULT_RETRY_BACKOFF_MS); assertFalse(manager.shouldResetProducerStateAfterResolvingSequences()); TopicPartition tp0 = new TopicPartition("foo", 0); TopicPartition tp1 = new TopicPartition("foo", 1); assertEquals(Integer.valueOf(0), manager.sequenceNumber(tp0)); assertEquals(Integer.valueOf(0), manager.sequenceNumber(tp1)); manager.incrementSequenceNumber(tp0, 1); manager.incrementSequenceNumber(tp1, 1); manager.maybeUpdateLastAckedSequence(tp0, 0); manager.maybeUpdateLastAckedSequence(tp1, 0); manager.markSequenceUnresolved(tp0); manager.markSequenceUnresolved(tp1); assertFalse(manager.shouldResetProducerStateAfterResolvingSequences()); manager.maybeUpdateLastAckedSequence(tp0, 5); manager.incrementSequenceNumber(tp0, 1); manager.markSequenceUnresolved(tp0); manager.markSequenceUnresolved(tp1); assertTrue(manager.shouldResetProducerStateAfterResolvingSequences()); }
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 testIsSendToPartitionAllowedWithPendingPartitionAfterFatalError() { final long pid = 13131L; final short epoch = 1; doInitTransactions(pid, epoch); transactionManager.beginTransaction(); transactionManager.maybeAddPartitionToTransaction(tp0); transactionManager.transitionToFatalError(new KafkaException()); assertFalse(transactionManager.isSendToPartitionAllowed(tp0)); assertTrue(transactionManager.hasFatalError()); }
synchronized boolean hasOngoingTransaction() { // transactions are considered ongoing once started until completion or a fatal error return currentState == State.IN_TRANSACTION || isCompleting() || hasAbortableError(); }
synchronized boolean isSendToPartitionAllowed(TopicPartition tp) { if (hasFatalError()) return false; return !isTransactional() || partitionsInTransaction.contains(tp); }
public synchronized void maybeAddPartitionToTransaction(TopicPartition topicPartition) { failIfNotReadyForSend(); if (isPartitionAdded(topicPartition) || isPartitionPendingAdd(topicPartition)) return; log.debug("Begin adding new partition {} to transaction", topicPartition); newPartitionsInTransaction.add(topicPartition); }
transactionManager.beginTransaction(); transactionManager.maybeAddPartitionToTransaction(unauthorizedPartition); sender.run(time.milliseconds()); assertTrue(transactionManager.hasAbortableError()); transactionManager.beginAbort(); sender.run(time.milliseconds()); assertTrue(responseFuture.isDone()); assertTrue(transactionManager.isReady()); assertFalse(transactionManager.hasPartitionsToAdd()); assertFalse(accumulator.hasIncomplete()); transactionManager.beginTransaction(); transactionManager.maybeAddPartitionToTransaction(tp0); assertTrue(transactionManager.isPartitionAdded(tp0)); assertFalse(transactionManager.hasPartitionsToAdd()); transactionManager.beginCommit(); prepareProduceResponse(Errors.NONE, pid, epoch); sender.run(time.milliseconds()); sender.run(time.milliseconds()); assertTrue(transactionManager.isReady());
if (transactionManager != null) { try { if (transactionManager.shouldResetProducerStateAfterResolvingSequences()) transactionManager.resetProducerId(); if (!transactionManager.isTransactional()) { } else if (transactionManager.hasUnresolvedSequences() && !transactionManager.hasFatalError()) { transactionManager.transitionToFatalError( new KafkaException("The client hasn't received acknowledgment for " + "some previously sent messages and can no longer retry them. It isn't safe to continue.")); } else if (transactionManager.hasInFlightTransactionalRequest() || maybeSendTransactionalRequest(now)) { if (transactionManager.hasFatalError() || !transactionManager.hasProducerId()) { RuntimeException lastError = transactionManager.lastError(); if (lastError != null) maybeAbortBatches(lastError); client.poll(retryBackoffMs, now); return; } else if (transactionManager.hasAbortableError()) { accumulator.abortUndrainedBatches(transactionManager.lastError()); transactionManager.authenticationFailed(e);
@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 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 testIsSendToPartitionAllowedWithPendingPartitionAfterAbortableError() { final long pid = 13131L; final short epoch = 1; doInitTransactions(pid, epoch); transactionManager.beginTransaction(); transactionManager.maybeAddPartitionToTransaction(tp0); transactionManager.transitionToAbortableError(new KafkaException()); assertFalse(transactionManager.isSendToPartitionAllowed(tp0)); assertTrue(transactionManager.hasAbortableError()); }
transactionManager.beginTransaction(); transactionManager.maybeAddPartitionToTransaction(tp0); assertFalse(transactionManager.transactionContainsPartition(tp0)); assertFalse(transactionManager.isSendToPartitionAllowed(tp0)); assertTrue(transactionManager.transactionContainsPartition(tp0)); assertTrue(transactionManager.isSendToPartitionAllowed(tp0)); assertFalse(responseFuture.isDone()); offsets.put(tp1, new OffsetAndMetadata(1)); final String consumerGroupId = "myconsumergroup"; TransactionalRequestResult addOffsetsResult = transactionManager.sendOffsetsToTransaction(offsets, consumerGroupId); assertFalse(transactionManager.hasPendingOffsetCommits()); assertTrue(transactionManager.hasPendingOffsetCommits()); // We should now have created and queued the offset commit request. assertNull(transactionManager.coordinator(CoordinatorType.GROUP)); assertNotNull(transactionManager.coordinator(CoordinatorType.GROUP)); assertTrue(transactionManager.hasPendingOffsetCommits()); assertFalse(transactionManager.hasPendingOffsetCommits()); transactionManager.beginCommit(); prepareEndTxnResponse(Errors.NONE, TransactionResult.COMMIT, pid, epoch); assertFalse(transactionManager.hasOngoingTransaction());
transactionManager.beginTransaction(); transactionManager.maybeAddPartitionToTransaction(tp0); assertFalse(transactionManager.transactionContainsPartition(tp0)); assertFalse(transactionManager.isSendToPartitionAllowed(tp0)); assertTrue(transactionManager.transactionContainsPartition(tp0)); assertTrue(transactionManager.isSendToPartitionAllowed(tp0)); assertFalse(responseFuture.isDone()); TransactionalRequestResult commitResult = transactionManager.beginCommit(); assertTrue(transactionManager.hasAbortableError()); assertTrue(transactionManager.hasOngoingTransaction()); assertFalse(transactionManager.isCompleting()); assertTrue(transactionManager.transactionContainsPartition(tp0)); TransactionalRequestResult abortResult = transactionManager.beginAbort(); assertFalse(transactionManager.hasOngoingTransaction()); assertFalse(transactionManager.transactionContainsPartition(tp0));
final short epoch = 1; doInitTransactions(pid, epoch); transactionManager.beginTransaction(); transactionManager.maybeAddPartitionToTransaction(tp1); prepareAddPartitionsToTxn(tp1, Errors.NONE); assertTrue(transactionManager.transactionContainsPartition(tp1)); transactionManager.maybeAddPartitionToTransaction(tp0); prepareAddPartitionsToTxn(tp0, Errors.TOPIC_AUTHORIZATION_FAILED); assertTrue(transactionManager.hasAbortableError()); assertTrue(transactionManager.isSendToPartitionAllowed(tp1)); assertTrue(transactionManager.hasAbortableError());
@Test public void testAddPartitionToTransactionOverridesRetryBackoffForConcurrentTransactions() { long pid = 13131L; short epoch = 1; TopicPartition partition = new TopicPartition("foo", 0); doInitTransactions(pid, epoch); transactionManager.beginTransaction(); transactionManager.maybeAddPartitionToTransaction(partition); assertTrue(transactionManager.hasPartitionsToAdd()); assertFalse(transactionManager.isPartitionAdded(partition)); assertTrue(transactionManager.isPartitionPendingAdd(partition)); prepareAddPartitionsToTxn(partition, Errors.CONCURRENT_TRANSACTIONS); sender.run(time.milliseconds()); TransactionManager.TxnRequestHandler handler = transactionManager.nextRequestHandler(false); assertNotNull(handler); assertEquals(20, handler.retryBackoffMs()); }
@Test public void testTopicAuthorizationFailureInAddPartitions() { final long pid = 13131L; final short epoch = 1; final TopicPartition tp0 = new TopicPartition("foo", 0); final TopicPartition tp1 = new TopicPartition("bar", 0); doInitTransactions(pid, epoch); transactionManager.beginTransaction(); transactionManager.maybeAddPartitionToTransaction(tp0); transactionManager.maybeAddPartitionToTransaction(tp1); Map<TopicPartition, Errors> errors = new HashMap<>(); errors.put(tp0, Errors.TOPIC_AUTHORIZATION_FAILED); errors.put(tp1, Errors.OPERATION_NOT_ATTEMPTED); prepareAddPartitionsToTxn(errors); sender.run(time.milliseconds()); assertTrue(transactionManager.hasError()); assertTrue(transactionManager.lastError() instanceof TopicAuthorizationException); assertFalse(transactionManager.isPartitionPendingAdd(tp0)); assertFalse(transactionManager.isPartitionPendingAdd(tp1)); assertFalse(transactionManager.isPartitionAdded(tp0)); assertFalse(transactionManager.isPartitionAdded(tp1)); assertFalse(transactionManager.hasPartitionsToAdd()); TopicAuthorizationException exception = (TopicAuthorizationException) transactionManager.lastError(); assertEquals(singleton(tp0.topic()), exception.unauthorizedTopics()); assertAbortableError(TopicAuthorizationException.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 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()); }
transactionManager.beginTransaction(); transactionManager.maybeAddPartitionToTransaction(tp0); transactionManager.beginCommit(); sender.run(time.milliseconds()); assertFalse(accumulator.hasUndrained()); assertTrue(accumulator.hasIncomplete()); assertFalse(transactionManager.hasInFlightTransactionalRequest()); assertFalse(responseFuture.isDone()); assertFalse(accumulator.hasUndrained()); assertTrue(accumulator.hasIncomplete()); assertFalse(transactionManager.hasInFlightTransactionalRequest()); assertFalse(responseFuture.isDone()); assertFalse(accumulator.hasUndrained()); assertFalse(accumulator.hasIncomplete()); assertFalse(transactionManager.hasInFlightTransactionalRequest()); assertTrue(transactionManager.hasInFlightTransactionalRequest()); sendEndTxnResponse(Errors.NONE, TransactionResult.COMMIT, pid, epoch); sender.run(time.milliseconds()); assertFalse(transactionManager.hasInFlightTransactionalRequest()); assertTrue(transactionManager.isReady());
transactionManager.beginTransaction(); transactionManager.maybeAddPartitionToTransaction(tp0); assertFalse(transactionManager.transactionContainsPartition(tp0)); assertFalse(transactionManager.isSendToPartitionAllowed(tp0)); assertTrue(transactionManager.transactionContainsPartition(tp0)); assertTrue(transactionManager.isSendToPartitionAllowed(tp0)); TransactionalRequestResult commitResult = transactionManager.beginCommit(); assertTrue(transactionManager.hasFatalError()); assertFalse(transactionManager.hasOngoingTransaction());