gharris1727 commented on code in PR #13433:
URL: https://github.com/apache/kafka/pull/13433#discussion_r1149838302


##########
connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java:
##########
@@ -289,29 +291,202 @@ public ConnectSchema schema() {
     @Override
     public boolean equals(Object o) {
         if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-        ConnectSchema schema = (ConnectSchema) o;
-        return Objects.equals(optional, schema.optional) &&
-                Objects.equals(version, schema.version) &&
-                Objects.equals(name, schema.name) &&
-                Objects.equals(doc, schema.doc) &&
-                Objects.equals(type, schema.type) &&
-                Objects.deepEquals(defaultValue, schema.defaultValue) &&
-                Objects.equals(fields, schema.fields) &&
-                Objects.equals(keySchema, schema.keySchema) &&
-                Objects.equals(valueSchema, schema.valueSchema) &&
-                Objects.equals(parameters, schema.parameters);
+        if (!(o instanceof Schema)) return false;
+        Schema schema = (Schema) o;
+        return equals(this, schema);
+    }
+
+    private static boolean equals(Schema left, Schema right) {
+        return equals(left, right, new IdentityHashMap<>());
+    }
+
+    private static boolean equals(Schema left, Schema right, 
IdentityHashMap<Schema, Schema> equivalentSchemas) {
+        if (left == right)
+            return true;
+
+        if (equivalentSchemas.containsKey(left)) {
+            // Use referential equality because object equality might cause a 
stack overflow
+            return equivalentSchemas.get(left) == right;
+        }
+
+        boolean shallowMatches = Objects.equals(left.isOptional(), 
right.isOptional()) &&
+                Objects.equals(left.version(), right.version()) &&
+                Objects.equals(left.name(), right.name()) &&
+                Objects.equals(left.doc(), right.doc()) &&
+                Objects.equals(left.type(), right.type()) &&
+                Objects.equals(left.parameters(), right.parameters());
+        if (!shallowMatches)
+            return false;
+
+        // Avoid mutating the passed-in map, since that may interfere with 
recursive calls higher up the stack
+        IdentityHashMap<Schema, Schema> equivalentSchemasCopy = new 
IdentityHashMap<>(equivalentSchemas);
+        equivalentSchemasCopy.put(left, right);

Review Comment:
   This is a very contrived example that I don't think is likely to happen in 
practice. It relies on setting up schemas which are equal _and_ referentially 
distinct _and_ form a referential cycle. In practice these three conditions 
don't appear together:
   
   Equal and referentially distinct schemas aren't going to form a referential 
cycle: If you're comparing two deserializations of the same schema, one must 
have been deserialized first and cannot contain a reference to the second.
   
   Schemas which are referentially distinct and form a cycle are not going to 
be equal: If some cycle has N nodes, then those N different schemas must be 
meaningfully different and have a different name or set of fields.
   
   And if your schemas are equal and form a referential cycle, the deserializer 
is going to make them referentially equal: All nodes in a graph will have a 
single schema that is referentially shared, otherwise it is performing needless 
work to copy the same schema multiple times.
   
   I don't think this needs to be changed, I just pointed it out as a 
limitation of the data structure choice.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to