This is an automated email from the ASF dual-hosted git repository.

divijv pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 4cf86c5d2f6 KAFKA-15492: Upgrade and enable spotbugs when building 
with Java 21 (#14533)
4cf86c5d2f6 is described below

commit 4cf86c5d2f6355c5684ecedeb1bf96e90877244d
Author: Ismael Juma <[email protected]>
AuthorDate: Thu Oct 12 05:09:10 2023 -0700

    KAFKA-15492: Upgrade and enable spotbugs when building with Java 21 (#14533)
    
    Spotbugs was temporarily disabled as part of KAFKA-15485 to support Kafka 
build with JDK 21. This PR upgrades the spotbugs version to 4.8.0 which adds 
support for JDK 21 and enables it's usage on build again.
    
    Reviewers: Divij Vaidya <[email protected]>
---
 build.gradle                                       | 18 ++++------
 .../consumer/internals/AbstractStickyAssignor.java |  6 +++-
 .../common/config/provider/FileConfigProvider.java |  3 +-
 .../kafka/metrics/LinuxIoMetricsCollector.scala    |  4 +--
 .../scala/kafka/server/PartitionMetadataFile.scala |  2 +-
 .../org/apache/kafka/message/MessageGenerator.java |  5 +--
 gradle/dependencies.gradle                         |  2 +-
 gradle/spotbugs-exclude.xml                        | 42 +++++++++++++++++++++-
 .../org/apache/kafka/raft/FileBasedStateStore.java |  2 +-
 .../org/apache/kafka/raft/ReplicatedCounter.java   |  5 +--
 .../apache/kafka/server/common/CheckpointFile.java |  2 +-
 .../streams/state/internals/OffsetCheckpoint.java  |  2 +-
 12 files changed, 67 insertions(+), 26 deletions(-)

diff --git a/build.gradle b/build.gradle
index 9df240a7433..cc2f6d0327b 100644
--- a/build.gradle
+++ b/build.gradle
@@ -235,10 +235,7 @@ subprojects {
 
   apply plugin: 'java-library'
   apply plugin: 'checkstyle'
-
-  // spotbugs doesn't support Java 21 yet
-  if (!JavaVersion.current().isCompatibleWith(JavaVersion.VERSION_21))
-    apply plugin: "com.github.spotbugs"
+  apply plugin: "com.github.spotbugs"
 
   // We use the shadow plugin for the jmh-benchmarks module and the `-all` jar 
can get pretty large, so
   // don't publish it
@@ -708,15 +705,12 @@ subprojects {
 
   test.dependsOn('checkstyleMain', 'checkstyleTest')
 
-  // spotbugs doesn't support Java 21 yet
-  if (!JavaVersion.current().isCompatibleWith(JavaVersion.VERSION_21)) {
-    spotbugs {
-      toolVersion = versions.spotbugs
-      excludeFilter = file("$rootDir/gradle/spotbugs-exclude.xml")
-      ignoreFailures = false
-    }
-    test.dependsOn('spotbugsMain')
+  spotbugs {
+    toolVersion = versions.spotbugs
+    excludeFilter = file("$rootDir/gradle/spotbugs-exclude.xml")
+    ignoreFailures = false
   }
+  test.dependsOn('spotbugsMain')
 
   tasks.withType(com.github.spotbugs.snom.SpotBugsTask) {
     reports {
diff --git 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignor.java
 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignor.java
index 0823752d159..ee48bed6df0 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignor.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignor.java
@@ -52,7 +52,7 @@ public abstract class AbstractStickyAssignor extends 
AbstractPartitionAssignor {
     private static final Logger log = 
LoggerFactory.getLogger(AbstractStickyAssignor.class);
 
     public static final int DEFAULT_GENERATION = -1;
-    public int maxGeneration = DEFAULT_GENERATION;
+    private int maxGeneration = DEFAULT_GENERATION;
 
     private PartitionMovements partitionMovements;
 
@@ -118,6 +118,10 @@ public abstract class AbstractStickyAssignor extends 
AbstractPartitionAssignor {
         return 
assignPartitions(partitionInfosWithoutRacks(partitionsPerTopic), subscriptions);
     }
 
+    public int maxGeneration() {
+        return maxGeneration;
+    }
+
     /**
      * Returns true iff all consumers have an identical subscription. Also 
fills out the passed in
      * {@code consumerToOwnedPartitions} with each consumer's previously owned 
and still-subscribed partitions,
diff --git 
a/clients/src/main/java/org/apache/kafka/common/config/provider/FileConfigProvider.java
 
b/clients/src/main/java/org/apache/kafka/common/config/provider/FileConfigProvider.java
index 41f011947e8..893ec2c49ae 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/config/provider/FileConfigProvider.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/config/provider/FileConfigProvider.java
@@ -23,6 +23,7 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.io.Reader;
+import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.util.Enumeration;
@@ -101,7 +102,7 @@ public class FileConfigProvider implements ConfigProvider {
 
     // visible for testing
     protected Reader reader(String path) throws IOException {
-        return Files.newBufferedReader(Paths.get(path));
+        return Files.newBufferedReader(Paths.get(path), 
StandardCharsets.UTF_8);
     }
 
     public void close() {
diff --git a/core/src/main/scala/kafka/metrics/LinuxIoMetricsCollector.scala 
b/core/src/main/scala/kafka/metrics/LinuxIoMetricsCollector.scala
index 5a41dbad73c..11602affb1f 100644
--- a/core/src/main/scala/kafka/metrics/LinuxIoMetricsCollector.scala
+++ b/core/src/main/scala/kafka/metrics/LinuxIoMetricsCollector.scala
@@ -18,10 +18,10 @@
 package kafka.metrics
 
 import java.nio.file.{Files, Paths}
-
 import org.apache.kafka.common.utils.Time
 import org.slf4j.Logger
 
+import java.nio.charset.StandardCharsets
 import scala.jdk.CollectionConverters._
 
 /**
@@ -68,7 +68,7 @@ class LinuxIoMetricsCollector(procRoot: String, val time: 
Time, val logger: Logg
     try {
       cachedReadBytes = -1
       cachedWriteBytes = -1
-      val lines = Files.readAllLines(path).asScala
+      val lines = Files.readAllLines(path, StandardCharsets.UTF_8).asScala
       lines.foreach(line => {
         if (line.startsWith(READ_BYTES_PREFIX)) {
           cachedReadBytes = line.substring(READ_BYTES_PREFIX.size).toLong
diff --git a/core/src/main/scala/kafka/server/PartitionMetadataFile.scala 
b/core/src/main/scala/kafka/server/PartitionMetadataFile.scala
index 154edf2bf88..97af1688482 100644
--- a/core/src/main/scala/kafka/server/PartitionMetadataFile.scala
+++ b/core/src/main/scala/kafka/server/PartitionMetadataFile.scala
@@ -138,7 +138,7 @@ class PartitionMetadataFile(val file: File,
   def read(): PartitionMetadata = {
     lock synchronized {
       try {
-        val reader = Files.newBufferedReader(path)
+        val reader = Files.newBufferedReader(path, StandardCharsets.UTF_8)
         try {
           val partitionBuffer = new 
PartitionMetadataReadBuffer(file.getAbsolutePath, reader)
           partitionBuffer.read()
diff --git 
a/generator/src/main/java/org/apache/kafka/message/MessageGenerator.java 
b/generator/src/main/java/org/apache/kafka/message/MessageGenerator.java
index 56f3f6ab0b2..ebffb4d88d2 100644
--- a/generator/src/main/java/org/apache/kafka/message/MessageGenerator.java
+++ b/generator/src/main/java/org/apache/kafka/message/MessageGenerator.java
@@ -27,6 +27,7 @@ import net.sourceforge.argparse4j.inf.ArgumentParser;
 import net.sourceforge.argparse4j.inf.Namespace;
 
 import java.io.BufferedWriter;
+import java.nio.charset.StandardCharsets;
 import java.nio.file.DirectoryStream;
 import java.nio.file.Files;
 import java.nio.file.Path;
@@ -238,7 +239,7 @@ public final class MessageGenerator {
                         String name = generator.outputName(spec) + JAVA_SUFFIX;
                         outputFileNames.add(name);
                         Path outputPath = Paths.get(outputDir, name);
-                        try (BufferedWriter writer = 
Files.newBufferedWriter(outputPath)) {
+                        try (BufferedWriter writer = 
Files.newBufferedWriter(outputPath, StandardCharsets.UTF_8)) {
                             generator.generateAndWrite(spec, writer);
                         }
                     }
@@ -252,7 +253,7 @@ public final class MessageGenerator {
         for (TypeClassGenerator typeClassGenerator : typeClassGenerators) {
             outputFileNames.add(typeClassGenerator.outputName());
             Path factoryOutputPath = Paths.get(outputDir, 
typeClassGenerator.outputName());
-            try (BufferedWriter writer = 
Files.newBufferedWriter(factoryOutputPath)) {
+            try (BufferedWriter writer = 
Files.newBufferedWriter(factoryOutputPath, StandardCharsets.UTF_8)) {
                 typeClassGenerator.generateAndWrite(writer);
             }
         }
diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle
index 9c1db35e6f4..6fabae34b73 100644
--- a/gradle/dependencies.gradle
+++ b/gradle/dependencies.gradle
@@ -161,7 +161,7 @@ versions += [
   scoverage: "1.9.3",
   slf4j: "1.7.36",
   snappy: "1.1.10.5",
-  spotbugs: "4.7.3",
+  spotbugs: "4.8.0",
   zinc: "1.9.2",
   zookeeper: "3.8.2",
   zstd: "1.5.5-6"
diff --git a/gradle/spotbugs-exclude.xml b/gradle/spotbugs-exclude.xml
index 6073c304439..9bcb95e1b26 100644
--- a/gradle/spotbugs-exclude.xml
+++ b/gradle/spotbugs-exclude.xml
@@ -46,6 +46,21 @@ For a detailed description of spotbugs bug categories, see 
https://spotbugs.read
         <Bug pattern="EQ_COMPARETO_USE_OBJECT_EQUALS"/>
     </Match>
 
+    <Match>
+        <!-- Disable warnings about constructors that throw exceptions.
+            CT_CONSTRUCTOR_THROW: Be wary of letting constructors throw 
exceptions -->
+        <Bug pattern="CT_CONSTRUCTOR_THROW"/>
+    </Match>
+
+    <Match>
+        <!-- Disable warnings about identifiers that conflict with standard 
library identifiers.
+            PI_DO_NOT_REUSE_PUBLIC_IDENTIFIERS_CLASS_NAMES: Do not reuse 
public identifiers from JSL as class name
+            PI_DO_NOT_REUSE_PUBLIC_IDENTIFIERS_FIELD_NAMES: Do not reuse 
public identifiers from JSL as field name
+            PI_DO_NOT_REUSE_PUBLIC_IDENTIFIERS_METHOD_NAMES: Do not reuse 
public identifiers from JSL as method name
+            -->
+        <Bug 
pattern="PI_DO_NOT_REUSE_PUBLIC_IDENTIFIERS_CLASS_NAMES,PI_DO_NOT_REUSE_PUBLIC_IDENTIFIERS_FIELD_NAMES,PI_DO_NOT_REUSE_PUBLIC_IDENTIFIERS_METHOD_NAMES"/>
+    </Match>
+
     <Match>
         <!-- Spotbugs tends to work a little bit better with Java than with 
Scala.  We suppress
              some categories of bug reports when using Scala, since spotbugs 
generates huge
@@ -70,7 +85,8 @@ For a detailed description of spotbugs bug categories, see 
https://spotbugs.read
             NP_ALWAYS_NULL: Null pointer dereference
             MS_CANNOT_BE_FINAL: Field isn't final and can't be protected from 
malicious code
             IC_INIT_CIRCULARITY: Initialization circularity
-            SE_NO_SUITABLE_CONSTRUCTOR: Class is Serializable but its 
superclass doesn't define a void constructor -->
+            SE_NO_SUITABLE_CONSTRUCTOR: Class is Serializable but its 
superclass doesn't define a void constructor
+            PA_PUBLIC_MUTABLE_OBJECT_ATTRIBUTE: Mutable object-type field is 
public -->
         <Source name="~.*\.scala" />
         <Or>
             <Bug pattern="NP_LOAD_OF_KNOWN_NULL_VALUE"/>
@@ -94,9 +110,16 @@ For a detailed description of spotbugs bug categories, see 
https://spotbugs.read
             <Bug pattern="SE_NO_SUITABLE_CONSTRUCTOR"/>
             <Bug pattern="DMI_RANDOM_USED_ONLY_ONCE"/>
             <Bug pattern="SSD_DO_NOT_USE_INSTANCE_LOCK_ON_SHARED_STATIC_DATA"/>
+            <Bug pattern="PA_PUBLIC_MUTABLE_OBJECT_ATTRIBUTE"/>
         </Or>
     </Match>
 
+    <Match>
+        <!-- disabled due to too many false positives
+            RV_EXCEPTION_NOT_THROWN: Exception created and dropped rather than 
thrown -->
+        <Bug pattern="RV_EXCEPTION_NOT_THROWN"/>
+    </Match>
+
     <!-- false positive in Java 11, related to 
https://github.com/spotbugs/spotbugs/issues/756 but more complex -->
     <Match>
         <Class 
name="org.apache.kafka.common.compress.KafkaLZ4BlockOutputStream"/>
@@ -270,6 +293,13 @@ For a detailed description of spotbugs bug categories, see 
https://spotbugs.read
         <Package name="~org\.apache\.kafka\.jmh\..*\.jmh_generated"/>
     </Match>
 
+    <Match>
+        <!-- JMH benchmarks often have fields that are modified by the 
framework.
+            SS_SHOULD_BE_STATIC: Unread field: should this field be static? -->
+        <Package name="~org\.apache\.kafka\.jmh\..*"/>
+        <Bug pattern="SS_SHOULD_BE_STATIC"/>
+    </Match>
+
     <Match>
         <!-- Suppress warnings about generated schema arrays. -->
         <Or>
@@ -367,6 +397,16 @@ For a detailed description of spotbugs bug categories, see 
https://spotbugs.read
         </Or>
     </Match>
 
+    <Match>
+        <!-- Public mutable fields are intentional in some Streams classes.
+         PA_PUBLIC_PRIMITIVE_ATTRIBUTE: Primitive field is public -->
+        <Or>
+            <Class name="org.apache.kafka.streams.kstream.Materialized"/>
+            <Class 
name="org.apache.kafka.streams.state.internals.LeftOrRightValueDeserializer"/>
+        </Or>
+        <Bug pattern="PA_PUBLIC_PRIMITIVE_ATTRIBUTE"/>
+    </Match>
+
     <Match>
         <!-- Suppress a spurious warning about locks not being released on all 
paths.
              This happens because there is an 'if' statement that checks if we 
have the lock before
diff --git a/raft/src/main/java/org/apache/kafka/raft/FileBasedStateStore.java 
b/raft/src/main/java/org/apache/kafka/raft/FileBasedStateStore.java
index 019fd147179..24bf4d2e7b1 100644
--- a/raft/src/main/java/org/apache/kafka/raft/FileBasedStateStore.java
+++ b/raft/src/main/java/org/apache/kafka/raft/FileBasedStateStore.java
@@ -71,7 +71,7 @@ public class FileBasedStateStore implements QuorumStateStore {
     }
 
     private QuorumStateData readStateFromFile(File file) {
-        try (final BufferedReader reader = 
Files.newBufferedReader(file.toPath())) {
+        try (final BufferedReader reader = 
Files.newBufferedReader(file.toPath(), StandardCharsets.UTF_8)) {
             final String line = reader.readLine();
             if (line == null) {
                 throw new EOFException("File ended prematurely.");
diff --git a/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java 
b/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java
index cceb65930ed..0b4fecd8d5a 100644
--- a/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java
+++ b/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java
@@ -29,10 +29,11 @@ import java.util.OptionalInt;
 import static java.util.Collections.singletonList;
 
 public class ReplicatedCounter implements RaftClient.Listener<Integer> {
+    private static final int SNAPSHOT_DELAY_IN_RECORDS = 10;
+
     private final int nodeId;
     private final Logger log;
     private final RaftClient<Integer> client;
-    private final int snapshotDelayInRecords = 10;
 
     private int committed = 0;
     private int uncommitted = 0;
@@ -107,7 +108,7 @@ public class ReplicatedCounter implements 
RaftClient.Listener<Integer> {
             }
             log.debug("Counter incremented from {} to {}", initialCommitted, 
committed);
 
-            if (lastOffsetSnapshotted + snapshotDelayInRecords < 
lastCommittedOffset) {
+            if (lastOffsetSnapshotted + SNAPSHOT_DELAY_IN_RECORDS < 
lastCommittedOffset) {
                 log.debug(
                     "Generating new snapshot with committed offset {} and 
epoch {} since the previous snapshot includes {}",
                     lastCommittedOffset,
diff --git 
a/server-common/src/main/java/org/apache/kafka/server/common/CheckpointFile.java
 
b/server-common/src/main/java/org/apache/kafka/server/common/CheckpointFile.java
index 747a16a6d29..818af8b3c1b 100644
--- 
a/server-common/src/main/java/org/apache/kafka/server/common/CheckpointFile.java
+++ 
b/server-common/src/main/java/org/apache/kafka/server/common/CheckpointFile.java
@@ -89,7 +89,7 @@ public class CheckpointFile<T> {
 
     public List<T> read() throws IOException {
         synchronized (lock) {
-            try (BufferedReader reader = 
Files.newBufferedReader(absolutePath)) {
+            try (BufferedReader reader = Files.newBufferedReader(absolutePath, 
StandardCharsets.UTF_8)) {
                 CheckpointReadBuffer<T> checkpointBuffer = new 
CheckpointReadBuffer<>(absolutePath.toString(), reader, version, formatter);
                 return checkpointBuffer.read();
             }
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/OffsetCheckpoint.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/OffsetCheckpoint.java
index 3ec238663dd..3ca0b7ea214 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/OffsetCheckpoint.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/OffsetCheckpoint.java
@@ -148,7 +148,7 @@ public class OffsetCheckpoint {
      */
     public Map<TopicPartition, Long> read() throws IOException {
         synchronized (lock) {
-            try (final BufferedReader reader = 
Files.newBufferedReader(file.toPath())) {
+            try (final BufferedReader reader = 
Files.newBufferedReader(file.toPath(), StandardCharsets.UTF_8)) {
                 final int version = readInt(reader);
                 switch (version) {
                     case 0:

Reply via email to