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 be846d833c)
Signed-off-by: Ewen Cheslack-Postava <me@ewencp.org>
This commit is contained in:
Gunnar Morling 2018-06-16 23:04:31 -07:00 committed by Ewen Cheslack-Postava
parent 689bac3c6f
commit 0f3affc0f4
2 changed files with 11 additions and 1 deletions

View File

@ -276,7 +276,7 @@ public class ConnectSchema implements Schema {
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) &&

View File

@ -269,6 +269,16 @@ public class ConnectSchemaTest {
assertNotEquals(s1, differentValueSchema);
}
@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