This is an automated email from the ASF dual-hosted git repository. kkarantasis pushed a commit to branch 2.6 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/2.6 by this push: new a5da4e2 KAFKA-8938: Improve allocations during Struct validation in ConnectSchema (#7384) a5da4e2 is described below commit a5da4e2ee95189cb3e3ffe81cfd52d4be4338153 Author: Auston AuthorDate: Tue Jun 9 14:55:56 2020 -0700 KAFKA-8938: Improve allocations during Struct validation in ConnectSchema (#7384) Struct value validation in Kafka Connect can be optimized to avoid creating an Iterator when the expectedClasses list is of size 1. This is a meaningful enhancement for high throughput connectors. Reviewers: Konstantine Karantasis --- .../java/org/apache/kafka/connect/data/ConnectSchema.java | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java b/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java index 66630d1..a465b12 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java @@ -229,12 +229,17 @@ public class ConnectSchema implements Schema { + " for field: \"" + name + "\""); boolean foundMatch = false; - for (Class expectedClass : expectedClasses) { - if (expectedClass.isInstance(value)) { - foundMatch = true; - break; + if (expectedClasses.size() == 1) { + foundMatch = expectedClasses.get(0).isInstance(value); + } else { + for (Class expectedClass : expectedClasses) { + if (expectedClass.isInstance(value)) { + foundMatch = true; + break; + } } } + if (!foundMatch) throw new DataException("Invalid Java object for schema type " + schema.type() + ": " + value.getClass()