Skip to content

Commit

Permalink
MINOR: Align assertFutureThrows method signature with JUnit conventions
Browse files Browse the repository at this point in the history
  • Loading branch information
Parkerhiphop committed Feb 6, 2025
1 parent aa8c576 commit 3527286
Show file tree
Hide file tree
Showing 25 changed files with 345 additions and 346 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,7 @@ public void testTopLevelErrorConstructor() throws InterruptedException {
partitionFutures.completeExceptionally(Errors.GROUP_AUTHORIZATION_FAILED.exception());
DeleteConsumerGroupOffsetsResult topLevelErrorResult =
new DeleteConsumerGroupOffsetsResult(partitionFutures, partitions);
TestUtils.assertFutureThrows(topLevelErrorResult.all(), GroupAuthorizationException.class);
TestUtils.assertFutureThrows(GroupAuthorizationException.class, topLevelErrorResult.all());
}

@Test
Expand All @@ -79,9 +79,9 @@ public void testPartitionMissingInResponseErrorConstructor() throws InterruptedE
DeleteConsumerGroupOffsetsResult missingPartitionResult =
new DeleteConsumerGroupOffsetsResult(partitionFutures, partitions);

TestUtils.assertFutureThrows(missingPartitionResult.all(), IllegalArgumentException.class);
TestUtils.assertFutureThrows(IllegalArgumentException.class, missingPartitionResult.all());
assertNull(missingPartitionResult.partitionResult(tpZero).get());
TestUtils.assertFutureThrows(missingPartitionResult.partitionResult(tpOne), IllegalArgumentException.class);
TestUtils.assertFutureThrows(IllegalArgumentException.class, missingPartitionResult.partitionResult(tpOne));
}

@Test
Expand Down Expand Up @@ -110,9 +110,9 @@ private DeleteConsumerGroupOffsetsResult createAndVerifyPartitionLevelError() th
DeleteConsumerGroupOffsetsResult partitionLevelErrorResult =
new DeleteConsumerGroupOffsetsResult(partitionFutures, partitions);

