KAFKA-7058: Comparing schema default values using Objects#deepEquals()
https://issues.apache.org/jira/browse/KAFKA-7058
* Summary of testing strategy: Added new unit test
Author: Gunnar Morling <gunnar.morling@googlemail.com>
Reviewers: Randall Hauch <rhauch@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes #5225 from gunnarmorling/KAFKA-7058
(cherry picked from commit be846d833caade74f1d0536ecf9d540855cde758)
Signed-off-by: Ewen Cheslack-Postava <me@ewencp.org>
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 d1fd9cd..08a0ea3 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
@@ -276,7 +276,7 @@
ConnectSchema schema = (ConnectSchema) o;
return Objects.equals(optional, schema.optional) &&
Objects.equals(type, schema.type) &&
- Objects.equals(defaultValue, schema.defaultValue) &&
+ Objects.deepEquals(defaultValue, schema.defaultValue) &&
Objects.equals(fields, schema.fields) &&
Objects.equals(keySchema, schema.keySchema) &&
Objects.equals(valueSchema, schema.valueSchema) &&
diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java
index f5c6e2f..e7dfa4c 100644
--- a/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java
+++ b/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java
@@ -270,6 +270,16 @@
}
@Test
+ public void testArrayDefaultValueEquality() {
+ ConnectSchema s1 = new ConnectSchema(Schema.Type.ARRAY, false, new String[] {"a", "b"}, null, null, null, null, null, null, SchemaBuilder.int8().build());
+ ConnectSchema s2 = new ConnectSchema(Schema.Type.ARRAY, false, new String[] {"a", "b"}, null, null, null, null, null, null, SchemaBuilder.int8().build());
+ ConnectSchema differentValueSchema = new ConnectSchema(Schema.Type.ARRAY, false, new String[] {"b", "c"}, null, null, null, null, null, null, SchemaBuilder.int8().build());
+
+ assertEquals(s1, s2);
+ assertNotEquals(s1, differentValueSchema);
+ }
+
+ @Test
public void testMapEquality() {
// Same as testArrayEquality, but for both key and value schemas
ConnectSchema s1 = new ConnectSchema(Schema.Type.MAP, false, null, null, null, null, null, null, SchemaBuilder.int8().build(), SchemaBuilder.int16().build());