MINOR: replace test "expected" parameter by assertThrows (#9520)

This PR includes following changes.

1. @Test(expected = Exception.class) is replaced by assertThrows
2. remove reference to org.scalatest.Assertions
3. change the magic code from 1 to 2 for testAppendAtInvalidOffset to test ZSTD
4. rename testMaybeAddPartitionToTransactionXXXX to testNotReadyForSendXXX
5. increase maxBlockMs from 1s to 3s to avoid unexpected timeout from TransactionsTest#testTimeout

Reviewers: Ismael Juma <ismael@confluent.io>
This commit is contained in:
Chia-Ping Tsai 2021-01-10 20:20:13 +08:00 committed by GitHub
parent 3697390402
commit 913a019d6c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
264 changed files with 2812 additions and 3145 deletions

View File

@ -20,11 +20,13 @@ import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.UnknownHostException;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.stream.Collectors;
import org.apache.kafka.common.config.ConfigException;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
import org.junit.Test;
@ -62,19 +64,20 @@ public class ClientUtilsTest {
validatedAddresses.forEach(address -> assertEquals(10000, address.getPort()));
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testInvalidConfig() {
ClientUtils.parseAndValidateAddresses(Arrays.asList("localhost:10000"), "random.value");
assertThrows(IllegalArgumentException.class,
() -> ClientUtils.parseAndValidateAddresses(Collections.singletonList("localhost:10000"), "random.value"));
}
@Test(expected = ConfigException.class)
@Test
public void testNoPort() {
checkWithoutLookup("127.0.0.1");
assertThrows(ConfigException.class, () -> checkWithoutLookup("127.0.0.1"));
}
@Test(expected = ConfigException.class)
@Test
public void testOnlyBadHostname() {
checkWithoutLookup("some.invalid.hostname.foo.bar.local:9999");
assertThrows(ConfigException.class, () -> checkWithoutLookup("some.invalid.hostname.foo.bar.local:9999"));
}
@Test
@ -95,9 +98,10 @@ public class ClientUtilsTest {
assertEquals(1, result.size());
}
@Test(expected = UnknownHostException.class)
public void testResolveUnknownHostException() throws UnknownHostException {
ClientUtils.resolve("some.invalid.hostname.foo.bar.local", ClientDnsLookup.USE_ALL_DNS_IPS);
@Test
public void testResolveUnknownHostException() {
assertThrows(UnknownHostException.class,
() -> ClientUtils.resolve("some.invalid.hostname.foo.bar.local", ClientDnsLookup.USE_ALL_DNS_IPS));
}
@Test

View File

@ -30,6 +30,7 @@ import java.util.Collections;
import java.util.List;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
public class InFlightRequestsTest {
@ -99,14 +100,14 @@ public class InFlightRequestsTest {
assertEquals(0, inFlightRequests.count());
}
@Test(expected = IllegalStateException.class)
@Test
public void testCompleteNextThrowsIfNoInflights() {
inFlightRequests.completeNext(dest);
assertThrows(IllegalStateException.class, () -> inFlightRequests.completeNext(dest));
}
@Test(expected = IllegalStateException.class)
@Test
public void testCompleteLastSentThrowsIfNoInFlights() {
inFlightRequests.completeLastSent(dest);
assertThrows(IllegalStateException.class, () -> inFlightRequests.completeLastSent(dest));
}
private int addRequest(String destination) {

View File

@ -76,10 +76,10 @@ public class MetadataTest {
Collections.emptyList());
}
@Test(expected = IllegalStateException.class)
@Test
public void testMetadataUpdateAfterClose() {
metadata.close();
metadata.updateWithCurrentRequestVersion(emptyMetadataResponse(), false, 1000);
assertThrows(IllegalStateException.class, () -> metadata.updateWithCurrentRequestVersion(emptyMetadataResponse(), false, 1000));
}
private static void checkTimeToNextUpdate(long refreshBackoffMs, long metadataExpireMs) {

View File

@ -63,6 +63,7 @@ import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
public class NetworkClientTest {
@ -119,13 +120,12 @@ public class NetworkClientTest {
selector.reset();
}
@Test(expected = IllegalStateException.class)
@Test
public void testSendToUnreadyNode() {
MetadataRequest.Builder builder = new MetadataRequest.Builder(Collections.singletonList("test"), true);
long now = time.milliseconds();
ClientRequest request = client.newClientRequest("5", builder, now, false);
client.send(request, now);
client.poll(1, time.milliseconds());
assertThrows(IllegalStateException.class, () -> client.send(request, now));
}
@Test

View File

@ -28,6 +28,7 @@ import org.junit.Test;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
public class NodeApiVersionsTest {
@ -103,28 +104,32 @@ public class NodeApiVersionsTest {
assertEquals(3, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 3, (short) 4));
}
@Test(expected = UnsupportedVersionException.class)
@Test
public void testLatestUsableVersionOutOfRangeLow() {
NodeApiVersions apiVersions = NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 1, (short) 2);
apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 3, (short) 4);
assertThrows(UnsupportedVersionException.class,
() -> apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 3, (short) 4));
}
@Test(expected = UnsupportedVersionException.class)
@Test
public void testLatestUsableVersionOutOfRangeHigh() {
NodeApiVersions apiVersions = NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 2, (short) 3);
apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 0, (short) 1);
assertThrows(UnsupportedVersionException.class,
() -> apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 0, (short) 1));
}
@Test(expected = UnsupportedVersionException.class)
@Test
public void testUsableVersionCalculationNoKnownVersions() {
NodeApiVersions versions = new NodeApiVersions(new ApiVersionsResponseKeyCollection());
versions.latestUsableVersion(ApiKeys.FETCH);
assertThrows(UnsupportedVersionException.class,
() -> versions.latestUsableVersion(ApiKeys.FETCH));
}
@Test(expected = UnsupportedVersionException.class)
@Test
public void testLatestUsableVersionOutOfRange() {
NodeApiVersions apiVersions = NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 300, (short) 300);
apiVersions.latestUsableVersion(ApiKeys.PRODUCE);
assertThrows(UnsupportedVersionException.class,
() -> apiVersions.latestUsableVersion(ApiKeys.PRODUCE));
}
@Test

View File

@ -221,20 +221,22 @@ public class KafkaConsumerTest {
consumer.close();
}
@Test(expected = KafkaException.class)
@Test
public void testInvalidSocketSendBufferSize() {
Map<String, Object> config = new HashMap<>();
config.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999");
config.put(ConsumerConfig.SEND_BUFFER_CONFIG, -2);
new KafkaConsumer<>(config, new ByteArrayDeserializer(), new ByteArrayDeserializer());
assertThrows(KafkaException.class,
() -> new KafkaConsumer<>(config, new ByteArrayDeserializer(), new ByteArrayDeserializer()));
}
@Test(expected = KafkaException.class)
@Test
public void testInvalidSocketReceiveBufferSize() {
Map<String, Object> config = new HashMap<>();
config.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999");
config.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, -2);
new KafkaConsumer<>(config, new ByteArrayDeserializer(), new ByteArrayDeserializer());
assertThrows(KafkaException.class,
() -> new KafkaConsumer<>(config, new ByteArrayDeserializer(), new ByteArrayDeserializer()));
}
@Test
@ -270,65 +272,69 @@ public class KafkaConsumerTest {
consumer.close();
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testSubscriptionOnNullTopicCollection() {
try (KafkaConsumer<byte[], byte[]> consumer = newConsumer(groupId)) {
consumer.subscribe((List<String>) null);
assertThrows(IllegalArgumentException.class, () -> consumer.subscribe((List<String>) null));
}
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testSubscriptionOnNullTopic() {
try (KafkaConsumer<byte[], byte[]> consumer = newConsumer(groupId)) {
consumer.subscribe(singletonList(null));
assertThrows(IllegalArgumentException.class, () -> consumer.subscribe(singletonList(null)));
}
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testSubscriptionOnEmptyTopic() {
try (KafkaConsumer<byte[], byte[]> consumer = newConsumer(groupId)) {
String emptyTopic = " ";
consumer.subscribe(singletonList(emptyTopic));
assertThrows(IllegalArgumentException.class, () -> consumer.subscribe(singletonList(emptyTopic)));
}
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testSubscriptionOnNullPattern() {
try (KafkaConsumer<byte[], byte[]> consumer = newConsumer(groupId)) {
consumer.subscribe((Pattern) null);
assertThrows(IllegalArgumentException.class,
() -> consumer.subscribe((Pattern) null));
}
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testSubscriptionOnEmptyPattern() {
try (KafkaConsumer<byte[], byte[]> consumer = newConsumer(groupId)) {
consumer.subscribe(Pattern.compile(""));
assertThrows(IllegalArgumentException.class,
() -> consumer.subscribe(Pattern.compile("")));
}
}
@Test(expected = IllegalStateException.class)
@Test
public void testSubscriptionWithEmptyPartitionAssignment() {
Properties props = new Properties();
props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999");
props.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "");
props.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId);
try (KafkaConsumer<byte[], byte[]> consumer = newConsumer(props)) {
consumer.subscribe(singletonList(topic));
assertThrows(IllegalStateException.class,
() -> consumer.subscribe(singletonList(topic)));
}
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testSeekNegative() {
try (KafkaConsumer<byte[], byte[]> consumer = newConsumer((String) null)) {
consumer.assign(singleton(new TopicPartition("nonExistTopic", 0)));
consumer.seek(new TopicPartition("nonExistTopic", 0), -1);
assertThrows(IllegalArgumentException.class,
() -> consumer.seek(new TopicPartition("nonExistTopic", 0), -1));
}
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testAssignOnNullTopicPartition() {
try (KafkaConsumer<byte[], byte[]> consumer = newConsumer((String) null)) {
consumer.assign(null);
assertThrows(IllegalArgumentException.class, () -> consumer.assign(null));
}
}
@ -341,17 +347,17 @@ public class KafkaConsumerTest {
}
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testAssignOnNullTopicInPartition() {
try (KafkaConsumer<byte[], byte[]> consumer = newConsumer((String) null)) {
consumer.assign(singleton(new TopicPartition(null, 0)));
assertThrows(IllegalArgumentException.class, () -> consumer.assign(singleton(new TopicPartition(null, 0))));
}
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testAssignOnEmptyTopicInPartition() {
try (KafkaConsumer<byte[], byte[]> consumer = newConsumer((String) null)) {
consumer.assign(singleton(new TopicPartition(" ", 0)));
assertThrows(IllegalArgumentException.class, () -> consumer.assign(singleton(new TopicPartition(" ", 0))));
}
}
@ -635,7 +641,7 @@ public class KafkaConsumerTest {
mockClient.updateMetadata(initialMetadata);
}
@Test(expected = NoOffsetForPartitionException.class)
@Test
public void testMissingOffsetNoResetPolicy() {
Time time = new MockTime();
SubscriptionState subscription = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE);
@ -656,7 +662,7 @@ public class KafkaConsumerTest {
// lookup committed offset and find nothing
client.prepareResponseFrom(offsetResponse(Collections.singletonMap(tp0, -1L), Errors.NONE), coordinator);
consumer.poll(Duration.ZERO);
assertThrows(NoOffsetForPartitionException.class, () -> consumer.poll(Duration.ZERO));
}
@Test
@ -1456,26 +1462,26 @@ public class KafkaConsumerTest {
consumer.close();
}
@Test(expected = IllegalStateException.class)
@Test
public void testPollWithNoSubscription() {
try (KafkaConsumer<byte[], byte[]> consumer = newConsumer((String) null)) {
consumer.poll(Duration.ZERO);
assertThrows(IllegalStateException.class, () -> consumer.poll(Duration.ZERO));
}
}
@Test(expected = IllegalStateException.class)
@Test
public void testPollWithEmptySubscription() {
try (KafkaConsumer<byte[], byte[]> consumer = newConsumer(groupId)) {
consumer.subscribe(Collections.emptyList());
consumer.poll(Duration.ZERO);
assertThrows(IllegalStateException.class, () -> consumer.poll(Duration.ZERO));
}
}
@Test(expected = IllegalStateException.class)
@Test
public void testPollWithEmptyUserAssignment() {
try (KafkaConsumer<byte[], byte[]> consumer = newConsumer(groupId)) {
consumer.assign(Collections.emptySet());
consumer.poll(Duration.ZERO);
assertThrows(IllegalStateException.class, () -> consumer.poll(Duration.ZERO));
}
}
@ -1756,49 +1762,49 @@ public class KafkaConsumerTest {
}
}
@Test(expected = AuthenticationException.class)
@Test
public void testPartitionsForAuthenticationFailure() {
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError();
consumer.partitionsFor("some other topic");
assertThrows(AuthenticationException.class, () -> consumer.partitionsFor("some other topic"));
}
@Test(expected = AuthenticationException.class)
@Test
public void testBeginningOffsetsAuthenticationFailure() {
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError();
consumer.beginningOffsets(Collections.singleton(tp0));
assertThrows(AuthenticationException.class, () -> consumer.beginningOffsets(Collections.singleton(tp0)));
}
@Test(expected = AuthenticationException.class)
@Test
public void testEndOffsetsAuthenticationFailure() {
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError();
consumer.endOffsets(Collections.singleton(tp0));
assertThrows(AuthenticationException.class, () -> consumer.endOffsets(Collections.singleton(tp0)));
}
@Test(expected = AuthenticationException.class)
@Test
public void testPollAuthenticationFailure() {
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError();
consumer.subscribe(singleton(topic));
consumer.poll(Duration.ZERO);
assertThrows(AuthenticationException.class, () -> consumer.poll(Duration.ZERO));
}
@Test(expected = AuthenticationException.class)
@Test
public void testOffsetsForTimesAuthenticationFailure() {
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError();
consumer.offsetsForTimes(singletonMap(tp0, 0L));
assertThrows(AuthenticationException.class, () -> consumer.offsetsForTimes(singletonMap(tp0, 0L)));
}
@Test(expected = AuthenticationException.class)
@Test
public void testCommitSyncAuthenticationFailure() {
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError();
Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>();
offsets.put(tp0, new OffsetAndMetadata(10L));
consumer.commitSync(offsets);
assertThrows(AuthenticationException.class, () -> consumer.commitSync(offsets));
}
@Test(expected = AuthenticationException.class)
@Test
public void testCommittedAuthenticationFailure() {
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError();
consumer.committed(Collections.singleton(tp0)).get(tp0);
assertThrows(AuthenticationException.class, () -> consumer.committed(Collections.singleton(tp0)).get(tp0));
}
@Test
@ -2392,7 +2398,7 @@ public class KafkaConsumerTest {
verify(consumer).close(Duration.ofSeconds(1));
}
@Test(expected = InvalidTopicException.class)
@Test
public void testSubscriptionOnInvalidTopic() {
Time time = new MockTime();
SubscriptionState subscription = new SubscriptionState(new LogContext(), OffsetResetStrategy.EARLIEST);
@ -2418,7 +2424,7 @@ public class KafkaConsumerTest {
KafkaConsumer<String, String> consumer = newConsumer(time, client, subscription, metadata, assignor, true, groupInstanceId);
consumer.subscribe(singleton(invalidTopicName), getConsumerRebalanceListener(consumer));
consumer.poll(Duration.ZERO);
assertThrows(InvalidTopicException.class, () -> consumer.poll(Duration.ZERO));
}
@Test
@ -2539,11 +2545,11 @@ public class KafkaConsumerTest {
assertFalse(consumerMetricPresent(consumer, "time-between-poll-max"));
}
@Test(expected = IllegalStateException.class)
@Test
public void testEnforceRebalanceWithManualAssignment() {
try (KafkaConsumer<byte[], byte[]> consumer = newConsumer((String) null)) {
consumer.assign(singleton(new TopicPartition("topic", 0)));
consumer.enforceRebalance();
assertThrows(IllegalStateException.class, consumer::enforceRebalance);
}
}

View File

@ -23,6 +23,7 @@ import java.io.IOException;
import java.util.Optional;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
/**
* This test case ensures OffsetAndMetadata class is serializable and is serialization compatible.
@ -31,9 +32,9 @@ import static org.junit.Assert.assertEquals;
*/
public class OffsetAndMetadataTest {
@Test(expected = IllegalArgumentException.class)
@Test
public void testInvalidNegativeOffset() {
new OffsetAndMetadata(-239L, Optional.of(15), "");
assertThrows(IllegalArgumentException.class, () -> new OffsetAndMetadata(-239L, Optional.of(15), ""));
}
@Test

View File

@ -306,13 +306,13 @@ public class ConsumerCoordinatorTest {
assertTrue(future.succeeded());
}
@Test(expected = GroupAuthorizationException.class)
@Test
public void testGroupDescribeUnauthorized() {
client.prepareResponse(groupCoordinatorResponse(node, Errors.GROUP_AUTHORIZATION_FAILED));
coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
assertThrows(GroupAuthorizationException.class, () -> coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE)));
}
@Test(expected = GroupAuthorizationException.class)
@Test
public void testGroupReadUnauthorized() {
subscriptions.subscribe(singleton(topic1), rebalanceListener);
@ -321,7 +321,7 @@ public class ConsumerCoordinatorTest {
client.prepareResponse(joinGroupLeaderResponse(0, "memberId", Collections.emptyMap(),
Errors.GROUP_AUTHORIZATION_FAILED));
coordinator.poll(time.timer(Long.MAX_VALUE));
assertThrows(GroupAuthorizationException.class, () -> coordinator.poll(time.timer(Long.MAX_VALUE)));
}
@Test
@ -671,7 +671,7 @@ public class ConsumerCoordinatorTest {
assertTrue(coordinator.coordinatorUnknown());
}
@Test(expected = ApiException.class)
@Test
public void testJoinGroupInvalidGroupId() {
final String consumerId = "leader";
@ -685,7 +685,7 @@ public class ConsumerCoordinatorTest {
client.prepareResponse(joinGroupLeaderResponse(0, consumerId, Collections.emptyMap(),
Errors.INVALID_GROUP_ID));
coordinator.poll(time.timer(Long.MAX_VALUE));
assertThrows(ApiException.class, () -> coordinator.poll(time.timer(Long.MAX_VALUE)));
}
@Test
@ -1274,7 +1274,7 @@ public class ConsumerCoordinatorTest {
assertTrue(received.get());
}
@Test(expected = KafkaException.class)
@Test
public void testUnexpectedErrorOnSyncGroup() {
subscriptions.subscribe(singleton(topic1), rebalanceListener);
@ -1284,7 +1284,7 @@ public class ConsumerCoordinatorTest {
// join initially, but let coordinator rebalance on sync
client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE));
client.prepareResponse(syncGroupResponse(Collections.emptyList(), Errors.UNKNOWN_SERVER_ERROR));
coordinator.joinGroupIfNeeded(time.timer(Long.MAX_VALUE));
assertThrows(KafkaException.class, () -> coordinator.joinGroupIfNeeded(time.timer(Long.MAX_VALUE)));
}
@Test
@ -1661,7 +1661,7 @@ public class ConsumerCoordinatorTest {
assertEquals(getAdded(owned, assigned), rebalanceListener.assigned);
}
@Test(expected = ApiException.class)
@Test
public void testInvalidSessionTimeout() {
subscriptions.subscribe(singleton(topic1), rebalanceListener);
@ -1670,7 +1670,7 @@ public class ConsumerCoordinatorTest {
// coordinator doesn't like the session timeout
client.prepareResponse(joinGroupFollowerResponse(0, consumerId, "", Errors.INVALID_SESSION_TIMEOUT));
coordinator.joinGroupIfNeeded(time.timer(Long.MAX_VALUE));
assertThrows(ApiException.class, () -> coordinator.joinGroupIfNeeded(time.timer(Long.MAX_VALUE)));
}
@Test
@ -2066,37 +2066,37 @@ public class ConsumerCoordinatorTest {
new OffsetAndMetadata(100L, "metadata")), time.timer(10000)));
}
@Test(expected = OffsetMetadataTooLarge.class)
@Test
public void testCommitOffsetMetadataTooLarge() {
// since offset metadata is provided by the user, we have to propagate the exception so they can handle it
client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.OFFSET_METADATA_TOO_LARGE);
coordinator.commitOffsetsSync(singletonMap(t1p,
new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE));
assertThrows(OffsetMetadataTooLarge.class, () -> coordinator.commitOffsetsSync(singletonMap(t1p,
new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE)));
}
@Test(expected = CommitFailedException.class)
@Test
public void testCommitOffsetIllegalGeneration() {
// we cannot retry if a rebalance occurs before the commit completed
client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.ILLEGAL_GENERATION);
coordinator.commitOffsetsSync(singletonMap(t1p,
new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE));
assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(t1p,
new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE)));
}
@Test(expected = CommitFailedException.class)
@Test
public void testCommitOffsetUnknownMemberId() {
// we cannot retry if a rebalance occurs before the commit completed
client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.UNKNOWN_MEMBER_ID);
coordinator.commitOffsetsSync(singletonMap(t1p,
new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE));
assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(t1p,
new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE)));
}
@Test
@ -2317,14 +2317,15 @@ public class ConsumerCoordinatorTest {
assertEquals(expectedGeneration, coordinator.generationIfStable());
}
@Test(expected = KafkaException.class)
@Test
public void testCommitOffsetSyncCallbackWithNonRetriableException() {
client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
// sync commit with invalid partitions should throw if we have no callback
prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.UNKNOWN_SERVER_ERROR);
coordinator.commitOffsetsSync(singletonMap(t1p, new OffsetAndMetadata(100L)), time.timer(Long.MAX_VALUE));
assertThrows(KafkaException.class, () -> coordinator.commitOffsetsSync(singletonMap(t1p,
new OffsetAndMetadata(100L)), time.timer(Long.MAX_VALUE)));
}
@Test
@ -2454,14 +2455,14 @@ public class ConsumerCoordinatorTest {
assertEquals(100L, subscriptions.position(t1p).offset);
}
@Test(expected = KafkaException.class)
@Test
public void testRefreshOffsetUnknownTopicOrPartition() {
client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
subscriptions.assignFromUser(singleton(t1p));
client.prepareResponse(offsetFetchResponse(t1p, Errors.UNKNOWN_TOPIC_OR_PARTITION, "", 100L));
coordinator.refreshCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE));
assertThrows(KafkaException.class, () -> coordinator.refreshCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE)));
}
@Test
@ -2710,19 +2711,20 @@ public class ConsumerCoordinatorTest {
}
}
@Test(expected = FencedInstanceIdException.class)
@Test
public void testCommitOffsetRequestSyncWithFencedInstanceIdException() {
client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
// sync commit with invalid partitions should throw if we have no callback
prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.FENCED_INSTANCE_ID);
coordinator.commitOffsetsSync(singletonMap(t1p, new OffsetAndMetadata(100L)), time.timer(Long.MAX_VALUE));
assertThrows(FencedInstanceIdException.class, () -> coordinator.commitOffsetsSync(singletonMap(t1p,
new OffsetAndMetadata(100L)), time.timer(Long.MAX_VALUE)));
}
@Test(expected = FencedInstanceIdException.class)
@Test
public void testCommitOffsetRequestAsyncWithFencedInstanceIdException() {
receiveFencedInstanceIdException();
assertThrows(FencedInstanceIdException.class, this::receiveFencedInstanceIdException);
}
@Test

View File

@ -49,6 +49,7 @@ import java.util.concurrent.atomic.AtomicInteger;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.ArgumentMatchers.anyLong;
@ -246,20 +247,20 @@ public class ConsumerNetworkClientTest {
}
}
@Test(expected = InvalidTopicException.class)
@Test
public void testInvalidTopicExceptionPropagatedFromMetadata() {
MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith("clusterId", 1,
Collections.singletonMap("topic", Errors.INVALID_TOPIC_EXCEPTION), Collections.emptyMap());
metadata.updateWithCurrentRequestVersion(metadataResponse, false, time.milliseconds());
consumerClient.poll(time.timer(Duration.ZERO));
assertThrows(InvalidTopicException.class, () -> consumerClient.poll(time.timer(Duration.ZERO)));
}
@Test(expected = TopicAuthorizationException.class)
@Test
public void testTopicAuthorizationExceptionPropagatedFromMetadata() {
MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith("clusterId", 1,
Collections.singletonMap("topic", Errors.TOPIC_AUTHORIZATION_FAILED), Collections.emptyMap());
metadata.updateWithCurrentRequestVersion(metadataResponse, false, time.milliseconds());
consumerClient.poll(time.timer(Duration.ZERO));
assertThrows(TopicAuthorizationException.class, () -> consumerClient.poll(time.timer(Duration.ZERO)));
}
@Test

View File

