[jira] [Commented] (KAFKA-16858) Flatten SMT throws NPE
[ https://issues.apache.org/jira/browse/KAFKA-16858?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17850866#comment-17850866 ] Adam Strickland commented on KAFKA-16858: - * {quote}Are you able to provide an anonymized form of your schema directly, rather than just a high-level "Array of Structs"? I'm wondering if your schema is capable of triggering the use of the mutable SchemaWrapper [https://github.com/confluentinc/schema-registry/blob/7b886f309c83041d4f2a5b41b5910f3b8002413a/protobuf-converter/src/main/java/io/confluent/connect/protobuf/ProtobufData.java#L1779] inside the ProtobufConverter.{quote} Yes; see [^proto.proto] * {quote}w.r.t. the variable validateValue depth: Are you saying that in _error cases_ the recursion depth is unpredictable, or in general? The validateValue should be called at every or almost every location in the tree of values, so I would expect to see lots of different recursion depths. Maybe you can share some more stacktraces as examples.{quote} We were only looking at error cases. I specifically recall seeing the same message break on the 3rd recursion and on the 7th; the array for the message in question contained 2 Structs. * {quote}So far in this investigation, I'm trying to find the source of the null in hopes that we can prevent it, and get well-formed data to the Flatten SMT. Regardless of the result of that investigation, I think we can consider this input malformed, and throw an intentional DataException instead of NullPointerException. Would that be an acceptable solution for you, or does this data need to make it all the way through the pipeline?{quote} Throwing a DataException would be preferable to the NPE. I'm not sure I understand what you mean by "need to make it all the way through the pipeline"... As it stands the particular problematic attribute is not something we care about right now, which is why we can simply squash the Exception. > 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, proto.proto > > > {{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)
[jira] [Commented] (KAFKA-16858) Flatten SMT throws NPE
[ https://issues.apache.org/jira/browse/KAFKA-16858?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17850530#comment-17850530 ] Greg Harris commented on KAFKA-16858: - Thanks so much for the additional context [~undone] this is a very odd bug. I managed to reproduce a Flatten NPE with mutable input objects rather than mocking: {noformat} xformValue.configure(Collections.emptyMap()); Schema innerSchema = SchemaBuilder.struct().optional(); Struct innerStruct = new Struct(innerSchema); AtomicReference valueSchema = new AtomicReference<>(innerSchema); Schema arraySchema = SchemaBuilder.array(new ConnectSchema(Schema.Type.ARRAY) { @Override public Schema valueSchema() { return valueSchema.get(); } }); Schema schema = SchemaBuilder.struct().field("field", arraySchema); Struct value = new Struct(schema).put("field", Collections.singletonList(Collections.singletonList(innerStruct))); valueSchema.set(null); SourceRecord record = new SourceRecord(null, null, "topic", 0, schema, value); xformValue.apply(record);{noformat} It throws this error: {noformat} Cannot invoke "org.apache.kafka.connect.data.Schema.name()" because "schema" is null java.lang.NullPointerException: Cannot invoke "org.apache.kafka.connect.data.Schema.name()" because "schema" is null at org.apache.kafka.connect.data.ConnectSchema.expectedClassesFor(ConnectSchema.java:268) at org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:224) at org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:213) 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:255) at org.apache.kafka.connect.data.Struct.put(Struct.java:216) at org.apache.kafka.connect.data.Struct.put(Struct.java:203) at org.apache.kafka.connect.transforms.Flatten.buildWithSchema(Flatten.java:250) at org.apache.kafka.connect.transforms.Flatten.applyWithSchema(Flatten.java:164) at org.apache.kafka.connect.transforms.Flatten.apply(Flatten.java:79){noformat} This is different from your stacktrace in two ways: # Instead of 4 buildWithSchema calls, there's only 1. This is because my test Struct is nested less deeply than your Struct, and a deeper test case behaves almost identically. # Instead of 3 validateValue calls, there's 5. This is because I'm using an a nested array "[[Struct]]" instead of your singly-nested array "[Struct]". This one is a bit more important, because the reproduction case doesn't work for singly-nested arrays. The difference is that a singly-nested array has it's valueSchema evaluated during `buildUpdatedSchema`, and the doubly-nested array has it's valueSchema evaluated during `buildWithSchema`. When the null valueSchema is evaluated during buildUpdatedSchema, it throws this exception instead: {noformat} valueSchema cannot be null. org.apache.kafka.connect.errors.SchemaBuilderException: valueSchema cannot be null. at app//org.apache.kafka.connect.data.SchemaBuilder.array(SchemaBuilder.java:363) at app//org.apache.kafka.connect.transforms.util.SchemaUtil.copySchemaBasics(SchemaUtil.java:29) at app//org.apache.kafka.connect.transforms.Flatten.convertFieldSchema(Flatten.java:225) at app//org.apache.kafka.connect.transforms.Flatten.buildUpdatedSchema(Flatten.java:201) at app//org.apache.kafka.connect.transforms.Flatten.applyWithSchema(Flatten.java:156) at app//org.apache.kafka.connect.transforms.Flatten.apply(Flatten.java:79){noformat} I was unable to reproduce this with non-mutable schemas, they trigger the NPE too early while the input Struct is being constructed. Some follow-ups: * Are you able to provide an anonymized form of your schema directly, rather than just a high-level "Array of Structs"? I'm wondering if your schema is capable of triggering the use of the mutable SchemaWrapper [https://github.com/confluentinc/schema-registry/blob/7b886f309c83041d4f2a5b41b5910f3b8002413a/protobuf-converter/src/main/java/io/confluent/connect/protobuf/ProtobufData.java#L1779] inside the ProtobufConverter. * I don't have an explanation of how this can happen for empty and non-present arrays, as it looks like validateValue(ConnectSchema:255) can only be triggered by non-empty lists. * w.r.t. the variable validateValue depth: Are you saying that in _error cases_ the recursion depth is unpredictable, or in general? The validateValue should be called at every or almost every location in the tree of values, so I would expect to see lots of different recursion depths. Maybe you can share some more stacktraces as examples. * So far in this investigation, I'm trying to find the source of the null in hopes that we can prevent it, a
[jira] [Commented] (KAFKA-16858) Flatten SMT throws NPE
[ https://issues.apache.org/jira/browse/KAFKA-16858?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17850375#comment-17850375 ] Adam Strickland commented on KAFKA-16858: - {quote}Do you see this fault with another SMT in the chain ahead of Flatten? {quote} No. It consistently shows up only from the Flatten SMT and always on the same attribute. {quote}What Converter (sink connector) or source connector implementation are you using? {quote} This is a sink connector, writing to Postgres using `io.confluent.connect.jdbc.JdbcSinkConnector`. The value converter is `io.confluent.connect.protobuf.ProtobufConverter`. Some other data points: * We use this same SMT in multiple connectors; we see the NPE in only one of them. * While it always throws from the same attribute, it doesn't *always* throw... For instance, the NPE was logged 52 times within the last 24 hours, out of 234 messages processed via that connector. * The connector we see it on has multiple SMTs; the Flatten is the 4th SMT applied. * For reference, the SMTs being applied prior to Flatten are, in order: ## `io.confluent.connect.transforms.Filter$Value`, ## `o.a.k.connect.transforms.ExtravtField$Value`, and ## another `o.a.k.connect.transforms.ExtravtField$Value` * The attribute it breaks on is an array of `Struct`s; the array can be empty and the attribute is itself optional. We have seen the break occur in 3 states (attribute missing, attribute present with empty list, and attribute present with a list). We have not seen it occur when the attribute is present but the value is `null`. However, I think that's simply a use case we do not have. * We have not used the trace logger you linked to, but at one point inserted our own logging statements at various levels in our custom transform; this was how we found the faulty attribute. One other thing we have noticed is the stack recurses all the way down into `ConnectSchema.validateValue` an unexpected number of times, resolving to `Schema\{STRUCT}` some number of times prior to returning `null`. The number of times is not consistent and does not correlate (as far as we can tell) with the number of items in the array or any other measure we have checked. > 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
[jira] [Commented] (KAFKA-16858) Flatten SMT throws NPE
[ https://issues.apache.org/jira/browse/KAFKA-16858?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17850361#comment-17850361 ] Adam Strickland commented on KAFKA-16858: - {quote}It really confused me that the stacktrace is in reverse order, is there a tool that does that by default? {quote} Oh yeah, sorry about that, I guess I've gotten used to it. It's from streaming the logs to DataDog and not accounting for the linebreaks in Java stacktraces. One of the many items on the list... > 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)
[jira] [Commented] (KAFKA-16858) Flatten SMT throws NPE
[ https://issues.apache.org/jira/browse/KAFKA-16858?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17850174#comment-17850174 ] Greg Harris commented on KAFKA-16858: - It looks like this should probably be a DataException. We use arrayOfNull() in Values/SimpleHeaderConverter when inferring the schema for an empty array, and generally assume that when a root value is schema'd, that all contained values (within a map, array, or struct) are also schema'd. If this validation passed, it would make null function as top-type, and allow non-schema'd data embedded inside of schema'd data. > 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)
[jira] [Commented] (KAFKA-16858) Flatten SMT throws NPE
[ https://issues.apache.org/jira/browse/KAFKA-16858?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17850173#comment-17850173 ] Greg Harris commented on KAFKA-16858: - 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)