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


##########
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:
   I think this is only acceptable for when comparing two schemas, and fails 
once you compare three or more.
   Consider the following test (A->B->C->A) which fails:
   ```
           SchemaBuilder builderA = SchemaBuilder.struct().optional()
                   .field("f1", Schema.INT32_SCHEMA);
           SchemaBuilder builderB = SchemaBuilder.struct().optional()
                   .field("f1", Schema.INT32_SCHEMA);
           SchemaBuilder builderC = SchemaBuilder.struct().optional()
                   .field("f1", Schema.INT32_SCHEMA);
           builderA.field("f2", builderB);
           builderB.field("f2", builderC);
           builderC.field("f2", builderA);
   
           Schema schemaA = builderA.build();
           Schema schemaB = builderB.build();
           Schema schemaC = builderC.build();
   
           assertEquals(schemaA, schemaB);
           assertEquals(schemaB, schemaC);
           assertEquals(schemaC, schemaA);
   ```
   compared to this mutually recursive test (A->B->A) which passes:
   ```
           SchemaBuilder builderA = SchemaBuilder.struct().optional()
                   .field("f1", Schema.INT32_SCHEMA);
           SchemaBuilder builderB = SchemaBuilder.struct().optional()
                   .field("f1", Schema.INT32_SCHEMA);
           builderA.field("f2", builderB);
           builderB.field("f2", builderA);
   
           Schema schemaA = builderA.build();
           Schema schemaB = builderB.build();
   
           assertEquals(schemaA, schemaB);
   ```



##########
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:
   This copy is certainly necessary. Because elements in this map only have 
shallow equality verified, it is only acceptable to use it to short-circuit in 
recursive calls when you know an earlier call is actually going to do all of 
the verification.
   
   WDYT about also passing around a second map that is only written to when the 
deep equality is true, and using that to reduce re-computation when multiple 
fields contain the same non-trivial schema. This would be acceptable to mutate 
during calls without copying.



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