Skip to content
Merged
Show file tree
Hide file tree
Changes from 4 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -3362,6 +3362,7 @@ project(':jmh-benchmarks') {
tasks.withType(JavaCompile) {
// Suppress warning caused by code generated by jmh: `warning: [cast] redundant cast to long`
options.compilerArgs << "-Xlint:-cast"
options.compilerArgs << "-Xlint:unchecked"
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This should be included by -Xlint:all, right?

https://github.com/apache/kafka/blob/trunk/build.gradle#L129

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Happy to remove it -- it seems something is off with my local setup -- I don't see the full warn output as you do, and adding this flag did make a difference for me... so 🤷

}

jar {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -311,7 +311,7 @@ public static <K, V> void sendAndAwaitAsyncCommit(
Optional<Map<TopicPartition, OffsetAndMetadata>> offsetsOpt
) throws InterruptedException {

var commitCallback = new RetryCommitCallback(consumer, offsetsOpt);
var commitCallback = new RetryCommitCallback<>(consumer, offsetsOpt);
sendAsyncCommit(consumer, commitCallback, offsetsOpt);

TestUtils.waitForCondition(() -> {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -137,6 +137,7 @@ private void createAndDeleteTopics() {
}
}

@SuppressWarnings("unchecked")
private Gauge<Integer> getGauge(String metricName) {
return KafkaYammerMetrics.defaultRegistry().allMetrics().entrySet().stream()
.filter(entry -> entry.getKey().getName().endsWith(metricName))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -123,6 +123,7 @@ public void testClientInstanceId(ClusterInstance clusterInstance) throws Interru
}
}

@SuppressWarnings("unchecked")
@ClusterTest(types = {Type.CO_KRAFT, Type.KRAFT})
public void testIntervalMsParser(ClusterInstance clusterInstance) {
List<String> alterOpts = asList("--bootstrap-server", clusterInstance.bootstrapServers(),
Expand Down Expand Up @@ -152,6 +153,7 @@ public void testMetrics(ClusterInstance clusterInstance) {
}
}

@SuppressWarnings("unchecked")
private static String[] toArray(List<String>... lists) {
return Stream.of(lists).flatMap(List::stream).toArray(String[]::new);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -168,6 +168,7 @@ public static void assertAssignment(boolean expectFailed,
"Wrong rebalance delay in " + assignment);
}

@SuppressWarnings("unchecked")
public static <T, R extends ConnectRecord<R>> TransformationChain<T, R> getTransformationChain(
RetryWithToleranceOperator<T> toleranceOperator,
List<Object> results) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1278,6 +1278,7 @@ public void testOnCompleteExecutionOnTimeout() {
assertEquals(1, delayedShareFetch.expiredRequestMeter().count());
}

@SuppressWarnings("unchecked")
@Test
public void testRemoteStorageFetchTryCompleteReturnsFalse() {
ReplicaManager replicaManager = mock(ReplicaManager.class);
Expand Down Expand Up @@ -1357,6 +1358,7 @@ public void testRemoteStorageFetchTryCompleteReturnsFalse() {
delayedShareFetch.lock().unlock();
}

@SuppressWarnings("unchecked")
@Test
public void testRemoteStorageFetchPartitionLeaderChanged() {
ReplicaManager replicaManager = mock(ReplicaManager.class);
Expand Down Expand Up @@ -1421,6 +1423,7 @@ public void testRemoteStorageFetchPartitionLeaderChanged() {
delayedShareFetch.lock().unlock();
}

@SuppressWarnings("unchecked")
@Test
public void testRemoteStorageFetchTryCompleteThrowsException() {
ReplicaManager replicaManager = mock(ReplicaManager.class);
Expand Down Expand Up @@ -1507,6 +1510,7 @@ public void testRemoteStorageFetchTryCompleteThrowsException() {
delayedShareFetch.lock().unlock();
}

@SuppressWarnings("unchecked")
@Test
public void testRemoteStorageFetchTryCompletionDueToBrokerBecomingOffline() {
ReplicaManager replicaManager = mock(ReplicaManager.class);
Expand Down Expand Up @@ -2197,6 +2201,7 @@ private static BiConsumer<SharePartitionKey, Throwable> mockExceptionHandler() {
return mock(BiConsumer.class);
}

@SuppressWarnings("unchecked")
static class DelayedShareFetchBuilder {
private ShareFetch shareFetch = mock(ShareFetch.class);
private ReplicaManager replicaManager = mock(ReplicaManager.class);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2470,7 +2470,6 @@ public void testSharePartitionInitializationExceptions() throws Exception {
}

@Test
@SuppressWarnings("unchecked")
public void testShareFetchProcessingExceptions() throws Exception {
String groupId = "grp";
TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0));
Expand Down Expand Up @@ -3232,6 +3231,7 @@ Compression.NONE, new SimpleRecord("test-key".getBytes(), "test-value".getBytes(
return CollectionConverters.asScala(logReadResults).toSeq();
}

@SuppressWarnings("unchecked")
static void mockReplicaManagerDelayedShareFetch(ReplicaManager replicaManager,
DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory) {
doAnswer(invocationOnMock -> {
Expand Down
2 changes: 2 additions & 0 deletions core/src/test/java/kafka/server/share/SharePartitionTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -112,6 +112,7 @@
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;

@SuppressWarnings("resource")
public class SharePartitionTest {

private static final String ACQUISITION_LOCK_NEVER_GOT_RELEASED = "Acquisition lock never got released.";
Expand Down Expand Up @@ -7993,6 +7994,7 @@ public void testAcquireWithReadCommittedIsolationLevel() {
assertNotNull(sharePartition.cachedState().get(50L).offsetState().get(69L).acquisitionLockTimeoutTask());
}

@SuppressWarnings({"rawtypes", "unchecked"})
@Test
public void testContainsAbortMarker() {
SharePartition sharePartition = SharePartitionBuilder.builder()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -160,6 +160,7 @@ public void testControlledShutdownCountMetric() {
);
}

@SuppressWarnings("unchecked") // suppress warning about Gauge typecast
@Test
public void testBrokerRegistrationStateMetrics() {
MetricsRegistry registry = new MetricsRegistry();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -164,6 +164,7 @@ public void testLastAppliedRecordMetrics() {
}
}

@SuppressWarnings("unchecked")
@Test
public void testTimeSinceLastHeartbeatReceivedMs() {
MetricsRegistry registry = new MetricsRegistry();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,35 +55,35 @@ public void testParseEmptyArgAsValid() {
List<String> argArray = List.of("my.empty.property=", "my.empty.property1");
Properties props = CommandLineUtils.parseKeyValueArgs(argArray);

assertEquals(props.getProperty("my.empty.property"), "", "Value of a key with missing value should be an empty string");
assertEquals(props.getProperty("my.empty.property1"), "", "Value of a key with missing value with no delimiter should be an empty string");
assertEquals("", props.getProperty("my.empty.property"), "Value of a key with missing value should be an empty string");
assertEquals("", props.getProperty("my.empty.property1"), "Value of a key with missing value with no delimiter should be an empty string");
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixing incorrect order for expected vs actual value.

}

@Test
public void testParseSingleArg() {
List<String> argArray = List.of("my.property=value");
Properties props = CommandLineUtils.parseKeyValueArgs(argArray);

assertEquals(props.getProperty("my.property"), "value", "Value of a single property should be 'value'");
assertEquals("value", props.getProperty("my.property"), "Value of a single property should be 'value'");
}

@Test
public void testParseArgs() {
List<String> argArray = List.of("first.property=first", "second.property=second");
Properties props = CommandLineUtils.parseKeyValueArgs(argArray);

assertEquals(props.getProperty("first.property"), "first", "Value of first property should be 'first'");
assertEquals(props.getProperty("second.property"), "second", "Value of second property should be 'second'");
assertEquals("first", props.getProperty("first.property"), "Value of first property should be 'first'");
assertEquals("second", props.getProperty("second.property"), "Value of second property should be 'second'");
}

@Test
public void testParseArgsWithMultipleDelimiters() {
List<String> argArray = List.of("first.property==first", "second.property=second=", "third.property=thi=rd");
Properties props = CommandLineUtils.parseKeyValueArgs(argArray);

assertEquals(props.getProperty("first.property"), "=first", "Value of first property should be '=first'");
assertEquals(props.getProperty("second.property"), "second=", "Value of second property should be 'second='");
assertEquals(props.getProperty("third.property"), "thi=rd", "Value of second property should be 'thi=rd'");
assertEquals("=first", props.getProperty("first.property"), "Value of first property should be '=first'");
assertEquals("second=", props.getProperty("second.property"), "Value of second property should be 'second='");
assertEquals("thi=rd", props.getProperty("third.property"), "Value of second property should be 'thi=rd'");
}

Properties props = new Properties();
Expand Down Expand Up @@ -272,6 +272,7 @@ public void testInitializeBootstrapPropertiesWithBothBootstraps() {
Optional.of("127.0.0.2:9094"), Optional.of("127.0.0.3:9095"))).getMessage());
}

@SuppressWarnings("unchecked")
private OptionSpec<String> createMockOptionSpec(String name) {
OptionSpec<String> spec = mock(OptionSpec.class);
when(spec.toString()).thenReturn("[" + name.replaceAll("--", "") + "]");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -479,6 +479,7 @@ public void testDescribeClientQuotasMatchExact() throws ExecutionException, Inte
}
}

@SuppressWarnings("unchecked")
private void testMatchEntities(ClientQuotaFilter filter, int expectedMatchSize, Predicate<ClientQuotaEntity> partition)
throws InterruptedException {
TestUtils.retryOnExceptionWithTimeout(5000L, () -> {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,7 @@
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;

@SuppressWarnings("resource")
@ClusterTestDefaults(brokers = 3)
public class TopicBasedRemoteLogMetadataManagerTest {
private static final int SEG_SIZE = 1048576;
Expand Down Expand Up @@ -106,6 +107,7 @@ public void testTopicDoesNotExist() throws ExecutionException, InterruptedExcept
}
}

@SuppressWarnings("unchecked")
@ClusterTest
public void testDoesTopicExistWithAdminClientExecutionError() throws ExecutionException, InterruptedException {
// Create a mock Admin client that throws an ExecutionException (not UnknownTopicOrPartitionException)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -381,6 +381,7 @@ void testRemoteStorageManagerWithUserDefinedConfigs() {
assertFalse(remoteStorageManagerConfig.containsKey("remote.storage.manager.y"));
}

@SuppressWarnings("unchecked")
@Test
void testRemoteLogMetadataManagerWithEndpointConfig() {
ArgumentCaptor<Map<String, Object>> capture = ArgumentCaptor.forClass(Map.class);
Expand All @@ -391,6 +392,7 @@ void testRemoteLogMetadataManagerWithEndpointConfig() {
assertEquals(brokerId, capture.getValue().get(ServerConfigs.BROKER_ID_CONFIG));
}

@SuppressWarnings("unchecked")
@Test
void testRemoteLogMetadataManagerWithEndpointConfigOverridden() throws IOException {
Properties props = new Properties();
Expand Down Expand Up @@ -430,6 +432,7 @@ public RemoteLogMetadataManager createRemoteLogMetadataManager() {
}
}

@SuppressWarnings("unchecked")
@Test
void testConfigure() {
ArgumentCaptor<Map<String, Object>> capture = ArgumentCaptor.forClass(Map.class);
Expand Down Expand Up @@ -2071,6 +2074,7 @@ public void testCandidateLogSegmentsSkipsSegmentsAfterLastStableOffset() {
assertEquals(expected, actual);
}

@SuppressWarnings("unchecked")
@Test
public void testRemoteSizeData() {
Supplier<RemoteLogManager.RetentionSizeData>[] invalidRetentionSizeData =
Expand All @@ -2087,6 +2091,7 @@ public void testRemoteSizeData() {
}
}

@SuppressWarnings("unchecked")
@Test
public void testRemoteSizeTime() {
Supplier<RemoteLogManager.RetentionTimeData>[] invalidRetentionTimeData =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1961,6 +1961,7 @@ public void configure(final Map<String, ?> configs) {

}

@SuppressWarnings("deprecation")
@Override
public ProductionExceptionHandlerResponse handle(final ErrorHandlerContext context, final ProducerRecord<byte[], byte[]> record, final Exception exception) {
sourceRawData[0] = context.sourceRawKey();
Expand All @@ -1978,15 +1979,14 @@ public ProductionExceptionHandlerResponse handle(final ErrorHandlerContext conte
assertNull(sourceRawData[1]);
}


@Test
public void shouldHaveRawDataDuringExceptionInSerialization() {
final byte[][] sourceRawData = new byte[][]{new byte[]{}, new byte[]{}};
try (final ErrorStringSerializer errorSerializer = new ErrorStringSerializer()) {
final RecordCollector collector = newRecordCollector(
new ProductionExceptionHandler() {
@SuppressWarnings({"deprecation", "rawtypes"})
@Override
@SuppressWarnings({"rawtypes", "unused"})
public ProductionExceptionHandlerResponse handleSerializationException(final ErrorHandlerContext context, final ProducerRecord record, final Exception exception, final SerializationExceptionOrigin origin) {
sourceRawData[0] = context.sourceRawKey();
sourceRawData[1] = context.sourceRawValue();
Expand Down