You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "urbandan (via GitHub)" <gi...@apache.org> on 2023/06/05 11:44:37 UTC

[GitHub] [kafka] urbandan commented on a diff in pull request #13433: KAFKA-12694, KAFKA-3910: Add cyclic schema support, fix default struct values

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


##########
connect/api/src/test/java/org/apache/kafka/connect/data/SchemaBuilderTest.java:
##########
@@ -36,6 +38,255 @@ public class SchemaBuilderTest {
     private static final String DOC = "doc";
     private static final Map<String, String> NO_PARAMS = null;
 
+    @Test
+    public void testDefaultValueStructSchema() {
+        SchemaBuilder builder = SchemaBuilder.struct()
+                .field("f1", Schema.BOOLEAN_SCHEMA);
+
+        Struct defaultValue = new Struct(builder.build()); // the Struct receives a schema, not a builder
+        defaultValue.put("f1", true);
+
+        builder.defaultValue(defaultValue)
+                .build();
+    }
+
+    @Test
+    public void testDefaultValueStructSchemaBuilder() {
+        SchemaBuilder builder = SchemaBuilder.struct()
+                .field("f1", Schema.BOOLEAN_SCHEMA);
+
+        Struct defaultValue = new Struct(builder);
+        defaultValue.put("f1", true);
+
+        builder.defaultValue(defaultValue).build();

Review Comment:
   Sounds good to me - I agree, your suggestion would fix this specific issue.
   
   I'm wondering though that this might be the consequence of a bigger, underlying issue - namely that the schema and the struct can both change after the default value check has been done. This might be out-of-scope for this change, but what if we also:
   
   1. added an "immutable" flag to Struct, which can be set to true to not allow further modifications on the Struct instance
   2. added an "immutable" flag to SchemaBuilder, too (does not allow further modifications on the builder)
   3. deprecated the current SchemaBuilder::defaultValue
   4. and added instead a new SchemaBuilder::buildWithDefault(Struct) which would require Struct::schema to be the same as "this" (the SchemaBuilder insance), validate the Struct, set the SchemaBuilder::immutable flag to true, set the Struct::immutable flag to true, and return a ConnectSchema instance:
   
   ```
   public ConnectSchema buildWithDefaultValue(Object value) {
       checkCanSet(DEFAULT_FIELD, defaultValue, value);
       checkNotNull(TYPE_FIELD, type, DEFAULT_FIELD);
       try {
           ConnectSchema.validateValue(this, value);
       } catch (DataException e) {
           throw new SchemaBuilderException("Invalid default value", e);
       }
       if (value instanceof Struct) {
       	Struct struct = (Struct) value;
   	if (struct.schema() != this) {
   	    throw new SchemaBuilderException("Default Struct value must use the same SchemaBuilder instance as schema");
   	}
   	struct.setImmutable();
       }
       defaultValue = value;
       this.immutable = true;
       return new ConnectSchema(type, isOptional(), defaultValue, name, version, doc,
                   parameters == null ? null : Collections.unmodifiableMap(parameters),
                   fields == null ? null : Collections.unmodifiableList(new ArrayList<>(fields.values())), keySchema, valueSchema);
   }
   ```
   
   This would ensure that:
   1. The Struct cannot be modified after it was set as a default value.
   2. The underlying schema referenced by the Struct (SchemaBuilder instance) cannot be modified after the struct default was validated and set.
   3. Deprecates the old default value setting procedure (clearly indicates the preferred solution)
   
   This would still not ensure that:
   1. The schema cannot be modified after the Struct was validated



-- 
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-unsubscribe@kafka.apache.org

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