Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

MINOR: Disallow unused local variables #18963

Open
wants to merge 2 commits into
base: trunk
Choose a base branch
from

Conversation

lucasbru
Copy link
Member

@lucasbru lucasbru commented Feb 19, 2025

Recently, we found a regression that could have been detected by static analysis, since a local variable wasn't being passed to a method during a refactoring, and was left unused. It was fixed in 7a749b5, but almost slipped into 4.0. Unused variables are typically detected by IDEs, but this is insufficient to prevent these kinds of bugs. This change enables unused local variable detection in checkstyle for Kafka.

A few notes on the usage:

  • There are two situations in which people actually want to have a local variable but not use it. First, there are for (Type ignored: collection) loops which have to loop collection.length number of times, but that do not use ignored in the loop body. These are typically still easier to read than a classical for loop. Second, some IDEs detect it if a return value of a function such as File.delete is not being used. In this case, people sometimes store the result in an unused local variable to make ignoring the return value explicit and to avoid the squiggly lines.
  • In Java 22, unsued local variables can be omitted by using a single underscore _. This is supported by checkstyle. In pre-22 versions, IntelliJ allows such variables to be named ignored to suppress the unused local variable warning. This pattern is often (but not consistently) used in the Kafka codebase. This is, however, not supported by checkstyle.

Since we cannot switch to Java 22, yet, and we want to use automated detection using checkstyle, we have to resort to prefixing the unused local variables with @SuppressWarnings("UnusedLocalVariable"). We have to apply this in 11 cases across the Kafka codebase. While not being pretty, I'd argue it's worth it to prevent bugs like the one fixed in 7a749b5.

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

@github-actions github-actions bot added streams core Kafka Broker producer consumer tools connect kraft mirror-maker-2 storage Pull requests that target the storage module tiered-storage Related to the Tiered Storage feature generator RPC and Record code generator clients labels Feb 19, 2025
@lucasbru lucasbru changed the title [WIP] MINOR: Forbid unused locals MINOR: Forbid unused locals Feb 25, 2025
@lucasbru lucasbru changed the title MINOR: Forbid unused locals MINOR: Disallow unused locals Feb 25, 2025
@lucasbru lucasbru changed the title MINOR: Disallow unused locals MINOR: Disallow unused local variables Feb 25, 2025
@@ -74,7 +74,7 @@ static void verifyVersionsMatchTopLevelMessage(
verifyVersionsMatchTopLevelMessage(what, topLevelMessage, field);
}
for (StructSpec struct : topLevelMessage.commonStructs()) {
for (FieldSpec field : topLevelMessage.fields()) {
for (FieldSpec field : struct.fields()) {
Copy link
Member Author

Choose a reason for hiding this comment

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

This actually looks like a bug to me

Copy link
Member

Choose a reason for hiding this comment

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

@cmccabe or @mannoopj can you comment?

Copy link
Member

@mumrah mumrah left a comment

Choose a reason for hiding this comment

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

Thanks for the patch @lucasbru! I'm always in favor of safety improvements :)

I don't love that we have to suppress warnings, but I understand we're a bit constrained due to our wide range of Java versions. It doesn't look like we have too many required suppressions.

For a few places where you have created an ignored variable and a suppression, I noticed something interesting. If we remove the assignment, I get an IDEA warning about ignoring the result of a method. However, it does not fail the checkstyle.

I think we should evaluate each of these individually to determine if its safe to ignore the result. I left comments on these inline.

Comment on lines +963 to 964
@SuppressWarnings("UnusedLocalVariable")
boolean ignored = copyQuotaManagerLockCondition.await(quotaTimeout().toMillis(), TimeUnit.MILLISECONDS);
Copy link
Member

Choose a reason for hiding this comment

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

@abhijeetk88 @satishd, is it safe to ignore the result of the await here?

@@ -74,7 +74,7 @@ static void verifyVersionsMatchTopLevelMessage(
verifyVersionsMatchTopLevelMessage(what, topLevelMessage, field);
}
for (StructSpec struct : topLevelMessage.commonStructs()) {
for (FieldSpec field : topLevelMessage.fields()) {
for (FieldSpec field : struct.fields()) {
Copy link
Member

Choose a reason for hiding this comment

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

@cmccabe or @mannoopj can you comment?

@@ -169,6 +169,7 @@ public LoadedLogOffsets load() throws IOException {
long offset = LogFileUtils.offsetFromFile(file);
if (offset >= minSwapFileOffset && offset < maxSwapFileOffset) {
logger.info("Deleting segment files {} that is compacted but has not been deleted yet.", file.getName());
@SuppressWarnings("UnusedLocalVariable")
boolean ignore = file.delete();
Copy link
Member

Choose a reason for hiding this comment

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

Shouldn't we actually check if the file was deleted?

@mimaison I see you worked the Java conversion of this class. WDYT about checking the result of delete and renameTo below?

@@ -885,7 +885,8 @@ public Set<StreamTask> drainRestoredActiveTasks(final Duration timeout) {
restoredActiveTasksLock.lock();
try {
while (restoredActiveTasks.isEmpty() && now <= deadline) {
final boolean elapsed = restoredActiveTasksCondition.await(deadline - now, TimeUnit.MILLISECONDS);
@SuppressWarnings("UnusedLocalVariable")
final boolean ignored = restoredActiveTasksCondition.await(deadline - now, TimeUnit.MILLISECONDS);
Copy link
Member

Choose a reason for hiding this comment

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

@cadonna do we need to check the result of the await here?

Copy link
Member Author

Choose a reason for hiding this comment

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

It really does not matter in that case whether we got interrupted or the timeout expired. We could consider just removing the local here. It will be flagged by intellij though.

@@ -79,7 +79,8 @@ public static void main(String[] args) {
shareConsumers.forEach(shareConsumer -> shareConsumersMetrics.add(shareConsumer.metrics()));
}
shareConsumers.forEach(shareConsumer -> {
Map<TopicIdPartition, Optional<KafkaException>> val = shareConsumer.commitSync();
@SuppressWarnings("UnusedLocalVariable")
Map<TopicIdPartition, Optional<KafkaException>> ignored = shareConsumer.commitSync();
Copy link
Member

Choose a reason for hiding this comment

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

@AndrewJSchofield do we need to check the result of commitSync?

Copy link
Collaborator

@kirktrue kirktrue left a comment

Choose a reason for hiding this comment

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

Nice improvement @lucasbru!

I know that in some places in Java you can use __ to ignore a parameter, but this seems like a more comprehensive solution. I'm actually surprised there are so few places with unused variables.

@mumrah
Copy link
Member

mumrah commented Feb 26, 2025

@lucasbru try updating this PR with trunk. That should fix the build scan errors

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
clients connect consumer core Kafka Broker generator RPC and Record code generator kraft mirror-maker-2 producer storage Pull requests that target the storage module streams tiered-storage Related to the Tiered Storage feature tools
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants