Skip to content

Commit

Permalink
KAFKA-4924: Fix Kafka Connect API findbugs warnings
Browse files Browse the repository at this point in the history
Author: Colin P. Mccabe <[email protected]>

Reviewers: Ewen Cheslack-Postava <[email protected]>

Closes apache#2715 from cmccabe/KAFKA-4924
  • Loading branch information
cmccabe authored and ewencp committed Mar 21, 2017
1 parent 6f7780c commit a7e3679
Show file tree
Hide file tree
Showing 3 changed files with 33 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ public class Decimal {
public static SchemaBuilder builder(int scale) {
return SchemaBuilder.bytes()
.name(LOGICAL_NAME)
.parameter(SCALE_FIELD, ((Integer) scale).toString())
.parameter(SCALE_FIELD, Integer.toString(scale))
.version(1);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -655,7 +655,8 @@ else if (value instanceof ByteBuffer)

throw new DataException("Couldn't convert " + value + " to JSON.");
} catch (ClassCastException e) {
throw new DataException("Invalid type for " + schema.type() + ": " + value.getClass());
String schemaTypeStr = (schema != null) ? schema.type().toString() : "unknown schema";
throw new DataException("Invalid type for " + schemaTypeStr + ": " + value.getClass());
}
}

Expand Down
30 changes: 30 additions & 0 deletions gradle/findbugs-exclude.xml
Original file line number Diff line number Diff line change
Expand Up @@ -25,4 +25,34 @@
<Match>
<Bug code="EI,EI2,MS"/>
</Match>

<Match>
<!-- Suppress a warning about some static initializers in Schema using instances of a
subclass. -->
<Or>
<Class name="org.apache.kafka.connect.data.Schema"/>
<Class name="org.apache.kafka.connect.data.SchemaBuilder"/>
</Or>
<Bug pattern="IC_SUPERCLASS_USES_SUBCLASS_DURING_INITIALIZATION"/>
</Match>

<Match>
<!-- Suppress warnings about unread protected fields in some public classes.
Although these are not read in Kafka Connect code, they are part of the API. -->
<Or>
<Class name="org.apache.kafka.connect.connector.Connector"/>
<Class name="org.apache.kafka.connect.sink.SinkTask"/>
<Class name="org.apache.kafka.connect.source.SourceTask"/>
</Or>
<Bug pattern="URF_UNREAD_PUBLIC_OR_PROTECTED_FIELD"/>
</Match>

<Match>
<!-- Suppress warnings about converting an integer number of
milliseconds to a java.util.Date object. We do this intentionally in
org.apache.kafka.connect.data.Time. -->
<Class name="org.apache.kafka.connect.data.Time"/>
<Method name="toLogical"/>
<Bug pattern="ICAST_INT_2_LONG_AS_INSTANT"/>
</Match>
</FindBugsFilter>

0 comments on commit a7e3679

Please sign in to comment.