Skip to content

Commit 1f4e041

Browse files
committed
comments / fewer suppressions
1 parent 26fbf50 commit 1f4e041

File tree

4 files changed

+11
-18
lines changed

4 files changed

+11
-18
lines changed

clients/src/main/java/org/apache/kafka/common/requests/DescribeUserScramCredentialsRequest.java

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -69,11 +69,11 @@ public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
6969
.setErrorCode(apiError.error().code())
7070
.setErrorMessage(apiError.message());
7171

72-
for (@SuppressWarnings("UnusedLocalVariable") DescribeUserScramCredentialsRequestData.UserName ignored : data.users()) {
72+
data.users().forEach(__ ->
7373
response.results().add(new DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult()
74-
.setErrorCode(apiError.error().code())
75-
.setErrorMessage(apiError.message()));
76-
}
74+
.setErrorCode(apiError.error().code())
75+
.setErrorMessage(apiError.message()))
76+
);
7777
return new DescribeUserScramCredentialsResponse(response);
7878
}
7979
}

clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -2881,11 +2881,11 @@ private void verifySessionPartitions() {
28812881
field.setAccessible(true);
28822882
LinkedHashMap<?, ?> sessionPartitions =
28832883
(LinkedHashMap<?, ?>) field.get(handler);
2884-
for (@SuppressWarnings("UnusedLocalVariable") Map.Entry<?, ?> ignored : sessionPartitions.entrySet()) {
2885-
// If `sessionPartitions` are modified on another thread, Thread.yield will increase the
2886-
// possibility of ConcurrentModificationException if appropriate synchronization is not used.
2887-
Thread.yield();
2888-
}
2884+
// If `sessionPartitions` are modified on another thread, Thread.yield will increase the
2885+
// possibility of ConcurrentModificationException if appropriate synchronization is not used.
2886+
sessionPartitions.forEach(
2887+
(key, value) -> Thread.yield()
2888+
);
28892889
} catch (Exception e) {
28902890
throw new RuntimeException(e);
28912891
}

connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -60,9 +60,7 @@ public void start(Map<String, String> props) {
6060
@Override
6161
public void put(Collection<SinkRecord> records) {
6262
super.put(records);
63-
for (@SuppressWarnings("UnusedLocalVariable") SinkRecord ignore : records) {
64-
count++;
65-
}
63+
count += records.size();
6664
}
6765

6866
}

tools/src/main/java/org/apache/kafka/tools/ShareConsumerPerformance.java

Lines changed: 1 addition & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -20,10 +20,8 @@
2020
import org.apache.kafka.clients.consumer.ConsumerRecord;
2121
import org.apache.kafka.clients.consumer.ConsumerRecords;
2222
import org.apache.kafka.clients.consumer.KafkaShareConsumer;
23-
import org.apache.kafka.common.KafkaException;
2423
import org.apache.kafka.common.Metric;
2524
import org.apache.kafka.common.MetricName;
26-
import org.apache.kafka.common.TopicIdPartition;
2725
import org.apache.kafka.common.serialization.ByteArrayDeserializer;
2826
import org.apache.kafka.common.utils.Exit;
2927
import org.apache.kafka.common.utils.Utils;
@@ -40,7 +38,6 @@
4038
import java.util.Collections;
4139
import java.util.List;
4240
import java.util.Map;
43-
import java.util.Optional;
4441
import java.util.Properties;
4542
import java.util.Set;
4643
import java.util.concurrent.ExecutorService;
@@ -79,8 +76,7 @@ public static void main(String[] args) {
7976
shareConsumers.forEach(shareConsumer -> shareConsumersMetrics.add(shareConsumer.metrics()));
8077
}
8178
shareConsumers.forEach(shareConsumer -> {
82-
@SuppressWarnings("UnusedLocalVariable")
83-
Map<TopicIdPartition, Optional<KafkaException>> ignored = shareConsumer.commitSync();
79+
shareConsumer.commitSync();
8480
shareConsumer.close(Duration.ofMillis(500));
8581
});
8682

@@ -118,7 +114,6 @@ private static void consume(List<KafkaShareConsumer<byte[], byte[]>> shareConsum
118114
AtomicLong bytesRead = new AtomicLong(0);
119115
List<ShareConsumerConsumption> shareConsumersConsumptionDetails = new ArrayList<>();
120116

121-
122117
ExecutorService executorService = Executors.newFixedThreadPool(shareConsumers.size());
123118
for (int i = 0; i < shareConsumers.size(); i++) {
124119
final int index = i;

0 commit comments

Comments
 (0)