rdblue commented on code in PR #12238:
URL: https://github.com/apache/iceberg/pull/12238#discussion_r1955075130
##########
core/src/test/java/org/apache/iceberg/avro/TestAvroSchemaProjection.java:
##########
@@ -150,4 +150,58 @@ public void projectWithMapSchemaChanged() {
.as("Result of buildAvroProjection is missing some IDs")
.isFalse();
}
+
+ @Test
+ public void projectWithVariantSchemaChanged() {
+ final org.apache.avro.Schema currentAvroSchema =
+ SchemaBuilder.record("myrecord")
+ .fields()
+ .name("f11")
+ .type()
+ .nullable()
+ .intType()
+ .noDefault()
+ .endRecord();
+
+ final org.apache.avro.Schema variantSchema =
+ SchemaBuilder.record("v")
+ .fields()
+ .name("metadata")
+ .type()
+ .bytesType()
+ .noDefault()
+ .name("value")
+ .type()
+ .bytesType()
+ .noDefault()
+ .endRecord();
+ Variant.get().addToSchema(variantSchema);
+
+ final org.apache.avro.Schema updatedAvroSchema =
+ SchemaBuilder.record("myrecord")
+ .fields()
+ .name("f11")
+ .type()
+ .nullable()
+ .intType()
+ .noDefault()
+ .name("f12")
+ .type(variantSchema)
+ .noDefault()
+ .endRecord();
+
+ final Schema currentIcebergSchema =
AvroSchemaUtil.toIceberg(currentAvroSchema);
+
+ // Getting the node ID in updatedAvroSchema allocated by converting into
iceberg schema and back
+ final org.apache.avro.Schema idAllocatedUpdatedAvroSchema =
+
AvroSchemaUtil.convert(AvroSchemaUtil.toIceberg(updatedAvroSchema).asStruct());
+
+ final org.apache.avro.Schema projectedAvroSchema =
+ AvroSchemaUtil.buildAvroProjection(
+ idAllocatedUpdatedAvroSchema, currentIcebergSchema,
Collections.emptyMap());
+
+ assertThat(AvroSchemaUtil.missingIds(projectedAvroSchema))
+ .as("Result of buildAvroProjection is missing some IDs")
+ .isFalse();
Review Comment:
I don't think this is sufficient to test changes to `BuildAvroProjection`.
For that class, we need to test that the variant can be selected. This asserts
that all of the projected fields (regardless of whether a variant was
projected) have IDs. And I don't think that the variant would be projected
because the `expected` schema is a conversion from the "current" Avro schema
that doesn't have the variant field.
There should be a test case that projects the variant field. I also
recommend having fewer schemas. You only need 3: an Iceberg schema for all
fields, that Iceberg schema converted to Avro (representing a file schema), and
an Iceberg schema that projects the Variant field (produced using
`schema.select`).
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]