TestUtils.assertFutureThrows(partitionLevelErrorResult.all(), UnknownTopicOrPartitionException.class);
TestUtils.assertFutureThrows(UnknownTopicOrPartitionException.class, partitionLevelErrorResult.all());
assertNull(partitionLevelErrorResult.partitionResult(tpZero).get());
TestUtils.assertFutureThrows(partitionLevelErrorResult.partitionResult(tpOne), UnknownTopicOrPartitionException.class);
TestUtils.assertFutureThrows(UnknownTopicOrPartitionException.class, partitionLevelErrorResult.partitionResult(tpOne));
return partitionLevelErrorResult;
}
}

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -42,9 +42,9 @@ public class ListTransactionsResultTest {
@Test
public void testAllFuturesFailIfLookupFails() {
future.completeExceptionally(new KafkaException());
assertFutureThrows(result.all(), KafkaException.class);
assertFutureThrows(result.allByBrokerId(), KafkaException.class);
assertFutureThrows(result.byBrokerId(), KafkaException.class);
assertFutureThrows(KafkaException.class, result.all());
assertFutureThrows(KafkaException.class, result.allByBrokerId());
assertFutureThrows(KafkaException.class, result.byBrokerId());
}

@Test
Expand Down Expand Up @@ -111,9 +111,9 @@ public void testPartialFailure() throws Exception {
assertEquals(broker1Listings, resultBrokerFutures.get(1).get());

// Everything else should fail
assertFutureThrows(result.all(), KafkaException.class);
assertFutureThrows(result.allByBrokerId(), KafkaException.class);
assertFutureThrows(resultBrokerFutures.get(2), KafkaException.class);
assertFutureThrows(KafkaException.class, result.all());
assertFutureThrows(KafkaException.class, result.allByBrokerId());
assertFutureThrows(KafkaException.class, resultBrokerFutures.get(2));
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,7 @@ public void testTopLevelErrorConstructor() throws InterruptedException {
memberFutures.completeExceptionally(Errors.GROUP_AUTHORIZATION_FAILED.exception());
RemoveMembersFromConsumerGroupResult topLevelErrorResult =
new RemoveMembersFromConsumerGroupResult(memberFutures, membersToRemove);
TestUtils.assertFutureThrows(topLevelErrorResult.all(), GroupAuthorizationException.class);
TestUtils.assertFutureThrows(GroupAuthorizationException.class, topLevelErrorResult.all());
}

@Test
Expand All @@ -78,9 +78,9 @@ public void testMemberMissingErrorInRequestConstructor() throws InterruptedExcep
RemoveMembersFromConsumerGroupResult missingMemberResult =
new RemoveMembersFromConsumerGroupResult(memberFutures, membersToRemove);

TestUtils.assertFutureThrows(missingMemberResult.all(), IllegalArgumentException.class);
TestUtils.assertFutureThrows(IllegalArgumentException.class, missingMemberResult.all());
assertNull(missingMemberResult.memberResult(instanceOne).get());
TestUtils.assertFutureThrows(missingMemberResult.memberResult(instanceTwo), IllegalArgumentException.class);
TestUtils.assertFutureThrows(IllegalArgumentException.class, missingMemberResult.memberResult(instanceTwo));
}

@Test
Expand Down Expand Up @@ -111,9 +111,9 @@ private RemoveMembersFromConsumerGroupResult createAndVerifyMemberLevelError() t
RemoveMembersFromConsumerGroupResult memberLevelErrorResult =
new RemoveMembersFromConsumerGroupResult(memberFutures, membersToRemove);

TestUtils.assertFutureThrows(memberLevelErrorResult.all(), FencedInstanceIdException.class);
TestUtils.assertFutureThrows(FencedInstanceIdException.class, memberLevelErrorResult.all());
assertNull(memberLevelErrorResult.memberResult(instanceOne).get());
TestUtils.assertFutureThrows(memberLevelErrorResult.memberResult(instanceTwo), FencedInstanceIdException.class);
TestUtils.assertFutureThrows(FencedInstanceIdException.class, memberLevelErrorResult.memberResult(instanceTwo));
return memberLevelErrorResult;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ public void testFatalLookupError() {

driver.onFailure(time.milliseconds(), spec, new UnknownServerException());
assertTrue(result.all().isDone());
TestUtils.assertFutureThrows(result.all(), UnknownServerException.class);
TestUtils.assertFutureThrows(UnknownServerException.class, result.all());
assertEquals(Collections.emptyList(), driver.poll());
}

Expand Down Expand Up @@ -200,7 +200,7 @@ public void testFatalFulfillmentError() throws Exception {

driver.onFailure(time.milliseconds(), requestSpec, new UnknownServerException());
assertTrue(future.isDone());
TestUtils.assertFutureThrows(future, UnknownServerException.class);
TestUtils.assertFutureThrows(UnknownServerException.class, future);
assertEquals(Collections.emptyList(), driver.poll());
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -367,7 +367,7 @@ public void testFatalLookupError() {

driver.onFailure(time.milliseconds(), spec, new UnknownServerException());
assertTrue(result.all().get(tp0).isDone());
TestUtils.assertFutureThrows(result.all().get(tp0), UnknownServerException.class);
TestUtils.assertFutureThrows(UnknownServerException.class, result.all().get(tp0));
assertEquals(Collections.emptyList(), driver.poll());
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -507,7 +507,7 @@ public void testCommitSyncFailsWithCommitFailedExceptionIfUnknownMemberId() {
NetworkClientDelegate.PollResult res = commitRequestManager.poll(time.milliseconds());
assertEquals(0, res.unsentRequests.size());
assertTrue(commitResult.isDone());
assertFutureThrows(commitResult, CommitFailedException.class);
assertFutureThrows(CommitFailedException.class, commitResult);
}

@Test
Expand All @@ -528,7 +528,7 @@ public void testCommitSyncFailsWithCommitFailedExceptionOnStaleMemberEpoch() {

// Commit should fail with CommitFailedException
assertTrue(commitResult.isDone());
assertFutureThrows(commitResult, CommitFailedException.class);
assertFutureThrows(CommitFailedException.class, commitResult);
}

/**
Expand Down Expand Up @@ -583,7 +583,7 @@ public void testCommitAsyncFailsWithRetriableOnCoordinatorDisconnected() {

// Commit should mark the coordinator unknown and fail with RetriableCommitFailedException.
assertTrue(commitResult.isDone());
assertFutureThrows(commitResult, RetriableCommitFailedException.class);
assertFutureThrows(RetriableCommitFailedException.class, commitResult);
assertCoordinatorDisconnectHandling();
}

Expand Down Expand Up @@ -806,10 +806,10 @@ public void testOffsetFetchRequestTimeoutRequests(final Errors error,
assertFalse(commitRequestManager.pendingRequests.unsentOffsetFetches.isEmpty());
NetworkClientDelegate.PollResult poll = commitRequestManager.poll(time.milliseconds());
mimicResponse(error, poll);
futures.forEach(f -> assertFutureThrows(f, expectedExceptionClass));
futures.forEach(f -> assertFutureThrows(expectedExceptionClass, f));
assertTrue(commitRequestManager.pendingRequests.unsentOffsetFetches.isEmpty());
} else {
futures.forEach(f -> assertFutureThrows(f, expectedExceptionClass));
futures.forEach(f -> assertFutureThrows(expectedExceptionClass, f));
assertEmptyPendingRequests(commitRequestManager);
}
}
Expand Down Expand Up @@ -931,7 +931,7 @@ public void testOffsetCommitRequestErroredRequestsNotRetriedForAsyncCommit(final
assertTrue(commitResult.isDone());
assertTrue(commitResult.isCompletedExceptionally());
if (error.exception() instanceof RetriableException) {
assertFutureThrows(commitResult, RetriableCommitFailedException.class);
assertFutureThrows(RetriableCommitFailedException.class, commitResult);
}

// We expect that the request should not have been retried on this async commit.
Expand Down Expand Up @@ -994,7 +994,7 @@ public void testOffsetCommitSyncFailedWithRetriableThrowsTimeoutWhenRetryTimeExp
assertEquals(0, res.unsentRequests.size());
assertTrue(commitResult.isDone());

assertFutureThrows(commitResult, expectedExceptionClass);
assertFutureThrows(expectedExceptionClass, commitResult);
}

/**
Expand Down Expand Up @@ -1022,7 +1022,7 @@ public void testOffsetCommitAsyncFailedWithRetriableThrowsRetriableCommitExcepti
assertExceptionHandling(commitRequestManager, retriableError, false);

// Request should complete with a RetriableCommitException
assertFutureThrows(commitResult, RetriableCommitFailedException.class);
assertFutureThrows(RetriableCommitFailedException.class, commitResult);
}

@ParameterizedTest
Expand Down Expand Up @@ -1372,7 +1372,7 @@ private void testRetriable(final CommitRequestManager commitRequestManager,
mimicResponse(error, poll);
futures.forEach(f -> {
assertTrue(f.isCompletedExceptionally());
assertFutureThrows(f, TimeoutException.class);
assertFutureThrows(TimeoutException.class, f);
});
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3412,7 +3412,7 @@ public void testPollWithCreateFetchRequestsError() {
assertFalse(future.isDone());

assertDoesNotThrow(() -> sendFetches(false));
assertFutureThrows(future, AuthenticationException.class);
assertFutureThrows(AuthenticationException.class, future);
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -129,7 +129,7 @@ public void testTimeoutBeforeSend() throws Exception {
time.sleep(REQUEST_TIMEOUT_MS);
ncd.poll(0, time.milliseconds());
assertTrue(unsentRequest.future().isDone());
TestUtils.assertFutureThrows(unsentRequest.future(), TimeoutException.class);
TestUtils.assertFutureThrows(TimeoutException.class, unsentRequest.future());
}
}

Expand All @@ -142,7 +142,7 @@ public void testTimeoutAfterSend() throws Exception {
time.sleep(REQUEST_TIMEOUT_MS);
ncd.poll(0, time.milliseconds());
assertTrue(unsentRequest.future().isDone());
TestUtils.assertFutureThrows(unsentRequest.future(), DisconnectException.class);
TestUtils.assertFutureThrows(DisconnectException.class, unsentRequest.future());
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -565,7 +565,7 @@ public void testResetOffsetsAuthorizationFailure() {
CompletableFuture<Void> nextReset = assertDoesNotThrow(() -> requestManager.resetPositionsIfNeeded());
assertEquals(0, requestManager.requestsToSend());
assertTrue(nextReset.isCompletedExceptionally());
assertFutureThrows(nextReset, TopicAuthorizationException.class);
assertFutureThrows(TopicAuthorizationException.class, nextReset);
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -438,7 +438,7 @@ public void testR2JPatternSubscriptionEventFailureWithMixedSubscriptionType() {
processor.process(event);

verify(subscriptionState).subscribe(pattern, listener);
Exception thrown = assertFutureThrows(event.future(), mixedSubscriptionError.getClass());
Exception thrown = assertFutureThrows(mixedSubscriptionError.getClass(), event.future());
assertEquals(mixedSubscriptionError, thrown);
}

Expand All @@ -462,7 +462,7 @@ public void testSyncCommitEventWithoutCommitRequestManager() {

setupProcessor(false);
processor.process(event);
assertFutureThrows(event.future(), KafkaException.class);
assertFutureThrows(KafkaException.class, event.future());
}

@Test
Expand All @@ -476,7 +476,7 @@ public void testSyncCommitEventWithException() {
processor.process(event);

verify(commitRequestManager).commitSync(Optional.empty(), 12345);
assertFutureThrows(event.future(), IllegalStateException.class);
assertFutureThrows(IllegalStateException.class, event.future());
}

@ParameterizedTest
Expand All @@ -499,7 +499,7 @@ public void testAsyncCommitEventWithoutCommitRequestManager() {

setupProcessor(false);
processor.process(event);
assertFutureThrows(event.future(), KafkaException.class);
assertFutureThrows(KafkaException.class, event.future());
}

@Test
Expand All @@ -513,7 +513,7 @@ public void testAsyncCommitEventWithException() {
processor.process(event);

verify(commitRequestManager).commitAsync(Optional.empty());
assertFutureThrows(event.future(), IllegalStateException.class);
assertFutureThrows(IllegalStateException.class, event.future());
}

private static Stream<Arguments> offsetsGenerator() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1294,7 +1294,7 @@ public void testInitTransactionsResponseAfterTimeout() throws Exception {
"Timed out while waiting for expected `InitProducerId` request to be sent");

time.sleep(maxBlockMs);
TestUtils.assertFutureThrows(future, TimeoutException.class);
TestUtils.assertFutureThrows(TimeoutException.class, future);

client.respond(initProducerIdResponse(1L, (short) 5, Errors.NONE));

Expand Down Expand Up @@ -1532,7 +1532,7 @@ public void testCommitTransactionWithRecordTooLargeException() throws Exception

client.prepareResponse(endTxnResponse(Errors.NONE));
producer.beginTransaction();
TestUtils.assertFutureThrows(producer.send(largeRecord), RecordTooLargeException.class);
TestUtils.assertFutureThrows(RecordTooLargeException.class, producer.send(largeRecord));
assertThrows(KafkaException.class, producer::commitTransaction);
}
}
Expand Down Expand Up @@ -1569,7 +1569,7 @@ public void testCommitTransactionWithMetadataTimeoutForMissingTopic() throws Exc
producer.initTransactions();
producer.beginTransaction();

TestUtils.assertFutureThrows(producer.send(record), TimeoutException.class);
TestUtils.assertFutureThrows(TimeoutException.class, producer.send(record));
assertThrows(KafkaException.class, producer::commitTransaction);
}
}
Expand Down Expand Up @@ -1606,7 +1606,7 @@ public void testCommitTransactionWithMetadataTimeoutForPartitionOutOfRange() thr
producer.initTransactions();
producer.beginTransaction();

TestUtils.assertFutureThrows(producer.send(record), TimeoutException.class);
TestUtils.assertFutureThrows(TimeoutException.class, producer.send(record));
assertThrows(KafkaException.class, producer::commitTransaction);
}
}
Expand Down Expand Up @@ -1645,7 +1645,7 @@ public void testCommitTransactionWithSendToInvalidTopic() throws Exception {
producer.initTransactions();
producer.beginTransaction();

TestUtils.assertFutureThrows(producer.send(record), InvalidTopicException.class);
TestUtils.assertFutureThrows(InvalidTopicException.class, producer.send(record));
assertThrows(KafkaException.class, producer::commitTransaction);
}
}
Expand Down Expand Up @@ -2080,7 +2080,7 @@ public void testSendToInvalidTopic() throws Exception {

assertEquals(Collections.singleton(invalidTopicName),
metadata.fetch().invalidTopics(), "Cluster has incorrect invalid topic list.");
TestUtils.assertFutureThrows(future, InvalidTopicException.class);
TestUtils.assertFutureThrows(InvalidTopicException.class, future);

producer.close(Duration.ofMillis(0));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -352,7 +352,7 @@ private void testCompleteExceptionally(

for (int i = 0; i < futures.size(); i++) {
FutureRecordMetadata future = futures.get(i);
RuntimeException caughtException = TestUtils.assertFutureThrows(future, RuntimeException.class);
RuntimeException caughtException = TestUtils.assertFutureThrows(RuntimeException.class, future);
RuntimeException expectedException = recordExceptions.apply(i);
assertEquals(expectedException, caughtException);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2534,15 +2534,15 @@ public void testRecordErrorPropagatedToApplication() throws InterruptedException

Integer index = futureEntry.getKey();
if (index == 0 || index == 2) {
InvalidRecordException exception = TestUtils.assertFutureThrows(future, InvalidRecordException.class);
InvalidRecordException exception = TestUtils.assertFutureThrows(InvalidRecordException.class, future);
assertInstanceOf(InvalidRecordException.class, exception);
assertEquals(index.toString(), exception.getMessage());
} else if (index == 3) {
InvalidRecordException exception = TestUtils.assertFutureThrows(future, InvalidRecordException.class);
InvalidRecordException exception = TestUtils.assertFutureThrows(InvalidRecordException.class, future);
assertInstanceOf(InvalidRecordException.class, exception);
assertEquals(Errors.INVALID_RECORD.message(), exception.getMessage());
} else {
KafkaException exception = TestUtils.assertFutureThrows(future, KafkaException.class);
KafkaException exception = TestUtils.assertFutureThrows(KafkaException.class, future);
assertEquals(KafkaException.class, exception.getClass());
}
}
Expand Down Expand Up @@ -2938,7 +2938,7 @@ public void testTransactionAbortedExceptionOnAbortWithoutError() throws Interrup
// drain all the unsent batches with a TransactionAbortedException.
sender.runOnce();
// Now attempt to fetch the result for the record.
TestUtils.assertFutureThrows(metadata, TransactionAbortedException.class);
TestUtils.assertFutureThrows(TransactionAbortedException.class, metadata);
}

@Test
Expand Down
Loading

0 comments on commit 3527286

Please sign in to comment.