YannByron commented on code in PR #5201:
URL: https://github.com/apache/hudi/pull/5201#discussion_r849280513
##########
hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java:
##########
@@ -159,23 +167,67 @@ public Schema getTableAvroSchema() throws Exception {
* @throws Exception
*/
public Schema getTableAvroSchema(boolean includeMetadataFields) throws
Exception {
+ Schema schema;
Option<Schema> schemaFromCommitMetadata =
getTableSchemaFromCommitMetadata(includeMetadataFields);
if (schemaFromCommitMetadata.isPresent()) {
- return schemaFromCommitMetadata.get();
- }
- Option<Schema> schemaFromTableConfig =
metaClient.getTableConfig().getTableCreateSchema();
- if (schemaFromTableConfig.isPresent()) {
- if (includeMetadataFields) {
- return HoodieAvroUtils.addMetadataFields(schemaFromTableConfig.get(),
hasOperationField);
+ schema = schemaFromCommitMetadata.get();
+ } else {
+ Option<Schema> schemaFromTableConfig =
metaClient.getTableConfig().getTableCreateSchema();
+ if (schemaFromTableConfig.isPresent()) {
+ if (includeMetadataFields) {
+ schema =
HoodieAvroUtils.addMetadataFields(schemaFromTableConfig.get(),
hasOperationField);
+ } else {
+ schema = schemaFromTableConfig.get();
+ }
} else {
- return schemaFromTableConfig.get();
+ if (includeMetadataFields) {
+ schema = getTableAvroSchemaFromDataFile();
+ } else {
+ schema =
HoodieAvroUtils.removeMetadataFields(getTableAvroSchemaFromDataFile());
+ }
}
}
- if (includeMetadataFields) {
- return getTableAvroSchemaFromDataFile();
- } else {
- return
HoodieAvroUtils.removeMetadataFields(getTableAvroSchemaFromDataFile());
+
+ Option<String[]> partitionFieldsOpt =
metaClient.getTableConfig().getPartitionFields();
+ if (metaClient.getTableConfig().isDropPartitionColumns()) {
+ schema = recreateSchemaWhenDropPartitionColumns(partitionFieldsOpt,
schema);
+ }
+ return schema;
+ }
+
+ public static Schema recreateSchemaWhenDropPartitionColumns(Option<String[]>
partitionFieldsOpt, Schema originSchema) {
+ // when hoodie.datasource.write.drop.partition.columns is true, partition
columns can't be persisted in data files.
+ // And there are no partition schema if the schema is parsed from data
files.
+ // Here we create partition Fields for this case, and use StringType as
the data type.
+ Schema schema = originSchema;
+ if (partitionFieldsOpt.isPresent() && partitionFieldsOpt.get().length !=
0) {
+ List<String> partitionFields = Arrays.asList(partitionFieldsOpt.get());
+
+ final Schema schema0 = originSchema;
+ boolean hasPartitionColNotInSchema = partitionFields.stream().anyMatch(
+ pt -> !HoodieAvroUtils.containsFieldInSchema(schema0, pt)
+ );
+ boolean hasPartitionColInSchema = partitionFields.stream().anyMatch(
+ pt -> HoodieAvroUtils.containsFieldInSchema(schema0, pt)
+ );
+ if (hasPartitionColNotInSchema && hasPartitionColInSchema) {
+ throw new HoodieIncompatibleSchemaException(
+ "Not support: Partial partition fields are still in the schema "
+ + "when enable
hoodie.datasource.write.drop.partition.columns");
+ }
+
+ if (hasPartitionColNotInSchema) {
+ // when hasPartitionColNotInSchema is true and hasPartitionColInSchema
is false, all partition columns
+ // are not in originSchema. So we create and add them.
+ List<Field> newFields = new ArrayList<>();
+ for (String partitionField: partitionFields) {
+ newFields.add(new Schema.Field(
+ partitionField, Schema.create(Schema.Type.STRING), "",
JsonProperties.NULL_VALUE));
Review Comment:
in some cases, the schema is out-of-date in hoodie.properties, even there is
no schema stored in hoodie.properties.
I prefer to have a unified place to manage the latest schema, rather than
the current logical that get schema from metadata, hoodie.properties, datafile
in order.
Like deltalake and iceberg do.
--
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]