yihua commented on code in PR #14265: URL: https://github.com/apache/hudi/pull/14265#discussion_r2547898812
########## hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaCompatibility.java: ########## @@ -0,0 +1,206 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.schema; + +import org.apache.hudi.avro.AvroSchemaUtils; +import org.apache.hudi.common.util.ValidationUtils; + +import java.util.Collections; +import java.util.Set; + +/** + * Utility class for checking HoodieSchema compatibility and evolution rules. + * This provides HoodieSchema-native methods that delegate to existing AvroSchemaUtils + * functionality while maintaining the architectural separation. + * + * <p>This class handles schema compatibility checking, which is crucial for: + * <ul> + * <li>Table schema evolution validation</li> + * <li>Writer schema compatibility with table schema</li> + * <li>Projection compatibility for query optimization</li> + * <li>Metadata field handling during schema checks</li> + * </ul> + */ +public final class HoodieSchemaCompatibility { + + // Prevent instantiation + private HoodieSchemaCompatibility() { + } + + /** + * Checks if writer schema is compatible with table schema for write operations. + * This is equivalent to AvroSchemaUtils.checkSchemaCompatible() but operates on HoodieSchemas. + * + * @param tableSchema the table schema to check against + * @param writerSchema the writer schema to validate + * @param shouldValidate whether to perform compatibility validation + * @param allowProjection whether to allow projection (fewer fields in writer) + * @throws IllegalArgumentException if schemas are null + * @throws org.apache.hudi.exception.SchemaCompatibilityException if schemas are incompatible + */ + public static void checkSchemaCompatible(HoodieSchema tableSchema, HoodieSchema writerSchema, + boolean shouldValidate, boolean allowProjection) { + ValidationUtils.checkArgument(tableSchema != null, "Table schema cannot be null"); + ValidationUtils.checkArgument(writerSchema != null, "Writer schema cannot be null"); + + // Delegate to AvroSchemaUtils for the actual compatibility check + AvroSchemaUtils.checkSchemaCompatible( + tableSchema.toAvroSchema(), + writerSchema.toAvroSchema(), + shouldValidate, + allowProjection, + Collections.emptySet()); // Default to no partition columns + } + + /** + * Checks if writer schema is compatible with table schema, excluding specified partition columns. + * This is equivalent to AvroSchemaUtils.checkSchemaCompatible() with partition column exclusions. + * + * @param tableSchema the table schema to check against + * @param writerSchema the writer schema to validate + * @param shouldValidate whether to perform compatibility validation + * @param allowProjection whether to allow projection (fewer fields in writer) + * @param partitionColumns set of partition column names to exclude from compatibility checks + * @throws IllegalArgumentException if schemas are null + * @throws org.apache.hudi.exception.SchemaCompatibilityException if schemas are incompatible + */ + public static void checkSchemaCompatible(HoodieSchema tableSchema, HoodieSchema writerSchema, + boolean shouldValidate, boolean allowProjection, + Set<String> partitionColumns) { + ValidationUtils.checkArgument(tableSchema != null, "Table schema cannot be null"); + ValidationUtils.checkArgument(writerSchema != null, "Writer schema cannot be null"); + + if (!shouldValidate) { + return; + } + + // Delegate to AvroSchemaUtils for the actual compatibility check + AvroSchemaUtils.checkSchemaCompatible( + tableSchema.toAvroSchema(), + writerSchema.toAvroSchema(), + shouldValidate, + allowProjection, + partitionColumns != null ? partitionColumns : Collections.emptySet()); + } + + /** + * Validates that the incoming schema represents a valid evolution of the table schema. + * This is equivalent to AvroSchemaUtils.checkValidEvolution() but operates on HoodieSchemas. + * + * @param incomingSchema the new schema being introduced + * @param tableSchema the existing table schema + * @throws IllegalArgumentException if schemas are null + * @throws org.apache.hudi.exception.SchemaCompatibilityException if evolution is invalid + */ + public static void checkValidEvolution(HoodieSchema incomingSchema, HoodieSchema tableSchema) { + ValidationUtils.checkArgument(incomingSchema != null, "Incoming schema cannot be null"); + ValidationUtils.checkArgument(tableSchema != null, "Table schema cannot be null"); + + // Delegate to AvroSchemaUtils for evolution validation + AvroSchemaUtils.checkValidEvolution(incomingSchema.toAvroSchema(), tableSchema.toAvroSchema()); + } + + /** + * Checks if two schemas are compatible in terms of data reading. + * This uses the same logic as AvroSchemaUtils.isSchemaCompatible() but for HoodieSchemas. + * + * @param readerSchema the schema used to read the data + * @param writerSchema the schema used to write the data + * @return true if reader schema can read data written with writer schema + * @throws IllegalArgumentException if schemas are null + */ + public static boolean isSchemaCompatible(HoodieSchema readerSchema, HoodieSchema writerSchema) { + ValidationUtils.checkArgument(readerSchema != null, "Reader schema cannot be null"); + ValidationUtils.checkArgument(writerSchema != null, "Writer schema cannot be null"); + + // Use HoodieSchemaUtils delegation for consistency + return AvroSchemaUtils.isSchemaCompatible(readerSchema.toAvroSchema(), writerSchema.toAvroSchema()); Review Comment: `isSchemaCompatible(readerSchema, writerSchema, true)` to avoid duplication? ########## hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaField.java: ########## @@ -0,0 +1,393 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.schema; + +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; + +import org.apache.avro.Schema; + +import java.io.Serializable; +import java.util.Map; +import java.util.Objects; + +/** + * Wrapper class for Avro Schema.Field that provides Hudi-specific field functionality + * while maintaining binary compatibility with Avro. + * + * <p>This class encapsulates an Avro Schema.Field and provides a consistent interface + * for field operations across the Hudi codebase. It maintains full compatibility with + * Avro by delegating all operations to the underlying Avro field.</p> + * + * <p>Usage example: + * <pre>{@code + * // Create from Avro field + * Schema.Field avroField = new Schema.Field("name", Schema.create(Schema.Type.STRING)); + * HoodieSchemaField hoodieField = HoodieSchemaField.fromAvroField(avroField); + * + * // Access field properties + * String name = hoodieField.name(); + * HoodieSchema schema = hoodieField.schema(); + * Option<Object> defaultValue = hoodieField.defaultVal(); + * }</pre></p> + * + * @since 1.2.0 + */ +public class HoodieSchemaField implements Serializable { + + private static final long serialVersionUID = 1L; + + private final Schema.Field avroField; + private final HoodieSchema hoodieSchema; Review Comment: ```suggestion private final HoodieSchema fieldSchema; ``` ########## hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaField.java: ########## @@ -0,0 +1,393 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.schema; + +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; + +import org.apache.avro.Schema; + +import java.io.Serializable; +import java.util.Map; +import java.util.Objects; + +/** + * Wrapper class for Avro Schema.Field that provides Hudi-specific field functionality + * while maintaining binary compatibility with Avro. + * + * <p>This class encapsulates an Avro Schema.Field and provides a consistent interface + * for field operations across the Hudi codebase. It maintains full compatibility with + * Avro by delegating all operations to the underlying Avro field.</p> + * + * <p>Usage example: + * <pre>{@code + * // Create from Avro field + * Schema.Field avroField = new Schema.Field("name", Schema.create(Schema.Type.STRING)); + * HoodieSchemaField hoodieField = HoodieSchemaField.fromAvroField(avroField); + * + * // Access field properties + * String name = hoodieField.name(); + * HoodieSchema schema = hoodieField.schema(); + * Option<Object> defaultValue = hoodieField.defaultVal(); + * }</pre></p> + * + * @since 1.2.0 + */ +public class HoodieSchemaField implements Serializable { + + private static final long serialVersionUID = 1L; + + private final Schema.Field avroField; + private final HoodieSchema hoodieSchema; + + /** + * Creates a new HoodieSchemaField wrapping the given Avro field. + * + * @param avroField the Avro field to wrap, cannot be null + */ + public HoodieSchemaField(Schema.Field avroField) { + ValidationUtils.checkArgument(avroField != null, "Avro field cannot be null"); + this.avroField = avroField; + this.hoodieSchema = HoodieSchema.fromAvroSchema(avroField.schema()); + } + + /** + * Factory method to create HoodieSchemaField from an Avro field. + * + * @param avroField the Avro field to wrap + * @return new HoodieSchemaField instance + * @throws IllegalArgumentException if avroField is null + */ + public static HoodieSchemaField fromAvroField(Schema.Field avroField) { + return new HoodieSchemaField(avroField); + } + + /** + * Creates a new HoodieSchemaField with the specified properties. + * + * @param name the name of the field + * @param schema the schema of the field + * @param doc the documentation string, can be null + * @param defaultVal the default value, can be null + * @return new HoodieSchemaField instance + */ + public static HoodieSchemaField of(String name, HoodieSchema schema, String doc, Object defaultVal) { + return of(name, schema, doc, defaultVal, HoodieFieldOrder.ASCENDING); + } + + /** + * Creates a new HoodieSchemaField with the specified properties, including field order. + * + * @param name the name of the field + * @param schema the schema of the field + * @param doc the documentation string, can be null + * @param defaultVal the default value, can be null + * @param order the field order for sorting + * @return new HoodieSchemaField instance + */ + public static HoodieSchemaField of(String name, HoodieSchema schema, String doc, Object defaultVal, HoodieFieldOrder order) { + ValidationUtils.checkArgument(name != null && !name.isEmpty(), "Field name cannot be null or empty"); + ValidationUtils.checkArgument(schema != null, "Field schema cannot be null"); + ValidationUtils.checkArgument(order != null, "Field order cannot be null"); + + Schema avroSchema = schema.getAvroSchema(); + ValidationUtils.checkState(avroSchema != null, "Schema's Avro schema cannot be null"); + + Schema.Field avroField = new Schema.Field(name, avroSchema, doc, defaultVal, order.toAvroOrder()); Review Comment: There is a compatibility issue with `new Schema.Field` constructor on Avro 1.12.0 used by Spark 4.0, so `HoodieAvroUtils.createNewSchemaField` needs to be used which handles the compatibility issue: https://github.com/apache/hudi/blob/89f622f8244974c56c6c9ec72377a4c3f7bec3f2/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java#L356 ########## hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaField.java: ########## @@ -0,0 +1,393 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.schema; + +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; + +import org.apache.avro.Schema; + +import java.io.Serializable; +import java.util.Map; +import java.util.Objects; + +/** + * Wrapper class for Avro Schema.Field that provides Hudi-specific field functionality + * while maintaining binary compatibility with Avro. + * + * <p>This class encapsulates an Avro Schema.Field and provides a consistent interface + * for field operations across the Hudi codebase. It maintains full compatibility with + * Avro by delegating all operations to the underlying Avro field.</p> + * + * <p>Usage example: + * <pre>{@code + * // Create from Avro field + * Schema.Field avroField = new Schema.Field("name", Schema.create(Schema.Type.STRING)); + * HoodieSchemaField hoodieField = HoodieSchemaField.fromAvroField(avroField); + * + * // Access field properties + * String name = hoodieField.name(); + * HoodieSchema schema = hoodieField.schema(); + * Option<Object> defaultValue = hoodieField.defaultVal(); + * }</pre></p> + * + * @since 1.2.0 + */ +public class HoodieSchemaField implements Serializable { + + private static final long serialVersionUID = 1L; + + private final Schema.Field avroField; + private final HoodieSchema hoodieSchema; + + /** + * Creates a new HoodieSchemaField wrapping the given Avro field. + * + * @param avroField the Avro field to wrap, cannot be null + */ + public HoodieSchemaField(Schema.Field avroField) { + ValidationUtils.checkArgument(avroField != null, "Avro field cannot be null"); + this.avroField = avroField; + this.hoodieSchema = HoodieSchema.fromAvroSchema(avroField.schema()); + } + + /** + * Factory method to create HoodieSchemaField from an Avro field. + * + * @param avroField the Avro field to wrap + * @return new HoodieSchemaField instance + * @throws IllegalArgumentException if avroField is null + */ + public static HoodieSchemaField fromAvroField(Schema.Field avroField) { + return new HoodieSchemaField(avroField); + } + + /** + * Creates a new HoodieSchemaField with the specified properties. + * + * @param name the name of the field + * @param schema the schema of the field + * @param doc the documentation string, can be null + * @param defaultVal the default value, can be null + * @return new HoodieSchemaField instance + */ + public static HoodieSchemaField of(String name, HoodieSchema schema, String doc, Object defaultVal) { + return of(name, schema, doc, defaultVal, HoodieFieldOrder.ASCENDING); + } + + /** + * Creates a new HoodieSchemaField with the specified properties, including field order. + * + * @param name the name of the field + * @param schema the schema of the field + * @param doc the documentation string, can be null + * @param defaultVal the default value, can be null + * @param order the field order for sorting + * @return new HoodieSchemaField instance + */ + public static HoodieSchemaField of(String name, HoodieSchema schema, String doc, Object defaultVal, HoodieFieldOrder order) { + ValidationUtils.checkArgument(name != null && !name.isEmpty(), "Field name cannot be null or empty"); + ValidationUtils.checkArgument(schema != null, "Field schema cannot be null"); + ValidationUtils.checkArgument(order != null, "Field order cannot be null"); + + Schema avroSchema = schema.getAvroSchema(); + ValidationUtils.checkState(avroSchema != null, "Schema's Avro schema cannot be null"); + + Schema.Field avroField = new Schema.Field(name, avroSchema, doc, defaultVal, order.toAvroOrder()); + return new HoodieSchemaField(avroField); + } + + /** + * Creates a new HoodieSchemaField with the specified name and schema. + * + * @param name the name of the field + * @param schema the schema of the field + * @return new HoodieSchemaField instance + */ + public static HoodieSchemaField of(String name, HoodieSchema schema) { + return of(name, schema, null, null); + } + + /** + * Creates a new HoodieSchemaField with the specified name, schema, and doc. + * + * @param name the name of the field + * @param schema the schema of the field + * @param doc the documentation string + * @return new HoodieSchemaField instance + */ + public static HoodieSchemaField of(String name, HoodieSchema schema, String doc) { + return of(name, schema, doc, null); + } + + /** + * Creates a metadata field for Hudi internal use. + * This is a convenience method for creating fields that are part of Hudi's metadata. + * + * @param name the metadata field name + * @param schema the metadata field schema + * @return new HoodieSchemaField configured as a metadata field + * @throws IllegalArgumentException if name is null/empty or schema is null + */ + public static HoodieSchemaField createMetadataField(String name, HoodieSchema schema) { + ValidationUtils.checkArgument(name != null && !name.isEmpty(), "Metadata field name cannot be null or empty"); + ValidationUtils.checkArgument(schema != null, "Metadata field schema cannot be null"); + + return HoodieSchemaField.of(name, schema, "Hudi metadata field: " + name, HoodieJsonProperties.NULL_VALUE); + } + + /** + * Returns the name of this field. + * + * @return the field name + */ + public String name() { + return avroField.name(); + } + + /** + * Returns the schema of this field. + * + * @return the field schema as HoodieSchema + */ + public HoodieSchema schema() { + return hoodieSchema; + } + + /** + * Returns the documentation string for this field. + * + * @return Option containing the documentation string, or Option.empty() if none + */ + public Option<String> doc() { + return Option.ofNullable(avroField.doc()); + } + + /** + * Returns the default value for this field. + * + * @return Option containing the default value, or Option.empty() if none + */ + public Option<Object> defaultVal() { + if (avroField != null && avroField.hasDefaultValue()) { + return Option.of(avroField.defaultVal()); + } + return Option.empty(); + } + + /** + * Returns the sort order for this field. + * + * @return the field order + */ + public HoodieFieldOrder order() { + return HoodieFieldOrder.fromAvroOrder(avroField.order()); + } + + /** + * Returns the position of this field within its enclosing record. + * + * @return the field position (0-based index) + */ + public int pos() { + return avroField.pos(); + } + + /** + * Checks if this field has a default value. + * + * @return true if the field has a default value + */ + public boolean hasDefaultValue() { + return avroField.hasDefaultValue(); + } + + /** + * Returns custom properties attached to this field. + * + * @return map of custom properties + */ + public Map<String, Object> getObjectProps() { + return avroField.getObjectProps(); + } + + /** + * Returns the value of a custom property. + * + * @param key the property key + * @return the property value, or null if not found + */ + public Object getProp(String key) { + return avroField.getProp(key); + } + + /** + * Adds a custom property to this field. + * + * @param key the property key + * @param value the property value + */ + public void addProp(String key, Object value) { + ValidationUtils.checkArgument(key != null && !key.isEmpty(), "Property key cannot be null or empty"); + avroField.addProp(key, value); + } + + /** + * Returns the underlying Avro field for compatibility purposes. + * + * <p>This method is provided for gradual migration and should be used + * sparingly. New code should prefer the HoodieSchemaField API.</p> + * + * @return the wrapped Avro Schema.Field + */ + public Schema.Field getAvroField() { + return avroField; + } + + /** + * Creates a copy of this field with a new name. + * + * @param newName the new name for the field + * @return new HoodieSchemaField with the specified name + */ + public HoodieSchemaField withName(String newName) { + ValidationUtils.checkArgument(newName != null && !newName.isEmpty(), "Field name cannot be null or empty"); + + Schema.Field newAvroField = new Schema.Field(newName, avroField.schema(), avroField.doc(), Review Comment: Same on all constructor usage of `Schema.Field` -- 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]
