private void maybeFailWithError() { if (hasError()) throw new KafkaException("Cannot execute transactional method because we are in an error state", lastError); }
public synchronized void failIfNotReadyForSend() { if (hasError()) throw new KafkaException("Cannot perform send because at least one previous transactional or " + "idempotent request has failed with errors.", lastError); if (isTransactional()) { if (!hasProducerId()) throw new IllegalStateException("Cannot perform a 'send' before completing a call to initTransactions " + "when transactions are enabled."); if (currentState != State.IN_TRANSACTION) throw new IllegalStateException("Cannot call send in state " + currentState); } }
private boolean maybeTerminateRequestWithError(TxnRequestHandler requestHandler) { if (hasError()) { if (hasAbortableError() && requestHandler instanceof FindCoordinatorHandler) // No harm letting the FindCoordinator request go through if we're expecting to abort return false; requestHandler.fail(lastError); return true; } return false; }
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()); }
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()); } }
@Test(expected = ExecutionException.class) public void testProducerFencedException() throws InterruptedException, ExecutionException { 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.INVALID_PRODUCER_EPOCH, pid, epoch); sender.run(time.milliseconds()); // Add partitions. sender.run(time.milliseconds()); // send produce. assertTrue(responseFuture.isDone()); assertTrue(transactionManager.hasError()); responseFuture.get(); }
private void verifyAddPartitionsFailsWithPartitionLevelError(final Errors error) throws InterruptedException { final long pid = 1L; 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()); prepareAddPartitionsToTxn(tp0, error); sender.run(time.milliseconds()); // attempt send addPartitions. assertTrue(transactionManager.hasError()); assertFalse(transactionManager.transactionContainsPartition(tp0)); }
@Test public void testTransactionalIdAuthorizationFailureInAddPartitions() { final long pid = 13131L; final short epoch = 1; final TopicPartition tp = new TopicPartition("foo", 0); doInitTransactions(pid, epoch); transactionManager.beginTransaction(); transactionManager.maybeAddPartitionToTransaction(tp); prepareAddPartitionsToTxn(tp, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED); sender.run(time.milliseconds()); assertTrue(transactionManager.hasError()); assertTrue(transactionManager.lastError() instanceof TransactionalIdAuthorizationException); assertFatalError(TransactionalIdAuthorizationException.class); }
@Test public void testUnsupportedForMessageFormatInProduceRequest() throws Exception { final long producerId = 343434L; TransactionManager transactionManager = new TransactionManager(); setupWithTransactionState(transactionManager); prepareAndReceiveInitProducerId(producerId, Errors.NONE); assertTrue(transactionManager.hasProducerId()); Future<RecordMetadata> future = accumulator.append(tp0, time.milliseconds(), "key".getBytes(), "value".getBytes(), null, null, MAX_BLOCK_TIMEOUT).future; client.prepareResponse(new MockClient.RequestMatcher() { @Override public boolean matches(AbstractRequest body) { return body instanceof ProduceRequest && ((ProduceRequest) body).hasIdempotentRecords(); } }, produceResponse(tp0, -1, Errors.UNSUPPORTED_FOR_MESSAGE_FORMAT, 0)); sender.run(time.milliseconds()); assertFutureFailure(future, UnsupportedForMessageFormatException.class); // unsupported for message format is not a fatal error assertFalse(transactionManager.hasError()); }
@Test public void testClusterAuthorizationExceptionInInitProducerIdRequest() throws Exception { final long producerId = 343434L; TransactionManager transactionManager = new TransactionManager(); setupWithTransactionState(transactionManager); prepareAndReceiveInitProducerId(producerId, Errors.CLUSTER_AUTHORIZATION_FAILED); assertFalse(transactionManager.hasProducerId()); assertTrue(transactionManager.hasError()); assertTrue(transactionManager.lastError() instanceof ClusterAuthorizationException); // cluster authorization is a fatal error for the producer assertSendFailure(ClusterAuthorizationException.class); }
@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 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 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 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 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 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); }
@Test public void testUnsupportedInitTransactions() { transactionManager.initializeTransactions(); prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId); sender.run(time.milliseconds()); // InitProducerRequest is queued sender.run(time.milliseconds()); // FindCoordinator is queued after peeking InitProducerRequest assertFalse(transactionManager.hasError()); assertNotNull(transactionManager.coordinator(CoordinatorType.TRANSACTION)); client.prepareUnsupportedVersionResponse(body -> { InitProducerIdRequest initProducerIdRequest = (InitProducerIdRequest) body; assertEquals(initProducerIdRequest.transactionalId(), transactionalId); assertEquals(initProducerIdRequest.transactionTimeoutMs(), transactionTimeoutMs); return true; }); sender.run(time.milliseconds()); // InitProducerRequest is dequeued assertTrue(transactionManager.hasFatalError()); assertTrue(transactionManager.lastError() instanceof UnsupportedVersionException); }