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


##########
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:
   > only the topmost put (which returned null) should be able to remove from 
the map.
   
   This would eliminate copying the map, but would still cause re-computation 
to happen in other branches.
   I don't think optimizing away the map copying is necessary, the overhead 
should be relatively minimal.



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