[ 
https://issues.apache.org/jira/browse/KAFKA-16858?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17850173#comment-17850173
 ] 

Greg Harris edited comment on KAFKA-16858 at 5/28/24 8:38 PM:
--------------------------------------------------------------

It really confused me that the stacktrace is in reverse order, is there a tool 
that does that by default?

I confirmed that ConnectSchema.validateValue() throws NPE when given a null 
schema, this is replicated with this assertion:

 
{noformat}
assertThrows(NullPointerException.class, () -> 
ConnectSchema.validateValue(null, new Object()));
{noformat}
I can also confirm that this NPE appears when using 
SchemaBuilder.arrayOfNull(), which would cause `valueSchema()` to return null. 
The exception only appears when the array is non-empty.

 

 
{noformat}
final String fieldName = "fieldName";
Schema testSchema = SchemaBuilder.struct()
        .field(fieldName, SchemaBuilder.arrayOfNull());
new Struct(testSchema).put(fieldName, Collections.emptyList());
assertThrows(NullPointerException.class, () -> 
        new Struct(testSchema).put(fieldName, Collections.singletonList(new 
Object())));
{noformat}
Looking at the way Flatten works, it's just copying schemas from input to 
output, so an arrayOfNull() can appear if the input contained an arrayOfNull(): 
[https://github.com/apache/kafka/blob/4d04eb83ea2cf67484d81e30603e976f63616428/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SchemaUtil.java#L29]
 

Since this is a type that SchemaBuilder can emit, we should probably handle it 
intentionally, either emitting a DataException, or passing the validation.

[~undone] For my own reference: Do you see this fault with another SMT in the 
chain ahead of Flatten? What Converter (sink connector) or source connector 
implementation are you using? There's also a very noisy log statement you could 
use to figure out where this null-array-schema is appearing: 
[https://github.com/apache/kafka/blob/4d04eb83ea2cf67484d81e30603e976f63616428/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java#L53]
 

 


was (Author: gharris1727):
It really confused me that the stacktrace is in reverse order, is there a tool 
that does that by default?

I confirmed that ConnectSchema.validateValue() throws NPE when given a null 
schema, this is replicated with this assertion:

 
{noformat}
assertThrows(NullPointerException.class, () -> 
ConnectSchema.validateValue(null, new Object()));
{noformat}
I can also confirm that this NPE appears when using 
SchemaBuilder.arrayOfNull(), which would cause `valueSchema()` to return null. 
The exception only appears when the array is non-empty.

 

 
{noformat}
final String fieldName = "fieldName";
Schema testSchema = SchemaBuilder.struct()
        .field(fieldName, SchemaBuilder.arrayOfNull());
new Struct(testSchema).put(fieldName, Collections.emptyList());
assertThrows(NullPointerException.class, () -> 
        new Struct(testSchema).put(fieldName, Collections.singletonList(new 
Object())));
{noformat}
Looking at the way Flatten works, it's just copying schemas from input to 
output, so an arrayOfNull() can appear if the input contained an arrayOfNull(): 
[https://github.com/apache/kafka/blob/4d04eb83ea2cf67484d81e30603e976f63616428/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SchemaUtil.java#L29]
 

Since this is a type that SchemaBuilder can emit, we should probably handle it 
intentionally, either emitting a DataException, or passing the validation.

[~undone] For my own reference: Do you see this fault with another SMT in the 
chain ahead of Flatten? What Converter implementation are you using? There's 
also a very noisy log statement you could use to figure out where this 
null-array-schema is appearing: 
[https://github.com/apache/kafka/blob/4d04eb83ea2cf67484d81e30603e976f63616428/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java#L53]
 

 

> Flatten SMT throws NPE
> ----------------------
>
>                 Key: KAFKA-16858
>                 URL: https://issues.apache.org/jira/browse/KAFKA-16858
>             Project: Kafka
>          Issue Type: Bug
>          Components: connect
>    Affects Versions: 3.6.0
>         Environment: Kafka 3.6 by way of CP 7.6.0
>            Reporter: Adam Strickland
>            Priority: Major
>         Attachments: FlattenTest.java
>
>
> {{ConnectSchema.expectedClassesFor}} sometimes will throw an NPE as part of a 
> call to an SMT chain.  Stack trace snippet:
> {{at 
> com.github.momenttechnology.kafka.connect.transforms.MomentFlatten.apply(MomentFlatten.java:84)}}
> {{at 
> com.github.momenttechnology.kafka.connect.transforms.MomentFlatten.applyWithSchema(MomentFlatten.java:173)}}
> {{at 
> com.github.momenttechnology.kafka.connect.transforms.MomentFlatten.buildWithSchema(MomentFlatten.java:280)}}
> {{at 
> com.github.momenttechnology.kafka.connect.transforms.MomentFlatten.buildWithSchema(MomentFlatten.java:280)}}
> {{at 
> com.github.momenttechnology.kafka.connect.transforms.MomentFlatten.buildWithSchema(MomentFlatten.java:280)}}
> {{at 
> com.github.momenttechnology.kafka.connect.transforms.MomentFlatten.buildWithSchema(MomentFlatten.java:274)}}
> {{at org.apache.kafka.connect.data.Struct.put(Struct.java:203)}}
> {{at org.apache.kafka.connect.data.Struct.put(Struct.java:216)}}
> {{at 
> org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:255)}}
> {{at 
> org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:213)}}
> {{at 
> org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:224)}}
> {{at 
> org.apache.kafka.connect.data.ConnectSchema.expectedClassesFor(ConnectSchema.java:268)}}
> (the above transform is a sub-class of 
> {{{}o.a.k.connect.transforms.Flatten{}}}; have confirmed that the error 
> occurs regardless).
> The field being transformed is an array of structs. If the call to 
> {{Schema#valueSchema()}} (o.a.k.connect.data.ConnectSchema.java:255) returns 
> {{{}null{}}}, the subsequent call to {{Schema#name()}} at 
> o.a.k.connect.data.ConnectSchema:268 throws an NPE.
> The strange thing that we have observed is that this doesn't always happen; 
> *sometimes* the struct's schema is found and sometimes it is not. We have 
> been unable to determine the root cause, but have constructed a test that 
> replicates the problem as observed (see attachment).
> In our case we have worked around the issue with the aforementioned sub-class 
> of {{{}Flatten{}}}, catching and logging the NPE on that specific use-case.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to