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


##########
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);

Review Comment:
   Since equality checking might be taking place in a connector pipeline on a 
per-record basis, it probably is worth trying to optimize this part a bit. I've 
been wracking my head trying to think of a case where reusing the map across 
stack frames/consecutive calls might cause problems, but haven't been able to 
think of any just yet (besides the counterexamples provided in the now-resolved 
comment about three-way cycles).



-- 
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