New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
KAFKA-5427: Transactional producer should allow FindCoordinator in error state #3297
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -80,6 +80,7 @@ | |
import static org.junit.Assert.assertEquals; | ||
import static org.junit.Assert.assertFalse; | ||
import static org.junit.Assert.assertNotNull; | ||
import static org.junit.Assert.assertNull; | ||
import static org.junit.Assert.assertTrue; | ||
import static org.junit.Assert.fail; | ||
|
||
|
@@ -1058,6 +1059,71 @@ public void testAllowAbortOnProduceFailure() throws InterruptedException { | |
assertTrue(transactionManager.isReady()); // make sure we are ready for a transaction now. | ||
} | ||
|
||
@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.beginAbortingTransaction(); | ||
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 testFindCoordinatorAllowedInAbortableErrorState() 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()); | ||
sender.run(time.milliseconds()); // Send AddPartitionsRequest | ||
|
||
transactionManager.transitionToAbortableError(new KafkaException()); | ||
sendAddPartitionsToTxnResponse(Errors.NOT_COORDINATOR, tp0, epoch, pid); | ||
sender.run(time.milliseconds()); // AddPartitions returns | ||
assertTrue(transactionManager.hasAbortableError()); | ||
|
||
assertNull(transactionManager.coordinator(CoordinatorType.TRANSACTION)); | ||
prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId); | ||
sender.run(time.milliseconds()); // FindCoordinator handled | ||
assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION)); | ||
assertTrue(transactionManager.hasAbortableError()); | ||
} | ||
|
||
@Test | ||
public void testCancelUnsentAddPartitionsAndProduceOnAbort() throws InterruptedException { | ||
final long pid = 13131L; | ||
|
@@ -1279,15 +1345,42 @@ public void shouldNotSendAbortTxnRequestWhenOnlyAddOffsetsRequestFailed() throws | |
|
||
TransactionalRequestResult abortResult = transactionManager.beginAbortingTransaction(); | ||
|
||
prepareAddOffsetsToTxnResponse(Errors.TOPIC_AUTHORIZATION_FAILED, consumerGroupId, pid, epoch); | ||
prepareAddOffsetsToTxnResponse(Errors.GROUP_AUTHORIZATION_FAILED, consumerGroupId, pid, epoch); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why was this changed? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The previous error is not expected for AddOffsets, which means it is a fatal error. The intent of the test case was apparently to trigger an abortable error. Note that I've added a test case for the fatal error case below. |
||
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() throws Exception { | ||
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.beginAbortingTransaction(); | ||
|
||
prepareAddOffsetsToTxnResponse(Errors.UNKNOWN, 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 testNoDrainWhenPartitionsPending() throws InterruptedException { | ||
final long pid = 13131L; | ||
|
@@ -1623,8 +1716,15 @@ public boolean matches(AbstractRequest body) { | |
}, new InitProducerIdResponse(0, error, pid, epoch), shouldDisconnect); | ||
} | ||
|
||
private void sendProduceResponse(Errors error, final long pid, final short epoch) { | ||
client.respond(produceRequestMatcher(pid, epoch), produceResponse(tp0, 0, error, 0)); | ||
} | ||
|
||
private void prepareProduceResponse(Errors error, final long pid, final short epoch) { | ||
client.prepareResponse(new MockClient.RequestMatcher() { | ||
client.prepareResponse(produceRequestMatcher(pid, epoch), produceResponse(tp0, 0, error, 0)); | ||
} | ||
private MockClient.RequestMatcher produceRequestMatcher(final long pid, final short epoch) { | ||
return new MockClient.RequestMatcher() { | ||
@Override | ||
public boolean matches(AbstractRequest body) { | ||
ProduceRequest produceRequest = (ProduceRequest) body; | ||
|
@@ -1640,12 +1740,24 @@ public boolean matches(AbstractRequest body) { | |
assertEquals(transactionalId, produceRequest.transactionalId()); | ||
return true; | ||
} | ||
}, produceResponse(tp0, 0, error, 0)); | ||
}; | ||
} | ||
|
||
private void prepareAddPartitionsToTxnResponse(Errors error, final TopicPartition topicPartition, | ||
final short epoch, final long pid) { | ||
client.prepareResponse(addPartitionsRequestMatcher(topicPartition, epoch, pid), | ||
new AddPartitionsToTxnResponse(0, singletonMap(topicPartition, error))); | ||
} | ||
|
||
private void prepareAddPartitionsToTxnResponse(Errors error, final TopicPartition topicPartition, final short epoch, final long pid) { | ||
client.prepareResponse(new MockClient.RequestMatcher() { | ||
private void sendAddPartitionsToTxnResponse(Errors error, final TopicPartition topicPartition, | ||
final short epoch, final long pid) { | ||
client.respond(addPartitionsRequestMatcher(topicPartition, epoch, pid), | ||
new AddPartitionsToTxnResponse(0, singletonMap(topicPartition, error))); | ||
} | ||
|
||
private MockClient.RequestMatcher addPartitionsRequestMatcher(final TopicPartition topicPartition, | ||
final short epoch, final long pid) { | ||
return new MockClient.RequestMatcher() { | ||
@Override | ||
public boolean matches(AbstractRequest body) { | ||
AddPartitionsToTxnRequest addPartitionsToTxnRequest = (AddPartitionsToTxnRequest) body; | ||
|
@@ -1655,7 +1767,7 @@ public boolean matches(AbstractRequest body) { | |
assertEquals(transactionalId, addPartitionsToTxnRequest.transactionalId()); | ||
return true; | ||
} | ||
}, new AddPartitionsToTxnResponse(0, singletonMap(topicPartition, error))); | ||
}; | ||
} | ||
|
||
private void prepareEndTxnResponse(Errors error, final TransactionResult result, final long pid, final short epoch) { | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
perhaps better to assert that
transactionManager.hasAbortableError
is false.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The
hasError
check is strictly stronger. It covers bothhasAbortableError
andhasFatalError
.