@ -1948,12 +1948,12 @@ public class FetcherTest {
assertEquals(initialUpdateResponse.topicMetadata().size(), allTopics.size());
}
@Test(expected = TimeoutException.class)
@Test
public void testGetAllTopicsTimeout() {
// since no response is prepared, the request should timeout
buildFetcher();
assignFromUser(singleton(tp0));
fetcher.getAllTopicMetadata(time.timer(50L));
assertThrows(TimeoutException.class, () -> fetcher.getAllTopicMetadata(time.timer(50L)));
}
@Test
@ -1969,13 +1969,13 @@ public class FetcherTest {
}
}
@Test(expected = InvalidTopicException.class)
@Test
public void testGetTopicMetadataInvalidTopic() {
buildFetcher();
assignFromUser(singleton(tp0));
client.prepareResponse(newMetadataResponse(topicName, Errors.INVALID_TOPIC_EXCEPTION));
fetcher.getTopicMetadata(
new MetadataRequest.Builder(Collections.singletonList(topicName), true), time.timer(5000L));
assertThrows(InvalidTopicException.class, () -> fetcher.getTopicMetadata(
new MetadataRequest.Builder(Collections.singletonList(topicName), true), time.timer(5000L)));
}
@Test
@ -2728,7 +2728,7 @@ public class FetcherTest {
Assert.assertNotNull(metadata.fetch().partitionCountForTopic(anotherTopic));
}
@Test(expected = TimeoutException.class)
@Test
public void testBatchedListOffsetsMetadataErrors() {
buildFetcher();
@ -2753,7 +2753,7 @@ public class FetcherTest {
offsetsToSearch.put(tp0, ListOffsetsRequest.EARLIEST_TIMESTAMP);
offsetsToSearch.put(tp1, ListOffsetsRequest.EARLIEST_TIMESTAMP);
fetcher.offsetsForTimes(offsetsToSearch, time.timer(0));
assertThrows(TimeoutException.class, () -> fetcher.offsetsForTimes(offsetsToSearch, time.timer(0)));
}
@Test

View File

@ -22,6 +22,7 @@ import java.util.concurrent.atomic.AtomicInteger;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
public class RequestFutureTest {
@ -52,52 +53,52 @@ public class RequestFutureTest {
assertNull(future.value());
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testRuntimeExceptionInComplete() {
RequestFuture<Exception> future = new RequestFuture<>();
future.complete(new RuntimeException());
assertThrows(IllegalArgumentException.class, () -> future.complete(new RuntimeException()));
}
@Test(expected = IllegalStateException.class)
@Test
public void invokeCompleteAfterAlreadyComplete() {
RequestFuture<Void> future = new RequestFuture<>();
future.complete(null);
future.complete(null);
assertThrows(IllegalStateException.class, () -> future.complete(null));
}
@Test(expected = IllegalStateException.class)
@Test
public void invokeCompleteAfterAlreadyFailed() {
RequestFuture<Void> future = new RequestFuture<>();
future.raise(new RuntimeException());
future.complete(null);
assertThrows(IllegalStateException.class, () -> future.complete(null));
}
@Test(expected = IllegalStateException.class)
@Test
public void invokeRaiseAfterAlreadyFailed() {
RequestFuture<Void> future = new RequestFuture<>();
future.raise(new RuntimeException());
future.raise(new RuntimeException());
assertThrows(IllegalStateException.class, () -> future.raise(new RuntimeException()));
}
@Test(expected = IllegalStateException.class)
@Test
public void invokeRaiseAfterAlreadyCompleted() {
RequestFuture<Void> future = new RequestFuture<>();
future.complete(null);
future.raise(new RuntimeException());
assertThrows(IllegalStateException.class, () -> future.raise(new RuntimeException()));
}
@Test(expected = IllegalStateException.class)
@Test
public void invokeExceptionAfterSuccess() {
RequestFuture<Void> future = new RequestFuture<>();
future.complete(null);
future.exception();
assertThrows(IllegalStateException.class, future::exception);
}
@Test(expected = IllegalStateException.class)
@Test
public void invokeValueAfterFailure() {
RequestFuture<Void> future = new RequestFuture<>();
future.raise(new RuntimeException());
future.value();
assertThrows(IllegalStateException.class, future::value);
}
@Test

View File

@ -46,6 +46,7 @@ import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UND
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
public class SubscriptionStateTest {
@ -254,13 +255,14 @@ public class SubscriptionStateTest {
assertTrue(state.isFetchable(tp0));
}
@Test(expected = IllegalStateException.class)
@Test
public void invalidPositionUpdate() {
state.subscribe(singleton(topic), rebalanceListener);
assertTrue(state.checkAssignmentMatchedSubscription(singleton(tp0)));
state.assignFromSubscribed(singleton(tp0));
state.position(tp0, new SubscriptionState.FetchPosition(0, Optional.empty(), leaderAndEpoch));
assertThrows(IllegalStateException.class, () -> state.position(tp0,
new SubscriptionState.FetchPosition(0, Optional.empty(), leaderAndEpoch)));
}
@Test
@ -276,33 +278,34 @@ public class SubscriptionStateTest {
assertFalse(state.checkAssignmentMatchedSubscription(Collections.singletonList(t1p0)));
}
@Test(expected = IllegalStateException.class)
@Test
public void cantChangePositionForNonAssignedPartition() {
state.position(tp0, new SubscriptionState.FetchPosition(1, Optional.empty(), leaderAndEpoch));
assertThrows(IllegalStateException.class, () -> state.position(tp0,
new SubscriptionState.FetchPosition(1, Optional.empty(), leaderAndEpoch)));
}
@Test(expected = IllegalStateException.class)
@Test
public void cantSubscribeTopicAndPattern() {
state.subscribe(singleton(topic), rebalanceListener);
state.subscribe(Pattern.compile(".*"), rebalanceListener);
assertThrows(IllegalStateException.class, () -> state.subscribe(Pattern.compile(".*"), rebalanceListener));
}
@Test(expected = IllegalStateException.class)
@Test
public void cantSubscribePartitionAndPattern() {
state.assignFromUser(singleton(tp0));
state.subscribe(Pattern.compile(".*"), rebalanceListener);
assertThrows(IllegalStateException.class, () -> state.subscribe(Pattern.compile(".*"), rebalanceListener));
}
@Test(expected = IllegalStateException.class)
@Test
public void cantSubscribePatternAndTopic() {
state.subscribe(Pattern.compile(".*"), rebalanceListener);
state.subscribe(singleton(topic), rebalanceListener);
assertThrows(IllegalStateException.class, () -> state.subscribe(singleton(topic), rebalanceListener));
}
@Test(expected = IllegalStateException.class)
@Test
public void cantSubscribePatternAndPartition() {
state.subscribe(Pattern.compile(".*"), rebalanceListener);
state.assignFromUser(singleton(tp0));
assertThrows(IllegalStateException.class, () -> state.assignFromUser(singleton(tp0)));
}
@Test

View File

@ -192,11 +192,11 @@ public class KafkaProducerTest {
new KafkaProducer<>(producerProps, new ByteArraySerializer(), new ByteArraySerializer()).close();
}
@Test(expected = ConfigException.class)
@Test
public void testNoSerializerProvided() {
Properties producerProps = new Properties();
producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9000");
new KafkaProducer(producerProps);
assertThrows(ConfigException.class, () -> new KafkaProducer(producerProps));
}
@Test
@ -357,20 +357,20 @@ public class KafkaProducerTest {
new KafkaProducer<>(config, new ByteArraySerializer(), new ByteArraySerializer()).close();
}
@Test(expected = KafkaException.class)
@Test
public void testInvalidSocketSendBufferSize() {
Map<String, Object> config = new HashMap<>();
config.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999");
config.put(ProducerConfig.SEND_BUFFER_CONFIG, -2);
new KafkaProducer<>(config, new ByteArraySerializer(), new ByteArraySerializer());
assertThrows(KafkaException.class, () -> new KafkaProducer<>(config, new ByteArraySerializer(), new ByteArraySerializer()));
}
@Test(expected = KafkaException.class)
@Test
public void testInvalidSocketReceiveBufferSize() {
Map<String, Object> config = new HashMap<>();
config.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999");
config.put(ProducerConfig.RECEIVE_BUFFER_CONFIG, -2);
new KafkaProducer<>(config, new ByteArraySerializer(), new ByteArraySerializer());
assertThrows(KafkaException.class, () -> new KafkaProducer<>(config, new ByteArraySerializer(), new ByteArraySerializer()));
}
private static KafkaProducer<String, String> producerWithOverrideNewSender(Map<String, Object> configs,

View File

@ -132,16 +132,13 @@ public class MockProducerTest {
public void shouldThrowOnInitTransactionIfProducerAlreadyInitializedForTransactions() {
buildMockProducer(true);
producer.initTransactions();
try {
producer.initTransactions();
fail("Should have thrown as producer is already initialized");
} catch (IllegalStateException e) { }
assertThrows(IllegalStateException.class, producer::initTransactions);
}
@Test(expected = IllegalStateException.class)
@Test
public void shouldThrowOnBeginTransactionIfTransactionsNotInitialized() {
buildMockProducer(true);
producer.beginTransaction();
assertThrows(IllegalStateException.class, producer::beginTransaction);
}
@Test
@ -152,44 +149,38 @@ public class MockProducerTest {
assertTrue(producer.transactionInFlight());
}
@Test(expected = IllegalStateException.class)
@Test
public void shouldThrowOnBeginTransactionsIfTransactionInflight() {
buildMockProducer(true);
producer.initTransactions();
producer.beginTransaction();
producer.beginTransaction();
assertThrows(IllegalStateException.class, () -> producer.beginTransaction());
}
@Test(expected = IllegalStateException.class)
@Test
public void shouldThrowOnSendOffsetsToTransactionIfTransactionsNotInitialized() {
buildMockProducer(true);
producer.sendOffsetsToTransaction(null, groupId);
assertThrows(IllegalStateException.class, () -> producer.sendOffsetsToTransaction(null, groupId));
}
@Test
public void shouldThrowOnSendOffsetsToTransactionTransactionIfNoTransactionGotStarted() {
buildMockProducer(true);
producer.initTransactions();
try {
producer.sendOffsetsToTransaction(null, groupId);
fail("Should have thrown as producer has no open transaction");
} catch (IllegalStateException e) { }
assertThrows(IllegalStateException.class, () -> producer.sendOffsetsToTransaction(null, groupId));
}
@Test(expected = IllegalStateException.class)
@Test
public void shouldThrowOnCommitIfTransactionsNotInitialized() {
buildMockProducer(true);
producer.commitTransaction();
assertThrows(IllegalStateException.class, producer::commitTransaction);
}
@Test
public void shouldThrowOnCommitTransactionIfNoTransactionGotStarted() {
buildMockProducer(true);
producer.initTransactions();
try {
producer.commitTransaction();
fail("Should have thrown as producer has no open transaction");
} catch (IllegalStateException e) { }
assertThrows(IllegalStateException.class, producer::commitTransaction);
}
@Test
@ -227,20 +218,17 @@ public class MockProducerTest {
assertThat(producer.commitCount(), equalTo(1L));
}
@Test(expected = IllegalStateException.class)
@Test
public void shouldThrowOnAbortIfTransactionsNotInitialized() {
buildMockProducer(true);
producer.abortTransaction();
assertThrows(IllegalStateException.class, () -> producer.abortTransaction());
}
@Test
public void shouldThrowOnAbortTransactionIfNoTransactionGotStarted() {
buildMockProducer(true);
producer.initTransactions();
try {
producer.abortTransaction();
fail("Should have thrown as producer has no open transaction");
} catch (IllegalStateException e) { }
assertThrows(IllegalStateException.class, producer::abortTransaction);
}
@Test
@ -254,10 +242,10 @@ public class MockProducerTest {
assertFalse(producer.transactionCommitted());
}
@Test(expected = IllegalStateException.class)
@Test
public void shouldThrowFenceProducerIfTransactionsNotInitialized() {
buildMockProducer(true);
producer.fenceProducer();
assertThrows(IllegalStateException.class, () -> producer.fenceProducer());
}
@Test
@ -265,10 +253,7 @@ public class MockProducerTest {
buildMockProducer(true);
producer.initTransactions();
producer.fenceProducer();
try {
producer.beginTransaction();
fail("Should have thrown as producer is fenced off");
} catch (ProducerFencedException e) { }
assertThrows(ProducerFencedException.class, producer::beginTransaction);
}
@Test
@ -276,12 +261,8 @@ public class MockProducerTest {
buildMockProducer(true);
producer.initTransactions();
producer.fenceProducer();
try {
producer.send(null);
fail("Should have thrown as producer is fenced off");
} catch (KafkaException e) {
assertTrue("The root cause of the exception should be ProducerFenced", e.getCause() instanceof ProducerFencedException);
}
Throwable e = assertThrows(KafkaException.class, () -> producer.send(null));
assertTrue("The root cause of the exception should be ProducerFenced", e.getCause() instanceof ProducerFencedException);
}
@Test
@ -289,10 +270,7 @@ public class MockProducerTest {
buildMockProducer(true);
producer.initTransactions();
producer.fenceProducer();
try {
producer.sendOffsetsToTransaction(null, groupId);
fail("Should have thrown as producer is fenced off");
} catch (ProducerFencedException e) { }
assertThrows(ProducerFencedException.class, () -> producer.sendOffsetsToTransaction(null, groupId));
}
@Test
@ -300,10 +278,7 @@ public class MockProducerTest {
buildMockProducer(true);
producer.initTransactions();
producer.fenceProducer();
try {
producer.sendOffsetsToTransaction(null, new ConsumerGroupMetadata(groupId));
fail("Should have thrown as producer is fenced off");
} catch (ProducerFencedException e) { }
assertThrows(ProducerFencedException.class, () -> producer.sendOffsetsToTransaction(null, new ConsumerGroupMetadata(groupId)));
}
@Test
@ -311,10 +286,7 @@ public class MockProducerTest {
buildMockProducer(true);
producer.initTransactions();
producer.fenceProducer();
try {
producer.commitTransaction();
fail("Should have thrown as producer is fenced off");
} catch (ProducerFencedException e) { }
assertThrows(ProducerFencedException.class, producer::commitTransaction);
}
@Test
@ -322,10 +294,7 @@ public class MockProducerTest {
buildMockProducer(true);
producer.initTransactions();
producer.fenceProducer();
try {
producer.abortTransaction();
fail("Should have thrown as producer is fenced off");
} catch (ProducerFencedException e) { }
assertThrows(ProducerFencedException.class, producer::abortTransaction);
}
@Test
@ -383,7 +352,7 @@ public class MockProducerTest {
}
@Test
public void shouldThrowOnAbortForNonAutoCompleteIfTransactionsAreEnabled() throws Exception {
public void shouldThrowOnAbortForNonAutoCompleteIfTransactionsAreEnabled() {
buildMockProducer(false);
producer.initTransactions();
producer.beginTransaction();
@ -453,12 +422,7 @@ public class MockProducerTest {
buildMockProducer(true);
producer.initTransactions();
producer.beginTransaction();
try {
String consumerGroupId = null;
producer.sendOffsetsToTransaction(Collections.emptyMap(), consumerGroupId);
fail("Should have thrown NullPointerException");
} catch (NullPointerException e) { }
assertThrows(NullPointerException.class, () -> producer.sendOffsetsToTransaction(Collections.emptyMap(), (String) null));
}
@Test
@ -466,11 +430,7 @@ public class MockProducerTest {
buildMockProducer(true);
producer.initTransactions();
producer.beginTransaction();
try {
producer.sendOffsetsToTransaction(Collections.emptyMap(), new ConsumerGroupMetadata(null));
fail("Should have thrown NullPointerException");
} catch (NullPointerException e) { }
assertThrows(NullPointerException.class, () -> producer.sendOffsetsToTransaction(Collections.emptyMap(), new ConsumerGroupMetadata(null)));
}
@Test
@ -478,7 +438,7 @@ public class MockProducerTest {
buildMockProducer(true);
producer.initTransactions();
producer.beginTransaction();
producer.sendOffsetsToTransaction(Collections.<TopicPartition, OffsetAndMetadata>emptyMap(), "groupId");
producer.sendOffsetsToTransaction(Collections.emptyMap(), "groupId");
assertFalse(producer.sentOffsets());
}
@ -669,7 +629,7 @@ public class MockProducerTest {
put(new TopicPartition(topic, 3), new OffsetAndMetadata(84L, null));
}
};
producer.sendOffsetsToTransaction(groupCommit, new ConsumerGroupMetadata(group2));
producer.sendOffsetsToTransaction(groupCommit2, new ConsumerGroupMetadata(group2));
producer.abortTransaction();
Map<String, Map<TopicPartition, OffsetAndMetadata>> expectedResult = new HashMap<>();
@ -682,96 +642,69 @@ public class MockProducerTest {
public void shouldThrowOnInitTransactionIfProducerIsClosed() {
buildMockProducer(true);
producer.close();
try {
producer.initTransactions();
fail("Should have thrown as producer is already closed");
} catch (IllegalStateException e) { }
assertThrows(IllegalStateException.class, producer::initTransactions);
}
@Test
public void shouldThrowOnSendIfProducerIsClosed() {
buildMockProducer(true);
producer.close();
try {
producer.send(null);
fail("Should have thrown as producer is already closed");
} catch (IllegalStateException e) { }
assertThrows(IllegalStateException.class, () -> producer.send(null));
}
@Test
public void shouldThrowOnBeginTransactionIfProducerIsClosed() {
buildMockProducer(true);
producer.close();
try {
producer.beginTransaction();
fail("Should have thrown as producer is already closed");
} catch (IllegalStateException e) { }
assertThrows(IllegalStateException.class, producer::beginTransaction);
}
@Test
public void shouldThrowSendOffsetsToTransactionByGroupIdIfProducerIsClosed() {
buildMockProducer(true);
producer.close();
try {
producer.sendOffsetsToTransaction(null, groupId);
fail("Should have thrown as producer is already closed");
} catch (IllegalStateException e) { }
assertThrows(IllegalStateException.class, () -> producer.sendOffsetsToTransaction(null, groupId));
}
@Test
public void shouldThrowSendOffsetsToTransactionByGroupMetadataIfProducerIsClosed() {
buildMockProducer(true);
producer.close();
try {
producer.sendOffsetsToTransaction(null, new ConsumerGroupMetadata(groupId));
fail("Should have thrown as producer is already closed");
} catch (IllegalStateException e) { }
assertThrows(IllegalStateException.class, () -> producer.sendOffsetsToTransaction(null, new ConsumerGroupMetadata(groupId)));
}
@Test
public void shouldThrowOnCommitTransactionIfProducerIsClosed() {
buildMockProducer(true);
producer.close();
try {
producer.commitTransaction();
fail("Should have thrown as producer is already closed");
} catch (IllegalStateException e) { }
assertThrows(IllegalStateException.class, producer::commitTransaction);
}
@Test
public void shouldThrowOnAbortTransactionIfProducerIsClosed() {
buildMockProducer(true);
producer.close();
try {
producer.abortTransaction();
fail("Should have thrown as producer is already closed");
} catch (IllegalStateException e) { }
assertThrows(IllegalStateException.class, producer::abortTransaction);
}
@Test
public void shouldThrowOnFenceProducerIfProducerIsClosed() {
buildMockProducer(true);
producer.close();
try {
producer.fenceProducer();
fail("Should have thrown as producer is already closed");
} catch (IllegalStateException e) { }
assertThrows(IllegalStateException.class, producer::fenceProducer);
}
@Test
public void shouldThrowOnFlushProducerIfProducerIsClosed() {
buildMockProducer(true);
producer.close();
try {
producer.flush();
fail("Should have thrown as producer is already closed");
} catch (IllegalStateException e) { }
assertThrows(IllegalStateException.class, producer::flush);
}
@Test
@SuppressWarnings("unchecked")
public void shouldThrowClassCastException() {
try (MockProducer<Integer, String> customProducer = new MockProducer<>(true, new IntegerSerializer(), new StringSerializer());) {
try (MockProducer<Integer, String> customProducer = new MockProducer<>(true, new IntegerSerializer(), new StringSerializer())) {
assertThrows(ClassCastException.class, () -> customProducer.send(new ProducerRecord(topic, "key1", "value1")));
}
}

View File

@ -18,6 +18,7 @@ package org.apache.kafka.clients.producer;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@ -64,11 +65,11 @@ public class RecordSendTest {
/**
* Test that an asynchronous request will eventually throw the right exception
*/
@Test(expected = ExecutionException.class)
@Test
public void testError() throws Exception {
FutureRecordMetadata future = new FutureRecordMetadata(asyncRequest(baseOffset, new CorruptRecordException(), 50L),
relOffset, RecordBatch.NO_TIMESTAMP, 0L, 0, 0, Time.SYSTEM);
future.get();
assertThrows(ExecutionException.class, future::get);
}
/**

View File

@ -91,13 +91,13 @@ public class BufferPoolTest {
/**
* Test that we cannot try to allocate more memory then we have in the whole pool
*/
@Test(expected = IllegalArgumentException.class)
@Test
public void testCantAllocateMoreMemoryThanWeHave() throws Exception {
BufferPool pool = new BufferPool(1024, 512, metrics, time, metricGroup);
ByteBuffer buffer = pool.allocate(1024, maxBlockTimeMs);
assertEquals(1024, buffer.limit());
pool.deallocate(buffer);
pool.allocate(1025, maxBlockTimeMs);
assertThrows(IllegalArgumentException.class, () -> pool.allocate(1025, maxBlockTimeMs));
}
/**
@ -155,11 +155,11 @@ public class BufferPoolTest {
* Test if BufferExhausted exception is thrown when there is not enough memory to allocate and the elapsed
* time is greater than the max specified block time.
*/
@Test(expected = BufferExhaustedException.class)
@Test
public void testBufferExhaustedExceptionIsThrown() throws Exception {
BufferPool pool = new BufferPool(2, 1, metrics, time, metricGroup);
pool.allocate(1, maxBlockTimeMs);
pool.allocate(2, maxBlockTimeMs);
assertThrows(BufferExhaustedException.class, () -> pool.allocate(2, maxBlockTimeMs));
}
/**

View File

@ -63,6 +63,7 @@ import static java.util.Arrays.asList;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@ -693,7 +694,7 @@ public class RecordAccumulatorTest {
assertTrue("The batch should have been drained.", drained.get(node1.id()).size() > 0);
}
@Test(expected = UnsupportedVersionException.class)
@Test
public void testIdempotenceWithOldMagic() throws InterruptedException {
// Simulate talking to an older broker, ie. one which supports a lower magic.
ApiVersions apiVersions = new ApiVersions();
@ -709,7 +710,8 @@ public class RecordAccumulatorTest {
RecordAccumulator accum = new RecordAccumulator(logContext, batchSize + DefaultRecordBatch.RECORD_BATCH_OVERHEAD,
CompressionType.NONE, lingerMs, retryBackoffMs, deliveryTimeoutMs, metrics, metricGrpName, time, apiVersions, transactionManager,
new BufferPool(totalSize, batchSize, metrics, time, metricGrpName));
accum.append(tp1, 0L, key, value, Record.EMPTY_HEADERS, null, 0, false, time.milliseconds());
assertThrows(UnsupportedVersionException.class,
() -> accum.append(tp1, 0L, key, value, Record.EMPTY_HEADERS, null, 0, false, time.milliseconds()));
}
@Test

View File

@ -213,9 +213,9 @@ public class TransactionManagerTest {
assertTrue(transactionManager.nextRequest(false).isEndTxn());
}
@Test(expected = IllegalStateException.class)
@Test
public void testFailIfNotReadyForSendNoProducerId() {
transactionManager.failIfNotReadyForSend();
assertThrows(IllegalStateException.class, () -> transactionManager.failIfNotReadyForSend());
}
@Test
@ -224,32 +224,32 @@ public class TransactionManagerTest {
transactionManager.failIfNotReadyForSend();
}
@Test(expected = KafkaException.class)
@Test
public void testFailIfNotReadyForSendIdempotentProducerFatalError() {
initializeTransactionManager(Optional.empty(), false);
transactionManager.transitionToFatalError(new KafkaException());
transactionManager.failIfNotReadyForSend();
assertThrows(KafkaException.class, () -> transactionManager.failIfNotReadyForSend());
}
@Test(expected = IllegalStateException.class)
@Test
public void testFailIfNotReadyForSendNoOngoingTransaction() {
doInitTransactions();
transactionManager.failIfNotReadyForSend();
assertThrows(IllegalStateException.class, () -> transactionManager.failIfNotReadyForSend());
}
@Test(expected = KafkaException.class)
@Test
public void testFailIfNotReadyForSendAfterAbortableError() {
doInitTransactions();
transactionManager.beginTransaction();
transactionManager.transitionToAbortableError(new KafkaException());
transactionManager.failIfNotReadyForSend();
assertThrows(KafkaException.class, transactionManager::failIfNotReadyForSend);
}
@Test(expected = KafkaException.class)
@Test
public void testFailIfNotReadyForSendAfterFatalError() {
doInitTransactions();
transactionManager.transitionToFatalError(new KafkaException());
transactionManager.failIfNotReadyForSend();
assertThrows(KafkaException.class, transactionManager::failIfNotReadyForSend);
}
@Test
@ -444,34 +444,30 @@ public class TransactionManagerTest {
assertEquals(DEFAULT_RETRY_BACKOFF_MS, handler.retryBackoffMs());
}
@Test(expected = IllegalStateException.class)
public void testMaybeAddPartitionToTransactionBeforeInitTransactions() {
transactionManager.failIfNotReadyForSend();
transactionManager.maybeAddPartitionToTransaction(new TopicPartition("foo", 0));
@Test
public void testNotReadyForSendBeforeInitTransactions() {
assertThrows(IllegalStateException.class, () -> transactionManager.failIfNotReadyForSend());
}
@Test(expected = IllegalStateException.class)
public void testMaybeAddPartitionToTransactionBeforeBeginTransaction() {
@Test
public void testNotReadyForSendBeforeBeginTransaction() {
doInitTransactions();
transactionManager.failIfNotReadyForSend();
transactionManager.maybeAddPartitionToTransaction(new TopicPartition("foo", 0));
assertThrows(IllegalStateException.class, () -> transactionManager.failIfNotReadyForSend());
}
@Test(expected = KafkaException.class)
public void testMaybeAddPartitionToTransactionAfterAbortableError() {
@Test
public void testNotReadyForSendAfterAbortableError() {
doInitTransactions();
transactionManager.beginTransaction();
transactionManager.transitionToAbortableError(new KafkaException());
transactionManager.failIfNotReadyForSend();
transactionManager.maybeAddPartitionToTransaction(new TopicPartition("foo", 0));
assertThrows(KafkaException.class, () -> transactionManager.failIfNotReadyForSend());
}
@Test(expected = KafkaException.class)
public void testMaybeAddPartitionToTransactionAfterFatalError() {
@Test
public void testNotReadyForSendAfterFatalError() {
doInitTransactions();
transactionManager.transitionToFatalError(new KafkaException());
transactionManager.failIfNotReadyForSend();
transactionManager.maybeAddPartitionToTransaction(new TopicPartition("foo", 0));
assertThrows(KafkaException.class, () -> transactionManager.failIfNotReadyForSend());
}
@Test
@ -3146,14 +3142,14 @@ public class TransactionManagerTest {
verifyCommitOrAbortTransactionRetriable(TransactionResult.COMMIT, TransactionResult.COMMIT);
}
@Test(expected = KafkaException.class)
public void testRetryAbortTransactionAfterCommitTimeout() throws InterruptedException {
verifyCommitOrAbortTransactionRetriable(TransactionResult.COMMIT, TransactionResult.ABORT);
@Test
public void testRetryAbortTransactionAfterCommitTimeout() {
assertThrows(KafkaException.class, () -> verifyCommitOrAbortTransactionRetriable(TransactionResult.COMMIT, TransactionResult.ABORT));
}
@Test(expected = KafkaException.class)
@Test
public void testRetryCommitTransactionAfterAbortTimeout() throws InterruptedException {
verifyCommitOrAbortTransactionRetriable(TransactionResult.ABORT, TransactionResult.COMMIT);
assertThrows(KafkaException.class, () -> verifyCommitOrAbortTransactionRetriable(TransactionResult.ABORT, TransactionResult.COMMIT));
}
@Test

View File

@ -28,6 +28,7 @@ import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertEquals;
@ -196,10 +197,10 @@ public class KafkaFutureTest {
allFuture.get();
}
@Test(expected = TimeoutException.class)
public void testFutureTimeoutWithZeroWait() throws Exception {
@Test
public void testFutureTimeoutWithZeroWait() {
final KafkaFutureImpl<String> future = new KafkaFutureImpl<>();
future.get(0, TimeUnit.MILLISECONDS);
assertThrows(TimeoutException.class, () -> future.get(0, TimeUnit.MILLISECONDS));
}
}

View File

@ -26,6 +26,7 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
public class AclBindingTest {
@ -124,18 +125,21 @@ public class AclBindingTest {
new AclBinding(new ResourcePattern(ResourceType.UNKNOWN, "foo", PatternType.LITERAL), ACL1.entry());
}
@Test(expected = IllegalArgumentException.class)
@Test
public void shouldThrowOnMatchPatternType() {
new AclBinding(new ResourcePattern(ResourceType.TOPIC, "foo", PatternType.MATCH), ACL1.entry());
assertThrows(IllegalArgumentException.class,
() -> new AclBinding(new ResourcePattern(ResourceType.TOPIC, "foo", PatternType.MATCH), ACL1.entry()));
}
@Test(expected = IllegalArgumentException.class)
@Test
public void shouldThrowOnAnyPatternType() {
new AclBinding(new ResourcePattern(ResourceType.TOPIC, "foo", PatternType.ANY), ACL1.entry());
assertThrows(IllegalArgumentException.class,
() -> new AclBinding(new ResourcePattern(ResourceType.TOPIC, "foo", PatternType.ANY), ACL1.entry()));
}
@Test(expected = IllegalArgumentException.class)
@Test
public void shouldThrowOnAnyResourceType() {
new AclBinding(new ResourcePattern(ResourceType.ANY, "foo", PatternType.LITERAL), ACL1.entry());
assertThrows(IllegalArgumentException.class,
() -> new AclBinding(new ResourcePattern(ResourceType.ANY, "foo", PatternType.LITERAL), ACL1.entry()));
}
}

View File

@ -22,24 +22,28 @@ import org.apache.kafka.common.resource.ResourcePattern;
import org.apache.kafka.common.resource.ResourceType;
import org.junit.Test;
import static org.junit.Assert.assertThrows;
public class ResourcePatternTest {
@Test(expected = IllegalArgumentException.class)
@Test
public void shouldThrowIfResourceTypeIsAny() {
new ResourcePattern(ResourceType.ANY, "name", PatternType.LITERAL);
assertThrows(IllegalArgumentException.class,
() -> new ResourcePattern(ResourceType.ANY, "name", PatternType.LITERAL));
}
@Test(expected = IllegalArgumentException.class)
@Test
public void shouldThrowIfPatternTypeIsMatch() {
new ResourcePattern(ResourceType.TOPIC, "name", PatternType.MATCH);
assertThrows(IllegalArgumentException.class, () -> new ResourcePattern(ResourceType.TOPIC, "name", PatternType.MATCH));
}
@Test(expected = IllegalArgumentException.class)
@Test
public void shouldThrowIfPatternTypeIsAny() {
new ResourcePattern(ResourceType.TOPIC, "name", PatternType.ANY);
assertThrows(IllegalArgumentException.class, () -> new ResourcePattern(ResourceType.TOPIC, "name", PatternType.ANY));
}
@Test(expected = NullPointerException.class)
@Test
public void shouldThrowIfResourceNameIsNull() {
new ResourcePattern(ResourceType.TOPIC, null, PatternType.ANY);
assertThrows(NullPointerException.class, () -> new ResourcePattern(ResourceType.TOPIC, null, PatternType.ANY));
}
}

View File

@ -41,6 +41,7 @@ import static java.util.Arrays.asList;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@ -84,9 +85,9 @@ public class ConfigDefTest {
assertEquals(Password.HIDDEN, vals.get("j").toString());
}
@Test(expected = ConfigException.class)
@Test
public void testInvalidDefault() {
new ConfigDef().define("a", Type.INT, "hello", Importance.HIGH, "docs");
assertThrows(ConfigException.class, () -> new ConfigDef().define("a", Type.INT, "hello", Importance.HIGH, "docs"));
}
@Test
@ -97,9 +98,9 @@ public class ConfigDefTest {
assertEquals(null, vals.get("a"));
}
@Test(expected = ConfigException.class)
@Test
public void testMissingRequired() {
new ConfigDef().define("a", Type.INT, Importance.HIGH, "docs").parse(new HashMap<String, Object>());
assertThrows(ConfigException.class, () -> new ConfigDef().define("a", Type.INT, Importance.HIGH, "docs").parse(new HashMap<String, Object>()));
}
@Test
@ -108,9 +109,10 @@ public class ConfigDefTest {
.parse(new HashMap<String, Object>());
}
@Test(expected = ConfigException.class)
@Test
public void testDefinedTwice() {
new ConfigDef().define("a", Type.STRING, Importance.HIGH, "docs").define("a", Type.INT, Importance.HIGH, "docs");
assertThrows(ConfigException.class, () -> new ConfigDef().define("a", Type.STRING,
Importance.HIGH, "docs").define("a", Type.INT, Importance.HIGH, "docs"));
}
@Test
@ -138,14 +140,16 @@ public class ConfigDefTest {
}
}
@Test(expected = ConfigException.class)
@Test
public void testInvalidDefaultRange() {
new ConfigDef().define("name", Type.INT, -1, Range.between(0, 10), Importance.HIGH, "docs");
assertThrows(ConfigException.class, () -> new ConfigDef().define("name", Type.INT, -1,
Range.between(0, 10), Importance.HIGH, "docs"));
}
@Test(expected = ConfigException.class)
@Test
public void testInvalidDefaultString() {
new ConfigDef().define("name", Type.STRING, "bad", ValidString.in("valid", "values"), Importance.HIGH, "docs");
assertThrows(ConfigException.class, () -> new ConfigDef().define("name", Type.STRING, "bad",
ValidString.in("valid", "values"), Importance.HIGH, "docs"));
}
@Test
@ -414,12 +418,12 @@ public class ConfigDefTest {
}
}
@Test(expected = ConfigException.class)
@Test
public void testMissingDependentConfigs() {
// Should not be possible to parse a config if a dependent config has not been defined
final ConfigDef configDef = new ConfigDef()
.define("parent", Type.STRING, Importance.HIGH, "parent docs", "group", 1, Width.LONG, "Parent", Collections.singletonList("child"));
configDef.parse(Collections.emptyMap());
assertThrows(ConfigException.class, () -> configDef.parse(Collections.emptyMap()));
}
@Test

View File

@ -17,6 +17,7 @@
package org.apache.kafka.common.config;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
import java.util.Collections;
import java.util.HashMap;
@ -66,59 +67,59 @@ public class SaslConfigsTest {
assertEquals(Short.valueOf("3600"), vals.get(SaslConfigs.SASL_LOGIN_REFRESH_BUFFER_SECONDS));
}
@Test(expected = ConfigException.class)
@Test
public void testSaslLoginRefreshWindowFactorMinValueIsReallyMinimum() {
Map<Object, Object> props = new HashMap<>();
props.put(SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_FACTOR, "0.499999");
new ConfigDef().withClientSaslSupport().parse(props);
assertThrows(ConfigException.class, () -> new ConfigDef().withClientSaslSupport().parse(props));
}
@Test(expected = ConfigException.class)
@Test
public void testSaslLoginRefreshWindowFactorMaxValueIsReallyMaximum() {
Map<Object, Object> props = new HashMap<>();
props.put(SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_FACTOR, "1.0001");
new ConfigDef().withClientSaslSupport().parse(props);
assertThrows(ConfigException.class, () -> new ConfigDef().withClientSaslSupport().parse(props));
}
@Test(expected = ConfigException.class)
@Test
public void testSaslLoginRefreshWindowJitterMinValueIsReallyMinimum() {
Map<Object, Object> props = new HashMap<>();
props.put(SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_JITTER, "-0.000001");
new ConfigDef().withClientSaslSupport().parse(props);
assertThrows(ConfigException.class, () -> new ConfigDef().withClientSaslSupport().parse(props));
}
@Test(expected = ConfigException.class)
@Test
public void testSaslLoginRefreshWindowJitterMaxValueIsReallyMaximum() {
Map<Object, Object> props = new HashMap<>();
props.put(SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_JITTER, "0.251");
new ConfigDef().withClientSaslSupport().parse(props);
assertThrows(ConfigException.class, () -> new ConfigDef().withClientSaslSupport().parse(props));
}
@Test(expected = ConfigException.class)
@Test
public void testSaslLoginRefreshMinPeriodSecondsMinValueIsReallyMinimum() {
Map<Object, Object> props = new HashMap<>();
props.put(SaslConfigs.SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS, "-1");
new ConfigDef().withClientSaslSupport().parse(props);
assertThrows(ConfigException.class, () -> new ConfigDef().withClientSaslSupport().parse(props));
}
@Test(expected = ConfigException.class)
@Test
public void testSaslLoginRefreshMinPeriodSecondsMaxValueIsReallyMaximum() {
Map<Object, Object> props = new HashMap<>();
props.put(SaslConfigs.SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS, "901");
new ConfigDef().withClientSaslSupport().parse(props);
assertThrows(ConfigException.class, () -> new ConfigDef().withClientSaslSupport().parse(props));
}
@Test(expected = ConfigException.class)
@Test
public void testSaslLoginRefreshBufferSecondsMinValueIsReallyMinimum() {
Map<Object, Object> props = new HashMap<>();
props.put(SaslConfigs.SASL_LOGIN_REFRESH_BUFFER_SECONDS, "-1");
new ConfigDef().withClientSaslSupport().parse(props);
assertThrows(ConfigException.class, () -> new ConfigDef().withClientSaslSupport().parse(props));
}
@Test(expected = ConfigException.class)
@Test
public void testSaslLoginRefreshBufferSecondsMaxValueIsReallyMaximum() {
Map<Object, Object> props = new HashMap<>();
props.put(SaslConfigs.SASL_LOGIN_REFRESH_BUFFER_SECONDS, "3601");
new ConfigDef().withClientSaslSupport().parse(props);
assertThrows(ConfigException.class, () -> new ConfigDef().withClientSaslSupport().parse(props));
}
}

View File

@ -20,72 +20,87 @@ import java.nio.ByteBuffer;
import java.util.concurrent.TimeUnit;
import org.apache.kafka.common.utils.Utils;
import org.junit.After;
import org.junit.Assert;
import org.junit.Test;
import static org.junit.Assert.assertThrows;
public class GarbageCollectedMemoryPoolTest {
@Test(expected = IllegalArgumentException.class)
public void testZeroSize() throws Exception {
new GarbageCollectedMemoryPool(0, 7, true, null);
}
private GarbageCollectedMemoryPool pool;
@Test(expected = IllegalArgumentException.class)
public void testNegativeSize() throws Exception {
new GarbageCollectedMemoryPool(-1, 7, false, null);
}
@Test(expected = IllegalArgumentException.class)
public void testZeroMaxAllocation() throws Exception {
new GarbageCollectedMemoryPool(100, 0, true, null);
}
@Test(expected = IllegalArgumentException.class)
public void testNegativeMaxAllocation() throws Exception {
new GarbageCollectedMemoryPool(100, -1, false, null);
}
@Test(expected = IllegalArgumentException.class)
public void testMaxAllocationLargerThanSize() throws Exception {
new GarbageCollectedMemoryPool(100, 101, true, null);
}
@Test(expected = IllegalArgumentException.class)
public void testAllocationOverMaxAllocation() throws Exception {
GarbageCollectedMemoryPool pool = new GarbageCollectedMemoryPool(1000, 10, false, null);
pool.tryAllocate(11);
}
@Test(expected = IllegalArgumentException.class)
public void testAllocationZero() throws Exception {
GarbageCollectedMemoryPool pool = new GarbageCollectedMemoryPool(1000, 10, true, null);
pool.tryAllocate(0);
}
@Test(expected = IllegalArgumentException.class)
public void testAllocationNegative() throws Exception {
GarbageCollectedMemoryPool pool = new GarbageCollectedMemoryPool(1000, 10, false, null);
pool.tryAllocate(-1);
}
@Test(expected = IllegalArgumentException.class)
public void testReleaseNull() throws Exception {
GarbageCollectedMemoryPool pool = new GarbageCollectedMemoryPool(1000, 10, true, null);
pool.release(null);
}
@Test(expected = IllegalArgumentException.class)
public void testReleaseForeignBuffer() throws Exception {
GarbageCollectedMemoryPool pool = new GarbageCollectedMemoryPool(1000, 10, true, null);
ByteBuffer fellOffATruck = ByteBuffer.allocate(1);
pool.release(fellOffATruck);
pool.close();
@After
public void releasePool() {
if (pool != null) pool.close();
}
@Test
public void testDoubleFree() throws Exception {
GarbageCollectedMemoryPool pool = new GarbageCollectedMemoryPool(1000, 10, false, null);
public void testZeroSize() {
assertThrows(IllegalArgumentException.class,
() -> new GarbageCollectedMemoryPool(0, 7, true, null));
}
@Test
public void testNegativeSize() {
assertThrows(IllegalArgumentException.class,
() -> new GarbageCollectedMemoryPool(-1, 7, false, null));
}
@Test
public void testZeroMaxAllocation() {
assertThrows(IllegalArgumentException.class,
() -> new GarbageCollectedMemoryPool(100, 0, true, null));
}
@Test
public void testNegativeMaxAllocation() {
assertThrows(IllegalArgumentException.class,
() -> new GarbageCollectedMemoryPool(100, -1, false, null));
}
@Test
public void testMaxAllocationLargerThanSize() {
assertThrows(IllegalArgumentException.class,
() -> new GarbageCollectedMemoryPool(100, 101, true, null));
}
@Test
public void testAllocationOverMaxAllocation() {
pool = new GarbageCollectedMemoryPool(1000, 10, false, null);
assertThrows(IllegalArgumentException.class, () -> pool.tryAllocate(11));
}
@Test
public void testAllocationZero() {
pool = new GarbageCollectedMemoryPool(1000, 10, true, null);
assertThrows(IllegalArgumentException.class, () -> pool.tryAllocate(0));
}
@Test
public void testAllocationNegative() {
pool = new GarbageCollectedMemoryPool(1000, 10, false, null);
assertThrows(IllegalArgumentException.class, () -> pool.tryAllocate(-1));
}
@Test
public void testReleaseNull() {
pool = new GarbageCollectedMemoryPool(1000, 10, true, null);
assertThrows(IllegalArgumentException.class, () -> pool.release(null));
}
@Test
public void testReleaseForeignBuffer() {
pool = new GarbageCollectedMemoryPool(1000, 10, true, null);
ByteBuffer fellOffATruck = ByteBuffer.allocate(1);
assertThrows(IllegalArgumentException.class, () -> pool.release(fellOffATruck));
}
@Test
public void testDoubleFree() {
pool = new GarbageCollectedMemoryPool(1000, 10, false, null);
ByteBuffer buffer = pool.tryAllocate(5);
Assert.assertNotNull(buffer);
pool.release(buffer); //so far so good
@ -100,8 +115,8 @@ public class GarbageCollectedMemoryPoolTest {
}
@Test
public void testAllocationBound() throws Exception {
GarbageCollectedMemoryPool pool = new GarbageCollectedMemoryPool(21, 10, false, null);
public void testAllocationBound() {
pool = new GarbageCollectedMemoryPool(21, 10, false, null);
ByteBuffer buf1 = pool.tryAllocate(10);
Assert.assertNotNull(buf1);
Assert.assertEquals(10, buf1.capacity());
@ -130,7 +145,7 @@ public class GarbageCollectedMemoryPoolTest {
long maxPool = maxHeap / 2;
long maxSingleAllocation = maxPool / 10;
Assert.assertTrue(maxSingleAllocation < Integer.MAX_VALUE / 2); //test JVM running with too much memory for this test logic (?)
GarbageCollectedMemoryPool pool = new GarbageCollectedMemoryPool(maxPool, (int) maxSingleAllocation, false, null);
pool = new GarbageCollectedMemoryPool(maxPool, (int) maxSingleAllocation, false, null);
//we will allocate 30 buffers from this pool, which is sized such that at-most
//11 should coexist and 30 do not fit in the JVM memory, proving that:

View File

@ -294,12 +294,13 @@ public class SimpleExampleMessageTest {
message -> assertEquals(myStruct, message.myStruct()), (short) 2);
}
@Test(expected = UnsupportedVersionException.class)
@Test
public void testMyStructUnsupportedVersion() {
SimpleExampleMessageData.MyStruct myStruct =
new SimpleExampleMessageData.MyStruct().setStructId(10);
// Check serialization throws exception for unsupported version
testRoundTrip(new SimpleExampleMessageData().setMyStruct(myStruct), (short) 1);
assertThrows(UnsupportedVersionException.class,
() -> testRoundTrip(new SimpleExampleMessageData().setMyStruct(myStruct), (short) 1));
}
/**

View File

@ -22,6 +22,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.assertThrows;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@ -196,12 +197,12 @@ public class MetricsTest {
assertNull(metrics.childrenSensors().get(grandchild));
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testBadSensorHierarchy() {
Sensor p = metrics.sensor("parent");
Sensor c1 = metrics.sensor("child1", p);
Sensor c2 = metrics.sensor("child2", p);
metrics.sensor("gc", c1, c2); // should fail
assertThrows(IllegalArgumentException.class, () -> metrics.sensor("gc", c1, c2));
}
@Test
@ -412,10 +413,11 @@ public class MetricsTest {
assertEquals(0.0, sampledTotal.measure(config, time.milliseconds()), EPS);
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testDuplicateMetricName() {
metrics.sensor("test").add(metrics.metricName("test", "grp1"), new Avg());
metrics.sensor("test2").add(metrics.metricName("test", "grp1"), new CumulativeSum());
assertThrows(IllegalArgumentException.class, () ->
metrics.sensor("test2").add(metrics.metricName("test", "grp1"), new CumulativeSum()));
}
@Test

View File

@ -33,6 +33,7 @@ import java.util.Arrays;
import java.util.Collections;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
public class FrequenciesTest {
@ -53,22 +54,22 @@ public class FrequenciesTest {
metrics.close();
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testFrequencyCenterValueAboveMax() {
new Frequencies(4, 1.0, 4.0,
freq("1", 1.0), freq("2", 20.0));
assertThrows(IllegalArgumentException.class,
() -> new Frequencies(4, 1.0, 4.0, freq("1", 1.0), freq("2", 20.0)));
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testFrequencyCenterValueBelowMin() {
new Frequencies(4, 1.0, 4.0,
freq("1", 1.0), freq("2", -20.0));
assertThrows(IllegalArgumentException.class,
() -> new Frequencies(4, 1.0, 4.0, freq("1", 1.0), freq("2", -20.0)));
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testMoreFrequencyParametersThanBuckets() {
new Frequencies(1, 1.0, 4.0,
freq("1", 1.0), freq("2", -20.0));
assertThrows(IllegalArgumentException.class,
() -> new Frequencies(1, 1.0, 4.0, freq("1", 1.0), freq("2", -20.0)));
}
@Test

View File

@ -177,18 +177,18 @@ public class SelectorTest {
/**
* Sending a request to a node without an existing connection should result in an exception
*/
@Test(expected = IllegalStateException.class)
public void testCantSendWithoutConnecting() throws Exception {
selector.send(createSend("0", "test"));
selector.poll(1000L);
@Test
public void testSendWithoutConnecting() {
assertThrows(IllegalStateException.class, () -> selector.send(createSend("0", "test")));
}
/**
* Sending a request to a node with a bad hostname should result in an exception during connect
*/
@Test(expected = IOException.class)
public void testNoRouteToHost() throws Exception {
selector.connect("0", new InetSocketAddress("some.invalid.hostname.foo.bar.local", server.port), BUFFER_SIZE, BUFFER_SIZE);
@Test
public void testNoRouteToHost() {
assertThrows(IOException.class,
() -> selector.connect("0", new InetSocketAddress("some.invalid.hostname.foo.bar.local", server.port), BUFFER_SIZE, BUFFER_SIZE));
}
/**
@ -375,10 +375,10 @@ public class SelectorTest {
}
}
@Test(expected = IllegalStateException.class)
@Test
public void testExistingConnectionId() throws IOException {
blockingConnect("0");
blockingConnect("0");
assertThrows(IllegalStateException.class, () -> blockingConnect("0"));
}
@Test

View File

@ -987,14 +987,14 @@ public class SslTransportLayerTest {
* Verifies that inter-broker listener with validation of truststore against keystore
* fails if certs from keystore are not trusted.
*/
@Test(expected = KafkaException.class)
@Test
public void testInterBrokerSslConfigValidationFailure() {
SecurityProtocol securityProtocol = SecurityProtocol.SSL;
sslServerConfigs.put(BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "required");
TestSecurityConfig config = new TestSecurityConfig(sslServerConfigs);
ListenerName listenerName = ListenerName.forSecurityProtocol(securityProtocol);
ChannelBuilders.serverChannelBuilder(listenerName, true, securityProtocol, config,
null, null, time, new LogContext());
assertThrows(KafkaException.class, () -> ChannelBuilders.serverChannelBuilder(listenerName, true, securityProtocol, config,
null, null, time, new LogContext()));
}
/**
@ -1205,10 +1205,10 @@ public class SslTransportLayerTest {
/**
* Tests invalid ssl.engine.factory plugin class
*/
@Test(expected = KafkaException.class)
@Test
public void testInvalidSslEngineFactory() {
sslClientConfigs.put(SslConfigs.SSL_ENGINE_FACTORY_CLASS_CONFIG, String.class);
createSelector(sslClientConfigs);
assertThrows(KafkaException.class, () -> createSelector(sslClientConfigs));
}
private void verifyInvalidReconfigure(ListenerReconfigurable reconfigurable,

View File

@ -25,18 +25,19 @@ import java.util.Set;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
public class ApiKeysTest {
@Test(expected = IllegalArgumentException.class)
@Test
public void testForIdWithInvalidIdLow() {
ApiKeys.forId(-1);
assertThrows(IllegalArgumentException.class, () -> ApiKeys.forId(-1));
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testForIdWithInvalidIdHigh() {
ApiKeys.forId(10000);
assertThrows(IllegalArgumentException.class, () -> ApiKeys.forId(10000));
}
@Test

View File

@ -25,6 +25,7 @@ import java.util.Arrays;
import java.util.List;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@ -85,7 +86,7 @@ public class AbstractLegacyRecordBatchTest {
}
}
@Test(expected = InvalidRecordException.class)
@Test
public void testInvalidWrapperOffsetV1() {
SimpleRecord[] simpleRecords = new SimpleRecord[] {
new SimpleRecord(1L, "a".getBytes(), "1".getBytes()),
@ -99,10 +100,10 @@ public class AbstractLegacyRecordBatchTest {
ByteBufferLegacyRecordBatch batch = new ByteBufferLegacyRecordBatch(records.buffer());
batch.setLastOffset(1L);
batch.iterator();
assertThrows(InvalidRecordException.class, batch::iterator);
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testSetNoTimestampTypeNotAllowed() {
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 0L,
CompressionType.GZIP, TimestampType.CREATE_TIME,
@ -110,10 +111,10 @@ public class AbstractLegacyRecordBatchTest {
new SimpleRecord(2L, "b".getBytes(), "2".getBytes()),
new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
ByteBufferLegacyRecordBatch batch = new ByteBufferLegacyRecordBatch(records.buffer());
batch.setMaxTimestamp(TimestampType.NO_TIMESTAMP_TYPE, RecordBatch.NO_TIMESTAMP);
assertThrows(IllegalArgumentException.class, () -> batch.setMaxTimestamp(TimestampType.NO_TIMESTAMP_TYPE, RecordBatch.NO_TIMESTAMP));
}
@Test(expected = UnsupportedOperationException.class)
@Test
public void testSetLogAppendTimeNotAllowedV0() {
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V0, 0L,
CompressionType.GZIP, TimestampType.CREATE_TIME,
@ -122,10 +123,10 @@ public class AbstractLegacyRecordBatchTest {
new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
long logAppendTime = 15L;
ByteBufferLegacyRecordBatch batch = new ByteBufferLegacyRecordBatch(records.buffer());
batch.setMaxTimestamp(TimestampType.LOG_APPEND_TIME, logAppendTime);
assertThrows(UnsupportedOperationException.class, () -> batch.setMaxTimestamp(TimestampType.LOG_APPEND_TIME, logAppendTime));
}
@Test(expected = UnsupportedOperationException.class)
@Test
public void testSetCreateTimeNotAllowedV0() {
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V0, 0L,
CompressionType.GZIP, TimestampType.CREATE_TIME,
@ -134,10 +135,10 @@ public class AbstractLegacyRecordBatchTest {
new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
long createTime = 15L;
ByteBufferLegacyRecordBatch batch = new ByteBufferLegacyRecordBatch(records.buffer());
batch.setMaxTimestamp(TimestampType.CREATE_TIME, createTime);
assertThrows(UnsupportedOperationException.class, () -> batch.setMaxTimestamp(TimestampType.CREATE_TIME, createTime));
}
@Test(expected = UnsupportedOperationException.class)
@Test
public void testSetPartitionLeaderEpochNotAllowedV0() {
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V0, 0L,
CompressionType.GZIP, TimestampType.CREATE_TIME,
@ -145,10 +146,10 @@ public class AbstractLegacyRecordBatchTest {
new SimpleRecord(2L, "b".getBytes(), "2".getBytes()),
new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
ByteBufferLegacyRecordBatch batch = new ByteBufferLegacyRecordBatch(records.buffer());
batch.setPartitionLeaderEpoch(15);
assertThrows(UnsupportedOperationException.class, () -> batch.setPartitionLeaderEpoch(15));
}
@Test(expected = UnsupportedOperationException.class)
@Test
public void testSetPartitionLeaderEpochNotAllowedV1() {
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 0L,
CompressionType.GZIP, TimestampType.CREATE_TIME,
@ -156,7 +157,7 @@ public class AbstractLegacyRecordBatchTest {
new SimpleRecord(2L, "b".getBytes(), "2".getBytes()),
new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
ByteBufferLegacyRecordBatch batch = new ByteBufferLegacyRecordBatch(records.buffer());
batch.setPartitionLeaderEpoch(15);
assertThrows(UnsupportedOperationException.class, () -> batch.setPartitionLeaderEpoch(15));
}
@Test

View File

@ -25,6 +25,7 @@ import java.util.Iterator;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
public class ByteBufferLogInputStreamTest {
@ -54,7 +55,7 @@ public class ByteBufferLogInputStreamTest {
assertFalse(iterator.hasNext());
}
@Test(expected = CorruptRecordException.class)
@Test
public void iteratorRaisesOnTooSmallRecords() {
ByteBuffer buffer = ByteBuffer.allocate(1024);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, CompressionType.NONE, TimestampType.CREATE_TIME, 0L);
@ -74,10 +75,10 @@ public class ByteBufferLogInputStreamTest {
ByteBufferLogInputStream logInputStream = new ByteBufferLogInputStream(buffer, Integer.MAX_VALUE);
assertNotNull(logInputStream.nextBatch());
logInputStream.nextBatch();
assertThrows(CorruptRecordException.class, logInputStream::nextBatch);
}
@Test(expected = CorruptRecordException.class)
@Test
public void iteratorRaisesOnInvalidMagic() {
ByteBuffer buffer = ByteBuffer.allocate(1024);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, CompressionType.NONE, TimestampType.CREATE_TIME, 0L);
@ -97,27 +98,25 @@ public class ByteBufferLogInputStreamTest {
ByteBufferLogInputStream logInputStream = new ByteBufferLogInputStream(buffer, Integer.MAX_VALUE);
assertNotNull(logInputStream.nextBatch());
logInputStream.nextBatch();
assertThrows(CorruptRecordException.class, logInputStream::nextBatch);
}
@Test(expected = CorruptRecordException.class)
@Test
public void iteratorRaisesOnTooLargeRecords() {
ByteBuffer buffer = ByteBuffer.allocate(1024);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, CompressionType.NONE, TimestampType.CREATE_TIME, 0L);
builder.append(15L, "a".getBytes(), "1".getBytes());
builder.append(20L, "b".getBytes(), "2".getBytes());
builder.close();
builder = MemoryRecords.builder(buffer, CompressionType.NONE, TimestampType.CREATE_TIME, 2L);
builder.append(30L, "c".getBytes(), "3".getBytes());
builder.append(40L, "d".getBytes(), "4".getBytes());
builder.close();
buffer.flip();
ByteBufferLogInputStream logInputStream = new ByteBufferLogInputStream(buffer, 25);
ByteBufferLogInputStream logInputStream = new ByteBufferLogInputStream(buffer, 60);
assertNotNull(logInputStream.nextBatch());
logInputStream.nextBatch();
assertThrows(CorruptRecordException.class, logInputStream::nextBatch);
}
}

View File

@ -18,17 +18,20 @@ package org.apache.kafka.common.record;
import org.apache.kafka.common.message.LeaderChangeMessage;
import org.apache.kafka.common.message.LeaderChangeMessage.Voter;
import org.junit.Assert;
import org.junit.Test;
import java.nio.ByteBuffer;
import java.util.Collections;
import static org.junit.Assert.assertThrows;
public class ControlRecordUtilsTest {
@Test
public void testInvalidControlRecordType() {
IllegalArgumentException thrown = Assert.assertThrows(
IllegalArgumentException thrown = assertThrows(
IllegalArgumentException.class, () -> testDeserializeRecord(ControlRecordType.COMMIT));
Assert.assertEquals("Expected LEADER_CHANGE control record type(3), but found COMMIT", thrown.getMessage());
}

View File

@ -32,6 +32,7 @@ import java.util.List;
import static org.apache.kafka.common.record.DefaultRecordBatch.RECORDS_COUNT_OFFSET;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
public class DefaultRecordBatchTest {
@ -175,7 +176,7 @@ public class DefaultRecordBatchTest {
assertEquals(actualSize, DefaultRecordBatch.sizeInBytes(Arrays.asList(records)));
}
@Test(expected = CorruptRecordException.class)
@Test
public void testInvalidRecordSize() {
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L,
CompressionType.NONE, TimestampType.CREATE_TIME,
@ -188,54 +189,46 @@ public class DefaultRecordBatchTest {
DefaultRecordBatch batch = new DefaultRecordBatch(buffer);
assertFalse(batch.isValid());
batch.ensureValid();
assertThrows(CorruptRecordException.class, batch::ensureValid);
}
@Test(expected = InvalidRecordException.class)
@Test
public void testInvalidRecordCountTooManyNonCompressedV2() {
long now = System.currentTimeMillis();
DefaultRecordBatch batch = recordsWithInvalidRecordCount(RecordBatch.MAGIC_VALUE_V2, now, CompressionType.NONE, 5);
// force iteration through the batch to execute validation
// batch validation is a part of normal workflow for LogValidator.validateMessagesAndAssignOffsets
for (Record record: batch) {
record.isValid();
}
assertThrows(InvalidRecordException.class, () -> batch.forEach(Record::isValid));
}
@Test(expected = InvalidRecordException.class)
@Test
public void testInvalidRecordCountTooLittleNonCompressedV2() {
long now = System.currentTimeMillis();
DefaultRecordBatch batch = recordsWithInvalidRecordCount(RecordBatch.MAGIC_VALUE_V2, now, CompressionType.NONE, 2);
// force iteration through the batch to execute validation
// batch validation is a part of normal workflow for LogValidator.validateMessagesAndAssignOffsets
for (Record record: batch) {
record.isValid();
}
assertThrows(InvalidRecordException.class, () -> batch.forEach(Record::isValid));
}
@Test(expected = InvalidRecordException.class)
@Test
public void testInvalidRecordCountTooManyCompressedV2() {
long now = System.currentTimeMillis();
DefaultRecordBatch batch = recordsWithInvalidRecordCount(RecordBatch.MAGIC_VALUE_V2, now, CompressionType.GZIP, 5);
// force iteration through the batch to execute validation
// batch validation is a part of normal workflow for LogValidator.validateMessagesAndAssignOffsets
for (Record record: batch) {
record.isValid();
}
assertThrows(InvalidRecordException.class, () -> batch.forEach(Record::isValid));
}
@Test(expected = InvalidRecordException.class)
@Test
public void testInvalidRecordCountTooLittleCompressedV2() {
long now = System.currentTimeMillis();
DefaultRecordBatch batch = recordsWithInvalidRecordCount(RecordBatch.MAGIC_VALUE_V2, now, CompressionType.GZIP, 2);
// force iteration through the batch to execute validation
// batch validation is a part of normal workflow for LogValidator.validateMessagesAndAssignOffsets
for (Record record: batch) {
record.isValid();
}
assertThrows(InvalidRecordException.class, () -> batch.forEach(Record::isValid));
}
@Test(expected = CorruptRecordException.class)
@Test
public void testInvalidCrc() {
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L,
CompressionType.NONE, TimestampType.CREATE_TIME,
@ -248,7 +241,7 @@ public class DefaultRecordBatchTest {
DefaultRecordBatch batch = new DefaultRecordBatch(buffer);
assertFalse(batch.isValid());
batch.ensureValid();
assertThrows(CorruptRecordException.class, batch::ensureValid);
}
@Test
@ -324,7 +317,7 @@ public class DefaultRecordBatchTest {
assertEquals(logAppendTime, record.timestamp());
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testSetNoTimestampTypeNotAllowed() {
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L,
CompressionType.NONE, TimestampType.CREATE_TIME,
@ -332,7 +325,7 @@ public class DefaultRecordBatchTest {
new SimpleRecord(2L, "b".getBytes(), "2".getBytes()),
new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
DefaultRecordBatch batch = new DefaultRecordBatch(records.buffer());
batch.setMaxTimestamp(TimestampType.NO_TIMESTAMP_TYPE, RecordBatch.NO_TIMESTAMP);
assertThrows(IllegalArgumentException.class, () -> batch.setMaxTimestamp(TimestampType.NO_TIMESTAMP_TYPE, RecordBatch.NO_TIMESTAMP));
}
@Test

View File

@ -33,6 +33,7 @@ import java.nio.ByteBuffer;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertThrows;
public class DefaultRecordTest {
@ -85,11 +86,11 @@ public class DefaultRecordTest {
}
}
@Test(expected = InvalidRecordException.class)
@Test
public void testBasicSerdeInvalidHeaderCountTooHigh() throws IOException {
Header[] headers = new Header[] {
new RecordHeader("foo", "value".getBytes()),
new RecordHeader("bar", (byte[]) null),
new RecordHeader("bar", null),
new RecordHeader("\"A\\u00ea\\u00f1\\u00fcC\"", "value".getBytes())
};
@ -107,18 +108,15 @@ public class DefaultRecordTest {
ByteBuffer buffer = out.buffer();
buffer.flip();
buffer.put(14, (byte) 8);
DefaultRecord logRecord = DefaultRecord.readFrom(buffer, baseOffset, baseTimestamp, baseSequence, null);
// force iteration through the record to validate the number of headers
assertEquals(DefaultRecord.sizeInBytes(offsetDelta, timestampDelta, record.key(), record.value(),
record.headers()), logRecord.sizeInBytes());
assertThrows(InvalidRecordException.class,
() -> DefaultRecord.readFrom(buffer, baseOffset, baseTimestamp, baseSequence, null));
}
@Test(expected = InvalidRecordException.class)
@Test
public void testBasicSerdeInvalidHeaderCountTooLow() throws IOException {
Header[] headers = new Header[] {
new RecordHeader("foo", "value".getBytes()),
new RecordHeader("bar", (byte[]) null),
new RecordHeader("bar", null),
new RecordHeader("\"A\\u00ea\\u00f1\\u00fcC\"", "value".getBytes())
};
@ -137,13 +135,11 @@ public class DefaultRecordTest {
buffer.flip();
buffer.put(14, (byte) 4);
DefaultRecord logRecord = DefaultRecord.readFrom(buffer, baseOffset, baseTimestamp, baseSequence, null);
// force iteration through the record to validate the number of headers
assertEquals(DefaultRecord.sizeInBytes(offsetDelta, timestampDelta, record.key(), record.value(),
record.headers()), logRecord.sizeInBytes());
assertThrows(InvalidRecordException.class,
() -> DefaultRecord.readFrom(buffer, baseOffset, baseTimestamp, baseSequence, null));
}
@Test(expected = InvalidRecordException.class)
@Test
public void testInvalidKeySize() {
byte attributes = 0;
long timestampDelta = 2;
@ -160,11 +156,12 @@ public class DefaultRecordTest {
buf.position(buf.limit());
buf.flip();
DefaultRecord.readFrom(buf, 0L, 0L, RecordBatch.NO_SEQUENCE, null);
assertThrows(InvalidRecordException.class,
() -> DefaultRecord.readFrom(buf, 0L, 0L, RecordBatch.NO_SEQUENCE, null));
}
@Test(expected = InvalidRecordException.class)
public void testInvalidKeySizePartial() throws IOException {
@Test
public void testInvalidKeySizePartial() {
byte attributes = 0;
long timestampDelta = 2;
int offsetDelta = 1;
@ -181,11 +178,12 @@ public class DefaultRecordTest {
buf.flip();
DataInputStream inputStream = new DataInputStream(new ByteBufferInputStream(buf));
DefaultRecord.readPartiallyFrom(inputStream, skipArray, 0L, 0L, RecordBatch.NO_SEQUENCE, null);
assertThrows(InvalidRecordException.class,
() -> DefaultRecord.readPartiallyFrom(inputStream, skipArray, 0L, 0L, RecordBatch.NO_SEQUENCE, null));
}
@Test(expected = InvalidRecordException.class)
public void testInvalidValueSize() throws IOException {
@Test
public void testInvalidValueSize() {
byte attributes = 0;
long timestampDelta = 2;
int offsetDelta = 1;
@ -202,10 +200,11 @@ public class DefaultRecordTest {
buf.position(buf.limit());
buf.flip();
DefaultRecord.readFrom(buf, 0L, 0L, RecordBatch.NO_SEQUENCE, null);
assertThrows(InvalidRecordException.class,
() -> DefaultRecord.readFrom(buf, 0L, 0L, RecordBatch.NO_SEQUENCE, null));
}
@Test(expected = InvalidRecordException.class)
@Test
public void testInvalidValueSizePartial() throws IOException {
byte attributes = 0;
long timestampDelta = 2;
@ -224,11 +223,12 @@ public class DefaultRecordTest {
buf.flip();
DataInputStream inputStream = new DataInputStream(new ByteBufferInputStream(buf));
DefaultRecord.readPartiallyFrom(inputStream, skipArray, 0L, 0L, RecordBatch.NO_SEQUENCE, null);
assertThrows(InvalidRecordException.class,
() -> DefaultRecord.readPartiallyFrom(inputStream, skipArray, 0L, 0L, RecordBatch.NO_SEQUENCE, null));
}
@Test(expected = InvalidRecordException.class)
public void testInvalidNumHeaders() throws IOException {
@Test
public void testInvalidNumHeaders() {
byte attributes = 0;
long timestampDelta = 2;
int offsetDelta = 1;
@ -245,11 +245,12 @@ public class DefaultRecordTest {
buf.position(buf.limit());
buf.flip();
DefaultRecord.readFrom(buf, 0L, 0L, RecordBatch.NO_SEQUENCE, null);
assertThrows(InvalidRecordException.class,
() -> DefaultRecord.readFrom(buf, 0L, 0L, RecordBatch.NO_SEQUENCE, null));
}
@Test(expected = InvalidRecordException.class)
public void testInvalidNumHeadersPartial() throws IOException {
@Test
public void testInvalidNumHeadersPartial() {
byte attributes = 0;
long timestampDelta = 2;
int offsetDelta = 1;
@ -267,10 +268,11 @@ public class DefaultRecordTest {
buf.flip();
DataInputStream inputStream = new DataInputStream(new ByteBufferInputStream(buf));
DefaultRecord.readPartiallyFrom(inputStream, skipArray, 0L, 0L, RecordBatch.NO_SEQUENCE, null);
assertThrows(InvalidRecordException.class,
() -> DefaultRecord.readPartiallyFrom(inputStream, skipArray, 0L, 0L, RecordBatch.NO_SEQUENCE, null));
}
@Test(expected = InvalidRecordException.class)
@Test
public void testInvalidHeaderKey() {
byte attributes = 0;
long timestampDelta = 2;
@ -289,11 +291,12 @@ public class DefaultRecordTest {
buf.position(buf.limit());
buf.flip();
DefaultRecord.readFrom(buf, 0L, 0L, RecordBatch.NO_SEQUENCE, null);
assertThrows(InvalidRecordException.class,
() -> DefaultRecord.readFrom(buf, 0L, 0L, RecordBatch.NO_SEQUENCE, null));
}
@Test(expected = InvalidRecordException.class)
public void testInvalidHeaderKeyPartial() throws IOException {
@Test
public void testInvalidHeaderKeyPartial() {
byte attributes = 0;
long timestampDelta = 2;
int offsetDelta = 1;
@ -312,10 +315,11 @@ public class DefaultRecordTest {
buf.flip();
DataInputStream inputStream = new DataInputStream(new ByteBufferInputStream(buf));
DefaultRecord.readPartiallyFrom(inputStream, skipArray, 0L, 0L, RecordBatch.NO_SEQUENCE, null);
assertThrows(InvalidRecordException.class,
() -> DefaultRecord.readPartiallyFrom(inputStream, skipArray, 0L, 0L, RecordBatch.NO_SEQUENCE, null));
}
@Test(expected = InvalidRecordException.class)
@Test
public void testNullHeaderKey() {
byte attributes = 0;
long timestampDelta = 2;
@ -334,11 +338,12 @@ public class DefaultRecordTest {
buf.position(buf.limit());
buf.flip();
DefaultRecord.readFrom(buf, 0L, 0L, RecordBatch.NO_SEQUENCE, null);
assertThrows(InvalidRecordException.class,
() -> DefaultRecord.readFrom(buf, 0L, 0L, RecordBatch.NO_SEQUENCE, null));
}
@Test(expected = InvalidRecordException.class)
public void testNullHeaderKeyPartial() throws IOException {
@Test
public void testNullHeaderKeyPartial() {
byte attributes = 0;
long timestampDelta = 2;
int offsetDelta = 1;
@ -357,10 +362,11 @@ public class DefaultRecordTest {
buf.flip();
DataInputStream inputStream = new DataInputStream(new ByteBufferInputStream(buf));
DefaultRecord.readPartiallyFrom(inputStream, skipArray, 0L, 0L, RecordBatch.NO_SEQUENCE, null);
assertThrows(InvalidRecordException.class,
() -> DefaultRecord.readPartiallyFrom(inputStream, skipArray, 0L, 0L, RecordBatch.NO_SEQUENCE, null));
}
@Test(expected = InvalidRecordException.class)
@Test
public void testInvalidHeaderValue() {
byte attributes = 0;
long timestampDelta = 2;
@ -381,11 +387,12 @@ public class DefaultRecordTest {
buf.position(buf.limit());
buf.flip();
DefaultRecord.readFrom(buf, 0L, 0L, RecordBatch.NO_SEQUENCE, null);
assertThrows(InvalidRecordException.class,
() -> DefaultRecord.readFrom(buf, 0L, 0L, RecordBatch.NO_SEQUENCE, null));
}
@Test(expected = InvalidRecordException.class)
public void testInvalidHeaderValuePartial() throws IOException {
@Test
public void testInvalidHeaderValuePartial() {
byte attributes = 0;
long timestampDelta = 2;
int offsetDelta = 1;
@ -406,10 +413,11 @@ public class DefaultRecordTest {
buf.flip();
DataInputStream inputStream = new DataInputStream(new ByteBufferInputStream(buf));
DefaultRecord.readPartiallyFrom(inputStream, skipArray, 0L, 0L, RecordBatch.NO_SEQUENCE, null);
assertThrows(InvalidRecordException.class,
() -> DefaultRecord.readPartiallyFrom(inputStream, skipArray, 0L, 0L, RecordBatch.NO_SEQUENCE, null));
}
@Test(expected = InvalidRecordException.class)
@Test
public void testUnderflowReadingTimestamp() {
byte attributes = 0;
int sizeOfBodyInBytes = 1;
@ -418,10 +426,11 @@ public class DefaultRecordTest {
buf.put(attributes);
buf.flip();
DefaultRecord.readFrom(buf, 0L, 0L, RecordBatch.NO_SEQUENCE, null);
assertThrows(InvalidRecordException.class,
() -> DefaultRecord.readFrom(buf, 0L, 0L, RecordBatch.NO_SEQUENCE, null));
}
@Test(expected = InvalidRecordException.class)
@Test
public void testUnderflowReadingVarlong() {
byte attributes = 0;
int sizeOfBodyInBytes = 2; // one byte for attributes, one byte for partial timestamp
@ -432,10 +441,11 @@ public class DefaultRecordTest {
buf.position(buf.limit() - 1);
buf.flip();
DefaultRecord.readFrom(buf, 0L, 0L, RecordBatch.NO_SEQUENCE, null);
assertThrows(InvalidRecordException.class,
() -> DefaultRecord.readFrom(buf, 0L, 0L, RecordBatch.NO_SEQUENCE, null));
}
@Test(expected = InvalidRecordException.class)
@Test
public void testInvalidVarlong() {
byte attributes = 0;
int sizeOfBodyInBytes = 11; // one byte for attributes, 10 bytes for max timestamp
@ -448,7 +458,8 @@ public class DefaultRecordTest {
buf.put(recordStartPosition + 10, Byte.MIN_VALUE); // use an invalid final byte
buf.flip();
DefaultRecord.readFrom(buf, 0L, 0L, RecordBatch.NO_SEQUENCE, null);
assertThrows(InvalidRecordException.class,
() -> DefaultRecord.readFrom(buf, 0L, 0L, RecordBatch.NO_SEQUENCE, null));
}
@Test

View File

@ -22,30 +22,34 @@ import org.junit.Test;
import java.nio.ByteBuffer;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
public class EndTransactionMarkerTest {
@Test(expected = IllegalArgumentException.class)
@Test
public void testUnknownControlTypeNotAllowed() {
new EndTransactionMarker(ControlRecordType.UNKNOWN, 24);
assertThrows(IllegalArgumentException.class,
() -> new EndTransactionMarker(ControlRecordType.UNKNOWN, 24));
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testCannotDeserializeUnknownControlType() {
EndTransactionMarker.deserializeValue(ControlRecordType.UNKNOWN, ByteBuffer.wrap(new byte[0]));
assertThrows(IllegalArgumentException.class,
() -> EndTransactionMarker.deserializeValue(ControlRecordType.UNKNOWN, ByteBuffer.wrap(new byte[0])));
}
@Test(expected = InvalidRecordException.class)
@Test
public void testIllegalNegativeVersion() {
ByteBuffer buffer = ByteBuffer.allocate(2);
buffer.putShort((short) -1);
buffer.flip();
EndTransactionMarker.deserializeValue(ControlRecordType.ABORT, buffer);
assertThrows(InvalidRecordException.class, () -> EndTransactionMarker.deserializeValue(ControlRecordType.ABORT, buffer));
}
@Test(expected = InvalidRecordException.class)
@Test
public void testNotEnoughBytes() {
EndTransactionMarker.deserializeValue(ControlRecordType.COMMIT, ByteBuffer.wrap(new byte[0]));
assertThrows(InvalidRecordException.class,
() -> EndTransactionMarker.deserializeValue(ControlRecordType.COMMIT, ByteBuffer.wrap(new byte[0])));
}
@Test

View File

@ -48,6 +48,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.assertThrows;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.ArgumentMatchers.anyLong;
@ -73,28 +74,29 @@ public class FileRecordsTest {
this.time = new MockTime();
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testAppendProtectsFromOverflow() throws Exception {
File fileMock = mock(File.class);
FileChannel fileChannelMock = mock(FileChannel.class);
when(fileChannelMock.size()).thenReturn((long) Integer.MAX_VALUE);
FileRecords records = new FileRecords(fileMock, fileChannelMock, 0, Integer.MAX_VALUE, false);
append(records, values);
assertThrows(IllegalArgumentException.class, () -> append(records, values));
}
@Test(expected = KafkaException.class)
@Test
public void testOpenOversizeFile() throws Exception {
File fileMock = mock(File.class);
FileChannel fileChannelMock = mock(FileChannel.class);
when(fileChannelMock.size()).thenReturn(Integer.MAX_VALUE + 5L);
new FileRecords(fileMock, fileChannelMock, 0, Integer.MAX_VALUE, false);
assertThrows(KafkaException.class, () -> new FileRecords(fileMock, fileChannelMock, 0, Integer.MAX_VALUE, false));
}
@Test(expected = IllegalArgumentException.class)
public void testOutOfRangeSlice() throws Exception {
this.fileRecords.slice(fileRecords.sizeInBytes() + 1, 15).sizeInBytes();
@Test
public void testOutOfRangeSlice() {
assertThrows(IllegalArgumentException.class,
() -> this.fileRecords.slice(fileRecords.sizeInBytes() + 1, 15).sizeInBytes());
}
/**

View File

@ -101,7 +101,7 @@ public class MemoryRecordsBuilderTest {
assertTrue(batches.get(0).isTransactional());
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testWriteTransactionalNotAllowedMagicV0() {
ByteBuffer buffer = ByteBuffer.allocate(128);
buffer.position(bufferOffset);
@ -110,11 +110,12 @@ public class MemoryRecordsBuilderTest {
short epoch = 15;
int sequence = 2342;
new MemoryRecordsBuilder(buffer, RecordBatch.MAGIC_VALUE_V0, compressionType, TimestampType.CREATE_TIME,
0L, 0L, pid, epoch, sequence, true, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
assertThrows(IllegalArgumentException.class, () -> new MemoryRecordsBuilder(buffer, RecordBatch.MAGIC_VALUE_V0,
compressionType, TimestampType.CREATE_TIME, 0L, 0L, pid, epoch, sequence,
true, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity()));
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testWriteTransactionalNotAllowedMagicV1() {
ByteBuffer buffer = ByteBuffer.allocate(128);
buffer.position(bufferOffset);
@ -123,11 +124,12 @@ public class MemoryRecordsBuilderTest {
short epoch = 15;
int sequence = 2342;
new MemoryRecordsBuilder(buffer, RecordBatch.MAGIC_VALUE_V1, compressionType, TimestampType.CREATE_TIME,
0L, 0L, pid, epoch, sequence, true, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
assertThrows(IllegalArgumentException.class, () -> new MemoryRecordsBuilder(buffer, RecordBatch.MAGIC_VALUE_V1,
compressionType, TimestampType.CREATE_TIME, 0L, 0L, pid, epoch, sequence,
true, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity()));
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testWriteControlBatchNotAllowedMagicV0() {
ByteBuffer buffer = ByteBuffer.allocate(128);
buffer.position(bufferOffset);
@ -136,11 +138,12 @@ public class MemoryRecordsBuilderTest {
short epoch = 15;
int sequence = 2342;
new MemoryRecordsBuilder(buffer, RecordBatch.MAGIC_VALUE_V0, compressionType, TimestampType.CREATE_TIME,
0L, 0L, pid, epoch, sequence, false, true, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
assertThrows(IllegalArgumentException.class, () -> new MemoryRecordsBuilder(buffer, RecordBatch.MAGIC_VALUE_V0,
compressionType, TimestampType.CREATE_TIME, 0L, 0L, pid, epoch, sequence,
false, true, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity()));
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testWriteControlBatchNotAllowedMagicV1() {
ByteBuffer buffer = ByteBuffer.allocate(128);
buffer.position(bufferOffset);
@ -149,11 +152,12 @@ public class MemoryRecordsBuilderTest {
short epoch = 15;
int sequence = 2342;
new MemoryRecordsBuilder(buffer, RecordBatch.MAGIC_VALUE_V1, compressionType, TimestampType.CREATE_TIME,
0L, 0L, pid, epoch, sequence, false, true, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
assertThrows(IllegalArgumentException.class, () -> new MemoryRecordsBuilder(buffer, RecordBatch.MAGIC_VALUE_V1,
compressionType, TimestampType.CREATE_TIME, 0L, 0L, pid, epoch, sequence,
false, true, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity()));
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testWriteTransactionalWithInvalidPID() {
ByteBuffer buffer = ByteBuffer.allocate(128);
buffer.position(bufferOffset);
@ -164,10 +168,10 @@ public class MemoryRecordsBuilderTest {
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType, TimestampType.CREATE_TIME,
0L, 0L, pid, epoch, sequence, true, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
builder.close();
assertThrows(IllegalArgumentException.class, builder::close);
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testWriteIdempotentWithInvalidEpoch() {
ByteBuffer buffer = ByteBuffer.allocate(128);
buffer.position(bufferOffset);
@ -178,10 +182,10 @@ public class MemoryRecordsBuilderTest {
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType, TimestampType.CREATE_TIME,
0L, 0L, pid, epoch, sequence, true, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
builder.close();
assertThrows(IllegalArgumentException.class, builder::close);
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testWriteIdempotentWithInvalidBaseSequence() {
ByteBuffer buffer = ByteBuffer.allocate(128);
buffer.position(bufferOffset);
@ -192,10 +196,10 @@ public class MemoryRecordsBuilderTest {
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType, TimestampType.CREATE_TIME,
0L, 0L, pid, epoch, sequence, true, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
builder.close();
assertThrows(IllegalArgumentException.class, builder::close);
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testWriteEndTxnMarkerNonTransactionalBatch() {
ByteBuffer buffer = ByteBuffer.allocate(128);
buffer.position(bufferOffset);
@ -206,10 +210,11 @@ public class MemoryRecordsBuilderTest {
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType, TimestampType.CREATE_TIME,
0L, 0L, pid, epoch, sequence, false, true, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
builder.appendEndTxnMarker(RecordBatch.NO_TIMESTAMP, new EndTransactionMarker(ControlRecordType.ABORT, 0));
assertThrows(IllegalArgumentException.class, () -> builder.appendEndTxnMarker(RecordBatch.NO_TIMESTAMP,
new EndTransactionMarker(ControlRecordType.ABORT, 0)));
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testWriteEndTxnMarkerNonControlBatch() {
ByteBuffer buffer = ByteBuffer.allocate(128);
buffer.position(bufferOffset);
@ -220,10 +225,11 @@ public class MemoryRecordsBuilderTest {
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType, TimestampType.CREATE_TIME,
0L, 0L, pid, epoch, sequence, true, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
builder.appendEndTxnMarker(RecordBatch.NO_TIMESTAMP, new EndTransactionMarker(ControlRecordType.ABORT, 0));
assertThrows(IllegalArgumentException.class, () -> builder.appendEndTxnMarker(RecordBatch.NO_TIMESTAMP,
new EndTransactionMarker(ControlRecordType.ABORT, 0)));
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testWriteLeaderChangeControlBatchWithoutLeaderEpoch() {
ByteBuffer buffer = ByteBuffer.allocate(128);
buffer.position(bufferOffset);
@ -233,10 +239,10 @@ public class MemoryRecordsBuilderTest {
0L, 0L,
RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE,
false, true, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
builder.appendLeaderChangeMessage(RecordBatch.NO_TIMESTAMP,
assertThrows(IllegalArgumentException.class, () -> builder.appendLeaderChangeMessage(RecordBatch.NO_TIMESTAMP,
new LeaderChangeMessage()
.setLeaderId(leaderId)
.setVoters(Collections.emptyList()));
.setVoters(Collections.emptyList())));
}
@Test
@ -504,20 +510,21 @@ public class MemoryRecordsBuilderTest {
}
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testAppendAtInvalidOffset() {
ByteBuffer buffer = ByteBuffer.allocate(1024);
buffer.position(bufferOffset);
long logAppendTime = System.currentTimeMillis();
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, RecordBatch.MAGIC_VALUE_V1, compressionType,
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, RecordBatch.MAGIC_VALUE_V2, compressionType,
TimestampType.CREATE_TIME, 0L, logAppendTime, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE,
false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
builder.appendWithOffset(0L, System.currentTimeMillis(), "a".getBytes(), null);
// offsets must increase monotonically
builder.appendWithOffset(0L, System.currentTimeMillis(), "b".getBytes(), null);
assertThrows(IllegalArgumentException.class, () -> builder.appendWithOffset(0L, System.currentTimeMillis(),
"b".getBytes(), null));
}
@Test

View File

@ -27,11 +27,11 @@ import java.io.OutputStream;
import java.nio.ByteBuffer;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.fail;
import static org.junit.Assert.assertThrows;
public class SimpleLegacyRecordTest {
@Test(expected = InvalidRecordException.class)
@Test
public void testCompressedIterationWithNullValue() throws Exception {
ByteBuffer buffer = ByteBuffer.allocate(128);
DataOutputStream out = new DataOutputStream(new ByteBufferOutputStream(buffer));
@ -40,13 +40,11 @@ public class SimpleLegacyRecordTest {
CompressionType.GZIP, TimestampType.CREATE_TIME);
buffer.flip();
MemoryRecords records = MemoryRecords.readableRecords(buffer);
if (records.records().iterator().hasNext())
fail("Iteration should have caused invalid record error");
assertThrows(InvalidRecordException.class, () -> records.records().iterator().hasNext());
}
@Test(expected = InvalidRecordException.class)
@Test
public void testCompressedIterationWithEmptyRecords() throws Exception {
ByteBuffer emptyCompressedValue = ByteBuffer.allocate(64);
OutputStream gzipOutput = CompressionType.GZIP.wrapForOutput(new ByteBufferOutputStream(emptyCompressedValue),
@ -63,27 +61,27 @@ public class SimpleLegacyRecordTest {
buffer.flip();
MemoryRecords records = MemoryRecords.readableRecords(buffer);
if (records.records().iterator().hasNext())
fail("Iteration should have caused invalid record error");
assertThrows(InvalidRecordException.class, () -> records.records().iterator().hasNext());
}
/* This scenario can happen if the record size field is corrupt and we end up allocating a buffer that is too small */
@Test(expected = CorruptRecordException.class)
@Test
public void testIsValidWithTooSmallBuffer() {
ByteBuffer buffer = ByteBuffer.allocate(2);
LegacyRecord record = new LegacyRecord(buffer);
assertFalse(record.isValid());
record.ensureValid();
assertThrows(CorruptRecordException.class, record::ensureValid);
}
@Test(expected = CorruptRecordException.class)
@Test
public void testIsValidWithChecksumMismatch() {
ByteBuffer buffer = ByteBuffer.allocate(4);
// set checksum
buffer.putInt(2);
LegacyRecord record = new LegacyRecord(buffer);
assertFalse(record.isValid());
record.ensureValid();
assertThrows(CorruptRecordException.class, record::ensureValid);
}
}

View File

@ -34,6 +34,7 @@ import java.util.List;
import java.util.stream.Collectors;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
public class CreateAclsRequestTest {
private static final short V0 = 0;
@ -51,14 +52,14 @@ public class CreateAclsRequestTest {
private static final AclBinding UNKNOWN_ACL1 = new AclBinding(new ResourcePattern(ResourceType.UNKNOWN, "unknown", PatternType.LITERAL),
new AccessControlEntry("User:*", "127.0.0.1", AclOperation.CREATE, AclPermissionType.ALLOW));
@Test(expected = UnsupportedVersionException.class)
@Test
public void shouldThrowOnV0IfNotLiteral() {
new CreateAclsRequest(data(PREFIXED_ACL1), V0);
assertThrows(UnsupportedVersionException.class, () -> new CreateAclsRequest(data(PREFIXED_ACL1), V0));
}
@Test(expected = IllegalArgumentException.class)
@Test
public void shouldThrowOnIfUnknown() {
new CreateAclsRequest(data(UNKNOWN_ACL1), V0);
assertThrows(IllegalArgumentException.class, () -> new CreateAclsRequest(data(UNKNOWN_ACL1), V0));
}
@Test

View File

@ -27,6 +27,7 @@ import java.util.HashMap;
import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
public class DeleteGroupsResponseTest {
@ -67,9 +68,9 @@ public class DeleteGroupsResponseTest {
assertEquals(expectedErrorCounts, deleteGroupsResponse.errorCounts());
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testGetErrorWithInvalidGroupId() {
deleteGroupsResponse.get("invalid-group-id");
assertThrows(IllegalArgumentException.class, () -> deleteGroupsResponse.get("invalid-group-id"));
}
@Test

View File

@ -39,6 +39,7 @@ import java.util.List;
import java.util.Set;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
public class DescribeAclsResponseTest {
private static final short V0 = 0;
@ -80,14 +81,16 @@ public class DescribeAclsResponseTest {
PatternType.LITERAL,
Collections.singletonList(DENY_READ_ACL));
@Test(expected = UnsupportedVersionException.class)
@Test
public void shouldThrowOnV0IfNotLiteral() {
buildResponse(10, Errors.NONE, Collections.singletonList(PREFIXED_ACL1)).serialize(V0);
assertThrows(UnsupportedVersionException.class,
() -> buildResponse(10, Errors.NONE, Collections.singletonList(PREFIXED_ACL1)).serialize(V0));
}
@Test(expected = IllegalArgumentException.class)
@Test
public void shouldThrowIfUnknown() {
buildResponse(10, Errors.NONE, Collections.singletonList(UNKNOWN_ACL)).serialize(V0);
assertThrows(IllegalArgumentException.class,
() -> buildResponse(10, Errors.NONE, Collections.singletonList(UNKNOWN_ACL)).serialize(V0));
}
@Test

View File

@ -20,15 +20,17 @@ import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.message.HeartbeatRequestData;
import org.junit.Test;
import static org.junit.Assert.assertThrows;
public class HeartbeatRequestTest {
@Test(expected = UnsupportedVersionException.class)
@Test
public void testRequestVersionCompatibilityFailBuild() {
new HeartbeatRequest.Builder(
new HeartbeatRequestData()
.setGroupId("groupId")
.setMemberId("consumerId")
.setGroupInstanceId("groupInstanceId")
).build((short) 2);
assertThrows(UnsupportedVersionException.class, () -> new HeartbeatRequest.Builder(
new HeartbeatRequestData()
.setGroupId("groupId")
.setMemberId("consumerId")
.setGroupInstanceId("groupInstanceId")
).build((short) 2));
}
}

View File

@ -28,6 +28,7 @@ import java.util.Arrays;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.fail;
public class JoinGroupRequestTest {
@ -68,15 +69,15 @@ public class JoinGroupRequestTest {
}
}
@Test(expected = UnsupportedVersionException.class)
@Test
public void testRequestVersionCompatibilityFailBuild() {
new JoinGroupRequest.Builder(
new JoinGroupRequestData()
.setGroupId("groupId")
.setMemberId("consumerId")
.setGroupInstanceId("groupInstanceId")
.setProtocolType("consumer")
).build((short) 4);
assertThrows(UnsupportedVersionException.class, () -> new JoinGroupRequest.Builder(
new JoinGroupRequestData()
.setGroupId("groupId")
.setMemberId("consumerId")
.setGroupInstanceId("groupInstanceId")
.setProtocolType("consumer")
).build((short) 4));
}
@Test

View File

@ -30,6 +30,7 @@ import java.util.Collections;
import java.util.List;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@ -121,8 +122,9 @@ public class LeaveGroupRequestTest {
}
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testBuildEmptyMembers() {
new LeaveGroupRequest.Builder(groupId, Collections.emptyList());
assertThrows(IllegalArgumentException.class,
() -> new LeaveGroupRequest.Builder(groupId, Collections.emptyList()));
}
}

View File

@ -617,13 +617,13 @@ public class RequestResponseTest {
}
}
@Test(expected = UnsupportedVersionException.class)
@Test
public void cannotUseFindCoordinatorV0ToFindTransactionCoordinator() {
FindCoordinatorRequest.Builder builder = new FindCoordinatorRequest.Builder(
new FindCoordinatorRequestData()
.setKeyType(CoordinatorType.TRANSACTION.id)
.setKey("foobar"));
builder.build((short) 0);
assertThrows(UnsupportedVersionException.class, () -> builder.build((short) 0));
}
@Test
@ -782,9 +782,10 @@ public class RequestResponseTest {
assertEquals(response.data().remainingPartitions(), deserialized.data().remainingPartitions());
}
@Test(expected = UnsupportedVersionException.class)
@Test
public void testCreateTopicRequestV0FailsIfValidateOnly() {
createCreateTopicRequest(0, true);
assertThrows(UnsupportedVersionException.class,
() -> createCreateTopicRequest(0, true));
}
@Test
@ -908,11 +909,11 @@ public class RequestResponseTest {
assertTrue(request.isValid());
}
@Test(expected = UnsupportedVersionException.class)
@Test
public void testListGroupRequestV3FailsWithStates() {
ListGroupsRequestData data = new ListGroupsRequestData()
.setStatesFilter(asList(ConsumerGroupState.STABLE.name()));
new ListGroupsRequest.Builder(data).build((short) 3);
assertThrows(UnsupportedVersionException.class, () -> new ListGroupsRequest.Builder(data).build((short) 3));
}
@Test

View File

@ -20,15 +20,17 @@ import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.message.SyncGroupRequestData;
import org.junit.Test;
import static org.junit.Assert.assertThrows;
public class SyncGroupRequestTest {
@Test(expected = UnsupportedVersionException.class)
@Test
public void testRequestVersionCompatibilityFailBuild() {
new SyncGroupRequest.Builder(
new SyncGroupRequestData()
.setGroupId("groupId")
.setMemberId("consumerId")
.setGroupInstanceId("groupInstanceId")
).build((short) 2);
assertThrows(UnsupportedVersionException.class, () -> new SyncGroupRequest.Builder(
new SyncGroupRequestData()
.setGroupId("groupId")
.setMemberId("consumerId")
.setGroupInstanceId("groupInstanceId")
).build((short) 2));
}
}

View File

@ -32,6 +32,7 @@ import javax.security.auth.login.Configuration;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.fail;
import org.apache.kafka.common.config.SaslConfigs;
@ -217,11 +218,11 @@ public class JaasContextTest {
Collections.emptyMap());
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testLoadForServerWithWrongListenerName() throws IOException {
writeConfiguration("Server", "test.LoginModule required;");
JaasContext.loadServerContext(new ListenerName("plaintext"), "SOME-MECHANISM",
Collections.emptyMap());
assertThrows(IllegalArgumentException.class, () -> JaasContext.loadServerContext(new ListenerName("plaintext"),
"SOME-MECHANISM", Collections.emptyMap()));
}
private AppConfigurationEntry configurationEntry(JaasContext.Type contextType, String jaasConfigProp) {

View File

@ -24,6 +24,7 @@ import java.util.HashMap;
import java.util.Map;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;
public class SaslExtensionsTest {
Map<String, String> map;
@ -35,10 +36,10 @@ public class SaslExtensionsTest {
this.map.put("who", "me");
}
@Test(expected = UnsupportedOperationException.class)
@Test
public void testReturnedMapIsImmutable() {
SaslExtensions extensions = new SaslExtensions(this.map);
extensions.map().put("hello", "test");
assertThrows(UnsupportedOperationException.class, () -> extensions.map().put("hello", "test"));
}
@Test

View File

@ -125,7 +125,6 @@ import org.apache.kafka.common.security.plain.internals.PlainServerCallbackHandl
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Utils;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
@ -1606,9 +1605,9 @@ public class SaslAuthenticatorTest {
ListOffsetsResponse response = new ListOffsetsResponse(data);
ByteBuffer buffer = RequestTestUtils.serializeResponseWithHeader(response, LIST_OFFSETS.latestVersion(), 0);
final RequestHeader header0 = new RequestHeader(LIST_OFFSETS, LIST_OFFSETS.latestVersion(), "id", SaslClientAuthenticator.MIN_RESERVED_CORRELATION_ID);
Assert.assertThrows(SchemaException.class, () -> NetworkClient.parseResponse(buffer.duplicate(), header0));
assertThrows(SchemaException.class, () -> NetworkClient.parseResponse(buffer.duplicate(), header0));
final RequestHeader header1 = new RequestHeader(LIST_OFFSETS, LIST_OFFSETS.latestVersion(), "id", 1);
Assert.assertThrows(IllegalStateException.class, () -> NetworkClient.parseResponse(buffer.duplicate(), header1));
assertThrows(IllegalStateException.class, () -> NetworkClient.parseResponse(buffer.duplicate(), header1));
}
/**

View File

@ -46,6 +46,7 @@ import org.mockito.Answers;
import static org.apache.kafka.common.security.scram.internals.ScramMechanism.SCRAM_SHA_256;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.fail;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.mock;
@ -55,7 +56,7 @@ import static org.mockito.Mockito.when;
public class SaslServerAuthenticatorTest {
@Test(expected = InvalidReceiveException.class)
@Test
public void testOversizeRequest() throws IOException {
TransportLayer transportLayer = mock(TransportLayer.class);
Map<String, ?> configs = Collections.singletonMap(BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG,
@ -67,7 +68,7 @@ public class SaslServerAuthenticatorTest {
invocation.<ByteBuffer>getArgument(0).putInt(SaslServerAuthenticator.MAX_RECEIVE_SIZE + 1);
return 4;
});
authenticator.authenticate();
assertThrows(InvalidReceiveException.class, authenticator::authenticate);
verify(transportLayer).read(any(ByteBuffer.class));
}

View File

@ -24,6 +24,7 @@ import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
public class OAuthBearerExtensionsValidatorCallbackTest {
@ -78,13 +79,13 @@ public class OAuthBearerExtensionsValidatorCallbackTest {
assertEquals("nothing", callback.ignoredExtensions().get("nothing"));
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testCannotValidateExtensionWhichWasNotGiven() {
Map<String, String> extensions = new HashMap<>();
extensions.put("hello", "bye");
OAuthBearerExtensionsValidatorCallback callback = new OAuthBearerExtensionsValidatorCallback(TOKEN, new SaslExtensions(extensions));
callback.valid("???");
assertThrows(IllegalArgumentException.class, () -> callback.valid("???"));
}
}

View File

@ -17,6 +17,7 @@
package org.apache.kafka.common.security.oauthbearer;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
import java.io.IOException;
import java.security.AccessController;
@ -61,18 +62,17 @@ public class OAuthBearerSaslClienCallbackHandlerTest {
};
}
@Test(expected = IOException.class)
public void testWithZeroTokens() throws Throwable {
@Test
public void testWithZeroTokens() {
OAuthBearerSaslClientCallbackHandler handler = createCallbackHandler();
try {
Subject.doAs(new Subject(), (PrivilegedExceptionAction<Void>) () -> {
PrivilegedActionException e = assertThrows(PrivilegedActionException.class, () -> Subject.doAs(new Subject(),
(PrivilegedExceptionAction<Void>) () -> {
OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback();
handler.handle(new Callback[] {callback});
return null;
});
} catch (PrivilegedActionException e) {
throw e.getCause();
}
}
));
assertEquals(IOException.class, e.getCause().getClass());
}
@Test()

View File

@ -17,6 +17,7 @@
package org.apache.kafka.common.security.oauthbearer.internals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
import org.apache.kafka.common.security.auth.SaslExtensions;
@ -55,11 +56,11 @@ public class OAuthBearerClientInitialResponseTest {
assertEquals(serverMessage, message);
}
@Test(expected = SaslException.class)
@Test
public void testThrowsSaslExceptionOnInvalidExtensionKey() throws Exception {
Map<String, String> extensions = new HashMap<>();
extensions.put("19", "42"); // keys can only be a-z
new OAuthBearerClientInitialResponse("123.345.567", new SaslExtensions(extensions));
assertThrows(SaslException.class, () -> new OAuthBearerClientInitialResponse("123.345.567", new SaslExtensions(extensions)));
}
@Test

View File

@ -18,6 +18,7 @@ package org.apache.kafka.common.security.oauthbearer.internals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertNull;
@ -154,8 +155,8 @@ public class OAuthBearerSaslServerTest {
* If the callback handler handles the `OAuthBearerExtensionsValidatorCallback`
* and finds an invalid extension, SaslServer should throw an authentication exception
*/
@Test(expected = SaslAuthenticationException.class)
public void throwsAuthenticationExceptionOnInvalidExtensions() throws Exception {
@Test
public void throwsAuthenticationExceptionOnInvalidExtensions() {
OAuthBearerUnsecuredValidatorCallbackHandler invalidHandler = new OAuthBearerUnsecuredValidatorCallbackHandler() {
@Override
public void handle(Callback[] callbacks) throws UnsupportedCallbackException {
@ -177,7 +178,8 @@ public class OAuthBearerSaslServerTest {
customExtensions.put("firstKey", "value");
customExtensions.put("secondKey", "value");
saslServer.evaluateResponse(clientInitialResponse(null, false, customExtensions));
assertThrows(SaslAuthenticationException.class,
() -> saslServer.evaluateResponse(clientInitialResponse(null, false, customExtensions)));
}
@Test
@ -187,9 +189,10 @@ public class OAuthBearerSaslServerTest {
assertTrue("Next challenge is not empty", nextChallenge.length == 0);
}
@Test(expected = SaslAuthenticationException.class)
public void authorizatonIdNotEqualsAuthenticationId() throws Exception {
saslServer.evaluateResponse(clientInitialResponse(USER + "x"));
@Test
public void authorizatonIdNotEqualsAuthenticationId() {
assertThrows(SaslAuthenticationException.class,
() -> saslServer.evaluateResponse(clientInitialResponse(USER + "x")));
}
@Test

View File

@ -17,6 +17,7 @@
package org.apache.kafka.common.security.oauthbearer.internals.unsecured;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
import java.nio.charset.StandardCharsets;
@ -81,24 +82,26 @@ public class OAuthBearerUnsecuredJwsTest {
assertEquals("", jws.splits().get(2));
}
@Test(expected = OAuthBearerIllegalTokenException.class)
@Test
public void missingPrincipal() {
String subject = null;
long issuedAt = 100;
Long expirationTime = null;
List<String> scope = Arrays.asList("scopeValue1", "scopeValue2");
String validCompactSerialization = compactSerialization(subject, issuedAt, expirationTime, scope);
new OAuthBearerUnsecuredJws(validCompactSerialization, "sub", "scope");
assertThrows(OAuthBearerIllegalTokenException.class,
() -> new OAuthBearerUnsecuredJws(validCompactSerialization, "sub", "scope"));
}
@Test(expected = OAuthBearerIllegalTokenException.class)
@Test
public void blankPrincipalName() {
String subject = " ";
long issuedAt = 100;
long expirationTime = issuedAt + 60 * 60;
List<String> scope = Arrays.asList("scopeValue1", "scopeValue2");
String validCompactSerialization = compactSerialization(subject, issuedAt, expirationTime, scope);
new OAuthBearerUnsecuredJws(validCompactSerialization, "sub", "scope");
assertThrows(OAuthBearerIllegalTokenException.class,
() -> new OAuthBearerUnsecuredJws(validCompactSerialization, "sub", "scope"));
}
private static String compactSerialization(String subject, Long issuedAt, Long expirationTime, List<String> scope) {

View File

@ -18,6 +18,7 @@ package org.apache.kafka.common.security.oauthbearer.internals.unsecured;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertThrows;
import java.io.IOException;
import java.util.Arrays;
@ -51,24 +52,24 @@ public class OAuthBearerUnsecuredLoginCallbackHandlerTest {
assertEquals("1", callback.extensions().map().get("testId"));
}
@Test(expected = IOException.class)
public void throwsErrorOnInvalidExtensionName() throws IOException, UnsupportedCallbackException {
@Test
public void throwsErrorOnInvalidExtensionName() {
Map<String, String> options = new HashMap<>();
options.put("unsecuredLoginExtension_test.Id", "1");
OAuthBearerUnsecuredLoginCallbackHandler callbackHandler = createCallbackHandler(options, new MockTime());
SaslExtensionsCallback callback = new SaslExtensionsCallback();
callbackHandler.handle(new Callback[] {callback});
assertThrows(IOException.class, () -> callbackHandler.handle(new Callback[] {callback}));
}
@Test(expected = IOException.class)
public void throwsErrorOnInvalidExtensionValue() throws IOException, UnsupportedCallbackException {
@Test
public void throwsErrorOnInvalidExtensionValue() {
Map<String, String> options = new HashMap<>();
options.put("unsecuredLoginExtension_testId", "Çalifornia");
OAuthBearerUnsecuredLoginCallbackHandler callbackHandler = createCallbackHandler(options, new MockTime());
SaslExtensionsCallback callback = new SaslExtensionsCallback();
callbackHandler.handle(new Callback[] {callback});
assertThrows(IOException.class, () -> callbackHandler.handle(new Callback[] {callback}));
}
@Test

View File

@ -65,9 +65,9 @@ public class PlainSaslServerTest {
assertEquals(0, nextChallenge.length);
}
@Test(expected = SaslAuthenticationException.class)
public void authorizatonIdNotEqualsAuthenticationId() throws Exception {
saslServer.evaluateResponse(saslMessage(USER_B, USER_A, PASSWORD_A));
@Test
public void authorizatonIdNotEqualsAuthenticationId() {
assertThrows(SaslAuthenticationException.class, () -> saslServer.evaluateResponse(saslMessage(USER_B, USER_A, PASSWORD_A)));
}
@Test

View File

@ -29,6 +29,7 @@ import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
@ -62,21 +63,21 @@ public class ScramCredentialUtilsTest {
assertNotEquals(ScramCredentialUtils.credentialToString(credential1), ScramCredentialUtils.credentialToString(credential2));
}
@Test(expected = IllegalArgumentException.class)
@Test
public void invalidCredential() {
ScramCredentialUtils.credentialFromString("abc");
assertThrows(IllegalArgumentException.class, () -> ScramCredentialUtils.credentialFromString("abc"));
}
@Test(expected = IllegalArgumentException.class)
@Test
public void missingFields() {
String cred = ScramCredentialUtils.credentialToString(formatter.generateCredential("password", 2048));
ScramCredentialUtils.credentialFromString(cred.substring(cred.indexOf(',')));
assertThrows(IllegalArgumentException.class, () -> ScramCredentialUtils.credentialFromString(cred.substring(cred.indexOf(','))));
}
@Test(expected = IllegalArgumentException.class)
@Test
public void extraneousFields() {
String cred = ScramCredentialUtils.credentialToString(formatter.generateCredential("password", 2048));
ScramCredentialUtils.credentialFromString(cred + ",a=test");
assertThrows(IllegalArgumentException.class, () -> ScramCredentialUtils.credentialFromString(cred + ",a=test"));
}
@Test

View File

@ -27,6 +27,8 @@ import org.apache.kafka.common.security.token.delegation.internals.DelegationTok
import org.junit.Before;
import org.junit.Test;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
public class ScramSaslServerTest {
@ -61,9 +63,9 @@ public class ScramSaslServerTest {
assertTrue("Next challenge is empty", nextChallenge.length > 0);
}
@Test(expected = SaslAuthenticationException.class)
public void authorizatonIdNotEqualsAuthenticationId() throws Exception {
saslServer.evaluateResponse(clientFirstMessage(USER_A, USER_B));
@Test
public void authorizatonIdNotEqualsAuthenticationId() {
assertThrows(SaslAuthenticationException.class, () -> saslServer.evaluateResponse(clientFirstMessage(USER_A, USER_B)));
}
private byte[] clientFirstMessage(String userName, String authorizationId) {

View File

@ -57,6 +57,7 @@ import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNotSame;
import static org.junit.Assert.assertSame;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.security.Security;
@ -111,7 +112,7 @@ public class SslFactoryTest {
Security.removeProvider(testProviderCreator.getProvider().getName());
}
@Test(expected = KafkaException.class)
@Test
public void testSslFactoryWithoutProviderClassConfiguration() {
// An exception is thrown as the algorithm is not registered through a provider
Map<String, Object> serverSslConfig = TestSslUtils.createSslConfig(
@ -120,10 +121,10 @@ public class SslFactoryTest {
tlsProtocol
);
SslFactory sslFactory = new SslFactory(Mode.SERVER);
sslFactory.configure(serverSslConfig);
assertThrows(KafkaException.class, () -> sslFactory.configure(serverSslConfig));
}
@Test(expected = KafkaException.class)
@Test
public void testSslFactoryWithIncorrectProviderClassConfiguration() {
// An exception is thrown as the algorithm is not registered through a provider
Map<String, Object> serverSslConfig = TestSslUtils.createSslConfig(
@ -134,7 +135,7 @@ public class SslFactoryTest {
serverSslConfig.put(SecurityConfig.SECURITY_PROVIDERS_CONFIG,
"com.fake.ProviderClass1,com.fake.ProviderClass2");
SslFactory sslFactory = new SslFactory(Mode.SERVER);
sslFactory.configure(serverSslConfig);
assertThrows(KafkaException.class, () -> sslFactory.configure(serverSslConfig));
}
@Test
@ -518,7 +519,7 @@ public class SslFactoryTest {
/**
* Tests invalid ssl.engine.factory configuration
*/
@Test(expected = ClassCastException.class)
@Test
public void testInvalidSslEngineFactory() throws Exception {
File trustStoreFile = File.createTempFile("truststore", ".jks");
Map<String, Object> clientSslConfig = sslConfigsBuilder(Mode.CLIENT)
@ -527,7 +528,7 @@ public class SslFactoryTest {
.build();
clientSslConfig.put(SslConfigs.SSL_ENGINE_FACTORY_CLASS_CONFIG, String.class);
SslFactory sslFactory = new SslFactory(Mode.CLIENT);
sslFactory.configure(clientSslConfig);
assertThrows(ClassCastException.class, () -> sslFactory.configure(clientSslConfig));
}
@Test

View File

@ -31,6 +31,7 @@ import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.core.IsNull.nullValue;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
public class SerializationTest {
@ -79,15 +80,15 @@ public class SerializationTest {
}
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testSerdeFromUnknown() {
Serdes.serdeFrom(DummyClass.class);
assertThrows(IllegalArgumentException.class, () -> Serdes.serdeFrom(DummyClass.class));
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testSerdeFromNotNull() {
try (Serde<Long> serde = Serdes.Long()) {
Serdes.serdeFrom(null, serde.deserializer());
assertThrows(IllegalArgumentException.class, () -> Serdes.serdeFrom(null, serde.deserializer()));
}
}
@ -107,25 +108,25 @@ public class SerializationTest {
}
}
@Test(expected = SerializationException.class)
@Test
public void floatDeserializerShouldThrowSerializationExceptionOnZeroBytes() {
try (Serde<Float> serde = Serdes.Float()) {
serde.deserializer().deserialize(topic, new byte[0]);
assertThrows(SerializationException.class, () -> serde.deserializer().deserialize(topic, new byte[0]));
}
}
@Test(expected = SerializationException.class)
@Test
public void floatDeserializerShouldThrowSerializationExceptionOnTooFewBytes() {
try (Serde<Float> serde = Serdes.Float()) {
serde.deserializer().deserialize(topic, new byte[3]);
assertThrows(SerializationException.class, () -> serde.deserializer().deserialize(topic, new byte[3]));
}
}
@Test(expected = SerializationException.class)
@Test
public void floatDeserializerShouldThrowSerializationExceptionOnTooManyBytes() {
try (Serde<Float> serde = Serdes.Float()) {
serde.deserializer().deserialize(topic, new byte[5]);
assertThrows(SerializationException.class, () -> serde.deserializer().deserialize(topic, new byte[5]));
}
}
@ -161,10 +162,10 @@ public class SerializationTest {
}
}
@Test(expected = IllegalArgumentException.class)
@Test
public void voidDeserializerShouldThrowOnNotNullValues() {
try (Serde<Void> serde = Serdes.Void()) {
serde.deserializer().deserialize(topic, new byte[5]);
assertThrows(IllegalArgumentException.class, () -> serde.deserializer().deserialize(topic, new byte[5]));
}
}

View File

@ -18,6 +18,7 @@ package org.apache.kafka.common.utils;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
import java.util.ArrayList;
@ -46,10 +47,10 @@ public class AbstractIteratorTest {
assertFalse(iter.hasNext());
}
@Test(expected = NoSuchElementException.class)
@Test
public void testEmptyIterator() {
Iterator<Object> iter = new ListIterator<Object>(Collections.emptyList());
iter.next();
Iterator<Object> iter = new ListIterator<>(Collections.emptyList());
assertThrows(NoSuchElementException.class, iter::next);
}
static class ListIterator<T> extends AbstractIterator<T> {

View File

@ -27,6 +27,7 @@ import java.nio.ByteBuffer;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
public class ByteUtilsTest {
private final byte x00 = 0x00;
@ -207,18 +208,18 @@ public class ByteUtilsTest {
assertVarlongSerde(Long.MIN_VALUE, new byte[] {xFF, xFF, xFF, xFF, xFF, xFF, xFF, xFF, xFF, x01});
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testInvalidVarint() {
// varint encoding has one overflow byte
ByteBuffer buf = ByteBuffer.wrap(new byte[] {xFF, xFF, xFF, xFF, xFF, x01});
ByteUtils.readVarint(buf);
assertThrows(IllegalArgumentException.class, () -> ByteUtils.readVarint(buf));
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testInvalidVarlong() {
// varlong encoding has one overflow byte
ByteBuffer buf = ByteBuffer.wrap(new byte[] {xFF, xFF, xFF, xFF, xFF, xFF, xFF, xFF, xFF, xFF, x01});
ByteUtils.readVarlong(buf);
assertThrows(IllegalArgumentException.class, () -> ByteUtils.readVarlong(buf));
}
@Test

View File

@ -36,6 +36,7 @@ import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
import static org.junit.Assert.assertThrows;
/**
* A unit test for ImplicitLinkedHashCollection.
@ -594,8 +595,7 @@ public class ImplicitLinkedHashCollectionTest {
assertTrue(coll.add(e3));
coll.moveToEnd(e1);
expectTraversal(coll.iterator(), 2, 3, 1);
Assert.assertThrows(RuntimeException.class, () ->
coll.moveToEnd(new TestElement(4, 4)));
assertThrows(RuntimeException.class, () -> coll.moveToEnd(new TestElement(4, 4)));
}
@Test

View File

@ -18,6 +18,7 @@ package org.apache.kafka.connect.connector;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.connect.errors.ConnectException;
import org.junit.Test;
import java.util.Collections;
@ -25,6 +26,7 @@ import java.util.List;
import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
public class ConnectorReconfigurationTest {
@ -36,10 +38,10 @@ public class ConnectorReconfigurationTest {
assertEquals(conn.configureOrder, 1);
}
@Test(expected = ConnectException.class)
@Test
public void testReconfigureStopException() {
TestConnector conn = new TestConnector(true);
conn.reconfigure(Collections.<String, String>emptyMap());
assertThrows(ConnectException.class, () -> conn.reconfigure(Collections.emptyMap()));
}
private static class TestConnector extends Connector {

View File

@ -31,6 +31,7 @@ import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;
public class ConnectSchemaTest {
private static final Schema MAP_INT_STRING_SCHEMA = SchemaBuilder.map(Schema.INT32_SCHEMA, Schema.STRING_SCHEMA).build();
@ -66,9 +67,10 @@ public class ConnectSchemaTest {
}
@Test(expected = DataException.class)
@Test
public void testFieldsOnlyValidForStructs() {
Schema.INT8_SCHEMA.fields();
assertThrows(DataException.class,
() -> Schema.INT8_SCHEMA.fields());
}
@Test
@ -109,128 +111,140 @@ public class ConnectSchemaTest {
// To avoid requiring excessive numbers of tests, these checks for invalid types use a similar type where possible
// to only include a single test for each type
@Test(expected = DataException.class)
@Test
public void testValidateValueMismatchInt8() {
ConnectSchema.validateValue(Schema.INT8_SCHEMA, 1);
assertThrows(DataException.class,
() -> ConnectSchema.validateValue(Schema.INT8_SCHEMA, 1));
}
@Test(expected = DataException.class)
@Test
public void testValidateValueMismatchInt16() {
ConnectSchema.validateValue(Schema.INT16_SCHEMA, 1);
assertThrows(DataException.class,
() -> ConnectSchema.validateValue(Schema.INT16_SCHEMA, 1));
}
@Test(expected = DataException.class)
@Test
public void testValidateValueMismatchInt32() {
ConnectSchema.validateValue(Schema.INT32_SCHEMA, (long) 1);
assertThrows(DataException.class,
() -> ConnectSchema.validateValue(Schema.INT32_SCHEMA, (long) 1));
}
@Test(expected = DataException.class)
@Test
public void testValidateValueMismatchInt64() {
ConnectSchema.validateValue(Schema.INT64_SCHEMA, 1);
assertThrows(DataException.class,
() -> ConnectSchema.validateValue(Schema.INT64_SCHEMA, 1));
}
@Test(expected = DataException.class)
@Test
public void testValidateValueMismatchFloat() {
ConnectSchema.validateValue(Schema.FLOAT32_SCHEMA, 1.0);
assertThrows(DataException.class,
() -> ConnectSchema.validateValue(Schema.FLOAT32_SCHEMA, 1.0));
}
@Test(expected = DataException.class)
@Test
public void testValidateValueMismatchDouble() {
ConnectSchema.validateValue(Schema.FLOAT64_SCHEMA, 1.f);
assertThrows(DataException.class,
() -> ConnectSchema.validateValue(Schema.FLOAT64_SCHEMA, 1.f));
}
@Test(expected = DataException.class)
@Test
public void testValidateValueMismatchBoolean() {
ConnectSchema.validateValue(Schema.BOOLEAN_SCHEMA, 1.f);
assertThrows(DataException.class,
() -> ConnectSchema.validateValue(Schema.BOOLEAN_SCHEMA, 1.f));
}
@Test(expected = DataException.class)
@Test
public void testValidateValueMismatchString() {
// CharSequence is a similar type (supertype of String), but we restrict to String.
CharBuffer cbuf = CharBuffer.wrap("abc");
ConnectSchema.validateValue(Schema.STRING_SCHEMA, cbuf);
assertThrows(DataException.class,
() -> ConnectSchema.validateValue(Schema.STRING_SCHEMA, cbuf));
}
@Test(expected = DataException.class)
@Test
public void testValidateValueMismatchBytes() {
ConnectSchema.validateValue(Schema.BYTES_SCHEMA, new Object[]{1, "foo"});
assertThrows(DataException.class,
() -> ConnectSchema.validateValue(Schema.BYTES_SCHEMA, new Object[]{1, "foo"}));
}
@Test(expected = DataException.class)
@Test
public void testValidateValueMismatchArray() {
ConnectSchema.validateValue(SchemaBuilder.array(Schema.INT32_SCHEMA).build(), Arrays.asList("a", "b", "c"));
assertThrows(DataException.class,
() -> ConnectSchema.validateValue(SchemaBuilder.array(Schema.INT32_SCHEMA).build(), Arrays.asList("a", "b", "c")));
}
@Test(expected = DataException.class)
@Test
public void testValidateValueMismatchArraySomeMatch() {
// Even if some match the right type, this should fail if any mismatch. In this case, type erasure loses
// the fact that the list is actually List<Object>, but we couldn't tell if only checking the first element
ConnectSchema.validateValue(SchemaBuilder.array(Schema.INT32_SCHEMA).build(), Arrays.asList(1, 2, "c"));
assertThrows(DataException.class,
() -> ConnectSchema.validateValue(SchemaBuilder.array(Schema.INT32_SCHEMA).build(), Arrays.asList(1, 2, "c")));
}
@Test(expected = DataException.class)
@Test
public void testValidateValueMismatchMapKey() {
ConnectSchema.validateValue(MAP_INT_STRING_SCHEMA, Collections.singletonMap("wrong key type", "value"));
assertThrows(DataException.class,
() -> ConnectSchema.validateValue(MAP_INT_STRING_SCHEMA, Collections.singletonMap("wrong key type", "value")));
}
@Test(expected = DataException.class)
@Test
public void testValidateValueMismatchMapValue() {
ConnectSchema.validateValue(MAP_INT_STRING_SCHEMA, Collections.singletonMap(1, 2));
assertThrows(DataException.class,
() -> ConnectSchema.validateValue(MAP_INT_STRING_SCHEMA, Collections.singletonMap(1, 2)));
}
@Test(expected = DataException.class)
@Test
public void testValidateValueMismatchMapSomeKeys() {
Map<Object, String> data = new HashMap<>();
data.put(1, "abc");
data.put("wrong", "it's as easy as one two three");
ConnectSchema.validateValue(MAP_INT_STRING_SCHEMA, data);
assertThrows(DataException.class,
() -> ConnectSchema.validateValue(MAP_INT_STRING_SCHEMA, data));
}
@Test(expected = DataException.class)
@Test
public void testValidateValueMismatchMapSomeValues() {
Map<Integer, Object> data = new HashMap<>();
data.put(1, "abc");
data.put(2, "wrong".getBytes());
ConnectSchema.validateValue(MAP_INT_STRING_SCHEMA, data);
assertThrows(DataException.class,
() -> ConnectSchema.validateValue(MAP_INT_STRING_SCHEMA, data));
}
@Test(expected = DataException.class)
@Test
public void testValidateValueMismatchStructWrongSchema() {
// Completely mismatching schemas
ConnectSchema.validateValue(
FLAT_STRUCT_SCHEMA,
new Struct(SchemaBuilder.struct().field("x", Schema.INT32_SCHEMA).build()).put("x", 1)
);
assertThrows(DataException.class, () -> ConnectSchema.validateValue(FLAT_STRUCT_SCHEMA,
new Struct(SchemaBuilder.struct().field("x", Schema.INT32_SCHEMA).build()).put("x", 1)));
}
@Test(expected = DataException.class)
@Test
public void testValidateValueMismatchStructWrongNestedSchema() {
// Top-level schema matches, but nested does not.
ConnectSchema.validateValue(
PARENT_STRUCT_SCHEMA,
new Struct(PARENT_STRUCT_SCHEMA)
.put("nested", new Struct(SchemaBuilder.struct().field("x", Schema.INT32_SCHEMA).build()).put("x", 1))
);
assertThrows(DataException.class, () -> ConnectSchema.validateValue(PARENT_STRUCT_SCHEMA,
new Struct(PARENT_STRUCT_SCHEMA)
.put("nested", new Struct(SchemaBuilder.struct()
.field("x", Schema.INT32_SCHEMA).build()).put("x", 1))));
}
@Test(expected = DataException.class)
@Test
public void testValidateValueMismatchDecimal() {
ConnectSchema.validateValue(Decimal.schema(2), new BigInteger("156"));
assertThrows(DataException.class, () -> ConnectSchema.validateValue(Decimal.schema(2), new BigInteger("156")));
}
@Test(expected = DataException.class)
@Test
public void testValidateValueMismatchDate() {
ConnectSchema.validateValue(Date.SCHEMA, 1000L);
assertThrows(DataException.class, () -> ConnectSchema.validateValue(Date.SCHEMA, 1000L));
}
@Test(expected = DataException.class)
@Test
public void testValidateValueMismatchTime() {
ConnectSchema.validateValue(Time.SCHEMA, 1000L);
assertThrows(DataException.class, () -> ConnectSchema.validateValue(Time.SCHEMA, 1000L));
}
@Test(expected = DataException.class)
@Test
public void testValidateValueMismatchTimestamp() {
ConnectSchema.validateValue(Timestamp.SCHEMA, 1000L);
assertThrows(DataException.class, () -> ConnectSchema.validateValue(Timestamp.SCHEMA, 1000L));
}
@Test

View File

@ -24,6 +24,7 @@ import java.util.GregorianCalendar;
import java.util.TimeZone;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
public class DateTest {
private static final GregorianCalendar EPOCH;
@ -54,14 +55,16 @@ public class DateTest {
assertEquals(10000, Date.fromLogical(Date.SCHEMA, EPOCH_PLUS_TEN_THOUSAND_DAYS.getTime()));
}
@Test(expected = DataException.class)
@Test
public void testFromLogicalInvalidSchema() {
Date.fromLogical(Date.builder().name("invalid").build(), EPOCH.getTime());
assertThrows(DataException.class,
() -> Date.fromLogical(Date.builder().name("invalid").build(), EPOCH.getTime()));
}
@Test(expected = DataException.class)
@Test
public void testFromLogicalInvalidHasTimeComponents() {
Date.fromLogical(Date.SCHEMA, EPOCH_PLUS_TIME_COMPONENT.getTime());
assertThrows(DataException.class,
() -> Date.fromLogical(Date.SCHEMA, EPOCH_PLUS_TIME_COMPONENT.getTime()));
}
@Test
@ -70,8 +73,9 @@ public class DateTest {
assertEquals(EPOCH_PLUS_TEN_THOUSAND_DAYS.getTime(), Date.toLogical(Date.SCHEMA, 10000));
}
@Test(expected = DataException.class)
@Test
public void testToLogicalInvalidSchema() {
Date.toLogical(Date.builder().name("invalid").build(), 0);
assertThrows(DataException.class,
() -> Date.toLogical(Date.builder().name("invalid").build(), 0));
}
}

View File

@ -28,6 +28,7 @@ import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;
public class SchemaBuilderTest {
private static final String NAME = "name";
@ -47,9 +48,9 @@ public class SchemaBuilderTest {
assertMetadata(schema, NAME, VERSION, DOC, NO_PARAMS);
}
@Test(expected = SchemaBuilderException.class)
@Test
public void testInt8BuilderInvalidDefault() {
SchemaBuilder.int8().defaultValue("invalid");
assertThrows(SchemaBuilderException.class, () -> SchemaBuilder.int8().defaultValue("invalid"));
}
@Test
@ -64,9 +65,9 @@ public class SchemaBuilderTest {
assertMetadata(schema, NAME, VERSION, DOC, NO_PARAMS);
}
@Test(expected = SchemaBuilderException.class)
@Test
public void testInt16BuilderInvalidDefault() {
SchemaBuilder.int16().defaultValue("invalid");
assertThrows(SchemaBuilderException.class, () -> SchemaBuilder.int16().defaultValue("invalid"));
}
@Test
@ -81,9 +82,9 @@ public class SchemaBuilderTest {
assertMetadata(schema, NAME, VERSION, DOC, NO_PARAMS);
}
@Test(expected = SchemaBuilderException.class)
@Test
public void testInt32BuilderInvalidDefault() {
SchemaBuilder.int32().defaultValue("invalid");
assertThrows(SchemaBuilderException.class, () -> SchemaBuilder.int32().defaultValue("invalid"));
}
@Test
@ -98,9 +99,9 @@ public class SchemaBuilderTest {
assertMetadata(schema, NAME, VERSION, DOC, NO_PARAMS);
}
@Test(expected = SchemaBuilderException.class)
@Test
public void testInt64BuilderInvalidDefault() {
SchemaBuilder.int64().defaultValue("invalid");
assertThrows(SchemaBuilderException.class, () -> SchemaBuilder.int64().defaultValue("invalid"));
}
@Test
@ -115,9 +116,9 @@ public class SchemaBuilderTest {
assertMetadata(schema, NAME, VERSION, DOC, NO_PARAMS);
}
@Test(expected = SchemaBuilderException.class)
@Test
public void testFloatBuilderInvalidDefault() {
SchemaBuilder.float32().defaultValue("invalid");
assertThrows(SchemaBuilderException.class, () -> SchemaBuilder.float32().defaultValue("invalid"));
}
@Test
@ -132,9 +133,9 @@ public class SchemaBuilderTest {
assertMetadata(schema, NAME, VERSION, DOC, NO_PARAMS);
}
@Test(expected = SchemaBuilderException.class)
@Test
public void testDoubleBuilderInvalidDefault() {
SchemaBuilder.float64().defaultValue("invalid");
assertThrows(SchemaBuilderException.class, () -> SchemaBuilder.float64().defaultValue("invalid"));
}
@Test
@ -149,9 +150,9 @@ public class SchemaBuilderTest {
assertMetadata(schema, NAME, VERSION, DOC, NO_PARAMS);
}
@Test(expected = SchemaBuilderException.class)
@Test
public void testBooleanBuilderInvalidDefault() {
SchemaBuilder.bool().defaultValue("invalid");
assertThrows(SchemaBuilderException.class, () -> SchemaBuilder.bool().defaultValue("invalid"));
}
@Test
@ -166,9 +167,9 @@ public class SchemaBuilderTest {
assertMetadata(schema, NAME, VERSION, DOC, NO_PARAMS);
}
@Test(expected = SchemaBuilderException.class)
@Test
public void testStringBuilderInvalidDefault() {
SchemaBuilder.string().defaultValue(true);
assertThrows(SchemaBuilderException.class, () -> SchemaBuilder.string().defaultValue(true));
}
@Test
@ -183,9 +184,9 @@ public class SchemaBuilderTest {
assertMetadata(schema, NAME, VERSION, DOC, NO_PARAMS);
}
@Test(expected = SchemaBuilderException.class)
@Test
public void testBytesBuilderInvalidDefault() {
SchemaBuilder.bytes().defaultValue("a string, not bytes");
assertThrows(SchemaBuilderException.class, () -> SchemaBuilder.bytes().defaultValue("a string, not bytes"));
}
@ -222,9 +223,9 @@ public class SchemaBuilderTest {
assertNoMetadata(schema);
}
@Test(expected = SchemaBuilderException.class)
@Test
public void testNonStructCantHaveFields() {
SchemaBuilder.int8().field("field", SchemaBuilder.int8().build());
assertThrows(SchemaBuilderException.class, () -> SchemaBuilder.int8().field("field", SchemaBuilder.int8().build()));
}
@ -243,10 +244,11 @@ public class SchemaBuilderTest {
assertNoMetadata(schema);
}
@Test(expected = SchemaBuilderException.class)
@Test
public void testArrayBuilderInvalidDefault() {
// Array, but wrong embedded type
SchemaBuilder.array(Schema.INT8_SCHEMA).defaultValue(Arrays.asList("string")).build();
assertThrows(SchemaBuilderException.class,
() -> SchemaBuilder.array(Schema.INT8_SCHEMA).defaultValue(Collections.singletonList("string")).build());
}
@Test
@ -274,12 +276,12 @@ public class SchemaBuilderTest {
assertNoMetadata(schema);
}
@Test(expected = SchemaBuilderException.class)
@Test
public void testMapBuilderInvalidDefault() {
// Map, but wrong embedded type
Map<Byte, String> defMap = Collections.singletonMap((byte) 5, "foo");
SchemaBuilder.map(Schema.INT8_SCHEMA, Schema.INT8_SCHEMA)
.defaultValue(defMap).build();
assertThrows(SchemaBuilderException.class, () -> SchemaBuilder.map(Schema.INT8_SCHEMA, Schema.INT8_SCHEMA)
.defaultValue(defMap).build());
}
@Test
@ -293,16 +295,13 @@ public class SchemaBuilderTest {
new Struct(emptyStructSchema);
}
@Test(expected = SchemaBuilderException.class)
@Test
public void testDuplicateFields() {
final Schema schema = SchemaBuilder.struct()
.name("testing")
.field("id", SchemaBuilder.string().doc("").build())
.field("id", SchemaBuilder.string().doc("").build())
.build();
final Struct struct = new Struct(schema)
.put("id", "testing");
struct.validate();
assertThrows(SchemaBuilderException.class, () -> SchemaBuilder.struct()
.name("testing")
.field("id", SchemaBuilder.string().doc("").build())
.field("id", SchemaBuilder.string().doc("").build())
.build());
}
@Test
@ -315,49 +314,44 @@ public class SchemaBuilderTest {
assertEquals("testing", schemaBuilder.name());
}
@Test(expected = SchemaBuilderException.class)
@Test
public void testDefaultFieldsDifferentValueOverwriting() {
final SchemaBuilder schemaBuilder = SchemaBuilder.string().name("testing").version(123);
schemaBuilder.name("testing");
schemaBuilder.version(456);
assertThrows(SchemaBuilderException.class, () -> schemaBuilder.version(456));
}
@Test(expected = SchemaBuilderException.class)
@Test
public void testFieldNameNull() {
Schema schema = SchemaBuilder.struct()
.field(null, Schema.STRING_SCHEMA)
.build();
assertThrows(SchemaBuilderException.class,
() -> SchemaBuilder.struct().field(null, Schema.STRING_SCHEMA).build());
}
@Test(expected = SchemaBuilderException.class)
@Test
public void testFieldSchemaNull() {
Schema schema = SchemaBuilder.struct()
.field("fieldName", null)
.build();
assertThrows(SchemaBuilderException.class,
() -> SchemaBuilder.struct().field("fieldName", null).build());
}
@Test(expected = SchemaBuilderException.class)
@Test
public void testArraySchemaNull() {
Schema schema = SchemaBuilder.array(null)
.build();
assertThrows(SchemaBuilderException.class, () -> SchemaBuilder.array(null).build());
}
@Test(expected = SchemaBuilderException.class)
@Test
public void testMapKeySchemaNull() {
Schema schema = SchemaBuilder.map(null, Schema.STRING_SCHEMA)
.build();
assertThrows(SchemaBuilderException.class, () -> SchemaBuilder.map(null, Schema.STRING_SCHEMA).build());
}
@Test(expected = SchemaBuilderException.class)
@Test
public void testMapValueSchemaNull() {
Schema schema = SchemaBuilder.map(Schema.STRING_SCHEMA, null)
.build();
assertThrows(SchemaBuilderException.class, () -> SchemaBuilder.map(Schema.STRING_SCHEMA, null).build());
}
@Test(expected = SchemaBuilderException.class)
@Test
public void testTypeNotNull() {
SchemaBuilder.type(null);
assertThrows(SchemaBuilderException.class, () -> SchemaBuilder.type(null));
}
private void assertTypeAndDefault(Schema schema, Schema.Type type, boolean optional, Object defaultValue) {

View File

@ -30,6 +30,7 @@ import java.util.List;
import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.fail;
public class SchemaProjectorTest {
@ -495,11 +496,11 @@ public class SchemaProjectorTest {
assertEquals(null, ((Struct) SchemaProjector.project(source, new Struct(source), target)).getInt64("id"));
}
@Test(expected = SchemaProjectorException.class)
@Test
public void testProjectMissingRequiredField() {
final Schema source = SchemaBuilder.struct().build();
final Schema target = SchemaBuilder.struct().field("id", SchemaBuilder.INT64_SCHEMA).build();
SchemaProjector.project(source, new Struct(source), target);
assertThrows(SchemaProjectorException.class, () -> SchemaProjector.project(source, new Struct(source), target));
}
private void verifyOptionalProjection(Schema source, Type targetType, Object value, Object defaultValue, Object expectedProjected, boolean optional) {

View File

@ -17,6 +17,7 @@
package org.apache.kafka.connect.data;
import org.apache.kafka.connect.errors.DataException;
import org.junit.Test;
import java.nio.ByteBuffer;
@ -114,38 +115,43 @@ public class StructTest {
// tests in SchemaTest. These are meant to ensure that we are invoking the same code path and that we do deeper
// inspection than just checking the class of the object
@Test(expected = DataException.class)
@Test
public void testInvalidFieldType() {
new Struct(FLAT_STRUCT_SCHEMA).put("int8", "should fail because this is a string, not int8");
assertThrows(DataException.class,
() -> new Struct(FLAT_STRUCT_SCHEMA).put("int8", "should fail because this is a string, not int8"));
}
@Test(expected = DataException.class)
@Test
public void testInvalidArrayFieldElements() {
new Struct(NESTED_SCHEMA).put("array", Arrays.asList("should fail since elements should be int8s"));
assertThrows(DataException.class,
() -> new Struct(NESTED_SCHEMA).put("array", Collections.singletonList("should fail since elements should be int8s")));
}
@Test(expected = DataException.class)
@Test
public void testInvalidMapKeyElements() {
new Struct(NESTED_SCHEMA).put("map", Collections.singletonMap("should fail because keys should be int8s", (byte) 12));
assertThrows(DataException.class,
() -> new Struct(NESTED_SCHEMA).put("map", Collections.singletonMap("should fail because keys should be int8s", (byte) 12)));
}
@Test(expected = DataException.class)
@Test
public void testInvalidStructFieldSchema() {
new Struct(NESTED_SCHEMA).put("nested", new Struct(MAP_SCHEMA));
assertThrows(DataException.class,
() -> new Struct(NESTED_SCHEMA).put("nested", new Struct(MAP_SCHEMA)));
}
@Test(expected = DataException.class)
@Test
public void testInvalidStructFieldValue() {
new Struct(NESTED_SCHEMA).put("nested", new Struct(NESTED_CHILD_SCHEMA));
assertThrows(DataException.class,
() -> new Struct(NESTED_SCHEMA).put("nested", new Struct(NESTED_CHILD_SCHEMA)));
}
@Test(expected = DataException.class)
@Test
public void testMissingFieldValidation() {
// Required int8 field
Schema schema = SchemaBuilder.struct().field("field", REQUIRED_FIELD_SCHEMA).build();
Struct struct = new Struct(schema);
struct.validate();
assertThrows(DataException.class, struct::validate);
}
@Test

View File

@ -24,6 +24,7 @@ import java.util.GregorianCalendar;
import java.util.TimeZone;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
public class TimeTest {
private static final GregorianCalendar EPOCH;
@ -56,14 +57,16 @@ public class TimeTest {
assertEquals(10000, Time.fromLogical(Time.SCHEMA, EPOCH_PLUS_TEN_THOUSAND_MILLIS.getTime()));
}
@Test(expected = DataException.class)
@Test
public void testFromLogicalInvalidSchema() {
Time.fromLogical(Time.builder().name("invalid").build(), EPOCH.getTime());
assertThrows(DataException.class,
() -> Time.fromLogical(Time.builder().name("invalid").build(), EPOCH.getTime()));
}
@Test(expected = DataException.class)
@Test
public void testFromLogicalInvalidHasDateComponents() {
Time.fromLogical(Time.SCHEMA, EPOCH_PLUS_DATE_COMPONENT.getTime());
assertThrows(DataException.class,
() -> Time.fromLogical(Time.SCHEMA, EPOCH_PLUS_DATE_COMPONENT.getTime()));
}
@Test
@ -72,8 +75,9 @@ public class TimeTest {
assertEquals(EPOCH_PLUS_TEN_THOUSAND_MILLIS.getTime(), Time.toLogical(Time.SCHEMA, 10000));
}
@Test(expected = DataException.class)
@Test
public void testToLogicalInvalidSchema() {
Time.toLogical(Time.builder().name("invalid").build(), 0);
assertThrows(DataException.class,
() -> Time.toLogical(Time.builder().name("invalid").build(), 0));
}
}

View File

@ -24,6 +24,7 @@ import java.util.GregorianCalendar;
import java.util.TimeZone;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
public class TimestampTest {
private static final GregorianCalendar EPOCH;
@ -56,9 +57,10 @@ public class TimestampTest {
assertEquals(TOTAL_MILLIS, Timestamp.fromLogical(Timestamp.SCHEMA, EPOCH_PLUS_MILLIS.getTime()));
}
@Test(expected = DataException.class)
@Test
public void testFromLogicalInvalidSchema() {
Timestamp.fromLogical(Timestamp.builder().name("invalid").build(), EPOCH.getTime());
assertThrows(DataException.class,
() -> Timestamp.fromLogical(Timestamp.builder().name("invalid").build(), EPOCH.getTime()));
}
@Test
@ -67,8 +69,9 @@ public class TimestampTest {
assertEquals(EPOCH_PLUS_MILLIS.getTime(), Timestamp.toLogical(Timestamp.SCHEMA, TOTAL_MILLIS));
}
@Test(expected = DataException.class)
@Test
public void testToLogicalInvalidSchema() {
Date.toLogical(Date.builder().name("invalid").build(), 0);
assertThrows(DataException.class,
() -> Date.toLogical(Date.builder().name("invalid").build(), 0));
}
}

View File

@ -37,6 +37,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.assertThrows;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@ -257,9 +258,9 @@ public class ValuesTest {
assertEquals(Boolean.TRUE, resultTrue.value());
}
@Test(expected = DataException.class)
@Test
public void shouldFailToParseInvalidBooleanValueString() {
Values.convertToBoolean(Schema.STRING_SCHEMA, "\"green\"");
assertThrows(DataException.class, () -> Values.convertToBoolean(Schema.STRING_SCHEMA, "\"green\""));
}
@Test
@ -542,50 +543,53 @@ public class ValuesTest {
/**
* This is technically invalid JSON, and we don't want to simply ignore the blank elements.
*/
@Test(expected = DataException.class)
@Test
public void shouldFailToConvertToListFromStringWithExtraDelimiters() {
Values.convertToList(Schema.STRING_SCHEMA, "[1, 2, 3,,,]");
assertThrows(DataException.class, () -> Values.convertToList(Schema.STRING_SCHEMA, "[1, 2, 3,,,]"));
}
/**
* Schema of type ARRAY requires a schema for the values, but Connect has no union or "any" schema type.
* Therefore, we can't represent this.
*/
@Test(expected = DataException.class)
@Test
public void shouldFailToConvertToListFromStringWithNonCommonElementTypeAndBlankElement() {
Values.convertToList(Schema.STRING_SCHEMA, "[1, 2, 3, \"four\",,,]");
assertThrows(DataException.class, () -> Values.convertToList(Schema.STRING_SCHEMA, "[1, 2, 3, \"four\",,,]"));
}
/**
* This is technically invalid JSON, and we don't want to simply ignore the blank entry.
*/
@Test(expected = DataException.class)
@Test
public void shouldFailToParseStringOfMapWithIntValuesWithBlankEntry() {
Values.convertToMap(Schema.STRING_SCHEMA, " { \"foo\" : 1234567890 ,, \"bar\" : 0, \"baz\" : -987654321 } ");
assertThrows(DataException.class,
() -> Values.convertToMap(Schema.STRING_SCHEMA, " { \"foo\" : 1234567890 ,, \"bar\" : 0, \"baz\" : -987654321 } "));
}
/**
* This is technically invalid JSON, and we don't want to simply ignore the malformed entry.
*/
@Test(expected = DataException.class)
@Test
public void shouldFailToParseStringOfMalformedMap() {
Values.convertToMap(Schema.STRING_SCHEMA, " { \"foo\" : 1234567890 , \"a\", \"bar\" : 0, \"baz\" : -987654321 } ");
assertThrows(DataException.class,
() -> Values.convertToMap(Schema.STRING_SCHEMA, " { \"foo\" : 1234567890 , \"a\", \"bar\" : 0, \"baz\" : -987654321 } "));
}
/**
* This is technically invalid JSON, and we don't want to simply ignore the blank entries.
*/
@Test(expected = DataException.class)
@Test
public void shouldFailToParseStringOfMapWithIntValuesWithOnlyBlankEntries() {
Values.convertToMap(Schema.STRING_SCHEMA, " { ,, , , } ");
assertThrows(DataException.class, () -> Values.convertToMap(Schema.STRING_SCHEMA, " { ,, , , } "));
}
/**
* This is technically invalid JSON, and we don't want to simply ignore the blank entry.
*/
@Test(expected = DataException.class)
@Test
public void shouldFailToParseStringOfMapWithIntValuesWithBlankEntries() {
Values.convertToMap(Schema.STRING_SCHEMA, " { \"foo\" : \"1234567890\" ,, \"bar\" : \"0\", \"baz\" : \"boz\" } ");
assertThrows(DataException.class,
() -> Values.convertToMap(Schema.STRING_SCHEMA, " { \"foo\" : \"1234567890\" ,, \"bar\" : \"0\", \"baz\" : \"boz\" } "));
}
@Test

View File

@ -79,9 +79,10 @@ public class ConnectHeadersTest {
other = "other key";
}
@Test(expected = NullPointerException.class)
@Test
public void shouldNotAllowNullKey() {
headers.add(null, "value", Schema.STRING_SCHEMA);
assertThrows(NullPointerException.class,
() -> headers.add(null, "value", Schema.STRING_SCHEMA));
}
protected void populate(Headers headers) {

View File

@ -23,6 +23,7 @@ import java.util.Collections;
import java.util.List;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
public class ConnectorUtilsTest {
@ -59,8 +60,9 @@ public class ConnectorUtilsTest {
Collections.emptyList()), grouped);
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testGroupPartitionsInvalidCount() {
ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 0);
assertThrows(IllegalArgumentException.class,
() -> ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 0));
}
}

View File

@ -22,6 +22,7 @@ import javax.security.auth.callback.CallbackHandler;
import javax.security.auth.callback.ChoiceCallback;
import javax.ws.rs.HttpMethod;
import javax.ws.rs.core.UriInfo;
import org.apache.kafka.common.security.JaasUtils;
import org.apache.kafka.connect.errors.ConnectException;
import org.easymock.EasyMock;
@ -46,6 +47,7 @@ import javax.security.auth.login.Configuration;
import javax.ws.rs.container.ContainerRequestContext;
import javax.ws.rs.core.Response;
import static org.junit.Assert.assertThrows;
import static org.powermock.api.easymock.PowerMock.replayAll;
@RunWith(PowerMockRunner.class)
@ -173,8 +175,8 @@ public class JaasBasicAuthFilterTest {
EasyMock.verify(requestContext);
}
@Test(expected = ConnectException.class)
public void testUnsupportedCallback() throws Exception {
@Test
public void testUnsupportedCallback() {
String authHeader = authHeader("basic", "user", "pwd");
CallbackHandler callbackHandler = new JaasBasicAuthFilter.BasicAuthCallBackHandler(authHeader);
Callback unsupportedCallback = new ChoiceCallback(
@ -184,7 +186,7 @@ public class JaasBasicAuthFilterTest {
1,
true
);
callbackHandler.handle(new Callback[] {unsupportedCallback});
assertThrows(ConnectException.class, () -> callbackHandler.handle(new Callback[] {unsupportedCallback}));
}
private String authHeader(String authorization, String username, String password) {

View File

@ -30,6 +30,7 @@ import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;
public class FileStreamSourceConnectorTest extends EasyMockSupport {
@ -98,10 +99,10 @@ public class FileStreamSourceConnectorTest extends EasyMockSupport {
EasyMock.verify(ctx);
}
@Test(expected = ConfigException.class)
@Test
public void testMultipleSourcesInvalid() {
sourceProperties.put(FileStreamSourceConnector.TOPIC_CONFIG, MULTIPLE_TOPICS);
connector.start(sourceProperties);
assertThrows(ConfigException.class, () -> connector.start(sourceProperties));
}
@Test
@ -114,22 +115,22 @@ public class FileStreamSourceConnectorTest extends EasyMockSupport {
EasyMock.verify(ctx);
}
@Test(expected = ConfigException.class)
@Test
public void testMissingTopic() {
sourceProperties.remove(FileStreamSourceConnector.TOPIC_CONFIG);
connector.start(sourceProperties);
assertThrows(ConfigException.class, () -> connector.start(sourceProperties));
}
@Test(expected = ConfigException.class)
@Test
public void testBlankTopic() {
// Because of trimming this tests is same as testing for empty string.
sourceProperties.put(FileStreamSourceConnector.TOPIC_CONFIG, " ");
connector.start(sourceProperties);
assertThrows(ConfigException.class, () -> connector.start(sourceProperties));
}
@Test(expected = ConfigException.class)
@Test
public void testInvalidBatchSize() {
sourceProperties.put(FileStreamSourceConnector.TASK_BATCH_SIZE_CONFIG, "abcd");
connector.start(sourceProperties);
assertThrows(ConfigException.class, () -> connector.start(sourceProperties));
}
}

View File

@ -823,10 +823,11 @@ public class JsonConverterTest {
assertNull(converted);
}
@Test(expected = DataException.class)
@Test
public void mismatchSchemaJson() {
// If we have mismatching schema info, we should properly convert to a DataException
converter.fromConnectData(TOPIC, Schema.FLOAT64_SCHEMA, true);
assertThrows(DataException.class,
() -> converter.fromConnectData(TOPIC, Schema.FLOAT64_SCHEMA, true));
}
@Test

View File

@ -30,6 +30,7 @@ import java.util.Collections;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
public class ByteArrayConverterTest {
@ -59,14 +60,16 @@ public class ByteArrayConverterTest {
);
}
@Test(expected = DataException.class)
@Test
public void testFromConnectBadSchema() {
converter.fromConnectData(TOPIC, Schema.INT32_SCHEMA, SAMPLE_BYTES);
assertThrows(DataException.class,
() -> converter.fromConnectData(TOPIC, Schema.INT32_SCHEMA, SAMPLE_BYTES));
}
@Test(expected = DataException.class)
@Test
public void testFromConnectInvalidValue() {
converter.fromConnectData(TOPIC, Schema.BYTES_SCHEMA, 12);
assertThrows(DataException.class,
() -> converter.fromConnectData(TOPIC, Schema.BYTES_SCHEMA, 12));
}
@Test

View File

@ -26,6 +26,7 @@ import org.junit.Test;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;
public abstract class NumberConverterTest<T extends Number> {
private static final String TOPIC = "topic";
@ -71,24 +72,25 @@ public abstract class NumberConverterTest<T extends Number> {
}
}
@Test(expected = DataException.class)
@Test
public void testDeserializingDataWithTooManyBytes() {
converter.toConnectData(TOPIC, new byte[10]);
assertThrows(DataException.class, () -> converter.toConnectData(TOPIC, new byte[10]));
}
@Test(expected = DataException.class)
@Test
public void testDeserializingHeaderWithTooManyBytes() {
converter.toConnectHeader(TOPIC, HEADER_NAME, new byte[10]);
assertThrows(DataException.class, () -> converter.toConnectHeader(TOPIC, HEADER_NAME, new byte[10]));
}
@Test(expected = DataException.class)
@Test
public void testSerializingIncorrectType() {
converter.fromConnectData(TOPIC, schema, "not a valid number");
assertThrows(DataException.class, () -> converter.fromConnectData(TOPIC, schema, "not a valid number"));
}
@Test(expected = DataException.class)
@Test
public void testSerializingIncorrectHeader() {
converter.fromConnectHeader(TOPIC, HEADER_NAME, schema, "not a valid number");
assertThrows(DataException.class,
() -> converter.fromConnectHeader(TOPIC, HEADER_NAME, schema, "not a valid number"));
}
@Test

View File

@ -69,6 +69,7 @@ import java.util.stream.Collectors;
import static org.apache.kafka.connect.runtime.AbstractHerder.keysWithVariableValues;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;
import static org.powermock.api.easymock.PowerMock.verifyAll;
import static org.powermock.api.easymock.PowerMock.replayAll;
import static org.easymock.EasyMock.strictMock;
@ -273,12 +274,12 @@ public class AbstractHerderTest {
}
@Test(expected = BadRequestException.class)
public void testConfigValidationEmptyConfig() throws Throwable {
AbstractHerder herder = createConfigValidationHerder(TestSourceConnector.class, noneConnectorClientConfigOverridePolicy);
@Test
public void testConfigValidationEmptyConfig() {
AbstractHerder herder = createConfigValidationHerder(TestSourceConnector.class, noneConnectorClientConfigOverridePolicy, 0);
replayAll();
herder.validateConnectorConfig(Collections.emptyMap(), false);
assertThrows(BadRequestException.class, () -> herder.validateConnectorConfig(Collections.emptyMap(), false));
verifyAll();
}
@ -312,8 +313,8 @@ public class AbstractHerderTest {
verifyAll();
}
@Test(expected = ConfigException.class)
public void testConfigValidationInvalidTopics() throws Throwable {
@Test
public void testConfigValidationInvalidTopics() {
AbstractHerder herder = createConfigValidationHerder(TestSinkConnector.class, noneConnectorClientConfigOverridePolicy);
replayAll();
@ -322,12 +323,12 @@ public class AbstractHerderTest {
config.put(SinkConnectorConfig.TOPICS_CONFIG, "topic1,topic2");
config.put(SinkConnectorConfig.TOPICS_REGEX_CONFIG, "topic.*");
herder.validateConnectorConfig(config, false);
assertThrows(ConfigException.class, () -> herder.validateConnectorConfig(config, false));
verifyAll();
}
@Test(expected = ConfigException.class)
@Test
public void testConfigValidationTopicsWithDlq() {
AbstractHerder herder = createConfigValidationHerder(TestSinkConnector.class, noneConnectorClientConfigOverridePolicy);
replayAll();
@ -337,12 +338,12 @@ public class AbstractHerderTest {
config.put(SinkConnectorConfig.TOPICS_CONFIG, "topic1");
config.put(SinkConnectorConfig.DLQ_TOPIC_NAME_CONFIG, "topic1");
herder.validateConnectorConfig(config, false);
assertThrows(ConfigException.class, () -> herder.validateConnectorConfig(config, false));
verifyAll();
}
@Test(expected = ConfigException.class)
@Test
public void testConfigValidationTopicsRegexWithDlq() {
AbstractHerder herder = createConfigValidationHerder(TestSinkConnector.class, noneConnectorClientConfigOverridePolicy);
replayAll();
@ -352,7 +353,7 @@ public class AbstractHerderTest {
config.put(SinkConnectorConfig.TOPICS_REGEX_CONFIG, "topic.*");
config.put(SinkConnectorConfig.DLQ_TOPIC_NAME_CONFIG, "topic1");
herder.validateConnectorConfig(config, false);
assertThrows(ConfigException.class, () -> herder.validateConnectorConfig(config, false));
verifyAll();
}
@ -789,6 +790,12 @@ public class AbstractHerderTest {
private AbstractHerder createConfigValidationHerder(Class<? extends Connector> connectorClass,
ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy) {
return createConfigValidationHerder(connectorClass, connectorClientConfigOverridePolicy, 1);
}
private AbstractHerder createConfigValidationHerder(Class<? extends Connector> connectorClass,
ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy,
int countOfCallingNewConnector) {
ConfigBackingStore configStore = strictMock(ConfigBackingStore.class);
@ -813,8 +820,11 @@ public class AbstractHerderTest {
} catch (ReflectiveOperationException e) {
throw new RuntimeException("Couldn't create connector", e);
}
EasyMock.expect(plugins.newConnector(connectorClass.getName())).andReturn(connector);
EasyMock.expect(plugins.compareAndSwapLoaders(connector)).andReturn(classLoader);
if (countOfCallingNewConnector > 0) {
EasyMock.expect(plugins.newConnector(connectorClass.getName())).andReturn(connector).times(countOfCallingNewConnector);
EasyMock.expect(plugins.compareAndSwapLoaders(connector)).andReturn(classLoader).times(countOfCallingNewConnector);
}
return herder;
}

View File

@ -36,6 +36,7 @@ import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNotSame;
import static org.junit.Assert.assertSame;
import static org.junit.Assert.assertThrows;
@SuppressWarnings("deprecation")
public class ConnectMetricsTest {
@ -67,9 +68,10 @@ public class ConnectMetricsTest {
assertNotNull(metrics.metrics());
}
@Test(expected = IllegalArgumentException.class)
@Test
public void testGettingGroupWithOddNumberOfTags() {
metrics.group("name", "k1", "v1", "k2", "v2", "extra");
assertThrows(IllegalArgumentException.class,
() -> metrics.group("name", "k1", "v1", "k2", "v2", "extra"));
}
@Test

View File

@ -153,7 +153,7 @@ public class ConnectorConfigTest<R extends ConnectRecord<R>> {
assertEquals(42, xform.magicNumber);
}
@Test(expected = ConfigException.class)
@Test
public void multipleTransformsOneDangling() {
Map<String, String> props = new HashMap<>();
props.put("name", "test");
@ -161,7 +161,7 @@ public class ConnectorConfigTest<R extends ConnectRecord<R>> {
props.put("transforms", "a, b");
props.put("transforms.a.type", SimpleTransformation.class.getName());
props.put("transforms.a.magic.number", "42");
new ConnectorConfig(MOCK_PLUGINS, props);
assertThrows(ConfigException.class, () -> new ConnectorConfig(MOCK_PLUGINS, props));
}
@Test

View File

@ -75,18 +75,18 @@ public class WorkerConfigTest {
new WorkerConfig(WorkerConfig.baseConfigDef(), props);
}
@Test(expected = ConfigException.class)
@Test
public void testAdminListenersNotAllowingEmptyStrings() {
Map<String, String> props = baseProps();
props.put(WorkerConfig.ADMIN_LISTENERS_CONFIG, "http://a.b:9999,");
new WorkerConfig(WorkerConfig.baseConfigDef(), props);
assertThrows(ConfigException.class, () -> new WorkerConfig(WorkerConfig.baseConfigDef(), props));
}
@Test(expected = ConfigException.class)
@Test
public void testAdminListenersNotAllowingBlankStrings() {
Map<String, String> props = baseProps();
props.put(WorkerConfig.ADMIN_LISTENERS_CONFIG, "http://a.b:9999, ,https://a.b:9999");
new WorkerConfig(WorkerConfig.baseConfigDef(), props);
assertThrows(ConfigException.class, () -> new WorkerConfig(WorkerConfig.baseConfigDef(), props));
}
@Test

View File

@ -91,6 +91,7 @@ import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_CREATION_ENABL
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
@PowerMockIgnore({"javax.management.*",
@ -541,7 +542,7 @@ public class WorkerSourceTaskTest extends ThreadedTest {
PowerMock.verifyAll();
}
@Test(expected = InvalidRecordException.class)
@Test
public void testSendRecordsCorruptTimestamp() throws Exception {
final Long timestamp = -3L;
createWorkerTask();
@ -555,8 +556,8 @@ public class WorkerSourceTaskTest extends ThreadedTest {
PowerMock.replayAll();
Whitebox.setInternalState(workerTask, "toSend", records);
Whitebox.invokeMethod(workerTask, "sendRecords");
assertEquals(null, sent.getValue().timestamp());
assertThrows(InvalidRecordException.class, () -> Whitebox.invokeMethod(workerTask, "sendRecords"));
assertFalse(sent.hasCaptured());
PowerMock.verifyAll();
}
@ -619,7 +620,7 @@ public class WorkerSourceTaskTest extends ThreadedTest {
PowerMock.verifyAll();
}
@Test(expected = ConnectException.class)
@Test
public void testSendRecordsProducerCallbackFail() throws Exception {
createWorkerTask();
@ -633,7 +634,7 @@ public class WorkerSourceTaskTest extends ThreadedTest {
PowerMock.replayAll();
Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2));
Whitebox.invokeMethod(workerTask, "sendRecords");
assertThrows(ConnectException.class, () -> Whitebox.invokeMethod(workerTask, "sendRecords"));
}
@Test

View File

@ -102,8 +102,8 @@ import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_F
import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_CREATION_ENABLE_CONFIG;
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.assertThrows;
import static org.junit.Assert.assertTrue;
@PowerMockIgnore({"javax.management.*",
@ -566,7 +566,7 @@ public class WorkerSourceTaskWithTopicCreationTest extends ThreadedTest {
PowerMock.verifyAll();
}
@Test(expected = InvalidRecordException.class)
@Test
public void testSendRecordsCorruptTimestamp() throws Exception {
final Long timestamp = -3L;
createWorkerTask();
@ -580,8 +580,8 @@ public class WorkerSourceTaskWithTopicCreationTest extends ThreadedTest {
PowerMock.replayAll();
Whitebox.setInternalState(workerTask, "toSend", records);
Whitebox.invokeMethod(workerTask, "sendRecords");
assertEquals(null, sent.getValue().timestamp());
assertThrows(InvalidRecordException.class, () -> Whitebox.invokeMethod(workerTask, "sendRecords"));
assertFalse(sent.hasCaptured());
PowerMock.verifyAll();
}
@ -644,7 +644,7 @@ public class WorkerSourceTaskWithTopicCreationTest extends ThreadedTest {
PowerMock.verifyAll();
}
@Test(expected = ConnectException.class)
@Test
public void testSendRecordsProducerCallbackFail() throws Exception {
createWorkerTask();
@ -658,11 +658,11 @@ public class WorkerSourceTaskWithTopicCreationTest extends ThreadedTest {
PowerMock.replayAll();
Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2));
Whitebox.invokeMethod(workerTask, "sendRecords");
assertThrows(ConnectException.class, () -> Whitebox.invokeMethod(workerTask, "sendRecords"));
}
@Test(expected = ConnectException.class)
public void testSendRecordsProducerSendFailsImmediately() throws Exception {
@Test
public void testSendRecordsProducerSendFailsImmediately() {
createWorkerTask();
SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD);
@ -677,7 +677,7 @@ public class WorkerSourceTaskWithTopicCreationTest extends ThreadedTest {
PowerMock.replayAll();
Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2));
Whitebox.invokeMethod(workerTask, "sendRecords");
assertThrows(ConnectException.class, () -> Whitebox.invokeMethod(workerTask, "sendRecords"));
}
@Test
@ -1057,8 +1057,8 @@ public class WorkerSourceTaskWithTopicCreationTest extends ThreadedTest {
PowerMock.verifyAll();
}
@Test(expected = ConnectException.class)
public void testTopicDescribeFails() throws Exception {
@Test
public void testTopicDescribeFails() {
createWorkerTask();
SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD);
@ -1071,10 +1071,10 @@ public class WorkerSourceTaskWithTopicCreationTest extends ThreadedTest {
PowerMock.replayAll();
Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2));
Whitebox.invokeMethod(workerTask, "sendRecords");
assertThrows(ConnectException.class, () -> Whitebox.invokeMethod(workerTask, "sendRecords"));
}
@Test(expected = ConnectException.class)
@Test
public void testTopicCreateFails() throws Exception {
createWorkerTask();
@ -1091,11 +1091,11 @@ public class WorkerSourceTaskWithTopicCreationTest extends ThreadedTest {
PowerMock.replayAll();
Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2));
Whitebox.invokeMethod(workerTask, "sendRecords");
assertNotNull(newTopicCapture.getValue());
assertThrows(ConnectException.class, () -> Whitebox.invokeMethod(workerTask, "sendRecords"));
assertTrue(newTopicCapture.hasCaptured());
}
@Test(expected = ConnectException.class)
@Test
public void testTopicCreateFailsWithExceptionWhenCreateReturnsFalse() throws Exception {
createWorkerTask();
@ -1111,8 +1111,8 @@ public class WorkerSourceTaskWithTopicCreationTest extends ThreadedTest {
PowerMock.replayAll();
Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2));
Whitebox.invokeMethod(workerTask, "sendRecords");
assertNotNull(newTopicCapture.getValue());
assertThrows(ConnectException.class, () -> Whitebox.invokeMethod(workerTask, "sendRecords"));
assertTrue(newTopicCapture.hasCaptured());
}
private void expectPreliminaryCalls() {

View File

@ -108,6 +108,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.assertThrows;
import static org.junit.Assert.fail;
@RunWith(PowerMockRunner.class)
@ -1186,7 +1187,7 @@ public class WorkerTest extends ThreadedTest {
null, allConnectorClientConfigOverridePolicy, CLUSTER_ID));
}
@Test(expected = ConnectException.class)
@Test
public void testConsumerConfigsClientOverridesWithNonePolicy() {
Map<String, String> props = new HashMap<>(workerProps);
props.put("consumer.auto.offset.reset", "latest");
@ -1199,8 +1200,8 @@ public class WorkerTest extends ThreadedTest {
EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX))
.andReturn(connConfig);
PowerMock.replayAll();
Worker.consumerConfigs(new ConnectorTaskId("test", 1), configWithOverrides, connectorConfig,
null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID);
assertThrows(ConnectException.class, () -> Worker.consumerConfigs(new ConnectorTaskId("test", 1),
configWithOverrides, connectorConfig, null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID));
}
@Test
@ -1230,7 +1231,7 @@ public class WorkerTest extends ThreadedTest {
null, allConnectorClientConfigOverridePolicy, CLUSTER_ID));
}
@Test(expected = ConnectException.class)
@Test
public void testAdminConfigsClientOverridesWithNonePolicy() {
Map<String, String> props = new HashMap<>(workerProps);
props.put("admin.client.id", "testid");
@ -1243,8 +1244,8 @@ public class WorkerTest extends ThreadedTest {
EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX))
.andReturn(connConfig);
PowerMock.replayAll();
Worker.adminConfigs(new ConnectorTaskId("test", 1), "", configWithOverrides, connectorConfig,
null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID);
assertThrows(ConnectException.class, () -> Worker.adminConfigs(new ConnectorTaskId("test", 1),
"", configWithOverrides, connectorConfig, null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID));
}

View File

@ -102,6 +102,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.assertThrows;
import static org.junit.Assert.fail;
@RunWith(PowerMockRunner.class)
@ -1169,21 +1170,21 @@ public class WorkerWithTopicCreationTest extends ThreadedTest {
null, allConnectorClientConfigOverridePolicy, CLUSTER_ID));
}
@Test(expected = ConnectException.class)
@Test
public void testConsumerConfigsClientOverridesWithNonePolicy() {
Map<String, String> props = new HashMap<>(workerProps);
props.put("consumer.auto.offset.reset", "latest");
props.put("consumer.max.poll.records", "5000");
WorkerConfig configWithOverrides = new StandaloneConfig(props);
Map<String, Object> connConfig = new HashMap<String, Object>();
Map<String, Object> connConfig = new HashMap<>();
connConfig.put("max.poll.records", "5000");
connConfig.put("max.poll.interval.ms", "1000");
EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX))
.andReturn(connConfig);
PowerMock.replayAll();
Worker.consumerConfigs(new ConnectorTaskId("test", 1), configWithOverrides, connectorConfig,
null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID);
assertThrows(ConnectException.class, () -> Worker.consumerConfigs(new ConnectorTaskId("test", 1),
configWithOverrides, connectorConfig, null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID));
}
@Test
@ -1212,7 +1213,7 @@ public class WorkerWithTopicCreationTest extends ThreadedTest {
null, allConnectorClientConfigOverridePolicy, CLUSTER_ID));
}
@Test(expected = ConnectException.class)
@Test
public void testAdminConfigsClientOverridesWithNonePolicy() {
Map<String, String> props = new HashMap<>(workerProps);
props.put("admin.client.id", "testid");
@ -1225,8 +1226,8 @@ public class WorkerWithTopicCreationTest extends ThreadedTest {
EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX))
.andReturn(connConfig);
PowerMock.replayAll();
Worker.adminConfigs(new ConnectorTaskId("test", 1), "", configWithOverrides, connectorConfig,
null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID);
assertThrows(ConnectException.class, () -> Worker.adminConfigs(new ConnectorTaskId("test", 1),
"", configWithOverrides, connectorConfig, null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID));
}
private void assertStatusMetrics(long expected, String metricName) {

View File

@ -66,33 +66,33 @@ public class DistributedConfigTest {
assertEquals(512 / 8, keyGenerator.generateKey().getEncoded().length);
}
@Test(expected = ConfigException.class)
@Test
public void shouldFailWithEmptyListOfVerificationAlgorithms() {
Map<String, String> configs = configs();
configs.put(DistributedConfig.INTER_WORKER_VERIFICATION_ALGORITHMS_CONFIG, "");
new DistributedConfig(configs);
assertThrows(ConfigException.class, () -> new DistributedConfig(configs));
}
@Test(expected = ConfigException.class)
@Test
public void shouldFailIfKeyAlgorithmNotInVerificationAlgorithmsList() {
Map<String, String> configs = configs();
configs.put(DistributedConfig.INTER_WORKER_KEY_GENERATION_ALGORITHM_CONFIG, "HmacSHA1");
configs.put(DistributedConfig.INTER_WORKER_VERIFICATION_ALGORITHMS_CONFIG, "HmacSHA256");
new DistributedConfig(configs);
assertThrows(ConfigException.class, () -> new DistributedConfig(configs));
}
@Test(expected = ConfigException.class)
@Test
public void shouldFailWithInvalidKeyAlgorithm() {
Map<String, String> configs = configs();
configs.put(DistributedConfig.INTER_WORKER_KEY_GENERATION_ALGORITHM_CONFIG, "not-actually-a-key-algorithm");
new DistributedConfig(configs);
assertThrows(ConfigException.class, () -> new DistributedConfig(configs));
}
@Test(expected = ConfigException.class)
@Test
public void shouldFailWithInvalidKeySize() {
Map<String, String> configs = configs();
configs.put(DistributedConfig.INTER_WORKER_KEY_SIZE_CONFIG, "0");
new DistributedConfig(configs);
assertThrows(ConfigException.class, () -> new DistributedConfig(configs));
}
@Test

View File

@ -61,6 +61,7 @@ import static org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter.ER
import static org.easymock.EasyMock.replay;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
@RunWith(PowerMockRunner.class)
@ -96,9 +97,9 @@ public class ErrorReporterTest {
}
}
@Test(expected = NullPointerException.class)
@Test
public void initializeDLQWithNullMetrics() {
new DeadLetterQueueReporter(producer, config(emptyMap()), TASK_ID, null);
assertThrows(NullPointerException.class, () -> new DeadLetterQueueReporter(producer, config(emptyMap()), TASK_ID, null));
}
@Test

View File

@ -68,6 +68,7 @@ import static org.easymock.EasyMock.replay;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
@RunWith(PowerMockRunner.class)
@ -116,14 +117,14 @@ public class RetryWithToleranceOperatorTest {
SinkTask.class, consumerRecord, new Throwable());
}
@Test(expected = ConnectException.class)
@Test
public void testExecuteFailedNoTolerance() {
RetryWithToleranceOperator retryWithToleranceOperator = new RetryWithToleranceOperator(0,
ERRORS_RETRY_MAX_DELAY_DEFAULT, NONE, SYSTEM);
retryWithToleranceOperator.metrics(errorHandlingMetrics);
retryWithToleranceOperator.executeFailed(Stage.TASK_PUT,
SinkTask.class, consumerRecord, new Throwable());
assertThrows(ConnectException.class, () -> retryWithToleranceOperator.executeFailed(Stage.TASK_PUT,
SinkTask.class, consumerRecord, new Throwable()));
}
@Test
@ -156,24 +157,24 @@ public class RetryWithToleranceOperatorTest {
testHandleExceptionInStage(Stage.TASK_POLL, new org.apache.kafka.connect.errors.RetriableException("Test"));
}
@Test(expected = ConnectException.class)
@Test
public void testThrowExceptionInTaskPut() {
testHandleExceptionInStage(Stage.TASK_PUT, new Exception());
assertThrows(ConnectException.class, () -> testHandleExceptionInStage(Stage.TASK_PUT, new Exception()));
}
@Test(expected = ConnectException.class)
@Test
public void testThrowExceptionInTaskPoll() {
testHandleExceptionInStage(Stage.TASK_POLL, new Exception());
assertThrows(ConnectException.class, () -> testHandleExceptionInStage(Stage.TASK_POLL, new Exception()));
}
@Test(expected = ConnectException.class)
@Test
public void testThrowExceptionInKafkaConsume() {
testHandleExceptionInStage(Stage.KAFKA_CONSUME, new Exception());
assertThrows(ConnectException.class, () -> testHandleExceptionInStage(Stage.KAFKA_CONSUME, new Exception()));
}
@Test(expected = ConnectException.class)
@Test
public void testThrowExceptionInKafkaProduce() {
testHandleExceptionInStage(Stage.KAFKA_PRODUCE, new Exception());
assertThrows(ConnectException.class, () -> testHandleExceptionInStage(Stage.KAFKA_PRODUCE, new Exception()));
}
private void testHandleExceptionInStage(Stage type, Exception ex) {

View File

@ -28,6 +28,7 @@ import java.util.Collections;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
public class DelegatingClassLoaderTest {
@ -50,13 +51,13 @@ public class DelegatingClassLoaderTest {
assertFalse(DelegatingClassLoader.serviceLoaderManifestForPlugin("resource/version.properties"));
}
@Test(expected = ClassNotFoundException.class)
@Test
public void testLoadingUnloadedPluginClass() throws ClassNotFoundException {
TestPlugins.assertAvailable();
DelegatingClassLoader classLoader = new DelegatingClassLoader(Collections.emptyList());
classLoader.initLoaders();
for (String pluginClassName : TestPlugins.pluginClasses()) {
classLoader.loadClass(pluginClassName);
assertThrows(ClassNotFoundException.class, () -> classLoader.loadClass(pluginClassName));
}
}

View File

@ -50,6 +50,7 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertSame;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
public class PluginsTest {
@ -184,15 +185,15 @@ public class PluginsTest {
assertTrue(headerConverter instanceof SimpleHeaderConverter);
}
@Test(expected = ConnectException.class)
@Test
public void shouldThrowIfPluginThrows() {
TestPlugins.assertAvailable();
plugins.newPlugin(
assertThrows(ConnectException.class, () -> plugins.newPlugin(
TestPlugins.ALWAYS_THROW_EXCEPTION,
new AbstractConfig(new ConfigDef(), Collections.emptyMap()),
Converter.class
);
));
}
@Test
@ -251,11 +252,11 @@ public class PluginsTest {
assertPluginClassLoaderAlwaysActive(samples);
}
@Test(expected = ConfigException.class)
@Test
public void shouldFailToFindConverterInCurrentClassloader() {
TestPlugins.assertAvailable();
props.put(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, TestPlugins.SAMPLING_CONVERTER);
createConfig();
assertThrows(ConfigException.class, this::createConfig);
}
@Test

View File

@ -34,6 +34,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.assertThrows;
import static org.junit.Assert.assertTrue;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.mock;
@ -71,14 +72,16 @@ public class InternalRequestSignatureTest {
assertNull(InternalRequestSignature.fromHeaders(REQUEST_BODY, internalRequestHeaders(ENCODED_SIGNATURE, null)));
}
@Test(expected = BadRequestException.class)
@Test
public void fromHeadersShouldThrowExceptionOnInvalidSignatureAlgorithm() {
InternalRequestSignature.fromHeaders(REQUEST_BODY, internalRequestHeaders(ENCODED_SIGNATURE, "doesn'texist"));
assertThrows(BadRequestException.class, () -> InternalRequestSignature.fromHeaders(REQUEST_BODY,
internalRequestHeaders(ENCODED_SIGNATURE, "doesn'texist")));
}
@Test(expected = BadRequestException.class)
@Test
public void fromHeadersShouldThrowExceptionOnInvalidBase64Signature() {
InternalRequestSignature.fromHeaders(REQUEST_BODY, internalRequestHeaders("not valid base 64", SIGNATURE_ALGORITHM));
assertThrows(BadRequestException.class, () -> InternalRequestSignature.fromHeaders(REQUEST_BODY,
internalRequestHeaders("not valid base 64", SIGNATURE_ALGORITHM)));
}
@Test
@ -89,10 +92,10 @@ public class InternalRequestSignatureTest {
assertNotNull(signature.keyAlgorithm());
}
@Test(expected = ConnectException.class)
@Test
public void addToRequestShouldThrowExceptionOnInvalidSignatureAlgorithm() {
Request request = mock(Request.class);
InternalRequestSignature.addToRequest(KEY, REQUEST_BODY, "doesn'texist", request);
assertThrows(ConnectException.class, () -> InternalRequestSignature.addToRequest(KEY, REQUEST_BODY, "doesn'texist", request));
}
@Test

View File

@ -79,6 +79,7 @@ import java.util.TreeSet;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
@RunWith(PowerMockRunner.class)
@ -334,79 +335,18 @@ public class ConnectorPluginsResourceTest {
PowerMock.verifyAll();
}
@Test(expected = BadRequestException.class)
public void testValidateConfigWithNonExistentName() throws Throwable {
Capture<Callback<ConfigInfos>> configInfosCallback = EasyMock.newCapture();
herder.validateConnectorConfig(EasyMock.eq(props), EasyMock.capture(configInfosCallback), EasyMock.anyBoolean());
PowerMock.expectLastCall().andAnswer((IAnswer<ConfigInfos>) () -> {
ConfigDef connectorConfigDef = ConnectorConfig.configDef();
List<ConfigValue> connectorConfigValues = connectorConfigDef.validate(props);
Connector connector = new ConnectorPluginsResourceTestConnector();
Config config = connector.validate(props);
ConfigDef configDef = connector.config();
Map<String, ConfigDef.ConfigKey> configKeys = configDef.configKeys();
List<ConfigValue> configValues = config.configValues();
Map<String, ConfigDef.ConfigKey> resultConfigKeys = new HashMap<>(configKeys);
resultConfigKeys.putAll(connectorConfigDef.configKeys());
configValues.addAll(connectorConfigValues);
ConfigInfos configInfos = AbstractHerder.generateResult(
ConnectorPluginsResourceTestConnector.class.getName(),
resultConfigKeys,
configValues,
Collections.singletonList("Test")
);
configInfosCallback.getValue().onCompletion(null, configInfos);
return null;
});
PowerMock.replayAll();
@Test
public void testValidateConfigWithNonExistentName() {
// make a request to connector-plugins resource using a non-loaded connector with the same
// simple name but different package.
String customClassname = "com.custom.package."
+ ConnectorPluginsResourceTestConnector.class.getSimpleName();
connectorPluginsResource.validateConfigs(customClassname, props);
PowerMock.verifyAll();
assertThrows(BadRequestException.class, () -> connectorPluginsResource.validateConfigs(customClassname, props));
}
@Test(expected = BadRequestException.class)
public void testValidateConfigWithNonExistentAlias() throws Throwable {
Capture<Callback<ConfigInfos>> configInfosCallback = EasyMock.newCapture();
herder.validateConnectorConfig(EasyMock.eq(props), EasyMock.capture(configInfosCallback), EasyMock.anyBoolean());
PowerMock.expectLastCall().andAnswer((IAnswer<ConfigInfos>) () -> {
ConfigDef connectorConfigDef = ConnectorConfig.configDef();
List<ConfigValue> connectorConfigValues = connectorConfigDef.validate(props);
Connector connector = new ConnectorPluginsResourceTestConnector();
Config config = connector.validate(props);
ConfigDef configDef = connector.config();
Map<String, ConfigDef.ConfigKey> configKeys = configDef.configKeys();
List<ConfigValue> configValues = config.configValues();
Map<String, ConfigDef.ConfigKey> resultConfigKeys = new HashMap<>(configKeys);
resultConfigKeys.putAll(connectorConfigDef.configKeys());
configValues.addAll(connectorConfigValues);
ConfigInfos configInfos = AbstractHerder.generateResult(
ConnectorPluginsResourceTestConnector.class.getName(),
resultConfigKeys,
configValues,
Collections.singletonList("Test")
);
configInfosCallback.getValue().onCompletion(null, configInfos);
return null;
});
PowerMock.replayAll();
connectorPluginsResource.validateConfigs("ConnectorPluginsTest", props);
PowerMock.verifyAll();
@Test
public void testValidateConfigWithNonExistentAlias() {
assertThrows(BadRequestException.class, () -> connectorPluginsResource.validateConfigs("ConnectorPluginsTest", props));
}
@Test

View File

@ -361,8 +361,8 @@ public class ConnectorsResourceTest {
PowerMock.verifyAll();
}
@Test(expected = AlreadyExistsException.class)
public void testCreateConnectorExists() throws Throwable {
@Test
public void testCreateConnectorExists() {
CreateConnectorRequest body = new CreateConnectorRequest(CONNECTOR_NAME, Collections.singletonMap(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME));
final Capture<Callback<Herder.Created<ConnectorInfo>>> cb = Capture.newInstance();
@ -371,7 +371,7 @@ public class ConnectorsResourceTest {
PowerMock.replayAll();
connectorsResource.createConnector(FORWARD, NULL_HEADERS, body);
assertThrows(AlreadyExistsException.class, () -> connectorsResource.createConnector(FORWARD, NULL_HEADERS, body));
PowerMock.verifyAll();
}
@ -463,7 +463,7 @@ public class ConnectorsResourceTest {
}
// Not found exceptions should pass through to caller so they can be processed for 404s
@Test(expected = NotFoundException.class)
@Test
public void testDeleteConnectorNotFound() throws Throwable {
final Capture<Callback<Herder.Created<ConnectorInfo>>> cb = Capture.newInstance();
herder.deleteConnectorConfig(EasyMock.eq(CONNECTOR_NAME), EasyMock.capture(cb));
@ -471,7 +471,7 @@ public class ConnectorsResourceTest {
PowerMock.replayAll();
connectorsResource.destroyConnector(CONNECTOR_NAME, NULL_HEADERS, FORWARD);
assertThrows(NotFoundException.class, () -> connectorsResource.destroyConnector(CONNECTOR_NAME, NULL_HEADERS, FORWARD));
PowerMock.verifyAll();
}
@ -506,15 +506,15 @@ public class ConnectorsResourceTest {
PowerMock.verifyAll();
}
@Test(expected = NotFoundException.class)
public void testGetConnectorConfigConnectorNotFound() throws Throwable {
@Test
public void testGetConnectorConfigConnectorNotFound() {
final Capture<Callback<Map<String, String>>> cb = Capture.newInstance();
herder.connectorConfig(EasyMock.eq(CONNECTOR_NAME), EasyMock.capture(cb));
expectAndCallbackException(cb, new NotFoundException("not found"));
PowerMock.replayAll();
connectorsResource.getConnectorConfig(CONNECTOR_NAME, NULL_HEADERS, FORWARD);
assertThrows(NotFoundException.class, () -> connectorsResource.getConnectorConfig(CONNECTOR_NAME, NULL_HEADERS, FORWARD));
PowerMock.verifyAll();
}
@ -603,19 +603,20 @@ public class ConnectorsResourceTest {
PowerMock.verifyAll();
}
@Test(expected = BadRequestException.class)
public void testPutConnectorConfigNameMismatch() throws Throwable {
@Test
public void testPutConnectorConfigNameMismatch() {
Map<String, String> connConfig = new HashMap<>(CONNECTOR_CONFIG);
connConfig.put(ConnectorConfig.NAME_CONFIG, "mismatched-name");
connectorsResource.putConnectorConfig(CONNECTOR_NAME, NULL_HEADERS, FORWARD, connConfig);
assertThrows(BadRequestException.class, () -> connectorsResource.putConnectorConfig(CONNECTOR_NAME,
NULL_HEADERS, FORWARD, connConfig));
}
@Test(expected = BadRequestException.class)
public void testCreateConnectorConfigNameMismatch() throws Throwable {
@Test
public void testCreateConnectorConfigNameMismatch() {
Map<String, String> connConfig = new HashMap<>();
connConfig.put(ConnectorConfig.NAME_CONFIG, "mismatched-name");
CreateConnectorRequest request = new CreateConnectorRequest(CONNECTOR_NAME, connConfig);
connectorsResource.createConnector(FORWARD, NULL_HEADERS, request);
assertThrows(BadRequestException.class, () -> connectorsResource.createConnector(FORWARD, NULL_HEADERS, request));
}
@Test
@ -632,7 +633,7 @@ public class ConnectorsResourceTest {
PowerMock.verifyAll();
}
@Test(expected = NotFoundException.class)
@Test
public void testGetConnectorTaskConfigsConnectorNotFound() throws Throwable {
final Capture<Callback<List<TaskInfo>>> cb = Capture.newInstance();
herder.taskConfigs(EasyMock.eq(CONNECTOR_NAME), EasyMock.capture(cb));
@ -640,7 +641,7 @@ public class ConnectorsResourceTest {
PowerMock.replayAll();
connectorsResource.getTaskConfigs(CONNECTOR_NAME, NULL_HEADERS, FORWARD);
assertThrows(NotFoundException.class, () -> connectorsResource.getTaskConfigs(CONNECTOR_NAME, NULL_HEADERS, FORWARD));
PowerMock.verifyAll();
}
@ -702,7 +703,7 @@ public class ConnectorsResourceTest {
);
}
@Test(expected = NotFoundException.class)
@Test
public void testPutConnectorTaskConfigsConnectorNotFound() throws Throwable {
final Capture<Callback<Void>> cb = Capture.newInstance();
herder.putTaskConfigs(
@ -715,20 +716,21 @@ public class ConnectorsResourceTest {
PowerMock.replayAll();
connectorsResource.putTaskConfigs(CONNECTOR_NAME, NULL_HEADERS, FORWARD, serializeAsBytes(TASK_CONFIGS));
assertThrows(NotFoundException.class, () -> connectorsResource.putTaskConfigs(CONNECTOR_NAME, NULL_HEADERS,
FORWARD, serializeAsBytes(TASK_CONFIGS)));
PowerMock.verifyAll();
}
@Test(expected = NotFoundException.class)
public void testRestartConnectorNotFound() throws Throwable {
@Test
public void testRestartConnectorNotFound() {
final Capture<Callback<Void>> cb = Capture.newInstance();
herder.restartConnector(EasyMock.eq(CONNECTOR_NAME), EasyMock.capture(cb));
expectAndCallbackException(cb, new NotFoundException("not found"));
PowerMock.replayAll();
connectorsResource.restartConnector(CONNECTOR_NAME, NULL_HEADERS, FORWARD);
assertThrows(NotFoundException.class, () -> connectorsResource.restartConnector(CONNECTOR_NAME, NULL_HEADERS, FORWARD));
PowerMock.verifyAll();
}
@ -768,7 +770,7 @@ public class ConnectorsResourceTest {
PowerMock.verifyAll();
}
@Test(expected = NotFoundException.class)
@Test
public void testRestartTaskNotFound() throws Throwable {
ConnectorTaskId taskId = new ConnectorTaskId(CONNECTOR_NAME, 0);
final Capture<Callback<Void>> cb = Capture.newInstance();
@ -777,7 +779,7 @@ public class ConnectorsResourceTest {
PowerMock.replayAll();
connectorsResource.restartTask(CONNECTOR_NAME, 0, NULL_HEADERS, FORWARD);
assertThrows(NotFoundException.class, () -> connectorsResource.restartTask(CONNECTOR_NAME, 0, NULL_HEADERS, FORWARD));
PowerMock.verifyAll();
}
@ -936,12 +938,9 @@ public class ConnectorsResourceTest {
}
private <T> void expectAndCallbackException(final Capture<Callback<T>> cb, final Throwable t) {
PowerMock.expectLastCall().andAnswer(new IAnswer<Void>() {
@Override
public Void answer() throws Throwable {
cb.getValue().onCompletion(t, null);
return null;
}
PowerMock.expectLastCall().andAnswer((IAnswer<Void>) () -> {
cb.getValue().onCompletion(t, null);
return null;
});
}

View File

@ -32,6 +32,7 @@ import java.util.Vector;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@ -77,7 +78,7 @@ public class LoggingResourceTest {
assertEquals("ERROR", level.get("level"));
}
@Test(expected = NotFoundException.class)
@Test
public void getUnknownLoggerTest() {
LoggingResource loggingResource = mock(LoggingResource.class);
Logger root = new Logger("root") {
@ -91,7 +92,7 @@ public class LoggingResourceTest {
when(loggingResource.currentLoggers()).thenReturn(loggers(a, b));
when(loggingResource.rootLogger()).thenReturn(root);
when(loggingResource.getLogger(any())).thenCallRealMethod();
loggingResource.getLogger("c");
assertThrows(NotFoundException.class, () -> loggingResource.getLogger("c"));
}
@Test
@ -154,7 +155,7 @@ public class LoggingResourceTest {
assertEquals(z.getLevel(), Level.DEBUG);
}
@Test(expected = BadRequestException.class)
@Test
public void setLevelWithEmptyArgTest() {
LoggingResource loggingResource = mock(LoggingResource.class);
Logger root = new Logger("root") {
@ -168,10 +169,10 @@ public class LoggingResourceTest {
when(loggingResource.currentLoggers()).thenReturn(loggers(a, b));
when(loggingResource.rootLogger()).thenReturn(root);
when(loggingResource.setLevel(any(), any())).thenCallRealMethod();
loggingResource.setLevel("@root", Collections.emptyMap());
assertThrows(BadRequestException.class, () -> loggingResource.setLevel("@root", Collections.emptyMap()));
}
@Test(expected = NotFoundException.class)
@Test
public void setLevelWithInvalidArgTest() {
LoggingResource loggingResource = mock(LoggingResource.class);
Logger root = new Logger("root") {
@ -185,7 +186,7 @@ public class LoggingResourceTest {
when(loggingResource.currentLoggers()).thenReturn(loggers(a, b));
when(loggingResource.rootLogger()).thenReturn(root);
when(loggingResource.setLevel(any(), any())).thenCallRealMethod();
loggingResource.setLevel("@root", Collections.singletonMap("level", "HIGH"));
assertThrows(NotFoundException.class, () -> loggingResource.setLevel("@root", Collections.singletonMap("level", "HIGH")));
}
private Enumeration<Logger> loggers(Logger... loggers) {

View File

@ -87,6 +87,7 @@ import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CO
import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@ -155,7 +156,7 @@ public class StandaloneHerderTest {
PowerMock.verifyAll();
}
@Test(expected = BadRequestException.class)
@Test
public void testCreateConnectorFailedValidation() throws Throwable {
// Basic validation should be performed and return an error, but should still evaluate the connector's config
connector = PowerMock.createMock(BogusSourceConnector.class);
@ -181,17 +182,12 @@ public class StandaloneHerderTest {
herder.putConnectorConfig(CONNECTOR_NAME, config, false, createCallback);
try {
createCallback.get(1000L, TimeUnit.SECONDS);
} catch (ExecutionException e) {
assertNotNull(e.getCause());
throw e.getCause();
} finally {
PowerMock.verifyAll();
}
ExecutionException exception = assertThrows(ExecutionException.class, () -> createCallback.get(1000L, TimeUnit.SECONDS));
assertEquals(BadRequestException.class, exception.getCause().getClass());
PowerMock.verifyAll();
}
@Test(expected = AlreadyExistsException.class)
@Test
public void testCreateConnectorAlreadyExists() throws Throwable {
connector = PowerMock.createMock(BogusSourceConnector.class);
// First addition should succeed
@ -218,15 +214,9 @@ public class StandaloneHerderTest {
// Second should fail
FutureCallback<Herder.Created<ConnectorInfo>> failedCreateCallback = new FutureCallback<>();
herder.putConnectorConfig(CONNECTOR_NAME, config, false, failedCreateCallback);
try {
failedCreateCallback.get(1000L, TimeUnit.SECONDS);
} catch (ExecutionException e) {
assertNotNull(e.getCause());
throw e.getCause();
} finally {
PowerMock.verifyAll();
}
ExecutionException exception = assertThrows(ExecutionException.class, () -> failedCreateCallback.get(1000L, TimeUnit.SECONDS));
assertEquals(AlreadyExistsException.class, exception.getCause().getClass());
PowerMock.verifyAll();
}
@Test
@ -602,17 +592,14 @@ public class StandaloneHerderTest {
PowerMock.verifyAll();
}
@Test(expected = UnsupportedOperationException.class)
@Test
public void testPutTaskConfigs() {
Callback<Void> cb = PowerMock.createMock(Callback.class);
PowerMock.replayAll();
herder.putTaskConfigs(CONNECTOR_NAME,
Arrays.asList(singletonMap("config", "value")),
cb,
null);
assertThrows(UnsupportedOperationException.class, () -> herder.putTaskConfigs(CONNECTOR_NAME,
singletonList(singletonMap("config", "value")), cb, null));
PowerMock.verifyAll();
}

View File

@ -42,6 +42,7 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.Callable;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
@RunWith(PowerMockRunner.class)
@ -164,7 +165,7 @@ public class OffsetStorageWriterTest {
PowerMock.verifyAll();
}
@Test(expected = ConnectException.class)
@Test
public void testAlreadyFlushing() {
@SuppressWarnings("unchecked")
final Callback<Void> callback = PowerMock.createMock(Callback.class);
@ -177,7 +178,7 @@ public class OffsetStorageWriterTest {
writer.offset(OFFSET_KEY, OFFSET_VALUE);
assertTrue(writer.beginFlush());
writer.doFlush(callback);
assertTrue(writer.beginFlush()); // should throw
assertThrows(ConnectException.class, writer::beginFlush);
PowerMock.verifyAll();
}

Some files were not shown because too many files have changed in this diff Show More