diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java index 18fd1018ad973..86efd7db1e086 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java @@ -18,6 +18,7 @@ package org.apache.hudi.client; +import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieIndexCommitMetadata; @@ -114,7 +115,6 @@ import java.util.function.BiConsumer; import java.util.function.BiFunction; -import static org.apache.hudi.avro.AvroSchemaUtils.getAvroRecordQualifiedName; import static org.apache.hudi.common.model.HoodieCommitMetadata.SCHEMA_KEY; import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS; import static org.apache.hudi.keygen.KeyGenUtils.getComplexKeygenErrorMessage; @@ -1657,7 +1657,7 @@ public void commitTableChange(InternalSchema newSchema, HoodieTableMetaClient me TableSchemaResolver schemaUtil = new TableSchemaResolver(metaClient); String historySchemaStr = schemaUtil.getTableHistorySchemaStrFromCommitMetadata().orElseGet( () -> SerDeHelper.inheritSchemas(getInternalSchema(schemaUtil), "")); - Schema schema = InternalSchemaConverter.convert(newSchema, getAvroRecordQualifiedName(config.getTableName())).toAvroSchema(); + Schema schema = InternalSchemaConverter.convert(newSchema, AvroSchemaUtils.getAvroRecordQualifiedName(config.getTableName())).toAvroSchema(); String commitActionType = CommitUtils.getCommitActionType(WriteOperationType.ALTER_SCHEMA, metaClient.getTableType()); String instantTime = startCommit(commitActionType, metaClient); config.setSchema(schema.toString()); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentSchemaEvolutionTableSchemaGetter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentSchemaEvolutionTableSchemaGetter.java index 8936b3c06057c..ab9c0ce426b77 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentSchemaEvolutionTableSchemaGetter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentSchemaEvolutionTableSchemaGetter.java @@ -18,9 +18,11 @@ package org.apache.hudi.client.transaction; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.TimelineLayout; @@ -30,27 +32,19 @@ import org.apache.hudi.common.util.VisibleForTesting; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.internal.schema.HoodieSchemaException; import org.apache.hudi.util.Lazy; import lombok.extern.slf4j.Slf4j; -import org.apache.avro.JsonProperties; import org.apache.avro.Schema; -import org.apache.avro.Schema.Field; import java.io.IOException; -import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; import java.util.HashSet; -import java.util.List; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Stream; -import static org.apache.hudi.avro.AvroSchemaUtils.appendFieldsToSchema; -import static org.apache.hudi.avro.AvroSchemaUtils.containsFieldInSchema; -import static org.apache.hudi.avro.AvroSchemaUtils.createNullableSchema; import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION; @@ -86,25 +80,27 @@ public ConcurrentSchemaEvolutionTableSchemaGetter(HoodieTableMetaClient metaClie * @param schema the input schema to process * @return the processed schema with partition columns handled appropriately */ - private Schema handlePartitionColumnsIfNeeded(Schema schema) { + private HoodieSchema handlePartitionColumnsIfNeeded(HoodieSchema schema) { if (metaClient.getTableConfig().shouldDropPartitionColumns()) { return metaClient.getTableConfig().getPartitionFields() - .map(partitionFields -> appendPartitionColumns(schema, Option.ofNullable(partitionFields))) - .or(() -> Option.of(schema)) - .get(); + .map(partitionFields -> TableSchemaResolver.appendPartitionColumns(schema, Option.ofNullable(partitionFields))) + .orElseGet(() -> schema); } return schema; } public Option getTableAvroSchemaIfPresent(boolean includeMetadataFields, Option instant) { return getTableAvroSchemaFromTimelineWithCache(instant) // Get table schema from schema evolution timeline. + .map(HoodieSchema::fromAvroSchema) .or(this::getTableCreateSchemaWithoutMetaField) // Fall back: read create schema from table config. - .map(tableSchema -> includeMetadataFields ? HoodieAvroUtils.addMetadataFields(tableSchema, false) : HoodieAvroUtils.removeMetadataFields(tableSchema)) - .map(this::handlePartitionColumnsIfNeeded); + .map(tableSchema -> includeMetadataFields ? HoodieSchemaUtils.addMetadataFields(tableSchema, false) : HoodieSchemaUtils.removeMetadataFields(tableSchema)) + .map(this::handlePartitionColumnsIfNeeded) + .map(HoodieSchema::toAvroSchema); } - private Option getTableCreateSchemaWithoutMetaField() { - return metaClient.getTableConfig().getTableCreateSchema(); + private Option getTableCreateSchemaWithoutMetaField() { + return metaClient.getTableConfig().getTableCreateSchema() + .map(HoodieSchema::fromAvroSchema); } private void setCachedLatestCommitWithValidSchema(Option instantOption) { @@ -199,36 +195,6 @@ Option> getLastCommitMetadataWithValidSchemaFromTime return Option.of(Pair.of(instantWithTableSchema.get(), tableSchemaAtInstant.get(instantWithTableSchema.get()))); } - public static Schema appendPartitionColumns(Schema dataSchema, Option partitionFields) { - // In cases when {@link DROP_PARTITION_COLUMNS} config is set true, partition columns - // won't be persisted w/in the data files, and therefore we need to append such columns - // when schema is parsed from data files - // - // Here we append partition columns with {@code StringType} as the data type - if (!partitionFields.isPresent() || partitionFields.get().length == 0) { - return dataSchema; - } - - boolean hasPartitionColNotInSchema = Arrays.stream(partitionFields.get()).anyMatch(pf -> !containsFieldInSchema(dataSchema, pf)); - boolean hasPartitionColInSchema = Arrays.stream(partitionFields.get()).anyMatch(pf -> containsFieldInSchema(dataSchema, pf)); - if (hasPartitionColNotInSchema && hasPartitionColInSchema) { - throw new HoodieSchemaException("Partition columns could not be partially contained w/in the data schema"); - } - - if (hasPartitionColNotInSchema) { - // when hasPartitionColNotInSchema is true and hasPartitionColInSchema is false, all partition columns - // are not in originSchema. So we create and add them. - List newFields = new ArrayList<>(); - for (String partitionField : partitionFields.get()) { - newFields.add(new Schema.Field( - partitionField, createNullableSchema(Schema.Type.STRING), "", JsonProperties.NULL_VALUE)); - } - return appendFieldsToSchema(dataSchema, newFields); - } - - return dataSchema; - } - /** * Get timeline in REVERSE order that only contains completed instants which POTENTIALLY evolve the table schema. * For types of instants that are included and not reflecting table schema at their instant completion time please refer diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java index b57b484936177..dfc2109386404 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java @@ -18,7 +18,6 @@ package org.apache.hudi.io; -import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.TaskContextSupplier; @@ -33,6 +32,7 @@ import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.schema.HoodieSchemaCache; +import org.apache.hudi.common.schema.HoodieSchemaField; import org.apache.hudi.common.schema.HoodieSchemaUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableVersion; @@ -56,7 +56,6 @@ import lombok.AccessLevel; import lombok.Getter; import lombok.extern.slf4j.Slf4j; -import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; import java.io.IOException; @@ -357,10 +356,10 @@ protected Option> getRecordMetadata(HoodieRecord record, Hoo Object eventTime = record.getColumnValueAsJava(schema.toAvroSchema(), eventTimeFieldName, props); if (eventTime != null) { // Append event_time. - Option field = AvroSchemaUtils.findNestedField(schema.toAvroSchema(), eventTimeFieldName); + Option field = HoodieSchemaUtils.findNestedField(schema, eventTimeFieldName); // Field should definitely exist. eventTime = record.convertColumnValueForLogicalType( - field.get().schema(), eventTime, keepConsistentLogicalTimestamp); + field.get().schema().toAvroSchema(), eventTime, keepConsistentLogicalTimestamp); Map metadata = recordMetadata.orElse(new HashMap<>()); metadata.put(METADATA_EVENT_TIME_KEY, String.valueOf(eventTime)); return Option.of(metadata); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java index 9c05ae30c505f..6f2b32c36a794 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java @@ -18,9 +18,6 @@ package org.apache.hudi.table; -import org.apache.hudi.avro.AvroSchemaCompatibility; -import org.apache.hudi.avro.AvroSchemaUtils; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieClusteringPlan; @@ -51,6 +48,10 @@ import org.apache.hudi.common.model.HoodieIndexMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaCompatibility; +import org.apache.hudi.common.schema.HoodieSchemaField; +import org.apache.hudi.common.schema.HoodieSchemaUtils; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableVersion; @@ -97,7 +98,6 @@ import lombok.Getter; import lombok.extern.slf4j.Slf4j; -import org.apache.avro.Schema; import java.io.FileNotFoundException; import java.io.IOException; @@ -117,7 +117,6 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.hudi.avro.AvroSchemaUtils.getNonNullTypeFromUnion; import static org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy.EAGER; import static org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy.LAZY; import static org.apache.hudi.common.table.HoodieTableConfig.TABLE_METADATA_PARTITIONS; @@ -927,13 +926,14 @@ public void validateSchema() throws HoodieUpsertException, HoodieInsertException try { TableSchemaResolver schemaResolver = new TableSchemaResolver(getMetaClient()); - Option existingTableSchema = schemaResolver.getTableAvroSchemaIfPresent(false); - if (!existingTableSchema.isPresent()) { + Option existingTableSchemaOpt = schemaResolver.getTableSchemaIfPresent(false); + if (existingTableSchemaOpt.isEmpty()) { return; } - Schema writerSchema = HoodieAvroUtils.createHoodieWriteSchema(config.getSchema()); - Schema tableSchema = HoodieAvroUtils.createHoodieWriteSchema(existingTableSchema.get()); - AvroSchemaUtils.checkSchemaCompatible(tableSchema, writerSchema, shouldValidate, allowProjection, getDropPartitionColNames()); + + HoodieSchema writerSchema = HoodieSchemaUtils.createHoodieWriteSchema(config.getSchema(), false); + HoodieSchema tableSchema = HoodieSchemaUtils.addMetadataFields(existingTableSchemaOpt.get()); + HoodieSchemaCompatibility.checkSchemaCompatible(tableSchema, writerSchema, shouldValidate, allowProjection, getDropPartitionColNames()); // Check secondary index column compatibility Option indexMetadata = metaClient.getIndexMetadata(); @@ -956,8 +956,8 @@ public void validateSchema() throws HoodieUpsertException, HoodieInsertException * @throws SchemaCompatibilityException if a secondary index column has incompatible evolution */ static void validateSecondaryIndexSchemaEvolution( - Schema tableSchema, - Schema writerSchema, + HoodieSchema tableSchema, + HoodieSchema writerSchema, HoodieIndexMetadata indexMetadata) throws SchemaCompatibilityException { // Filter for secondary index definitions @@ -984,21 +984,25 @@ static void validateSecondaryIndexSchemaEvolution( for (Map.Entry entry : columnToIndexName.entrySet()) { String columnName = entry.getKey(); String indexName = entry.getValue(); + + Option tableFieldOpt = tableSchema.getField(columnName); - Schema.Field tableField = tableSchema.getField(columnName); - - if (tableField == null) { + if (tableFieldOpt.isEmpty()) { // This shouldn't happen as indexed columns should exist in table schema log.warn("Secondary index '{}' references non-existent column: {}", indexName, columnName); continue; } - + + HoodieSchemaField tableField = tableFieldOpt.get(); + // Use AvroSchemaCompatibility's field lookup logic to handle aliases - Schema.Field writerField = AvroSchemaCompatibility.lookupWriterField(writerSchema, tableField); + HoodieSchemaField writerField = HoodieSchemaCompatibility.lookupWriterField(writerSchema, tableField); if (writerField != null && !tableField.schema().equals(writerField.schema())) { // Check if this is just making the field nullable/non-nullable, which is safe from SI perspective - if (getNonNullTypeFromUnion(tableField.schema()).equals(getNonNullTypeFromUnion(writerField.schema()))) { + HoodieSchema nonNullTableField = tableField.schema().getNonNullType(); + HoodieSchema nonNullWriterField = writerField.schema().getNonNullType(); + if (nonNullTableField.equals(nonNullWriterField)) { continue; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java index 885f81eb31d02..d2282e1c5f250 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java @@ -22,6 +22,7 @@ import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaCompatibility; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.util.InternalSchemaCache; @@ -57,8 +58,6 @@ import java.util.function.Function; import java.util.stream.Collectors; -import static org.apache.hudi.avro.AvroSchemaUtils.isStrictProjectionOf; - @NoArgsConstructor(access = AccessLevel.PRIVATE) @Slf4j public class HoodieMergeHelper extends BaseMergeHelper { @@ -96,7 +95,7 @@ public void runMerge(HoodieTable table, // - Its field-set is a proper subset (of the reader schema) // - There's no schema evolution transformation necessary boolean isPureProjection = schemaEvolutionTransformerOpt.isEmpty() - && isStrictProjectionOf(readerSchema.toAvroSchema(), writerSchema.toAvroSchema()); + && HoodieSchemaCompatibility.isStrictProjectionOf(readerSchema, writerSchema); // Check whether we will need to rewrite target (already merged) records into the // writer's schema boolean shouldRewriteInWriterSchema = !isPureProjection diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/TestHoodieTableSchemaEvolution.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/TestHoodieTableSchemaEvolution.java index 3143beb6ed0b9..793a21d741ecb 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/TestHoodieTableSchemaEvolution.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/TestHoodieTableSchemaEvolution.java @@ -20,10 +20,10 @@ import org.apache.hudi.common.model.HoodieIndexDefinition; import org.apache.hudi.common.model.HoodieIndexMetadata; +import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.exception.SchemaCompatibilityException; import org.apache.hudi.metadata.MetadataPartitionType; -import org.apache.avro.Schema; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; @@ -58,8 +58,8 @@ public class TestHoodieTableSchemaEvolution { @Test public void testNoSecondaryIndexes() { // When there are no secondary indexes, any schema evolution should be allowed - Schema tableSchema = new Schema.Parser().parse(TABLE_SCHEMA); - Schema writerSchema = new Schema.Parser().parse(TABLE_SCHEMA.replace("\"age\", \"type\": \"int\"", "\"age\", \"type\": \"long\"")); + HoodieSchema tableSchema = HoodieSchema.parse(TABLE_SCHEMA); + HoodieSchema writerSchema = HoodieSchema.parse(TABLE_SCHEMA.replace("\"age\", \"type\": \"int\"", "\"age\", \"type\": \"long\"")); HoodieIndexMetadata indexMetadata = new HoodieIndexMetadata(new HashMap<>()); @@ -70,8 +70,8 @@ public void testNoSecondaryIndexes() { @Test public void testNoSchemaChange() { // When schema doesn't change, validation should pass even with secondary indexes - Schema tableSchema = new Schema.Parser().parse(TABLE_SCHEMA); - Schema writerSchema = new Schema.Parser().parse(TABLE_SCHEMA); + HoodieSchema tableSchema = HoodieSchema.parse(TABLE_SCHEMA); + HoodieSchema writerSchema = HoodieSchema.parse(TABLE_SCHEMA); HoodieIndexDefinition indexDef = createSecondaryIndexDefinition("secondary_index_age", "age"); HoodieIndexMetadata indexMetadata = createIndexMetadata(indexDef); @@ -83,8 +83,8 @@ public void testNoSchemaChange() { @Test public void testNonIndexedColumnEvolution() { // Evolution of non-indexed columns should be allowed - Schema tableSchema = new Schema.Parser().parse(TABLE_SCHEMA); - Schema writerSchema = new Schema.Parser().parse(TABLE_SCHEMA.replace("\"name\", \"type\": \"string\"", "\"name\", \"type\": [\"null\", \"string\"]")); + HoodieSchema tableSchema = HoodieSchema.parse(TABLE_SCHEMA); + HoodieSchema writerSchema = HoodieSchema.parse(TABLE_SCHEMA.replace("\"name\", \"type\": \"string\"", "\"name\", \"type\": [\"null\", \"string\"]")); HoodieIndexDefinition indexDef = createSecondaryIndexDefinition("secondary_index_age", "age"); HoodieIndexMetadata indexMetadata = createIndexMetadata(indexDef); @@ -101,8 +101,8 @@ public void testIndexedColumnTypeEvolution(String fieldName, String originalType String evolvedSchema = TABLE_SCHEMA.replace("\"" + fieldName + "\", \"type\": \"" + originalType + "\"", "\"" + fieldName + "\", \"type\": \"" + evolvedType + "\""); - Schema tableSchema = new Schema.Parser().parse(originalSchema); - Schema writerSchema = new Schema.Parser().parse(evolvedSchema); + HoodieSchema tableSchema = HoodieSchema.parse(originalSchema); + HoodieSchema writerSchema = HoodieSchema.parse(evolvedSchema); HoodieIndexDefinition indexDef = createSecondaryIndexDefinition("secondary_index_" + fieldName, fieldName); HoodieIndexMetadata indexMetadata = createIndexMetadata(indexDef); @@ -126,8 +126,8 @@ private static Stream provideInvalidSchemaEvolutions() { @Test public void testMultipleIndexesOnSameColumn() { // When a column has multiple indexes, error should mention at least one - Schema tableSchema = new Schema.Parser().parse(TABLE_SCHEMA); - Schema writerSchema = new Schema.Parser().parse(TABLE_SCHEMA.replace("\"age\", \"type\": \"int\"", "\"age\", \"type\": \"long\"")); + HoodieSchema tableSchema = HoodieSchema.parse(TABLE_SCHEMA); + HoodieSchema writerSchema = HoodieSchema.parse(TABLE_SCHEMA.replace("\"age\", \"type\": \"int\"", "\"age\", \"type\": \"long\"")); HoodieIndexDefinition indexDef1 = createSecondaryIndexDefinition("secondary_index_age_1", "age"); HoodieIndexDefinition indexDef2 = createSecondaryIndexDefinition("secondary_index_age_2", "age"); @@ -148,8 +148,8 @@ public void testMultipleIndexesOnSameColumn() { @Test public void testCompoundIndex() { // Test index on multiple columns - if any column evolves, should fail - Schema tableSchema = new Schema.Parser().parse(TABLE_SCHEMA); - Schema writerSchema = new Schema.Parser().parse(TABLE_SCHEMA.replace("\"age\", \"type\": \"int\"", "\"age\", \"type\": \"long\"")); + HoodieSchema tableSchema = HoodieSchema.parse(TABLE_SCHEMA); + HoodieSchema writerSchema = HoodieSchema.parse(TABLE_SCHEMA.replace("\"age\", \"type\": \"int\"", "\"age\", \"type\": \"long\"")); HoodieIndexDefinition indexDef = createSecondaryIndexDefinition("secondary_index_compound", "name", "age"); HoodieIndexMetadata indexMetadata = createIndexMetadata(indexDef); @@ -180,8 +180,8 @@ public void testFieldWithAlias() { + " {\"name\": \"new_name\", \"type\": \"string\"}" // Field renamed using alias + "]}"; - Schema tableSchema = new Schema.Parser().parse(tableSchemaStr); - Schema writerSchema = new Schema.Parser().parse(writerSchemaStr); + HoodieSchema tableSchema = HoodieSchema.parse(tableSchemaStr); + HoodieSchema writerSchema = HoodieSchema.parse(writerSchemaStr); HoodieIndexDefinition indexDef = createSecondaryIndexDefinition("secondary_index_name", "old_name"); HoodieIndexMetadata indexMetadata = createIndexMetadata(indexDef); @@ -197,8 +197,8 @@ public void testNullableFieldEvolution() { String evolvedSchema = TABLE_SCHEMA.replace("\"name\", \"type\": \"string\"", "\"name\", \"type\": [\"null\", \"string\"], \"default\": null"); - Schema tableSchema = new Schema.Parser().parse(TABLE_SCHEMA); - Schema writerSchema = new Schema.Parser().parse(evolvedSchema); + HoodieSchema tableSchema = HoodieSchema.parse(TABLE_SCHEMA); + HoodieSchema writerSchema = HoodieSchema.parse(evolvedSchema); HoodieIndexDefinition indexDef = createSecondaryIndexDefinition("secondary_index_name", "name"); HoodieIndexMetadata indexMetadata = createIndexMetadata(indexDef); @@ -211,8 +211,8 @@ public void testNullableFieldEvolution() { @Test public void testMissingIndexedColumnInTableSchema() { // Edge case: index references a column that doesn't exist in table schema - Schema tableSchema = new Schema.Parser().parse(TABLE_SCHEMA); - Schema writerSchema = new Schema.Parser().parse(TABLE_SCHEMA); + HoodieSchema tableSchema = HoodieSchema.parse(TABLE_SCHEMA); + HoodieSchema writerSchema = HoodieSchema.parse(TABLE_SCHEMA); HoodieIndexDefinition indexDef = createSecondaryIndexDefinition("secondary_index_nonexistent", "nonexistent_column"); HoodieIndexMetadata indexMetadata = createIndexMetadata(indexDef); @@ -225,8 +225,8 @@ public void testMissingIndexedColumnInTableSchema() { @Test public void testNonSecondaryIndexDefinitions() { // Test that non-secondary index definitions are ignored - Schema tableSchema = new Schema.Parser().parse(TABLE_SCHEMA); - Schema writerSchema = new Schema.Parser().parse(TABLE_SCHEMA.replace("\"age\", \"type\": \"int\"", "\"age\", \"type\": \"long\"")); + HoodieSchema tableSchema = HoodieSchema.parse(TABLE_SCHEMA); + HoodieSchema writerSchema = HoodieSchema.parse(TABLE_SCHEMA.replace("\"age\", \"type\": \"int\"", "\"age\", \"type\": \"long\"")); // Create an expression index (not secondary index) HoodieIndexDefinition expressionIndexDef = HoodieIndexDefinition.newBuilder() @@ -261,14 +261,14 @@ public void testFixedTypeEvolution() { + " {\"name\": \"fixed_field\", \"type\": {\"type\": \"fixed\", \"name\": \"FixedField\", \"size\": 16}}" + "]}"; - Schema tableSchema = new Schema.Parser().parse(fixed8Schema); - Schema writerSchema = new Schema.Parser().parse(fixed16Schema); + HoodieSchema tableSchema = HoodieSchema.parse(fixed8Schema); + HoodieSchema writerSchema = HoodieSchema.parse(fixed16Schema); HoodieIndexDefinition indexDef = createSecondaryIndexDefinition("secondary_index_fixed", "fixed_field"); HoodieIndexMetadata indexMetadata = createIndexMetadata(indexDef); - final Schema tableSchemaFixed1 = tableSchema; - final Schema writerSchemaFixed1 = writerSchema; + final HoodieSchema tableSchemaFixed1 = tableSchema; + final HoodieSchema writerSchemaFixed1 = writerSchema; final HoodieIndexMetadata indexMetadataFixed = indexMetadata; SchemaCompatibilityException exception = assertThrows(SchemaCompatibilityException.class, () -> HoodieTable.validateSecondaryIndexSchemaEvolution(tableSchemaFixed1, writerSchemaFixed1, indexMetadataFixed)); @@ -277,11 +277,11 @@ public void testFixedTypeEvolution() { assertTrue(exception.getMessage().contains("secondary index")); // Fixed size decrease - tableSchema = new Schema.Parser().parse(fixed16Schema); - writerSchema = new Schema.Parser().parse(fixed8Schema); + tableSchema = HoodieSchema.parse(fixed16Schema); + writerSchema = HoodieSchema.parse(fixed8Schema); - final Schema tableSchemaFixed2 = tableSchema; - final Schema writerSchemaFixed2 = writerSchema; + final HoodieSchema tableSchemaFixed2 = tableSchema; + final HoodieSchema writerSchemaFixed2 = writerSchema; exception = assertThrows(SchemaCompatibilityException.class, () -> HoodieTable.validateSecondaryIndexSchemaEvolution(tableSchemaFixed2, writerSchemaFixed2, indexMetadataFixed)); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java index 845bcd6ecacdc..af1e7a8c45f05 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java @@ -19,13 +19,12 @@ package org.apache.hudi.io.storage; import org.apache.hudi.SparkAdapterSupport$; -import org.apache.hudi.avro.AvroSchemaUtils; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieSparkRecord; import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaUtils; import org.apache.hudi.common.util.FileFormatUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ParquetReaderIterator; @@ -116,14 +115,13 @@ public ClosableIterator> getRecordIterator(HoodieSchem @Override public ClosableIterator> getRecordIterator(HoodieSchema schema) throws IOException { - //TODO boundary to revisit in later pr to use HoodieSchema directly - ClosableIterator iterator = getUnsafeRowIterator(schema.getAvroSchema()); + ClosableIterator iterator = getUnsafeRowIterator(schema); return new CloseableMappingIterator<>(iterator, data -> unsafeCast(new HoodieSparkRecord(data))); } @Override public ClosableIterator getRecordKeyIterator() throws IOException { - Schema schema = HoodieAvroUtils.getRecordKeySchema(); + HoodieSchema schema = HoodieSchemaUtils.getRecordKeySchema(); ClosableIterator iterator = getUnsafeRowIterator(schema); return new CloseableMappingIterator<>(iterator, data -> { HoodieSparkRecord record = unsafeCast(new HoodieSparkRecord(data)); @@ -131,7 +129,7 @@ public ClosableIterator getRecordKeyIterator() throws IOException { }); } - public ClosableIterator getUnsafeRowIterator(Schema requestedSchema) throws IOException { + public ClosableIterator getUnsafeRowIterator(HoodieSchema requestedSchema) throws IOException { return getUnsafeRowIterator(requestedSchema, Collections.emptyList()); } @@ -141,10 +139,10 @@ public ClosableIterator getUnsafeRowIterator(Schema requestedSchema) * Currently, the filter must only contain field references related to the primary key, as the primary key does not involve schema evolution. * If it is necessary to expand to push down more fields in the future, please consider the issue of schema evolution carefully */ - public ClosableIterator getUnsafeRowIterator(Schema requestedSchema, List readFilters) throws IOException { - Schema nonNullSchema = AvroSchemaUtils.getNonNullTypeFromUnion(requestedSchema); - StructType structSchema = HoodieInternalRowUtils.getCachedSchema(nonNullSchema); - Option messageSchema = Option.of(getAvroSchemaConverter(storage.getConf().unwrapAs(Configuration.class)).convert(nonNullSchema)); + public ClosableIterator getUnsafeRowIterator(HoodieSchema requestedSchema, List readFilters) throws IOException { + HoodieSchema nonNullSchema = HoodieSchemaUtils.getNonNullTypeFromUnion(requestedSchema); + StructType structSchema = HoodieInternalRowUtils.getCachedSchema(nonNullSchema.toAvroSchema()); + Option messageSchema = Option.of(getAvroSchemaConverter(storage.getConf().unwrapAs(Configuration.class)).convert(nonNullSchema.toAvroSchema())); boolean enableTimestampFieldRepair = storage.getConf().getBoolean(ENABLE_LOGICAL_TIMESTAMP_REPAIR, true); StructType dataStructType = convertToStruct(enableTimestampFieldRepair ? SchemaRepair.repairLogicalTypes(getFileSchema(), messageSchema) : getFileSchema()); SparkBasicSchemaEvolution evolution = new SparkBasicSchemaEvolution(dataStructType, structSchema, SQLConf.get().sessionLocalTimeZone()); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java index 14d72336c84a9..091dddac24ee2 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java @@ -25,6 +25,8 @@ import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchema.TimePrecision; +import org.apache.hudi.common.schema.HoodieSchemaType; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.ValidationUtils; @@ -32,9 +34,6 @@ import org.apache.hudi.internal.schema.utils.SerDeHelper; import lombok.Getter; -import org.apache.avro.LogicalType; -import org.apache.avro.LogicalTypes; -import org.apache.avro.Schema; import org.apache.hadoop.conf.Configuration; import org.apache.parquet.avro.AvroWriteSupport; import org.apache.parquet.hadoop.api.WriteSupport; @@ -77,7 +76,6 @@ import scala.Enumeration; import scala.Function1; -import static org.apache.hudi.avro.AvroSchemaUtils.getNonNullTypeFromUnion; import static org.apache.hudi.common.config.HoodieStorageConfig.PARQUET_FIELD_ID_WRITE_ENABLED; import static org.apache.hudi.config.HoodieWriteConfig.ALLOW_OPERATION_METADATA_FIELD; import static org.apache.hudi.config.HoodieWriteConfig.AVRO_SCHEMA_STRING; @@ -106,7 +104,7 @@ */ public class HoodieRowParquetWriteSupport extends WriteSupport { - private static final Schema MAP_KEY_SCHEMA = Schema.create(Schema.Type.STRING); + private static final HoodieSchema MAP_KEY_SCHEMA = HoodieSchema.create(HoodieSchemaType.STRING); private static final String MAP_REPEATED_NAME = "key_value"; private static final String MAP_KEY_NAME = "key"; private static final String MAP_VALUE_NAME = "value"; @@ -120,7 +118,7 @@ public class HoodieRowParquetWriteSupport extends WriteSupport { private final Function1 timestampRebaseFunction = DataSourceUtils.createTimestampRebaseFuncInWrite(datetimeRebaseMode, "Parquet"); private final boolean writeLegacyListFormat; private final ValueWriter[] rootFieldWriters; - private final Schema avroSchema; + private final HoodieSchema schema; private final StructType structType; private RecordConsumer recordConsumer; @@ -134,22 +132,26 @@ public HoodieRowParquetWriteSupport(Configuration conf, StructType structType, O || Boolean.parseBoolean(config.getStringOrDefault(AvroWriteSupport.WRITE_OLD_LIST_STRUCTURE, "false")); this.structType = structType; // The avro schema is used to determine the precision for timestamps - this.avroSchema = SerDeHelper.fromJson(config.getString(INTERNAL_SCHEMA_STRING)).map(internalSchema -> InternalSchemaConverter.convert(internalSchema, "spark_schema")) + this.schema = SerDeHelper.fromJson(config.getString(INTERNAL_SCHEMA_STRING)).map(internalSchema -> InternalSchemaConverter.convert(internalSchema, "spark_schema")) .orElseGet(() -> { String schemaString = Option.ofNullable(config.getString(WRITE_SCHEMA_OVERRIDE)).orElseGet(() -> config.getString(AVRO_SCHEMA_STRING)); HoodieSchema parsedSchema = HoodieSchema.parse(schemaString); return HoodieSchema.addMetadataFields(parsedSchema, config.getBooleanOrDefault(ALLOW_OPERATION_METADATA_FIELD)); - }).toAvroSchema(); + }); ParquetWriteSupport.setSchema(structType, hadoopConf); - this.rootFieldWriters = getFieldWriters(structType, avroSchema); + this.rootFieldWriters = getFieldWriters(structType, schema); this.hadoopConf = hadoopConf; this.bloomFilterWriteSupportOpt = bloomFilterOpt.map(HoodieBloomFilterRowWriteSupport::new); } - private ValueWriter[] getFieldWriters(StructType schema, Schema avroSchema) { + private ValueWriter[] getFieldWriters(StructType schema, HoodieSchema hoodieSchema) { return Arrays.stream(schema.fields()).map(field -> { - Schema.Field avroField = avroSchema == null ? null : avroSchema.getField(field.name()); - return makeWriter(avroField == null ? null : avroField.schema(), field.dataType()); + HoodieSchema fieldSchema = Option.ofNullable(hoodieSchema) + .flatMap(s -> s.getField(field.name())) + // Note: Cannot use HoodieSchemaField::schema method reference due to Java 17 compilation ambiguity + .map(f -> f.schema()) + .orElse(null); + return makeWriter(fieldSchema, field.dataType()); }).toArray(ValueWriter[]::new); } @@ -163,7 +165,7 @@ public WriteContext init(Configuration configuration) { } Configuration configurationCopy = new Configuration(configuration); configurationCopy.set(AvroWriteSupport.WRITE_OLD_LIST_STRUCTURE, Boolean.toString(writeLegacyListFormat)); - MessageType messageType = convert(structType, avroSchema); + MessageType messageType = convert(structType, schema); return new WriteContext(messageType, metadata); } @@ -223,9 +225,8 @@ private void writeFields(InternalRow row, StructType schema, ValueWriter[] field } } - private ValueWriter makeWriter(Schema avroSchema, DataType dataType) { - Schema resolvedSchema = avroSchema == null ? null : getNonNullTypeFromUnion(avroSchema); - LogicalType logicalType = resolvedSchema != null ? resolvedSchema.getLogicalType() : null; + private ValueWriter makeWriter(HoodieSchema schema, DataType dataType) { + HoodieSchema resolvedSchema = schema == null ? null : schema.getNonNullType(); if (dataType == DataTypes.BooleanType) { return (row, ordinal) -> recordConsumer.addBoolean(row.getBoolean(ordinal)); @@ -238,20 +239,36 @@ private ValueWriter makeWriter(Schema avroSchema, DataType dataType) { } else if (dataType == DataTypes.LongType || dataType instanceof DayTimeIntervalType) { return (row, ordinal) -> recordConsumer.addLong(row.getLong(ordinal)); } else if (dataType == DataTypes.TimestampType) { - if (logicalType == null || logicalType.getName().equals(LogicalTypes.timestampMicros().getName())) { - return (row, ordinal) -> recordConsumer.addLong((long) timestampRebaseFunction.apply(row.getLong(ordinal))); - } else if (logicalType.getName().equals(LogicalTypes.timestampMillis().getName())) { - return (row, ordinal) -> recordConsumer.addLong(DateTimeUtils.microsToMillis((long) timestampRebaseFunction.apply(row.getLong(ordinal)))); + if (resolvedSchema != null && resolvedSchema.getType() == HoodieSchemaType.TIMESTAMP) { + HoodieSchema.Timestamp timestampSchema = (HoodieSchema.Timestamp) resolvedSchema; + if (timestampSchema.getPrecision() == TimePrecision.MICROS) { + return (row, ordinal) -> recordConsumer.addLong((long) timestampRebaseFunction.apply(row.getLong(ordinal))); + } else if (timestampSchema.getPrecision() == TimePrecision.MILLIS) { + return (row, ordinal) -> recordConsumer.addLong(DateTimeUtils.microsToMillis((long) timestampRebaseFunction.apply(row.getLong(ordinal)))); + } else { + throw new UnsupportedOperationException("Unsupported timestamp precision for TimestampType: " + timestampSchema.getPrecision()); + } } else { - throw new UnsupportedOperationException("Unsupported Avro logical type for TimestampType: " + logicalType); + // Default to micros precision when no timestamp schema is available + return (row, ordinal) -> recordConsumer.addLong((long) timestampRebaseFunction.apply(row.getLong(ordinal))); } } else if (SparkAdapterSupport$.MODULE$.sparkAdapter().isTimestampNTZType(dataType)) { - if (logicalType == null || logicalType.getName().equals(LogicalTypes.localTimestampMicros().getName())) { - return (row, ordinal) -> recordConsumer.addLong(row.getLong(ordinal)); - } else if (logicalType.getName().equals(LogicalTypes.localTimestampMillis().getName())) { - return (row, ordinal) -> recordConsumer.addLong(DateTimeUtils.microsToMillis(row.getLong(ordinal))); + if (resolvedSchema != null && resolvedSchema.getType() == HoodieSchemaType.TIMESTAMP) { + HoodieSchema.Timestamp timestampSchema = (HoodieSchema.Timestamp) resolvedSchema; + if (!timestampSchema.isUtcAdjusted()) { + if (timestampSchema.getPrecision() == TimePrecision.MICROS) { + return (row, ordinal) -> recordConsumer.addLong(row.getLong(ordinal)); + } else if (timestampSchema.getPrecision() == TimePrecision.MILLIS) { + return (row, ordinal) -> recordConsumer.addLong(DateTimeUtils.microsToMillis(row.getLong(ordinal))); + } else { + throw new UnsupportedOperationException("Unsupported timestamp precision for TimestampNTZType: " + timestampSchema.getPrecision()); + } + } else { + throw new UnsupportedOperationException("TimestampNTZType requires local timestamp schema, but got UTC-adjusted: " + timestampSchema.getName()); + } } else { - throw new UnsupportedOperationException("Unsupported Avro logical type for TimestampNTZType: " + logicalType); + // Default to micros precision when no timestamp schema is available + return (row, ordinal) -> recordConsumer.addLong(row.getLong(ordinal)); } } else if (dataType == DataTypes.FloatType) { return (row, ordinal) -> recordConsumer.addFloat(row.getFloat(ordinal)); @@ -406,29 +423,31 @@ public static HoodieRowParquetWriteSupport getHoodieRowParquetWriteSupport(Confi * Constructs the Parquet schema based on the given Spark schema and Avro schema. * The Avro schema is used to determine the precision of timestamps. * @param structType Spark StructType - * @param avroSchema Avro schema + * @param schema Hoodie schema * @return Parquet MessageType with field ids propagated from Spark StructType if available */ - private MessageType convert(StructType structType, Schema avroSchema) { + private MessageType convert(StructType structType, HoodieSchema schema) { return Types.buildMessage() .addFields(Arrays.stream(structType.fields()).map(field -> { - Schema.Field avroField = avroSchema.getField(field.name()); - return convertField(avroField == null ? null : avroField.schema(), field); + // Note: Cannot use HoodieSchemaField::schema method reference due to Java 17 compilation ambiguity + HoodieSchema fieldSchema = schema.getField(field.name()) + .map(f -> f.schema()) + .orElse(null); + return convertField(fieldSchema, field); }).toArray(Type[]::new)) .named("spark_schema"); } - private Type convertField(Schema avroFieldSchema, StructField structField) { - Type type = convertField(avroFieldSchema, structField, structField.nullable() ? OPTIONAL : REQUIRED); + private Type convertField(HoodieSchema fieldSchema, StructField structField) { + Type type = convertField(fieldSchema, structField, structField.nullable() ? OPTIONAL : REQUIRED); if (ParquetUtils.hasFieldId(structField)) { return type.withId(ParquetUtils.getFieldId(structField)); } return type; } - private Type convertField(Schema avroFieldSchema, StructField structField, Type.Repetition repetition) { - Schema resolvedSchema = avroFieldSchema == null ? null : getNonNullTypeFromUnion(avroFieldSchema); - LogicalType logicalType = resolvedSchema != null ? resolvedSchema.getLogicalType() : null; + private Type convertField(HoodieSchema fieldSchema, StructField structField, Type.Repetition repetition) { + HoodieSchema resolvedSchema = fieldSchema == null ? null : fieldSchema.getNonNullType(); DataType dataType = structField.dataType(); if (dataType == DataTypes.BooleanType) { @@ -444,24 +463,42 @@ private Type convertField(Schema avroFieldSchema, StructField structField, Type. } else if (dataType == DataTypes.LongType || dataType instanceof DayTimeIntervalType) { return Types.primitive(INT64, repetition).named(structField.name()); } else if (dataType == DataTypes.TimestampType) { - if (logicalType == null || logicalType.getName().equals(LogicalTypes.timestampMicros().getName())) { + if (resolvedSchema != null && resolvedSchema.getType() == HoodieSchemaType.TIMESTAMP) { + HoodieSchema.Timestamp timestampSchema = (HoodieSchema.Timestamp) resolvedSchema; + if (timestampSchema.getPrecision() == TimePrecision.MICROS) { + return Types.primitive(INT64, repetition) + .as(LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MICROS)).named(structField.name()); + } else if (timestampSchema.getPrecision() == TimePrecision.MILLIS) { + return Types.primitive(INT64, repetition) + .as(LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MILLIS)).named(structField.name()); + } else { + throw new UnsupportedOperationException("Unsupported timestamp precision for TimestampType: " + timestampSchema.getPrecision()); + } + } else { + // Default to micros precision when no timestamp schema is available return Types.primitive(INT64, repetition) .as(LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MICROS)).named(structField.name()); - } else if (logicalType.getName().equals(LogicalTypes.timestampMillis().getName())) { - return Types.primitive(INT64, repetition) - .as(LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MILLIS)).named(structField.name()); - } else { - throw new UnsupportedOperationException("Unsupported timestamp type: " + logicalType); } } else if (SparkAdapterSupport$.MODULE$.sparkAdapter().isTimestampNTZType(dataType)) { - if (logicalType == null || logicalType.getName().equals(LogicalTypes.localTimestampMicros().getName())) { + if (resolvedSchema != null && resolvedSchema.getType() == HoodieSchemaType.TIMESTAMP) { + HoodieSchema.Timestamp timestampSchema = (HoodieSchema.Timestamp) resolvedSchema; + if (!timestampSchema.isUtcAdjusted()) { + if (timestampSchema.getPrecision() == TimePrecision.MICROS) { + return Types.primitive(INT64, repetition) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MICROS)).named(structField.name()); + } else if (timestampSchema.getPrecision() == TimePrecision.MILLIS) { + return Types.primitive(INT64, repetition) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS)).named(structField.name()); + } else { + throw new UnsupportedOperationException("Unsupported timestamp precision for TimestampNTZType: " + timestampSchema.getPrecision()); + } + } else { + throw new UnsupportedOperationException("TimestampNTZType requires local timestamp schema, but got UTC-adjusted: " + timestampSchema.getName()); + } + } else { + // Default to micros precision when no timestamp schema is available return Types.primitive(INT64, repetition) .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MICROS)).named(structField.name()); - } else if (logicalType.getName().equals(LogicalTypes.localTimestampMillis().getName())) { - return Types.primitive(INT64, repetition) - .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS)).named(structField.name()); - } else { - throw new UnsupportedOperationException("Unsupported timestamp type: " + logicalType); } } else if (dataType == DataTypes.FloatType) { return Types.primitive(FLOAT, repetition).named(structField.name()); @@ -483,13 +520,13 @@ private Type convertField(Schema avroFieldSchema, StructField structField, Type. } else if (dataType instanceof ArrayType) { ArrayType arrayType = (ArrayType) dataType; DataType elementType = arrayType.elementType(); - Schema avroElementSchema = resolvedSchema == null ? null : resolvedSchema.getElementType(); + HoodieSchema elementSchema = resolvedSchema == null ? null : resolvedSchema.getElementType(); if (!writeLegacyListFormat) { return Types .buildGroup(repetition).as(LogicalTypeAnnotation.listType()) .addField( Types.repeatedGroup() - .addField(convertField(avroElementSchema, new StructField("element", elementType, arrayType.containsNull(), Metadata.empty()))) + .addField(convertField(elementSchema, new StructField("element", elementType, arrayType.containsNull(), Metadata.empty()))) .named("list")) .named(structField.name()); } else if ((arrayType.containsNull())) { @@ -498,18 +535,18 @@ private Type convertField(Schema avroFieldSchema, StructField structField, Type. .addField(Types .buildGroup(REPEATED) // "array" is the name chosen by parquet-hive (1.7.0 and prior version) - .addField(convertField(avroElementSchema, new StructField("array", elementType, true, Metadata.empty()))) + .addField(convertField(elementSchema, new StructField("array", elementType, true, Metadata.empty()))) .named("bag")) .named(structField.name()); } else { return Types .buildGroup(repetition).as(LogicalTypeAnnotation.listType()) // "array" is the name chosen by parquet-avro (1.7.0 and prior version) - .addField(convertField(avroElementSchema, new StructField("array", elementType, false, Metadata.empty()), REPEATED)) + .addField(convertField(elementSchema, new StructField("array", elementType, false, Metadata.empty()), REPEATED)) .named(structField.name()); } } else if (dataType instanceof MapType) { - Schema avroValueSchema = resolvedSchema == null ? null : resolvedSchema.getValueType(); + HoodieSchema valueSchema = resolvedSchema == null ? null : resolvedSchema.getValueType(); MapType mapType = (MapType) dataType; return Types .buildGroup(repetition).as(LogicalTypeAnnotation.mapType()) @@ -517,14 +554,18 @@ private Type convertField(Schema avroFieldSchema, StructField structField, Type. Types .repeatedGroup() .addField(convertField(MAP_KEY_SCHEMA, new StructField(MAP_KEY_NAME, DataTypes.StringType, false, Metadata.empty()))) - .addField(convertField(avroValueSchema, new StructField(MAP_VALUE_NAME, mapType.valueType(), mapType.valueContainsNull(), Metadata.empty()))) + .addField(convertField(valueSchema, new StructField(MAP_VALUE_NAME, mapType.valueType(), mapType.valueContainsNull(), Metadata.empty()))) .named(MAP_REPEATED_NAME)) .named(structField.name()); } else if (dataType instanceof StructType) { Types.GroupBuilder groupBuilder = Types.buildGroup(repetition); Arrays.stream(((StructType) dataType).fields()).forEach(field -> { - Schema.Field avroField = resolvedSchema == null ? null : resolvedSchema.getField(field.name()); - groupBuilder.addField(convertField(avroField == null ? null : avroField.schema(), field)); + // Note: Cannot use HoodieSchemaField::schema method reference due to Java 17 compilation ambiguity + HoodieSchema nestedFieldSchema = Option.ofNullable(resolvedSchema) + .flatMap(s -> s.getField(field.name())) + .map(f -> f.schema()) + .orElse(null); + groupBuilder.addField(convertField(nestedFieldSchema, field)); }); return groupBuilder.named(structField.name()); } else { diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala index 3a644682e0e39..96092fe22be82 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala @@ -85,7 +85,7 @@ class SparkFileFormatInternalRowReaderContext(baseFileReader: SparkColumnarFileR if (FSUtils.isLogFile(filePath)) { // NOTE: now only primary key based filtering is supported for log files new HoodieSparkFileReaderFactory(storage).newParquetFileReader(filePath) - .asInstanceOf[HoodieSparkParquetReader].getUnsafeRowIterator(requiredSchema.toAvroSchema, readFilters.asJava).asInstanceOf[ClosableIterator[InternalRow]] + .asInstanceOf[HoodieSparkParquetReader].getUnsafeRowIterator(requiredSchema, readFilters.asJava).asInstanceOf[ClosableIterator[InternalRow]] } else { // partition value is empty because the spark parquet reader will append the partition columns to // each row if they are given. That is the only usage of the partition values in the reader. diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaComparatorForRecordProjection.java b/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaComparatorForRecordProjection.java index 2a75936e93a43..a426930ea09ab 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaComparatorForRecordProjection.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaComparatorForRecordProjection.java @@ -23,8 +23,6 @@ import java.util.List; -import static org.apache.hudi.avro.AvroSchemaUtils.getNonNullTypeFromUnion; - public class AvroSchemaComparatorForRecordProjection extends AvroSchemaComparatorForSchemaEvolution { private static final AvroSchemaComparatorForRecordProjection INSTANCE = new AvroSchemaComparatorForRecordProjection(); @@ -41,7 +39,7 @@ protected boolean schemaEqualsInternal(Schema s1, Schema s2) { if (s1 == null || s2 == null) { return false; } - return super.schemaEqualsInternal(getNonNullTypeFromUnion(s1), getNonNullTypeFromUnion(s2)); + return super.schemaEqualsInternal(AvroSchemaUtils.getNonNullTypeFromUnion(s1), AvroSchemaUtils.getNonNullTypeFromUnion(s2)); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java index 66c2a19e55c01..c4ea3709c115c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java @@ -97,10 +97,6 @@ import static org.apache.avro.Schema.Type.ARRAY; import static org.apache.avro.Schema.Type.MAP; import static org.apache.avro.Schema.Type.UNION; -import static org.apache.hudi.avro.AvroSchemaUtils.createNewSchemaFromFieldsWithReference; -import static org.apache.hudi.avro.AvroSchemaUtils.createNullableSchema; -import static org.apache.hudi.avro.AvroSchemaUtils.isNullable; -import static org.apache.hudi.avro.AvroSchemaUtils.getNonNullTypeFromUnion; import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes; import static org.apache.hudi.common.util.ValidationUtils.checkState; @@ -126,7 +122,7 @@ public class HoodieAvroUtils { private static final Properties PROPERTIES = new Properties(); // All metadata fields are optional strings. - public static final Schema METADATA_FIELD_SCHEMA = createNullableSchema(Schema.Type.STRING); + public static final Schema METADATA_FIELD_SCHEMA = AvroSchemaUtils.createNullableSchema(Schema.Type.STRING); public static final Schema RECORD_KEY_SCHEMA = initRecordKeySchema(); @@ -320,7 +316,7 @@ public static Schema addMetadataFields(Schema schema, boolean withOperationField parentFields.add(newField); } } - return createNewSchemaFromFieldsWithReference(schema, parentFields); + return AvroSchemaUtils.createNewSchemaFromFieldsWithReference(schema, parentFields); } /** @@ -399,7 +395,7 @@ public static Schema removeFields(Schema schema, Set fieldsToRemove) { .map(HoodieAvroUtils::createNewSchemaField) .collect(Collectors.toList()); - return createNewSchemaFromFieldsWithReference(schema, filteredFields); + return AvroSchemaUtils.createNewSchemaFromFieldsWithReference(schema, filteredFields); } public static String addMetadataColumnTypes(String hiveColumnTypes) { @@ -418,7 +414,7 @@ public static Schema makeFieldNonNull(Schema schema, String fieldName, Object fi } }) .collect(Collectors.toList()); - return createNewSchemaFromFieldsWithReference(schema, filteredFields); + return AvroSchemaUtils.createNewSchemaFromFieldsWithReference(schema, filteredFields); } private static Schema initRecordKeySchema() { @@ -810,7 +806,7 @@ public static Schema getNestedFieldSchemaFromRecord(GenericRecord record, String Object val = valueNode.get(part); if (i == parts.length - 1) { - return getNonNullTypeFromUnion(valueNode.getSchema().getField(part).schema()); + return AvroSchemaUtils.getNonNullTypeFromUnion(valueNode.getSchema().getField(part).schema()); } else { if (!(val instanceof GenericRecord)) { throw new HoodieException("Cannot find a record at part value :" + part); @@ -836,11 +832,11 @@ public static Schema getNestedFieldSchemaFromWriteSchema(Schema writeSchema, Str String part = parts[i]; try { // Resolve nullable/union schema to the actual schema - currentSchema = getNonNullTypeFromUnion(currentSchema.getField(part).schema()); + currentSchema = AvroSchemaUtils.getNonNullTypeFromUnion(currentSchema.getField(part).schema()); if (i == parts.length - 1) { // Return the schema for the final part - return getNonNullTypeFromUnion(currentSchema); + return AvroSchemaUtils.getNonNullTypeFromUnion(currentSchema); } } catch (Exception e) { throw new HoodieException("Failed to get schema. Not a valid field name: " + fieldName); @@ -877,11 +873,11 @@ public static Object convertValueForSpecificDataTypes(Schema fieldSchema, if (fieldSchema == null) { return fieldValue; } else if (fieldValue == null) { - checkState(isNullable(fieldSchema)); + checkState(AvroSchemaUtils.isNullable(fieldSchema)); return null; } - return convertValueForAvroLogicalTypes(getNonNullTypeFromUnion(fieldSchema), fieldValue, consistentLogicalTimestampEnabled); + return convertValueForAvroLogicalTypes(AvroSchemaUtils.getNonNullTypeFromUnion(fieldSchema), fieldValue, consistentLogicalTimestampEnabled); } /** @@ -1115,7 +1111,7 @@ private static Object rewriteRecordWithNewSchemaInternal(Object oldRecord, newRecord.put(i, rewriteRecordWithNewSchema(indexedRecord.get(oldField.pos()), oldField.schema(), newField.schema(), renameCols, fieldNames, false)); } else if (newField.defaultVal() instanceof JsonProperties.Null) { newRecord.put(i, null); - } else if (!isNullable(newField.schema()) && newField.defaultVal() == null) { + } else if (!AvroSchemaUtils.isNullable(newField.schema()) && newField.defaultVal() == null) { throw new SchemaCompatibilityException("Field " + createFullName(fieldNames) + " has no default value and is non-nullable"); } else { newRecord.put(i, newField.defaultVal()); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaCompatibility.java b/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaCompatibility.java index 79a26abb9dd3d..ab2f04b2ae9e5 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaCompatibility.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaCompatibility.java @@ -18,8 +18,11 @@ package org.apache.hudi.common.schema; +import org.apache.hudi.avro.AvroSchemaCompatibility; import org.apache.hudi.avro.AvroSchemaUtils; +import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.internal.schema.HoodieSchemaException; import java.util.Collections; import java.util.Set; @@ -112,13 +115,36 @@ public static void checkValidEvolution(HoodieSchema incomingSchema, HoodieSchema * 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 + * @param prevSchema previous instance of the schema + * @param newSchema new instance of the schema * @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) { - return isSchemaCompatible(readerSchema, writerSchema, true); + public static boolean isSchemaCompatible(HoodieSchema prevSchema, HoodieSchema newSchema) { + return isSchemaCompatible(prevSchema, newSchema, true); + } + + /** + * Establishes whether {@code newSchema} is compatible w/ {@code prevSchema}, as + * defined by Avro's {@link AvroSchemaCompatibility}. + * From avro's compatibility standpoint, prevSchema is writer schema and new schema is reader schema. + * {@code newSchema} is considered compatible to {@code prevSchema}, iff data written using {@code prevSchema} + * could be read by {@code newSchema} + * + * @param prevSchema previous instance of the schema + * @param newSchema new instance of the schema + * @param checkNaming controls whether schemas fully-qualified names should be checked + * @param allowProjection whether to allow fewer fields in reader schema + * @return true if reader schema can read data written with writer schema + * @throws IllegalArgumentException if schemas are null + */ + public static boolean isSchemaCompatible(HoodieSchema prevSchema, HoodieSchema newSchema, + boolean checkNaming, boolean allowProjection) { + ValidationUtils.checkArgument(prevSchema != null, "Prev schema cannot be null"); + ValidationUtils.checkArgument(newSchema != null, "New schema cannot be null"); + + // Use HoodieSchemaUtils delegation for consistency + return AvroSchemaUtils.isSchemaCompatible(prevSchema.toAvroSchema(), newSchema.toAvroSchema(), checkNaming, allowProjection); } /** @@ -191,4 +217,36 @@ public static boolean areSchemasProjectionEquivalent(HoodieSchema schema1, Hoodi } return AvroSchemaUtils.areSchemasProjectionEquivalent(schema1.toAvroSchema(), schema2.toAvroSchema()); } + + /** + * Identifies the writer field that corresponds to the specified reader field. + * This function is adapted from AvroSchemaCompatibility#lookupWriterField + * + *

+ * Matching includes reader name aliases. + *

+ * + * @param writerSchema Schema of the record where to look for the writer field. + * @param readerField Reader field to identify the corresponding writer field + * of. + * @return the writer field, if any does correspond, or None. + */ + public static HoodieSchemaField lookupWriterField(final HoodieSchema writerSchema, final HoodieSchemaField readerField) { + ValidationUtils.checkArgument(writerSchema.getType() == HoodieSchemaType.RECORD, writerSchema + " is not a record"); + Option directOpt = writerSchema.getField(readerField.name()); + // Check aliases + for (final String readerFieldAliasName : readerField.getAvroField().aliases()) { + final Option writerFieldOpt = writerSchema.getField(readerFieldAliasName); + if (writerFieldOpt.isPresent()) { + if (directOpt.isPresent()) { + // Multiple matches found, fail fast + throw new HoodieSchemaException(String.format( + "Reader record field %s matches multiple fields in writer record schema %s", readerField, writerSchema)); + } + directOpt = writerFieldOpt; + } + } + + return directOpt.orElse(null); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaUtils.java index 4f30a1665e29e..4a2b458044d97 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaUtils.java @@ -483,6 +483,19 @@ public static HoodieSchema appendFieldsToSchemaDedupNested(HoodieSchema schema, newFields.stream().map(HoodieSchemaField::getAvroField).collect(Collectors.toList()))); } + /** + * Appends provided new fields at the end of the given schema + * + * NOTE: No deduplication is made, this method simply appends fields at the end of the list + * of the source schema as is + * + * This is equivalent to {@link AvroSchemaUtils#appendFieldsToSchema(Schema, List)} but operates on HoodieSchema. + */ + public static HoodieSchema appendFieldsToSchema(HoodieSchema schema, List newFields) { + return HoodieSchema.fromAvroSchema(AvroSchemaUtils.appendFieldsToSchema(schema.toAvroSchema(), + newFields.stream().map(HoodieSchemaField::getAvroField).collect(Collectors.toList()))); + } + /** * Create a new schema but maintain all meta info from the old schema. * This is equivalent to {@link AvroSchemaUtils#createNewSchemaFromFieldsWithReference(Schema, List)} but operates on HoodieSchema. diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java index fe417f8f9d20d..2309ef5ab7432 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java @@ -26,6 +26,9 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaField; +import org.apache.hudi.common.schema.HoodieSchemaType; +import org.apache.hudi.common.schema.HoodieSchemaUtils; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.HoodieLogFormat.Reader; import org.apache.hudi.common.table.log.block.HoodieDataBlock; @@ -48,9 +51,7 @@ import org.apache.hudi.storage.StoragePath; import org.apache.hudi.util.Lazy; -import org.apache.avro.JsonProperties; import org.apache.avro.Schema; -import org.apache.avro.Schema.Field; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,10 +67,6 @@ import java.util.function.Supplier; import java.util.stream.Stream; -import static org.apache.hudi.avro.AvroSchemaUtils.appendFieldsToSchema; -import static org.apache.hudi.avro.AvroSchemaUtils.containsFieldInSchema; -import static org.apache.hudi.avro.AvroSchemaUtils.createNullableSchema; - /** * Helper class to read schema from data files and log files and to convert it between different formats. */ @@ -143,6 +140,20 @@ public HoodieSchema getTableSchema(boolean includeMetadataFields) throws Excepti return HoodieSchema.fromAvroSchema(avroSchema); } + /** + * Fetches tables schema in Avro format as of the given instant as HoodieSchema. + * + * @param timestamp as of which table's schema will be fetched + */ + public HoodieSchema getTableSchema(String timestamp) throws Exception { + Schema avroSchema = getTableAvroSchema(timestamp); + return HoodieSchema.fromAvroSchema(avroSchema); + } + + public Option getTableSchemaIfPresent(boolean includeMetadataFields) { + return getTableAvroSchemaInternal(includeMetadataFields, Option.empty()).map(HoodieSchema::fromAvroSchema); + } + /** * Gets full schema (user + metadata) for a hoodie table in Avro format. * @@ -225,8 +236,10 @@ private Option getTableAvroSchemaInternal(boolean includeMetadataFields, // TODO partition columns have to be appended in all read-paths if (metaClient.getTableConfig().shouldDropPartitionColumns() && schema.isPresent()) { + HoodieSchema hoodieSchema = HoodieSchema.fromAvroSchema(schema.get()); return metaClient.getTableConfig().getPartitionFields() - .map(partitionFields -> appendPartitionColumns(schema.get(), Option.ofNullable(partitionFields))) + .map(partitionFields -> appendPartitionColumns(hoodieSchema, Option.ofNullable(partitionFields))) + .map(HoodieSchema::toAvroSchema) .or(() -> schema); } @@ -526,7 +539,7 @@ private Schema fetchSchemaFromFiles(Stream filePaths) { }).filter(Objects::nonNull).findFirst().orElse(null); } - public static Schema appendPartitionColumns(Schema dataSchema, Option partitionFields) { + public static HoodieSchema appendPartitionColumns(HoodieSchema dataSchema, Option partitionFields) { // In cases when {@link DROP_PARTITION_COLUMNS} config is set true, partition columns // won't be persisted w/in the data files, and therefore we need to append such columns // when schema is parsed from data files @@ -536,8 +549,8 @@ public static Schema appendPartitionColumns(Schema dataSchema, Option return dataSchema; } - boolean hasPartitionColNotInSchema = Arrays.stream(partitionFields.get()).anyMatch(pf -> !containsFieldInSchema(dataSchema, pf)); - boolean hasPartitionColInSchema = Arrays.stream(partitionFields.get()).anyMatch(pf -> containsFieldInSchema(dataSchema, pf)); + boolean hasPartitionColNotInSchema = Arrays.stream(partitionFields.get()).anyMatch(pf -> dataSchema.getField(pf).isEmpty()); + boolean hasPartitionColInSchema = Arrays.stream(partitionFields.get()).anyMatch(pf -> dataSchema.getField(pf).isPresent()); if (hasPartitionColNotInSchema && hasPartitionColInSchema) { throw new HoodieSchemaException("Partition columns could not be partially contained w/in the data schema"); } @@ -545,12 +558,12 @@ public static Schema appendPartitionColumns(Schema dataSchema, Option if (hasPartitionColNotInSchema) { // when hasPartitionColNotInSchema is true and hasPartitionColInSchema is false, all partition columns // are not in originSchema. So we create and add them. - List newFields = new ArrayList<>(); + List newFields = new ArrayList<>(); for (String partitionField: partitionFields.get()) { - newFields.add(new Schema.Field( - partitionField, createNullableSchema(Schema.Type.STRING), "", JsonProperties.NULL_VALUE)); + newFields.add(HoodieSchemaField.of( + partitionField, HoodieSchema.createNullable(HoodieSchemaType.STRING), "", HoodieSchema.NULL_VALUE)); } - return appendFieldsToSchema(dataSchema, newFields); + return HoodieSchemaUtils.appendFieldsToSchema(dataSchema, newFields); } return dataSchema; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java index 7ea9fac74cd30..64b19a7bdd912 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java @@ -188,7 +188,8 @@ private ClosableIterator makeBootstrapBaseFileIterator(HoodieBaseFile baseFil throw new IllegalArgumentException("Filegroup reader is doing bootstrap merge but we are not reading from the start of the base file"); } PartitionPathParser partitionPathParser = new PartitionPathParser(); - Object[] partitionValues = partitionPathParser.getPartitionFieldVals(partitionPathFields, inputSplit.getPartitionPath(), readerContext.getSchemaHandler().getTableSchema()); + Object[] partitionValues = partitionPathParser.getPartitionFieldVals(partitionPathFields, inputSplit.getPartitionPath(), + readerContext.getSchemaHandler().getTableSchema()); // filter out the partition values that are not required by the data schema List> partitionPathFieldsAndValues = partitionPathFields.map(partitionFields -> { HoodieSchema dataSchema = dataFileIterator.get().getRight(); diff --git a/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroReaderContext.java b/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroReaderContext.java index cfbef55610d4f..fefab767bc7c1 100644 --- a/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroReaderContext.java +++ b/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroReaderContext.java @@ -53,8 +53,8 @@ class TestHoodieAvroReaderContext { private static final HoodieSchema LIMITED_SKELETON_SCHEMA = getLimitedSkeletonSchema(); private static final HoodieSchema BASE_SCHEMA = getBaseSchema(); private static final HoodieSchema LIMITED_BASE_SCHEMA = getLimitedBaseSchema(); - private static final Schema FULL_MERGED_SCHEMA = AvroSchemaUtils.mergeSchemas(SKELETON_SCHEMA.toAvroSchema(), BASE_SCHEMA.toAvroSchema()); - private static final Schema LIMTIED_MERGED_SCHEMA = AvroSchemaUtils.mergeSchemas(LIMITED_SKELETON_SCHEMA.toAvroSchema(), LIMITED_BASE_SCHEMA.toAvroSchema()); + private static final HoodieSchema FULL_MERGED_SCHEMA = HoodieSchemaUtils.mergeSchemas(SKELETON_SCHEMA, BASE_SCHEMA); + private static final HoodieSchema LIMITED_MERGED_SCHEMA = HoodieSchemaUtils.mergeSchemas(LIMITED_SKELETON_SCHEMA, LIMITED_BASE_SCHEMA); private final StorageConfiguration storageConfig = mock(StorageConfiguration.class); private final HoodieTableConfig tableConfig = mock(HoodieTableConfig.class); @@ -308,9 +308,9 @@ private IndexedRecord createBaseRecord(String field1, String field2, double fiel private IndexedRecord createFullMergedRecord(String skeletonField1, String skeletonField2, int skeletonField3, String baseField1, String baseField2, double baseField3) { - GenericRecord nested = new GenericData.Record(FULL_MERGED_SCHEMA.getFields().get(5).schema()); + GenericRecord nested = new GenericData.Record(FULL_MERGED_SCHEMA.getFields().get(5).schema().toAvroSchema()); nested.put(0, baseField3); - GenericRecord record = new GenericData.Record(FULL_MERGED_SCHEMA); + GenericRecord record = new GenericData.Record(FULL_MERGED_SCHEMA.toAvroSchema()); record.put(0, skeletonField1); record.put(1, skeletonField2); record.put(2, skeletonField3); @@ -321,9 +321,9 @@ private IndexedRecord createFullMergedRecord(String skeletonField1, String skele } private IndexedRecord createLimitedMergedRecord(String skeletonField2, String baseField1, double baseField3) { - GenericRecord nested = new GenericData.Record(LIMTIED_MERGED_SCHEMA.getFields().get(2).schema()); + GenericRecord nested = new GenericData.Record(LIMITED_MERGED_SCHEMA.getFields().get(2).schema().toAvroSchema()); nested.put(0, baseField3); - GenericRecord record = new GenericData.Record(LIMTIED_MERGED_SCHEMA); + GenericRecord record = new GenericData.Record(LIMITED_MERGED_SCHEMA.toAvroSchema()); record.put(0, skeletonField2); record.put(1, baseField1); record.put(2, nested); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieCommitMetadata.java b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieCommitMetadata.java index 65f1e6eaa073d..a8459a716266c 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieCommitMetadata.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieCommitMetadata.java @@ -18,6 +18,11 @@ package org.apache.hudi.common.model; +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaCompatibility; +import org.apache.hudi.common.schema.HoodieSchemaField; +import org.apache.hudi.common.schema.HoodieSchemaType; +import org.apache.hudi.common.schema.HoodieSchemaUtils; import org.apache.hudi.common.table.timeline.CommitMetadataSerDe; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; @@ -29,7 +34,6 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; -import org.apache.avro.Schema; import org.apache.avro.reflect.ReflectData; import org.junit.jupiter.api.Test; import org.slf4j.Logger; @@ -45,8 +49,6 @@ import java.util.Map; import java.util.stream.Collectors; -import static org.apache.hudi.avro.AvroSchemaUtils.isSchemaCompatible; -import static org.apache.hudi.avro.HoodieAvroUtils.createNewSchemaField; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.convertMetadataToByteArray; import static org.apache.hudi.common.testutils.HoodieTestUtils.COMMIT_METADATA_SER_DE; import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; @@ -206,50 +208,50 @@ private org.apache.hudi.avro.model.HoodieWriteStat createWriteStat(String fileId @Test public void testSchemaEqualityForHoodieCommitMetaData() { // Step 1: Get the schema from the Avro auto-generated class - Schema avroSchema = org.apache.hudi.avro.model.HoodieCommitMetadata.SCHEMA$; + HoodieSchema schema = HoodieSchema.fromAvroSchema(org.apache.hudi.avro.model.HoodieCommitMetadata.SCHEMA$); // Step 2: Convert the POJO class to an Avro schema - Schema pojoSchema = ReflectData.get().getSchema(org.apache.hudi.common.model.HoodieCommitMetadata.class); + HoodieSchema pojoSchema = HoodieSchema.fromAvroSchema(ReflectData.get().getSchema(org.apache.hudi.common.model.HoodieCommitMetadata.class)); // Step 3: Validate schemas // We need to replace ENUM with STRING to workaround inherit type mismatch of java ENUM when coverted to avro object. - assertTrue(isSchemaCompatible(replaceEnumWithString(pojoSchema), avroSchema, false, false)); + assertTrue(HoodieSchemaCompatibility.isSchemaCompatible(replaceEnumWithString(pojoSchema), schema, false, false)); } @Test public void testSchemaEqualityForHoodieReplaceCommitMetaData() { // Step 1: Get the schema from the Avro auto-generated class - Schema avroSchema = org.apache.hudi.avro.model.HoodieReplaceCommitMetadata.SCHEMA$; + HoodieSchema schema = HoodieSchema.fromAvroSchema(org.apache.hudi.avro.model.HoodieReplaceCommitMetadata.SCHEMA$); // Step 2: Convert the POJO class to an Avro schema - Schema pojoSchema = ReflectData.get().getSchema(org.apache.hudi.common.model.HoodieReplaceCommitMetadata.class); + HoodieSchema pojoSchema = HoodieSchema.fromAvroSchema(ReflectData.get().getSchema(org.apache.hudi.common.model.HoodieReplaceCommitMetadata.class)); // Step 3: Validate schemas - // We need to replace ENUM with STRING to workaround inherit type mismatch of java ENUM when coverted to avro object. - assertTrue(isSchemaCompatible(replaceEnumWithString(pojoSchema), avroSchema, false, false)); + // We need to replace ENUM with STRING to workaround inherit type mismatch of java ENUM when converted to avro object. + assertTrue(HoodieSchemaCompatibility.isSchemaCompatible(replaceEnumWithString(pojoSchema), schema, false, false)); } // Utility method that search for all ENUM fields and replace it with STRING. - private Schema replaceEnumWithString(Schema schema) { - if (schema.getType() == Schema.Type.ENUM) { - return Schema.create(Schema.Type.STRING); - } else if (schema.getType() == Schema.Type.RECORD) { - List newFields = new ArrayList<>(); - for (Schema.Field field : schema.getFields()) { - Schema newFieldSchema = replaceEnumWithString(field.schema()); - newFields.add(createNewSchemaField(field.name(), newFieldSchema, field.doc(), field.defaultVal())); + private HoodieSchema replaceEnumWithString(HoodieSchema schema) { + if (schema.getType() == HoodieSchemaType.ENUM) { + return HoodieSchema.create(HoodieSchemaType.STRING); + } else if (schema.getType() == HoodieSchemaType.RECORD) { + List newFields = new ArrayList<>(); + for (HoodieSchemaField field : schema.getFields()) { + HoodieSchema newFieldSchema = replaceEnumWithString(field.schema()); + newFields.add(HoodieSchemaUtils.createNewSchemaField(field.name(), newFieldSchema, field.doc().orElse(null), field.defaultVal().orElse(null))); } - return Schema.createRecord(schema.getName(), schema.getDoc(), schema.getNamespace(), false, newFields); - } else if (schema.getType() == Schema.Type.UNION) { - List types = new ArrayList<>(); - for (Schema type : schema.getTypes()) { + return HoodieSchema.createRecord(schema.getName(), schema.getDoc().orElse(null), schema.getNamespace().orElse(null), false, newFields); + } else if (schema.getType() == HoodieSchemaType.UNION) { + List types = new ArrayList<>(); + for (HoodieSchema type : schema.getTypes()) { types.add(replaceEnumWithString(type)); } - return Schema.createUnion(types); - } else if (schema.getType() == Schema.Type.ARRAY) { - return Schema.createArray(replaceEnumWithString(schema.getElementType())); - } else if (schema.getType() == Schema.Type.MAP) { - return Schema.createMap(replaceEnumWithString(schema.getValueType())); + return HoodieSchema.createUnion(types); + } else if (schema.getType() == HoodieSchemaType.ARRAY) { + return HoodieSchema.createArray(replaceEnumWithString(schema.getElementType())); + } else if (schema.getType() == HoodieSchemaType.MAP) { + return HoodieSchema.createMap(replaceEnumWithString(schema.getValueType())); } return schema; } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/schema/TestHoodieSchemaCompatibility.java b/hudi-common/src/test/java/org/apache/hudi/common/schema/TestHoodieSchemaCompatibility.java index c7e36d52c12be..523547aa3ce7f 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/schema/TestHoodieSchemaCompatibility.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/schema/TestHoodieSchemaCompatibility.java @@ -587,18 +587,165 @@ public void testIsSchemaCompatibleValidation() { } @Test - public void testCompatibilityConsistency() { - // Test that compatibility checks are consistent between Avro and Hoodie utilities + public void testIsSchemaCompatibleWithCheckNamingAndAllowProjection() { + // Test with checkNaming=true, allowProjection=true HoodieSchema baseSchema = HoodieSchema.parse(SIMPLE_SCHEMA); HoodieSchema evolvedSchema = HoodieSchema.parse(EVOLVED_SCHEMA); - Schema baseAvro = baseSchema.toAvroSchema(); - Schema evolvedAvro = evolvedSchema.toAvroSchema(); + // Evolved schema should be compatible with base schema (can read data written with base schema) + assertTrue(HoodieSchemaCompatibility.isSchemaCompatible(baseSchema, evolvedSchema, true, true)); - // Results should be consistent - boolean avroCompatible = AvroSchemaUtils.isSchemaCompatible(baseAvro, evolvedAvro); - boolean hoodieCompatible = HoodieSchemaCompatibility.isSchemaCompatible(baseSchema, evolvedSchema); + // Base schema should also be compatible with evolved schema if projection is allowed + assertTrue(HoodieSchemaCompatibility.isSchemaCompatible(evolvedSchema, baseSchema, true, true)); + } + + @Test + public void testIsSchemaCompatibleWithCheckNamingNoProjection() { + // Test with checkNaming=true, allowProjection=false + HoodieSchema fullSchema = HoodieSchema.fromAvroSchema(FULL_SCHEMA); + HoodieSchema shortSchema = HoodieSchema.fromAvroSchema(SHORT_SCHEMA); + + // Short schema is not compatible with full schema when projection is not allowed + assertFalse(HoodieSchemaCompatibility.isSchemaCompatible(fullSchema, shortSchema, true, false)); + + // Full schema should be compatible with short schema (has all fields) + assertTrue(HoodieSchemaCompatibility.isSchemaCompatible(shortSchema, fullSchema, true, false)); + } + + @Test + public void testIsSchemaCompatibleWithoutCheckNaming() { + // Test with checkNaming=false, allowProjection=true + // Create schemas with different names but same structure + String schema1 = "{\"type\":\"record\",\"name\":\"Record1\",\"namespace\":\"ns1\",\"fields\":[" + + "{\"name\":\"field1\",\"type\":\"int\"}]}"; + String schema2 = "{\"type\":\"record\",\"name\":\"Record2\",\"namespace\":\"ns2\",\"fields\":[" + + "{\"name\":\"field1\",\"type\":\"int\"}]}"; + + HoodieSchema s1 = HoodieSchema.parse(schema1); + HoodieSchema s2 = HoodieSchema.parse(schema2); + + // With checkNaming=false, schemas with different names should be compatible + assertTrue(HoodieSchemaCompatibility.isSchemaCompatible(s1, s2, false, true)); + assertTrue(HoodieSchemaCompatibility.isSchemaCompatible(s2, s1, false, true)); + } + + @Test + public void testIsSchemaCompatibleWithTypePromotion() { + // Test type promotion scenarios + String intSchema = "{\"type\":\"record\",\"name\":\"R\",\"fields\":[" + + "{\"name\":\"num\",\"type\":\"int\"}]}"; + String longSchema = "{\"type\":\"record\",\"name\":\"R\",\"fields\":[" + + "{\"name\":\"num\",\"type\":\"long\"}]}"; + + HoodieSchema intS = HoodieSchema.parse(intSchema); + HoodieSchema longS = HoodieSchema.parse(longSchema); + + // Long can read int (type promotion) + assertTrue(HoodieSchemaCompatibility.isSchemaCompatible(intS, longS, true, true)); + + // Int cannot read long (no type demotion) + assertFalse(HoodieSchemaCompatibility.isSchemaCompatible(longS, intS, true, true)); + } + + @Test + public void testIsSchemaCompatibleWithNestedSchemas() { + // Test with nested record schemas + String nestedSchema1 = "{\"type\":\"record\",\"name\":\"Outer\",\"fields\":[" + + "{\"name\":\"inner\",\"type\":{\"type\":\"record\",\"name\":\"Inner\",\"fields\":[" + + "{\"name\":\"field1\",\"type\":\"string\"}," + + "{\"name\":\"field2\",\"type\":\"int\"}]}}]}"; + + String nestedSchema2 = "{\"type\":\"record\",\"name\":\"Outer\",\"fields\":[" + + "{\"name\":\"inner\",\"type\":{\"type\":\"record\",\"name\":\"Inner\",\"fields\":[" + + "{\"name\":\"field1\",\"type\":\"string\"}," + + "{\"name\":\"field2\",\"type\":\"int\"}," + + "{\"name\":\"field3\",\"type\":[\"null\",\"string\"],\"default\":null}]}}]}"; + + HoodieSchema s1 = HoodieSchema.parse(nestedSchema1); + HoodieSchema s2 = HoodieSchema.parse(nestedSchema2); + + // Schema with added nullable field should be compatible + assertTrue(HoodieSchemaCompatibility.isSchemaCompatible(s1, s2, true, true)); + + // Reverse should also work with projection allowed + assertTrue(HoodieSchemaCompatibility.isSchemaCompatible(s2, s1, true, true)); + } - assertEquals(avroCompatible, hoodieCompatible); + @Test + public void testIsSchemaCompatibleWithArrays() { + // Test with array type changes + String arrayIntSchema = "{\"type\":\"record\",\"name\":\"R\",\"fields\":[" + + "{\"name\":\"arr\",\"type\":{\"type\":\"array\",\"items\":\"int\"}}]}"; + String arrayLongSchema = "{\"type\":\"record\",\"name\":\"R\",\"fields\":[" + + "{\"name\":\"arr\",\"type\":{\"type\":\"array\",\"items\":\"long\"}}]}"; + + HoodieSchema intArray = HoodieSchema.parse(arrayIntSchema); + HoodieSchema longArray = HoodieSchema.parse(arrayLongSchema); + + // Array element type promotion + assertTrue(HoodieSchemaCompatibility.isSchemaCompatible(intArray, longArray, true, true)); + } + + @Test + public void testIsSchemaCompatibleWithMaps() { + // Test with map value type changes + String mapIntSchema = "{\"type\":\"record\",\"name\":\"R\",\"fields\":[" + + "{\"name\":\"map\",\"type\":{\"type\":\"map\",\"values\":\"int\"}}]}"; + String mapLongSchema = "{\"type\":\"record\",\"name\":\"R\",\"fields\":[" + + "{\"name\":\"map\",\"type\":{\"type\":\"map\",\"values\":\"long\"}}]}"; + + HoodieSchema intMap = HoodieSchema.parse(mapIntSchema); + HoodieSchema longMap = HoodieSchema.parse(mapLongSchema); + + // Map value type promotion + assertTrue(HoodieSchemaCompatibility.isSchemaCompatible(intMap, longMap, true, true)); + } + + @SuppressWarnings("DataFlowIssue") + @Test + public void testIsSchemaCompatibleValidationBothParams() { + HoodieSchema schema = HoodieSchema.parse(SIMPLE_SCHEMA); + + // Should throw on null previous schema + assertThrows(IllegalArgumentException.class, () -> HoodieSchemaCompatibility.isSchemaCompatible(null, schema, true, true)); + + // Should throw on null new schema + assertThrows(IllegalArgumentException.class, () -> HoodieSchemaCompatibility.isSchemaCompatible(schema, null, true, true)); + + // Should throw on both null + assertThrows(IllegalArgumentException.class, () -> HoodieSchemaCompatibility.isSchemaCompatible(null, null, true, true)); + } + + @Test + public void testIsSchemaCompatibleConsistencyWithAvro() { + // Verify HoodieSchemaCompatibility results match AvroSchemaUtils for various scenarios + HoodieSchema s1 = HoodieSchema.parse(SIMPLE_SCHEMA); + HoodieSchema s2 = HoodieSchema.parse(EVOLVED_SCHEMA); + + // Test all combinations of checkNaming and allowProjection + for (boolean checkNaming : Arrays.asList(true, false)) { + for (boolean allowProjection : Arrays.asList(true, false)) { + boolean avroResult = AvroSchemaUtils.isSchemaCompatible( + s1.toAvroSchema(), s2.toAvroSchema(), checkNaming, allowProjection); + boolean hoodieResult = HoodieSchemaCompatibility.isSchemaCompatible( + s1, s2, checkNaming, allowProjection); + + assertEquals(avroResult, hoodieResult, + String.format("Results should match for checkNaming=%s, allowProjection=%s", + checkNaming, allowProjection)); + } + } + } + + @Test + public void testIsSchemaCompatibleIdenticalSchemas() { + // Test with identical schemas + HoodieSchema schema = HoodieSchema.parse(SOURCE_SCHEMA); + + // Identical schemas should always be compatible + assertTrue(HoodieSchemaCompatibility.isSchemaCompatible(schema, schema, true, true)); + assertTrue(HoodieSchemaCompatibility.isSchemaCompatible(schema, schema, true, false)); + assertTrue(HoodieSchemaCompatibility.isSchemaCompatible(schema, schema, false, true)); + assertTrue(HoodieSchemaCompatibility.isSchemaCompatible(schema, schema, false, false)); } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/TestPartitionPathParser.java b/hudi-common/src/test/java/org/apache/hudi/common/table/TestPartitionPathParser.java index 1476c1ecd2c9a..57ab085aaf75d 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/TestPartitionPathParser.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/TestPartitionPathParser.java @@ -77,7 +77,7 @@ private static Stream fieldCases() { Arguments.of("default", HoodieSchema.create(HoodieSchemaType.INT), null), Arguments.of("2025-01-03", HoodieSchema.create(HoodieSchemaType.STRING), "2025-01-03"), Arguments.of("value1", HoodieSchema.create(HoodieSchemaType.BYTES), "value1".getBytes(StandardCharsets.UTF_8)), - Arguments.of("value1", HoodieSchema.createFixed("fixed", "docs", null, 50), "value1".getBytes(StandardCharsets.UTF_8)) + Arguments.of("value1", HoodieSchema.createFixed("fixed", null, "docs",50), "value1".getBytes(StandardCharsets.UTF_8)) ); } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java b/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java index 0bc955dfa293b..b1ff1656157dc 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java @@ -279,10 +279,10 @@ public void testReadLogFilesOnlyInMergeOnReadTable(RecordMergeMode recordMergeMo } } - protected static List> hoodieRecordsToIndexedRecords(List hoodieRecords, Schema schema) { + protected static List> hoodieRecordsToIndexedRecords(List hoodieRecords, HoodieSchema schema) { return hoodieRecords.stream().map(r -> { try { - Option avroIndexedRecordOption = r.toIndexedRecord(schema, CollectionUtils.emptyProps()); + Option avroIndexedRecordOption = r.toIndexedRecord(schema.toAvroSchema(), CollectionUtils.emptyProps()); if (avroIndexedRecordOption.isPresent()) { // eager deser avroIndexedRecordOption.get().getData().get(0); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java index c7162d54d3373..cc84de43e382c 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java @@ -19,7 +19,6 @@ package org.apache.hudi.common.testutils; -import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.common.model.HoodieAvroIndexedRecord; @@ -29,6 +28,9 @@ import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaField; +import org.apache.hudi.common.schema.HoodieSchemaType; +import org.apache.hudi.common.schema.HoodieSchemaUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator; @@ -91,7 +93,6 @@ import java.util.stream.IntStream; import java.util.stream.Stream; -import static org.apache.hudi.avro.HoodieAvroUtils.createNewSchemaField; import static org.apache.hudi.common.testutils.HoodieTestUtils.COMMIT_METADATA_SER_DE; import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes; @@ -260,7 +261,7 @@ public class HoodieTestDataGenerator implements AutoCloseable { private final String[] partitionPaths; //maintains the count of existing keys schema wise private Map numKeysBySchema; - private Option extendedSchema = Option.empty(); + private Option extendedSchema = Option.empty(); public HoodieTestDataGenerator(long seed) { this(seed, DEFAULT_PARTITION_PATHS, new HashMap<>()); @@ -516,6 +517,13 @@ public GenericRecord generateGenericRecord(String rowKey, String partitionPath, return generateGenericRecord(rowKey, partitionPath, riderName, driverName, timestamp, false, false); } + /** + * Get the Avro schema to use, considering extendedSchema if present. + */ + private Schema getEffectiveAvroSchema() { + return extendedSchema.map(HoodieSchema::toAvroSchema).orElse(AVRO_SCHEMA); + } + /** * Populate rec with values for TRIP_SCHEMA_PREFIX */ @@ -557,7 +565,7 @@ private void generateExtraSchemaValues(GenericRecord rec) { rec.put("current_ts", randomMillis); BigDecimal bigDecimal = new BigDecimal(String.format(Locale.ENGLISH, "%5f", rand.nextFloat())); - Schema decimalSchema = AVRO_SCHEMA.getField("height").schema(); + Schema decimalSchema = getEffectiveAvroSchema().getField("height").schema(); Conversions.DecimalConversion decimalConversions = new Conversions.DecimalConversion(); GenericFixed genericFixed = decimalConversions.toFixed(bigDecimal, decimalSchema, LogicalTypes.decimal(10, 6)); rec.put("height", genericFixed); @@ -574,7 +582,7 @@ private void generateMapTypeValues(GenericRecord rec) { * Populate rec with values for FARE_NESTED_SCHEMA */ private void generateFareNestedValues(GenericRecord rec) { - GenericRecord fareRecord = new GenericData.Record(extendedSchema.orElse(AVRO_SCHEMA).getField("fare").schema()); + GenericRecord fareRecord = new GenericData.Record(getEffectiveAvroSchema().getField("fare").schema()); fareRecord.put("amount", rand.nextDouble() * 100); fareRecord.put("currency", "USD"); if (extendedSchema.isPresent()) { @@ -603,14 +611,13 @@ private void generateEventLSNValue(GenericRecord rec) { * Populate rec with values for TIP_NESTED_SCHEMA */ private void generateTipNestedValues(GenericRecord rec) { + Schema schemaToUse = getEffectiveAvroSchema(); // TODO [HUDI-9603] remove this check - if (extendedSchema.isPresent()) { - if (extendedSchema.get().getField("tip_history") == null) { - return; - } + if (schemaToUse.getField("tip_history") == null) { + return; } - GenericArray tipHistoryArray = new GenericData.Array<>(1, AVRO_SCHEMA.getField("tip_history").schema()); - Schema tipSchema = new Schema.Parser().parse(AVRO_SCHEMA.getField("tip_history").schema().toString()).getElementType(); + GenericArray tipHistoryArray = new GenericData.Array<>(1, schemaToUse.getField("tip_history").schema()); + Schema tipSchema = new Schema.Parser().parse(schemaToUse.getField("tip_history").schema().toString()).getElementType(); GenericRecord tipRecord = new GenericData.Record(tipSchema); tipRecord.put("amount", rand.nextDouble() * 100); tipRecord.put("currency", "USD"); @@ -635,7 +642,8 @@ private void generateTripSuffixValues(GenericRecord rec, boolean isDeleteRecord) public GenericRecord generateGenericRecord(String rowKey, String partitionPath, String riderName, String driverName, long timestamp, boolean isDeleteRecord, boolean isFlattened) { - GenericRecord rec = new GenericData.Record(extendedSchema.orElseGet(() -> isFlattened ? FLATTENED_AVRO_SCHEMA : AVRO_SCHEMA)); + Schema schemaToUse = extendedSchema.isPresent() ? getEffectiveAvroSchema() : (isFlattened ? FLATTENED_AVRO_SCHEMA : AVRO_SCHEMA); + GenericRecord rec = new GenericData.Record(schemaToUse); generateTripPrefixValues(rec, rowKey, partitionPath, riderName, driverName, timestamp); if (isFlattened) { generateFareFlattenedValues(rec); @@ -1570,7 +1578,7 @@ public String toString() { } public static class SchemaEvolutionConfigs { - public Schema schema = AVRO_SCHEMA; + public HoodieSchema schema = HOODIE_SCHEMA; public boolean nestedSupport = true; public boolean mapSupport = true; public boolean arraySupport = true; @@ -1604,30 +1612,30 @@ public static class SchemaEvolutionConfigs { } private enum SchemaEvolutionTypePromotionCase { - INT_TO_INT(Schema.Type.INT, Schema.Type.INT, config -> true), - INT_TO_LONG(Schema.Type.INT, Schema.Type.LONG, config -> config.intToLongSupport), - INT_TO_FLOAT(Schema.Type.INT, Schema.Type.FLOAT, config -> config.intToFloatSupport), - INT_TO_DOUBLE(Schema.Type.INT, Schema.Type.DOUBLE, config -> config.intToDoubleSupport), - INT_TO_STRING(Schema.Type.INT, Schema.Type.STRING, config -> config.intToStringSupport), - LONG_TO_LONG(Schema.Type.LONG, Schema.Type.LONG, config -> true), - LONG_TO_FLOAT(Schema.Type.LONG, Schema.Type.FLOAT, config -> config.longToFloatSupport), - LONG_TO_DOUBLE(Schema.Type.LONG, Schema.Type.DOUBLE, config -> config.longToDoubleSupport), - LONG_TO_STRING(Schema.Type.LONG, Schema.Type.STRING, config -> config.longToStringSupport), - FLOAT_TO_FLOAT(Schema.Type.FLOAT, Schema.Type.FLOAT, config -> true), - FLOAT_TO_DOUBLE(Schema.Type.FLOAT, Schema.Type.DOUBLE, config -> config.floatToDoubleSupport), - FLOAT_TO_STRING(Schema.Type.FLOAT, Schema.Type.STRING, config -> config.floatToStringSupport), - DOUBLE_TO_DOUBLE(Schema.Type.DOUBLE, Schema.Type.DOUBLE, config -> true), - DOUBLE_TO_STRING(Schema.Type.DOUBLE, Schema.Type.STRING, config -> config.doubleToStringSupport), - STRING_TO_STRING(Schema.Type.STRING, Schema.Type.STRING, config -> true), - STRING_TO_BYTES(Schema.Type.STRING, Schema.Type.BYTES, config -> config.stringToBytesSupport), - BYTES_TO_BYTES(Schema.Type.BYTES, Schema.Type.BYTES, config -> true), - BYTES_TO_STRING(Schema.Type.BYTES, Schema.Type.STRING, config -> config.bytesToStringSupport); - - public final Schema.Type before; - public final Schema.Type after; + INT_TO_INT(HoodieSchemaType.INT, HoodieSchemaType.INT, config -> true), + INT_TO_LONG(HoodieSchemaType.INT, HoodieSchemaType.LONG, config -> config.intToLongSupport), + INT_TO_FLOAT(HoodieSchemaType.INT, HoodieSchemaType.FLOAT, config -> config.intToFloatSupport), + INT_TO_DOUBLE(HoodieSchemaType.INT, HoodieSchemaType.DOUBLE, config -> config.intToDoubleSupport), + INT_TO_STRING(HoodieSchemaType.INT, HoodieSchemaType.STRING, config -> config.intToStringSupport), + LONG_TO_LONG(HoodieSchemaType.LONG, HoodieSchemaType.LONG, config -> true), + LONG_TO_FLOAT(HoodieSchemaType.LONG, HoodieSchemaType.FLOAT, config -> config.longToFloatSupport), + LONG_TO_DOUBLE(HoodieSchemaType.LONG, HoodieSchemaType.DOUBLE, config -> config.longToDoubleSupport), + LONG_TO_STRING(HoodieSchemaType.LONG, HoodieSchemaType.STRING, config -> config.longToStringSupport), + FLOAT_TO_FLOAT(HoodieSchemaType.FLOAT, HoodieSchemaType.FLOAT, config -> true), + FLOAT_TO_DOUBLE(HoodieSchemaType.FLOAT, HoodieSchemaType.DOUBLE, config -> config.floatToDoubleSupport), + FLOAT_TO_STRING(HoodieSchemaType.FLOAT, HoodieSchemaType.STRING, config -> config.floatToStringSupport), + DOUBLE_TO_DOUBLE(HoodieSchemaType.DOUBLE, HoodieSchemaType.DOUBLE, config -> true), + DOUBLE_TO_STRING(HoodieSchemaType.DOUBLE, HoodieSchemaType.STRING, config -> config.doubleToStringSupport), + STRING_TO_STRING(HoodieSchemaType.STRING, HoodieSchemaType.STRING, config -> true), + STRING_TO_BYTES(HoodieSchemaType.STRING, HoodieSchemaType.BYTES, config -> config.stringToBytesSupport), + BYTES_TO_BYTES(HoodieSchemaType.BYTES, HoodieSchemaType.BYTES, config -> true), + BYTES_TO_STRING(HoodieSchemaType.BYTES, HoodieSchemaType.STRING, config -> config.bytesToStringSupport); + + public final HoodieSchemaType before; + public final HoodieSchemaType after; public final Predicate isEnabled; - SchemaEvolutionTypePromotionCase(Schema.Type before, Schema.Type after, Predicate isEnabled) { + SchemaEvolutionTypePromotionCase(HoodieSchemaType before, HoodieSchemaType after, Predicate isEnabled) { this.before = before; this.after = after; this.isEnabled = isEnabled; @@ -1635,7 +1643,7 @@ private enum SchemaEvolutionTypePromotionCase { } public void extendSchema(SchemaEvolutionConfigs configs, boolean isBefore) { - List baseFields = new ArrayList<>(); + List baseFields = new ArrayList<>(); for (SchemaEvolutionTypePromotionCase evolution : SchemaEvolutionTypePromotionCase.values()) { if (evolution.isEnabled.test(configs)) { baseFields.add(isBefore ? evolution.before : evolution.after); @@ -1644,7 +1652,7 @@ public void extendSchema(SchemaEvolutionConfigs configs, boolean isBefore) { // Add new field if we are testing adding new fields if (!isBefore && configs.addNewFieldSupport) { - baseFields.add(Schema.Type.BOOLEAN); + baseFields.add(HoodieSchemaType.BOOLEAN); } this.extendedSchema = Option.of(generateExtendedSchema(configs, new ArrayList<>(baseFields))); @@ -1658,69 +1666,72 @@ public void extendSchemaAfterEvolution(SchemaEvolutionConfigs configs) { extendSchema(configs, false); } - public Schema getExtendedSchema() { + public HoodieSchema getExtendedSchema() { return extendedSchema.orElseThrow(IllegalArgumentException::new); } - private static Schema generateExtendedSchema(SchemaEvolutionConfigs configs, List baseFields) { + private static HoodieSchema generateExtendedSchema(SchemaEvolutionConfigs configs, List baseFields) { return generateExtendedSchema(configs.schema, configs, baseFields, "customField", true); } - private static Schema generateExtendedSchema(Schema baseSchema, SchemaEvolutionConfigs configs, List baseFields, String fieldPrefix, boolean toplevel) { - List fields = baseSchema.getFields(); - List finalFields = new ArrayList<>(fields.size() + baseFields.size()); + private static HoodieSchema generateExtendedSchema(HoodieSchema baseSchema, SchemaEvolutionConfigs configs, List baseFields, String fieldPrefix, boolean toplevel) { + List fields = baseSchema.getFields(); + List finalFields = new ArrayList<>(fields.size() + baseFields.size()); boolean addedFields = false; - for (Schema.Field field : fields) { - if (configs.nestedSupport && field.name().equals("fare") && field.schema().getType() == Schema.Type.RECORD) { - finalFields.add(createNewSchemaField(field.name(), generateExtendedSchema(field.schema(), configs, baseFields, "customFare", false), field.doc(), field.defaultVal())); + for (HoodieSchemaField field : fields) { + if (configs.nestedSupport && field.name().equals("fare") && field.schema().getType() == HoodieSchemaType.RECORD) { + finalFields.add(HoodieSchemaUtils.createNewSchemaField(field.name(), + generateExtendedSchema(field.schema(), configs, baseFields, "customFare", false), field.doc().orElse(null), field.defaultVal().orElse(null))); } else if (configs.anyArraySupport || !field.name().equals("tip_history")) { //TODO: [HUDI-9603] remove the if condition when the issue is fixed if (field.name().equals("_hoodie_is_deleted")) { addedFields = true; - addFields(configs, finalFields, baseFields, fieldPrefix, baseSchema.getNamespace(), toplevel); + addFields(configs, finalFields, baseFields, fieldPrefix, baseSchema.getNamespace().orElse(null), toplevel); } - finalFields.add(createNewSchemaField(field)); + finalFields.add(HoodieSchemaUtils.createNewSchemaField(field)); } } if (!addedFields) { - addFields(configs, finalFields, baseFields, fieldPrefix, baseSchema.getNamespace(), toplevel); + addFields(configs, finalFields, baseFields, fieldPrefix, baseSchema.getNamespace().orElse(null), toplevel); } - Schema finalSchema = Schema.createRecord(baseSchema.getName(), baseSchema.getDoc(), - baseSchema.getNamespace(), baseSchema.isError()); - finalSchema.setFields(finalFields); + HoodieSchema finalSchema = HoodieSchema.createRecord(baseSchema.getName(), baseSchema.getDoc().orElse(null), + baseSchema.getNamespace().orElse(null), baseSchema.isError(), finalFields); return finalSchema; } - private static void addFields(SchemaEvolutionConfigs configs, List finalFields, List baseFields, String fieldPrefix, String namespace, boolean toplevel) { + private static void addFields(SchemaEvolutionConfigs configs, List finalFields, List baseFields, String fieldPrefix, String namespace, boolean toplevel) { if (toplevel) { if (configs.mapSupport) { - List mapFields = new ArrayList<>(baseFields.size()); + List mapFields = new ArrayList<>(baseFields.size()); addFieldsHelper(mapFields, baseFields, fieldPrefix + "Map"); - finalFields.add(new Schema.Field(fieldPrefix + "Map", Schema.createMap(Schema.createRecord("customMapRecord", "", namespace, false, mapFields)), "", null)); + finalFields.add(HoodieSchemaField.of(fieldPrefix + "Map", + HoodieSchema.createMap(HoodieSchema.createRecord("customMapRecord", "", namespace, false, mapFields)), "", null)); } if (configs.arraySupport) { - List arrayFields = new ArrayList<>(baseFields.size()); + List arrayFields = new ArrayList<>(baseFields.size()); addFieldsHelper(arrayFields, baseFields, fieldPrefix + "Array"); - finalFields.add(new Schema.Field(fieldPrefix + "Array", Schema.createArray(Schema.createRecord("customArrayRecord", "", namespace, false, arrayFields)), "", null)); + finalFields.add(HoodieSchemaField.of(fieldPrefix + "Array", + HoodieSchema.createArray(HoodieSchema.createRecord("customArrayRecord", "", namespace, false, arrayFields)), "", null)); } } addFieldsHelper(finalFields, baseFields, fieldPrefix); } - private static void addFieldsHelper(List finalFields, List baseFields, String fieldPrefix) { + private static void addFieldsHelper(List finalFields, List baseFields, String fieldPrefix) { for (int i = 0; i < baseFields.size(); i++) { - if (baseFields.get(i) == Schema.Type.BOOLEAN) { + if (baseFields.get(i) == HoodieSchemaType.BOOLEAN) { // boolean fields are added fields - finalFields.add(new Schema.Field(fieldPrefix + i, AvroSchemaUtils.createNullableSchema(Schema.Type.BOOLEAN), "", null)); + finalFields.add(HoodieSchemaField.of(fieldPrefix + i, HoodieSchema.createNullable(HoodieSchemaType.BOOLEAN), "", null)); } else { - finalFields.add(new Schema.Field(fieldPrefix + i, Schema.create(baseFields.get(i)), "", null)); + finalFields.add(HoodieSchemaField.of(fieldPrefix + i, HoodieSchema.create(baseFields.get(i)), "", null)); } } } private void generateCustomValues(GenericRecord rec, String customPrefix) { - for (Schema.Field field : rec.getSchema().getFields()) { + HoodieSchema recordSchema = HoodieSchema.fromAvroSchema(rec.getSchema()); + for (HoodieSchemaField field : recordSchema.getFields()) { if (field.name().startsWith(customPrefix)) { switch (field.schema().getType()) { case INT: @@ -1742,7 +1753,7 @@ private void generateCustomValues(GenericRecord rec, String customPrefix) { rec.put(field.name(), ByteBuffer.wrap(getUTF8Bytes(genPseudoRandomUUID(rand).toString()))); break; case UNION: - if (!AvroSchemaUtils.getNonNullTypeFromUnion(field.schema()).getType().equals(Schema.Type.BOOLEAN)) { + if (field.schema().getNonNullType().getType() != HoodieSchemaType.BOOLEAN) { throw new IllegalStateException("Union should only be boolean"); } rec.put(field.name(), rand.nextBoolean()); @@ -1763,18 +1774,18 @@ private void generateCustomValues(GenericRecord rec, String customPrefix) { } } - private GenericArray genArray(Schema arraySchema, String customPrefix) { - GenericArray customArray = new GenericData.Array<>(1, arraySchema); - Schema arrayElementSchema = arraySchema.getElementType(); - GenericRecord customRecord = new GenericData.Record(arrayElementSchema); + private GenericArray genArray(HoodieSchema arraySchema, String customPrefix) { + GenericArray customArray = new GenericData.Array<>(1, arraySchema.toAvroSchema()); + HoodieSchema arrayElementSchema = arraySchema.getElementType(); + GenericRecord customRecord = new GenericData.Record(arrayElementSchema.toAvroSchema()); generateCustomValues(customRecord, customPrefix); customArray.add(customRecord); return customArray; } - private Map genMap(Schema mapSchema, String customPrefix) { - Schema mapElementSchema = mapSchema.getValueType(); - GenericRecord customRecord = new GenericData.Record(mapElementSchema); + private Map genMap(HoodieSchema mapSchema, String customPrefix) { + HoodieSchema mapElementSchema = mapSchema.getValueType(); + GenericRecord customRecord = new GenericData.Record(mapElementSchema.toAvroSchema()); generateCustomValues(customRecord, customPrefix); return Collections.singletonMap("customMapKey", customRecord); } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FlinkRowDataReaderContext.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FlinkRowDataReaderContext.java index eda1e3c4864c3..8310dbe4f52a8 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FlinkRowDataReaderContext.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FlinkRowDataReaderContext.java @@ -18,7 +18,6 @@ package org.apache.hudi.table.format; -import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.client.model.BootstrapRowData; import org.apache.hudi.client.model.CommitTimeFlinkRecordMerger; import org.apache.hudi.client.model.EventTimeFlinkRecordMerger; @@ -50,7 +49,6 @@ import org.apache.hudi.util.RecordKeyToRowDataConverter; import org.apache.hudi.util.RowDataAvroQueryContexts; -import org.apache.avro.Schema; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.utils.JoinedRowData; import org.apache.flink.table.types.DataType; @@ -109,7 +107,7 @@ public ClosableIterator getFileRecordIterator( .getReaderFactory(HoodieRecord.HoodieRecordType.FLINK) .getFileReader(tableConfig, filePath, HoodieFileFormat.PARQUET, Option.empty()); DataType rowType = RowDataAvroQueryContexts.fromAvroSchema(dataSchema.toAvroSchema()).getRowType(); - return rowDataParquetReader.getRowDataIterator(schemaManager, rowType, requiredSchema, getSafePredicates(requiredSchema.toAvroSchema())); + return rowDataParquetReader.getRowDataIterator(schemaManager, rowType, requiredSchema, getSafePredicates(requiredSchema)); } @Override @@ -209,8 +207,8 @@ public void setSchemaHandler(FileGroupReaderSchemaHandler schemaHandler ((FlinkRecordContext) recordContext).setRecordKeyRowConverter(recordKeyRowConverter); } - private List getSafePredicates(Schema requiredSchema) { - boolean hasRowIndexField = AvroSchemaUtils.containsFieldInSchema(requiredSchema, ROW_INDEX_TEMPORARY_COLUMN_NAME); + private List getSafePredicates(HoodieSchema requiredSchema) { + boolean hasRowIndexField = requiredSchema.getField(ROW_INDEX_TEMPORARY_COLUMN_NAME).isPresent(); if (!getHasLogFiles() && !getNeedsBootstrapMerge()) { return allPredicates; } else if (!getHasLogFiles() && hasRowIndexField) { diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestTableSchemaResolver.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestTableSchemaResolver.java index 2b824214e1166..fea3853c171d1 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestTableSchemaResolver.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestTableSchemaResolver.java @@ -18,13 +18,14 @@ package org.apache.hudi.common.table; -import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaField; +import org.apache.hudi.common.schema.HoodieSchemaType; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.block.HoodieDataBlock; import org.apache.hudi.common.table.log.block.HoodieLogBlock; @@ -83,25 +84,25 @@ class TestTableSchemaResolver { @Test void testRecreateSchemaWhenDropPartitionColumns() { - Schema originSchema = new Schema.Parser().parse(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA); + HoodieSchema originSchema = HoodieSchema.parse(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA); // case2 String[] pts1 = new String[0]; - Schema s2 = TableSchemaResolver.appendPartitionColumns(originSchema, Option.of(pts1)); + HoodieSchema s2 = TableSchemaResolver.appendPartitionColumns(originSchema, Option.of(pts1)); assertEquals(originSchema, s2); // case3: partition_path is in originSchema String[] pts2 = {"partition_path"}; - Schema s3 = TableSchemaResolver.appendPartitionColumns(originSchema, Option.of(pts2)); + HoodieSchema s3 = TableSchemaResolver.appendPartitionColumns(originSchema, Option.of(pts2)); assertEquals(originSchema, s3); // case4: user_partition is not in originSchema String[] pts3 = {"user_partition"}; - Schema s4 = TableSchemaResolver.appendPartitionColumns(originSchema, Option.of(pts3)); + HoodieSchema s4 = TableSchemaResolver.appendPartitionColumns(originSchema, Option.of(pts3)); assertNotEquals(originSchema, s4); assertTrue(s4.getFields().stream().anyMatch(f -> f.name().equals("user_partition"))); - Schema.Field f = s4.getField("user_partition"); - assertEquals(f.schema(), AvroSchemaUtils.createNullableSchema(Schema.Type.STRING)); + HoodieSchemaField f = s4.getField("user_partition").get(); + assertEquals(f.schema(), HoodieSchema.createNullable(HoodieSchemaType.STRING)); // case5: user_partition is in originSchema, but partition_path is in originSchema String[] pts4 = {"user_partition", "partition_path"}; diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java index dc7681e7fb12f..ac1e91944de9b 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java @@ -26,6 +26,9 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaField; +import org.apache.hudi.common.schema.HoodieSchemaType; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.collection.ClosableIterator; @@ -63,7 +66,6 @@ import java.util.UUID; import java.util.stream.Collectors; -import static org.apache.hudi.avro.AvroSchemaUtils.createNullableSchema; import static org.apache.hudi.avro.HoodieAvroUtils.METADATA_FIELD_SCHEMA; import static org.apache.hudi.metadata.HoodieIndexVersion.V1; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -249,16 +251,16 @@ public void testReadColumnStatsFromMetadata() throws Exception { String recordKeyField = "id"; String partitionPathField = "partition"; String dataField = "data"; - Schema schema = getSchema(recordKeyField, partitionPathField, dataField); + HoodieSchema schema = getSchema(recordKeyField, partitionPathField, dataField); BloomFilter filter = BloomFilterFactory .createBloomFilter(1000, 0.0001, 10000, BloomFilterTypeCode.SIMPLE.name()); HoodieAvroWriteSupport writeSupport = - new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, Option.of(filter), new Properties()); + new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema.toAvroSchema()), schema.toAvroSchema(), Option.of(filter), new Properties()); try (ParquetWriter writer = new ParquetWriter(new Path(filePath), writeSupport, CompressionCodecName.GZIP, 120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE)) { valueList.forEach(entry -> { - GenericRecord rec = new GenericData.Record(schema); + GenericRecord rec = new GenericData.Record(schema.toAvroSchema()); rec.put(recordKeyField, entry.getLeft().getLeft()); rec.put(partitionPathField, partitionPath); if (entry.getRight()) { @@ -294,21 +296,20 @@ public void testReadColumnStatsFromMetadata() throws Exception { fileName, partitionPathField, partitionPath, partitionPath, 0, totalCount); } - private Schema getSchema(String recordKeyField, String partitionPathField, String dataField) { - List toBeAddedFields = new ArrayList<>(); - Schema recordSchema = Schema.createRecord("HoodieRecord", "", "", false); + private HoodieSchema getSchema(String recordKeyField, String partitionPathField, String dataField) { + List toBeAddedFields = new ArrayList<>(); - Schema.Field recordKeySchemaField = - new Schema.Field(recordKeyField, createNullableSchema(Schema.Type.STRING), "", JsonProperties.NULL_VALUE); - Schema.Field partitionPathSchemaField = - new Schema.Field(partitionPathField, createNullableSchema(Schema.Type.STRING), "", JsonProperties.NULL_VALUE); - Schema.Field dataSchemaField = - new Schema.Field(dataField, createNullableSchema(Schema.Type.STRING), "", JsonProperties.NULL_VALUE); + HoodieSchemaField recordKeySchemaField = + HoodieSchemaField.of(recordKeyField, HoodieSchema.createNullable(HoodieSchemaType.STRING), "", HoodieSchema.NULL_VALUE); + HoodieSchemaField partitionPathSchemaField = + HoodieSchemaField.of(partitionPathField, HoodieSchema.createNullable(HoodieSchemaType.STRING), "", HoodieSchema.NULL_VALUE); + HoodieSchemaField dataSchemaField = + HoodieSchemaField.of(dataField, HoodieSchema.createNullable(HoodieSchemaType.STRING), "", HoodieSchema.NULL_VALUE); toBeAddedFields.add(recordKeySchemaField); toBeAddedFields.add(partitionPathSchemaField); toBeAddedFields.add(dataSchemaField); - recordSchema.setFields(toBeAddedFields); + HoodieSchema recordSchema = HoodieSchema.createRecord("HoodieRecord", "", "", false, toBeAddedFields); return recordSchema; } @@ -478,24 +479,23 @@ public void testReadSchemaHash_MatchesDirectSchemaRead() throws Exception { private void writeParquetFileWithExtendedSchema(String filePath, List rowKeys) throws Exception { // Create an extended schema with an additional field - Schema extendedSchema = Schema.createRecord("record", "", "", false); - List fields = new ArrayList<>(); - fields.add(new Schema.Field("_row_key", Schema.create(Schema.Type.STRING), "", (Object) null)); - fields.add(new Schema.Field("time", Schema.create(Schema.Type.LONG), "", (Object) null)); - fields.add(new Schema.Field("number", Schema.create(Schema.Type.LONG), "", (Object) null)); - fields.add(new Schema.Field("extra_field", createNullableSchema(Schema.Type.STRING), "", JsonProperties.NULL_VALUE)); // Additional field - extendedSchema.setFields(fields); + List fields = new ArrayList<>(); + fields.add(HoodieSchemaField.of("_row_key", HoodieSchema.create(HoodieSchemaType.STRING), "", (Object) null)); + fields.add(HoodieSchemaField.of("time", HoodieSchema.create(HoodieSchemaType.LONG), "", (Object) null)); + fields.add(HoodieSchemaField.of("number", HoodieSchema.create(HoodieSchemaType.LONG), "", (Object) null)); + fields.add(HoodieSchemaField.of("extra_field", HoodieSchema.createNullable(HoodieSchemaType.STRING), "", HoodieSchema.NULL_VALUE)); // Additional field + HoodieSchema extendedSchema = HoodieSchema.createRecord("record", "", "", false, fields); BloomFilter filter = BloomFilterFactory.createBloomFilter(1000, 0.0001, -1, BloomFilterTypeCode.SIMPLE.name()); HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport( - new AvroSchemaConverter().convert(extendedSchema), extendedSchema, Option.of(filter), new Properties()); + new AvroSchemaConverter().convert(extendedSchema.toAvroSchema()), extendedSchema.toAvroSchema(), Option.of(filter), new Properties()); ParquetWriter writer = new ParquetWriter(new Path(filePath), writeSupport, CompressionCodecName.GZIP, 120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE); for (String rowKey : rowKeys) { - GenericRecord record = new GenericData.Record(extendedSchema); + GenericRecord record = new GenericData.Record(extendedSchema.toAvroSchema()); record.put("_row_key", rowKey); record.put("time", 1234567L); record.put("number", 12345L); diff --git a/hudi-hadoop-mr/pom.xml b/hudi-hadoop-mr/pom.xml index 28f5b161ae124..aeb162152322a 100644 --- a/hudi-hadoop-mr/pom.xml +++ b/hudi-hadoop-mr/pom.xml @@ -56,6 +56,13 @@ kryo-shaded + + + org.projectlombok + lombok + + + org.apache.avro diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HiveRecordContext.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HiveRecordContext.java index 5812770e36f4a..8f52b290354b0 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HiveRecordContext.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HiveRecordContext.java @@ -28,7 +28,7 @@ import org.apache.hudi.common.table.read.BufferedRecord; import org.apache.hudi.hadoop.utils.HiveAvroSerializer; import org.apache.hudi.hadoop.utils.HiveJavaTypeConverter; -import org.apache.hudi.hadoop.utils.HoodieArrayWritableAvroUtils; +import org.apache.hudi.hadoop.utils.HoodieArrayWritableSchemaUtils; import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils; import org.apache.avro.Schema; @@ -163,6 +163,6 @@ public ArrayWritable toBinaryRow(HoodieSchema schema, ArrayWritable record) { @Override public UnaryOperator projectRecord(HoodieSchema from, HoodieSchema to, Map renamedColumns) { - return record -> HoodieArrayWritableAvroUtils.rewriteRecordWithNewSchema(record, from.toAvroSchema(), to.toAvroSchema(), renamedColumns); + return record -> HoodieArrayWritableSchemaUtils.rewriteRecordWithNewSchema(record, from, to, renamedColumns); } } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieFileGroupReaderBasedRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieFileGroupReaderBasedRecordReader.java index bf60b21401619..3668b1ab8d435 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieFileGroupReaderBasedRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieFileGroupReaderBasedRecordReader.java @@ -35,7 +35,7 @@ import org.apache.hudi.common.util.VisibleForTesting; import org.apache.hudi.common.util.collection.ClosableIterator; import org.apache.hudi.hadoop.realtime.RealtimeSplit; -import org.apache.hudi.hadoop.utils.HoodieArrayWritableAvroUtils; +import org.apache.hudi.hadoop.utils.HoodieArrayWritableSchemaUtils; import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils; import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils; import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration; @@ -163,7 +163,7 @@ public HoodieFileGroupReaderBasedRecordReader(HiveReaderCreator readerCreator, HoodieSchema outputSchema = HoodieSchemaUtils.generateProjectionSchema(tableSchema, Stream.concat(tableSchema.getFields().stream().map(f -> f.name().toLowerCase(Locale.ROOT)).filter(n -> !partitionColumns.contains(n)), partitionColumns.stream()).collect(Collectors.toList())); - this.reverseProjection = HoodieArrayWritableAvroUtils.getReverseProjection(requestedSchema, outputSchema); + this.reverseProjection = HoodieArrayWritableSchemaUtils.getReverseProjection(requestedSchema, outputSchema); } @VisibleForTesting @@ -249,9 +249,9 @@ public JobConf getJobConf() { private static HoodieSchema getLatestTableSchema(HoodieTableMetaClient metaClient, JobConf jobConf, String latestCommitTime) { TableSchemaResolver tableSchemaResolver = new TableSchemaResolver(metaClient); try { - Schema schema = tableSchemaResolver.getTableAvroSchema(latestCommitTime); + HoodieSchema schema = tableSchemaResolver.getTableSchema(latestCommitTime); // Add partitioning fields to writer schema for resulting row to contain null values for these fields - return HoodieSchema.fromAvroSchema(HoodieRealtimeRecordReaderUtils.addPartitionFields(schema, getPartitionFieldNames(jobConf))); + return HoodieRealtimeRecordReaderUtils.addPartitionFields(schema, getPartitionFieldNames(jobConf)); } catch (Exception e) { throw new RuntimeException("Unable to get table schema", e); } @@ -334,6 +334,9 @@ public static HoodieSchema createRequestedSchema(HoodieSchema tableSchema, JobCo // The READ_COLUMN_NAMES_CONF_STR includes all columns from the query, including those used in the WHERE clause, // so any column referenced in the filter (non-partition) will appear twice if already present in the project schema, // here distinct() is used here to deduplicate the read columns. - Arrays.stream(jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR).split(",")).filter(c -> !partitionColumns.contains(c)).distinct().collect(Collectors.toList())); + Arrays.stream(jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR).split(",")) + .filter(c -> !partitionColumns.contains(c)) + .distinct() + .collect(Collectors.toList())); } } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/SchemaEvolutionContext.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/SchemaEvolutionContext.java index 4796b8ed29f2f..a30f690b0f5e6 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/SchemaEvolutionContext.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/SchemaEvolutionContext.java @@ -19,6 +19,8 @@ package org.apache.hudi.hadoop; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaField; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.util.InternalSchemaCache; @@ -206,12 +208,12 @@ public void doEvolutionForRealtimeInputFormat(AbstractRealtimeRecordReader realt String partitionFields = job.get(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS, ""); List partitioningFields = !partitionFields.isEmpty() ? Arrays.stream(partitionFields.split("/")).collect(Collectors.toList()) : new ArrayList<>(); - Schema writerSchema = InternalSchemaConverter.convert(internalSchemaOption.get(), tableAvroSchema.getName()).toAvroSchema(); + HoodieSchema writerSchema = InternalSchemaConverter.convert(internalSchemaOption.get(), tableAvroSchema.getName()); writerSchema = HoodieRealtimeRecordReaderUtils.addPartitionFields(writerSchema, partitioningFields); - Map schemaFieldsMap = HoodieRealtimeRecordReaderUtils.getNameToFieldMap(writerSchema); + Map schemaFieldsMap = HoodieRealtimeRecordReaderUtils.getNameToFieldMap(writerSchema); // we should get HoodieParquetInputFormat#HIVE_TMP_COLUMNS,since serdeConstants#LIST_COLUMNS maybe change by HoodieParquetInputFormat#setColumnNameList - Schema hiveSchema = realtimeRecordReader.constructHiveOrderedSchema(writerSchema, schemaFieldsMap, job.get(HIVE_TMP_COLUMNS)); - Schema readerSchema = InternalSchemaConverter.convert(prunedInternalSchema, tableAvroSchema.getName()).toAvroSchema(); + HoodieSchema hiveSchema = realtimeRecordReader.constructHiveOrderedSchema(writerSchema, schemaFieldsMap, job.get(HIVE_TMP_COLUMNS)); + HoodieSchema readerSchema = InternalSchemaConverter.convert(prunedInternalSchema, tableAvroSchema.getName()); // setUp evolution schema realtimeRecordReader.setWriterSchema(writerSchema); realtimeRecordReader.setReaderSchema(readerSchema); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/AbstractRealtimeRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/AbstractRealtimeRecordReader.java index 15550fd38bba3..913e670095273 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/AbstractRealtimeRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/AbstractRealtimeRecordReader.java @@ -22,6 +22,9 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodiePayloadProps; import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaField; +import org.apache.hudi.common.schema.HoodieSchemaUtils; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; @@ -33,8 +36,8 @@ import org.apache.hudi.hadoop.utils.HiveAvroSerializer; import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils; -import org.apache.avro.Schema; -import org.apache.avro.Schema.Field; +import lombok.Getter; +import lombok.Setter; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.ql.io.parquet.serde.ArrayWritableObjectInspector; import org.apache.hadoop.hive.serde.serdeConstants; @@ -55,7 +58,6 @@ import java.util.Set; import java.util.stream.Collectors; -import static org.apache.hudi.avro.HoodieAvroUtils.createNewSchemaField; import static org.apache.hudi.common.util.StringUtils.EMPTY_STRING; /** @@ -64,20 +66,29 @@ public abstract class AbstractRealtimeRecordReader { private static final Logger LOG = LoggerFactory.getLogger(AbstractRealtimeRecordReader.class); + @Getter protected final RealtimeSplit split; + @Getter protected final JobConf jobConf; protected final boolean usesCustomPayload; protected TypedProperties payloadProps = new TypedProperties(); // Schema handles - private Schema readerSchema; - private Schema writerSchema; - private Schema hiveSchema; + @Getter + @Setter + private HoodieSchema readerSchema; + @Getter + @Setter + private HoodieSchema writerSchema; + @Getter + @Setter + private HoodieSchema hiveSchema; private final HoodieTableMetaClient metaClient; protected SchemaEvolutionContext schemaEvolutionContext; // support merge operation protected boolean supportPayload; // handle hive type to avro record protected HiveAvroSerializer serializer; + @Getter private final boolean supportTimestamp; public AbstractRealtimeRecordReader(RealtimeSplit split, JobConf job) { @@ -156,18 +167,18 @@ private void prepareHiveAvroSerializer() { */ private void init() throws Exception { LOG.info("Getting writer schema from table avro schema "); - writerSchema = new TableSchemaResolver(metaClient).getTableAvroSchema(); + writerSchema = new TableSchemaResolver(metaClient).getTableSchema(); // Add partitioning fields to writer schema for resulting row to contain null values for these fields String partitionFields = jobConf.get(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS, ""); List partitioningFields = - partitionFields.length() > 0 ? Arrays.stream(partitionFields.split("/")).collect(Collectors.toList()) + !partitionFields.isEmpty() ? Arrays.stream(partitionFields.split("/")).collect(Collectors.toList()) : new ArrayList<>(); writerSchema = HoodieRealtimeRecordReaderUtils.addPartitionFields(writerSchema, partitioningFields); List projectionFields = HoodieRealtimeRecordReaderUtils.orderFields(jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, EMPTY_STRING), jobConf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, EMPTY_STRING), partitioningFields); - Map schemaFieldsMap = HoodieRealtimeRecordReaderUtils.getNameToFieldMap(writerSchema); + Map schemaFieldsMap = HoodieRealtimeRecordReaderUtils.getNameToFieldMap(writerSchema); hiveSchema = constructHiveOrderedSchema(writerSchema, schemaFieldsMap, jobConf.get(hive_metastoreConstants.META_TABLE_COLUMNS, EMPTY_STRING)); // TODO(vc): In the future, the reader schema should be updated based on log files & be able // to null out fields not present before @@ -177,16 +188,16 @@ private void init() throws Exception { split.getDeltaLogPaths(), split.getPath(), projectionFields)); } - public Schema constructHiveOrderedSchema(Schema writerSchema, Map schemaFieldsMap, String hiveColumnString) { + public HoodieSchema constructHiveOrderedSchema(HoodieSchema writerSchema, Map schemaFieldsMap, String hiveColumnString) { String[] hiveColumns = hiveColumnString.isEmpty() ? new String[0] : hiveColumnString.split(","); LOG.info("Hive Columns : " + hiveColumnString); - List hiveSchemaFields = new ArrayList<>(); + List hiveSchemaFields = new ArrayList<>(); for (String columnName : hiveColumns) { - Field field = schemaFieldsMap.get(columnName.toLowerCase()); + HoodieSchemaField field = schemaFieldsMap.get(columnName.toLowerCase()); if (field != null) { - hiveSchemaFields.add(createNewSchemaField(field)); + hiveSchemaFields.add(HoodieSchemaUtils.createNewSchemaField(field)); } else { // Hive has some extra virtual columns like BLOCK__OFFSET__INSIDE__FILE which do not exist in table schema. // They will get skipped as they won't be found in the original schema. @@ -194,50 +205,13 @@ public Schema constructHiveOrderedSchema(Schema writerSchema, Map } } - Schema hiveSchema = Schema.createRecord(writerSchema.getName(), writerSchema.getDoc(), writerSchema.getNamespace(), - writerSchema.isError()); - hiveSchema.setFields(hiveSchemaFields); + HoodieSchema hiveSchema = HoodieSchema.createRecord(writerSchema.getName(), writerSchema.getDoc().orElse(null), + writerSchema.getNamespace().orElse(null), writerSchema.isError(), hiveSchemaFields); LOG.debug("HIVE Schema is :{}", hiveSchema); return hiveSchema; } - protected Schema getLogScannerReaderSchema() { + protected HoodieSchema getLogScannerReaderSchema() { return usesCustomPayload ? writerSchema : readerSchema; } - - public Schema getReaderSchema() { - return readerSchema; - } - - public Schema getWriterSchema() { - return writerSchema; - } - - public Schema getHiveSchema() { - return hiveSchema; - } - - public boolean isSupportTimestamp() { - return supportTimestamp; - } - - public RealtimeSplit getSplit() { - return split; - } - - public JobConf getJobConf() { - return jobConf; - } - - public void setReaderSchema(Schema readerSchema) { - this.readerSchema = readerSchema; - } - - public void setWriterSchema(Schema writerSchema) { - this.writerSchema = writerSchema; - } - - public void setHiveSchema(Schema hiveSchema) { - this.hiveSchema = hiveSchema; - } } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java index 7449f4d7326d3..fe0c91f6f3a15 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java @@ -86,7 +86,7 @@ public RealtimeCompactedRecordReader(RealtimeSplit split, JobConf job, .map(HoodieVirtualKeyInfo::getRecordKeyFieldIndex) .orElse(HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS); this.orderingFields = ConfigUtils.getOrderingFields(payloadProps); - HoodieSchema logScannerReaderSchema = HoodieSchema.fromAvroSchema(getLogScannerReaderSchema()); + HoodieSchema logScannerReaderSchema = getLogScannerReaderSchema(); this.deleteContext = new DeleteContext(payloadProps, logScannerReaderSchema).withReaderSchema(logScannerReaderSchema); } @@ -103,7 +103,7 @@ private HoodieMergedLogRecordScanner getMergedLogRecordScanner() throws IOExcept split.getPath().toString(), HadoopFSUtils.getStorageConf(jobConf))) .withBasePath(split.getBasePath()) .withLogFilePaths(split.getDeltaLogPaths()) - .withReaderSchema(HoodieSchema.fromAvroSchema(getLogScannerReaderSchema())) + .withReaderSchema(getLogScannerReaderSchema()) .withLatestInstantTime(split.getMaxCommitTime()) .withMaxMemorySizeInBytes(HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes(jobConf)) .withReverseReader(false) @@ -122,9 +122,9 @@ private HoodieMergedLogRecordScanner getMergedLogRecordScanner() throws IOExcept private Option buildGenericRecordwithCustomPayload(HoodieRecord record) throws IOException { if (usesCustomPayload) { - return record.toIndexedRecord(getWriterSchema(), payloadProps); + return record.toIndexedRecord(getWriterSchema().toAvroSchema(), payloadProps); } else { - return record.toIndexedRecord(getReaderSchema(), payloadProps); + return record.toIndexedRecord(getReaderSchema().toAvroSchema(), payloadProps); } } @@ -169,12 +169,12 @@ private void setUpWritable(Option rec, ArrayWritable ar if (usesCustomPayload) { // If using a custom payload, return only the projection fields. The readerSchema is a schema derived from // the writerSchema with only the projection fields - recordToReturn = HoodieAvroUtils.rewriteRecord((GenericRecord) rec.get().getData(), getReaderSchema()); + recordToReturn = HoodieAvroUtils.rewriteRecord((GenericRecord) rec.get().getData(), getReaderSchema().toAvroSchema()); } // we assume, a later safe record in the log, is newer than what we have in the map & // replace it. Since we want to return an arrayWritable which is the same length as the elements in the latest // schema, we use writerSchema to create the arrayWritable from the latest generic record - ArrayWritable aWritable = (ArrayWritable) HoodieRealtimeRecordReaderUtils.avroToArrayWritable(recordToReturn, getHiveSchema(), isSupportTimestamp()); + ArrayWritable aWritable = (ArrayWritable) HoodieRealtimeRecordReaderUtils.avroToArrayWritable(recordToReturn, getHiveSchema().toAvroSchema(), isSupportTimestamp()); Writable[] replaceValue = aWritable.get(); if (LOG.isDebugEnabled()) { LOG.debug(String.format("key %s, base values: %s, log values: %s", key, HoodieRealtimeRecordReaderUtils.arrayWritableToString(arrayWritable), @@ -205,10 +205,10 @@ private Option mergeRecord(HoodieRecord newRecord, A // for presto engine, the hiveSchema will be: col1,col2, but the writerSchema will be col1,col2,par // so to be compatible with hive and presto, we should rewrite oldRecord before we call combineAndGetUpdateValue, // once presto on hudi have its own mor reader, we can remove the rewrite logical. - GenericRecord genericRecord = HiveAvroSerializer.rewriteRecordIgnoreResultCheck(oldRecord, getLogScannerReaderSchema()); + GenericRecord genericRecord = HiveAvroSerializer.rewriteRecordIgnoreResultCheck(oldRecord, getLogScannerReaderSchema().toAvroSchema()); RecordContext recordContext = AvroRecordContext.getFieldAccessorInstance(); BufferedRecord record = BufferedRecords.fromEngineRecord(genericRecord, HoodieSchema.fromAvroSchema(genericRecord.getSchema()), recordContext, orderingFields, newRecord.getRecordKey(), false); - BufferedRecord newBufferedRecord = BufferedRecords.fromHoodieRecord(newRecord, HoodieSchema.fromAvroSchema(getLogScannerReaderSchema()), + BufferedRecord newBufferedRecord = BufferedRecords.fromHoodieRecord(newRecord, HoodieSchema.fromAvroSchema(getLogScannerReaderSchema().toAvroSchema()), recordContext, payloadProps, orderingFields, deleteContext); BufferedRecord mergeResult = merger.merge(record, newBufferedRecord, recordContext, payloadProps); if (mergeResult.isDelete()) { @@ -218,7 +218,7 @@ private Option mergeRecord(HoodieRecord newRecord, A } private GenericRecord convertArrayWritableToHoodieRecord(ArrayWritable arrayWritable) { - GenericRecord record = serializer.serialize(arrayWritable, getHiveSchema()); + GenericRecord record = serializer.serialize(arrayWritable, getHiveSchema().toAvroSchema()); return record; } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeUnmergedRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeUnmergedRecordReader.java index 171eca08e99b2..46f8b677de227 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeUnmergedRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeUnmergedRecordReader.java @@ -19,7 +19,6 @@ package org.apache.hudi.hadoop.realtime; import org.apache.hudi.common.config.HoodieMemoryConfig; -import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.table.log.HoodieUnMergedLogRecordScanner; import org.apache.hudi.common.util.DefaultSizeEstimator; import org.apache.hudi.common.util.Functions; @@ -82,7 +81,7 @@ public RealtimeUnmergedRecordReader(RealtimeSplit split, JobConf job, split.getPath().toString(), HadoopFSUtils.getStorageConf(this.jobConf))) .withBasePath(split.getBasePath()) .withLogFilePaths(split.getDeltaLogPaths()) - .withReaderSchema(HoodieSchema.fromAvroSchema(getReaderSchema())) + .withReaderSchema(getReaderSchema()) .withLatestInstantTime(split.getMaxCommitTime()) .withReverseReader(false) .withBufferSize(this.jobConf.getInt(HoodieMemoryConfig.MAX_DFS_STREAM_BUFFER_SIZE.key(), @@ -109,8 +108,8 @@ private List> getParallelProducers( HoodieUnMergedLogRecordScanner scanner = scannerBuilder.withLogRecordScannerCallback(record -> { // convert Hoodie log record to Hadoop AvroWritable and buffer - GenericRecord rec = (GenericRecord) record.toIndexedRecord(getReaderSchema(), payloadProps).get().getData(); - ArrayWritable aWritable = (ArrayWritable) HoodieRealtimeRecordReaderUtils.avroToArrayWritable(rec, getHiveSchema(), isSupportTimestamp()); + GenericRecord rec = (GenericRecord) record.toIndexedRecord(getReaderSchema().toAvroSchema(), payloadProps).get().getData(); + ArrayWritable aWritable = (ArrayWritable) HoodieRealtimeRecordReaderUtils.avroToArrayWritable(rec, getHiveSchema().toAvroSchema(), isSupportTimestamp()); queue.insertRecord(aWritable); }) .build(); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HiveAvroSerializer.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HiveAvroSerializer.java index 5ce8fa08f827e..16690ac360b5d 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HiveAvroSerializer.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HiveAvroSerializer.java @@ -65,8 +65,6 @@ import java.util.Set; import java.util.stream.Collectors; -import static org.apache.hudi.avro.AvroSchemaUtils.getNonNullTypeFromUnion; -import static org.apache.hudi.avro.AvroSchemaUtils.resolveUnionSchema; import static org.apache.hudi.avro.HoodieAvroUtils.isMetadataField; /** @@ -194,7 +192,7 @@ private FieldContext extractFieldFromRecord(ArrayWritable record, StructObjectIn int fieldIdx = schemaField.pos(); TypeInfo fieldTypeInfo = fieldTypes.get(fieldIdx); - Schema fieldSchema = getNonNullTypeFromUnion(schemaField.schema()); + Schema fieldSchema = AvroSchemaUtils.getNonNullTypeFromUnion(schemaField.schema()); StructField structField = structObjectInspector.getStructFieldRef(fieldName); if (structField == null) { @@ -288,7 +286,7 @@ private Object serialize(TypeInfo typeInfo, ObjectInspector fieldOI, Object stru return null; } - schema = getNonNullTypeFromUnion(schema); + schema = AvroSchemaUtils.getNonNullTypeFromUnion(schema); /* Because we use Hive's 'string' type when Avro calls for enum, we have to expressly check for enum-ness */ if (Schema.Type.ENUM.equals(schema.getType())) { @@ -429,7 +427,7 @@ private Object serializeList(ListTypeInfo typeInfo, ListObjectInspector fieldOI, ObjectInspector listElementObjectInspector = fieldOI.getListElementObjectInspector(); // NOTE: We have to resolve nullable schema, since Avro permits array elements // to be null - Schema arrayNestedType = getNonNullTypeFromUnion(schema.getElementType()); + Schema arrayNestedType = AvroSchemaUtils.getNonNullTypeFromUnion(schema.getElementType()); Schema elementType; if (listElementObjectInspector.getCategory() == ObjectInspector.Category.PRIMITIVE) { elementType = arrayNestedType; @@ -495,7 +493,7 @@ private static void copyOldValueOrSetDefault(GenericRecord oldRecord, GenericRec Object newFieldValue; if (fieldValue instanceof GenericRecord) { GenericRecord record = (GenericRecord) fieldValue; - newFieldValue = rewriteRecordIgnoreResultCheck(record, resolveUnionSchema(field.schema(), record.getSchema().getFullName())); + newFieldValue = rewriteRecordIgnoreResultCheck(record, AvroSchemaUtils.resolveUnionSchema(field.schema(), record.getSchema().getFullName())); } else { newFieldValue = fieldValue; } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieArrayWritableAvroUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieArrayWritableSchemaUtils.java similarity index 72% rename from hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieArrayWritableAvroUtils.java rename to hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieArrayWritableSchemaUtils.java index e65ca612ca5ca..a177ccd2eaa20 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieArrayWritableAvroUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieArrayWritableSchemaUtils.java @@ -19,17 +19,17 @@ package org.apache.hudi.hadoop.utils; -import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaCompatibility; import org.apache.hudi.common.schema.HoodieSchemaField; +import org.apache.hudi.common.schema.HoodieSchemaType; +import org.apache.hudi.common.schema.HoodieSchemaUtils; +import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; -import org.apache.hudi.exception.HoodieAvroSchemaException; import org.apache.hudi.exception.SchemaCompatibilityException; +import org.apache.hudi.internal.schema.HoodieSchemaException; -import org.apache.avro.JsonProperties; -import org.apache.avro.LogicalTypes; -import org.apache.avro.Schema; import org.apache.hadoop.hive.common.type.HiveDecimal; import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; @@ -55,33 +55,31 @@ import java.util.Objects; import java.util.function.UnaryOperator; -import static org.apache.hudi.avro.AvroSchemaUtils.areSchemasProjectionEquivalent; -import static org.apache.hudi.avro.AvroSchemaUtils.isNullable; import static org.apache.hudi.avro.HoodieAvroUtils.createFullName; import static org.apache.hudi.avro.HoodieAvroUtils.createNamePrefix; import static org.apache.hudi.avro.HoodieAvroUtils.getOldFieldNameWithRenaming; import static org.apache.hudi.avro.HoodieAvroUtils.toJavaDate; import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes; -public class HoodieArrayWritableAvroUtils { +public class HoodieArrayWritableSchemaUtils { - public static ArrayWritable rewriteRecordWithNewSchema(ArrayWritable writable, Schema oldSchema, Schema newSchema, Map renameCols) { + public static ArrayWritable rewriteRecordWithNewSchema(ArrayWritable writable, HoodieSchema oldSchema, HoodieSchema newSchema, Map renameCols) { return (ArrayWritable) rewriteRecordWithNewSchema(writable, oldSchema, newSchema, renameCols, new LinkedList<>()); } - private static Writable rewriteRecordWithNewSchema(Writable writable, Schema oldAvroSchema, Schema newAvroSchema, Map renameCols, Deque fieldNames) { + private static Writable rewriteRecordWithNewSchema(Writable writable, HoodieSchema oldSchema, HoodieSchema newSchema, Map renameCols, Deque fieldNames) { if (writable == null) { return null; } - Schema oldSchema = AvroSchemaUtils.getNonNullTypeFromUnion(oldAvroSchema); - Schema newSchema = AvroSchemaUtils.getNonNullTypeFromUnion(newAvroSchema); - if (areSchemasProjectionEquivalent(oldSchema, newSchema)) { + HoodieSchema oldSchemaNonNull = HoodieSchemaUtils.getNonNullTypeFromUnion(oldSchema); + HoodieSchema newSchemaNonNull = HoodieSchemaUtils.getNonNullTypeFromUnion(newSchema); + if (HoodieSchemaCompatibility.areSchemasProjectionEquivalent(oldSchemaNonNull, newSchemaNonNull)) { return writable; } - return rewriteRecordWithNewSchemaInternal(writable, oldSchema, newSchema, renameCols, fieldNames); + return rewriteRecordWithNewSchemaInternal(writable, oldSchemaNonNull, newSchemaNonNull, renameCols, fieldNames); } - private static Writable rewriteRecordWithNewSchemaInternal(Writable writable, Schema oldSchema, Schema newSchema, Map renameCols, Deque fieldNames) { + private static Writable rewriteRecordWithNewSchemaInternal(Writable writable, HoodieSchema oldSchema, HoodieSchema newSchema, Map renameCols, Deque fieldNames) { switch (newSchema.getType()) { case RECORD: if (!(writable instanceof ArrayWritable)) { @@ -89,41 +87,42 @@ private static Writable rewriteRecordWithNewSchemaInternal(Writable writable, Sc } ArrayWritable arrayWritable = (ArrayWritable) writable; - List fields = newSchema.getFields(); + List fields = newSchema.getFields(); // projection will keep the size from the "from" schema because it gets recycled // and if the size changes the reader will fail boolean noFieldsRenaming = renameCols.isEmpty(); String namePrefix = createNamePrefix(noFieldsRenaming, fieldNames); Writable[] values = new Writable[Math.max(fields.size(), arrayWritable.get().length)]; for (int i = 0; i < fields.size(); i++) { - Schema.Field newField = newSchema.getFields().get(i); + HoodieSchemaField newField = newSchema.getFields().get(i); String newFieldName = newField.name(); fieldNames.push(newFieldName); - Schema.Field oldField = noFieldsRenaming + Option oldFieldOpt = noFieldsRenaming ? oldSchema.getField(newFieldName) : oldSchema.getField(getOldFieldNameWithRenaming(namePrefix, newFieldName, renameCols)); - if (oldField != null) { + if (oldFieldOpt.isPresent()) { + HoodieSchemaField oldField = oldFieldOpt.get(); values[i] = rewriteRecordWithNewSchema(arrayWritable.get()[oldField.pos()], oldField.schema(), newField.schema(), renameCols, fieldNames); - } else if (newField.defaultVal() instanceof JsonProperties.Null) { + } else if (newField.defaultVal().isPresent() && newField.defaultVal().get().equals(HoodieSchema.NULL_VALUE)) { values[i] = NullWritable.get(); - } else if (!isNullable(newField.schema()) && newField.defaultVal() == null) { + } else if (!newField.schema().isNullable() && newField.defaultVal().isEmpty()) { throw new SchemaCompatibilityException("Field " + createFullName(fieldNames) + " has no default value and is non-nullable"); - } else if (newField.defaultVal() != null) { - switch (AvroSchemaUtils.getNonNullTypeFromUnion(newField.schema()).getType()) { + } else if (newField.defaultVal().isPresent()) { + switch (HoodieSchemaUtils.getNonNullTypeFromUnion(newField.schema()).getType()) { case BOOLEAN: - values[i] = new BooleanWritable((Boolean) newField.defaultVal()); + values[i] = new BooleanWritable((Boolean) newField.defaultVal().get()); break; case INT: - values[i] = new IntWritable((Integer) newField.defaultVal()); + values[i] = new IntWritable((Integer) newField.defaultVal().get()); break; case LONG: - values[i] = new LongWritable((Long) newField.defaultVal()); + values[i] = new LongWritable((Long) newField.defaultVal().get()); break; case FLOAT: - values[i] = new FloatWritable((Float) newField.defaultVal()); + values[i] = new FloatWritable((Float) newField.defaultVal().get()); break; case DOUBLE: - values[i] = new DoubleWritable((Double) newField.defaultVal()); + values[i] = new DoubleWritable((Double) newField.defaultVal().get()); break; case STRING: values[i] = new Text(newField.defaultVal().toString()); @@ -140,10 +139,10 @@ private static Writable rewriteRecordWithNewSchemaInternal(Writable writable, Sc if ((writable instanceof BytesWritable)) { return writable; } - if (oldSchema.getType() != Schema.Type.STRING && oldSchema.getType() != Schema.Type.ENUM) { - throw new SchemaCompatibilityException(String.format("Only ENUM or STRING type can be converted ENUM type. Schema type was %s", oldSchema.getType().getName())); + if (oldSchema.getType() != HoodieSchemaType.STRING && oldSchema.getType() != HoodieSchemaType.ENUM) { + throw new SchemaCompatibilityException(String.format("Only ENUM or STRING type can be converted ENUM type. Schema type was %s", oldSchema.getType())); } - if (oldSchema.getType() == Schema.Type.STRING) { + if (oldSchema.getType() == HoodieSchemaType.STRING) { return new BytesWritable(((Text) writable).copyBytes()); } return writable; @@ -178,7 +177,7 @@ private static Writable rewriteRecordWithNewSchemaInternal(Writable writable, Sc } } - public static Writable rewritePrimaryType(Writable writable, Schema oldSchema, Schema newSchema) { + public static Writable rewritePrimaryType(Writable writable, HoodieSchema oldSchema, HoodieSchema newSchema) { if (oldSchema.getType() == newSchema.getType()) { switch (oldSchema.getType()) { case NULL: @@ -191,13 +190,14 @@ public static Writable rewritePrimaryType(Writable writable, Schema oldSchema, S case STRING: return writable; case FIXED: + case DECIMAL: if (oldSchema.getFixedSize() != newSchema.getFixedSize()) { // Check whether this is a [[Decimal]]'s precision change - if (oldSchema.getLogicalType() instanceof LogicalTypes.Decimal) { - LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) oldSchema.getLogicalType(); + if (oldSchema.getType() == HoodieSchemaType.DECIMAL) { + HoodieSchema.Decimal decimal = (HoodieSchema.Decimal) oldSchema; return HiveDecimalUtils.enforcePrecisionScale((HiveDecimalWritable) writable, new DecimalTypeInfo(decimal.getPrecision(), decimal.getScale())); } else { - throw new HoodieAvroSchemaException("Fixed type size change is not currently supported"); + throw new HoodieSchemaException("Fixed type size change is not currently supported"); } } @@ -209,93 +209,95 @@ public static Writable rewritePrimaryType(Writable writable, Schema oldSchema, S if (Objects.equals(oldSchema.getFullName(), newSchema.getFullName())) { return writable; } else { - LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) oldSchema.getLogicalType(); + HoodieSchema.Decimal decimal = (HoodieSchema.Decimal) oldSchema; return HiveDecimalUtils.enforcePrecisionScale((HiveDecimalWritable) writable, new DecimalTypeInfo(decimal.getPrecision(), decimal.getScale())); } default: - throw new HoodieAvroSchemaException("Unknown schema type: " + newSchema.getType()); + throw new HoodieSchemaException("Unknown schema type: " + newSchema.getType()); } } else { return rewritePrimaryTypeWithDiffSchemaType(writable, oldSchema, newSchema); } } - private static Writable rewritePrimaryTypeWithDiffSchemaType(Writable writable, Schema oldSchema, Schema newSchema) { + private static Writable rewritePrimaryTypeWithDiffSchemaType(Writable writable, HoodieSchema oldSchema, HoodieSchema newSchema) { switch (newSchema.getType()) { case NULL: case BOOLEAN: - break; case INT: - if (newSchema.getLogicalType() == LogicalTypes.date() && oldSchema.getType() == Schema.Type.STRING) { + break; + case DATE: + if (oldSchema.getType() == HoodieSchemaType.STRING) { return HoodieHiveUtils.getDateWriteable((HoodieAvroUtils.fromJavaDate(java.sql.Date.valueOf(writable.toString())))); } break; case LONG: - if (oldSchema.getType() == Schema.Type.INT) { + if (oldSchema.getType() == HoodieSchemaType.INT) { return new LongWritable(((IntWritable) writable).get()); } break; case FLOAT: - if ((oldSchema.getType() == Schema.Type.INT) - || (oldSchema.getType() == Schema.Type.LONG)) { - return oldSchema.getType() == Schema.Type.INT + if ((oldSchema.getType() == HoodieSchemaType.INT) + || (oldSchema.getType() == HoodieSchemaType.LONG)) { + return oldSchema.getType() == HoodieSchemaType.INT ? new FloatWritable(((IntWritable) writable).get()) : new FloatWritable(((LongWritable) writable).get()); } break; case DOUBLE: - if (oldSchema.getType() == Schema.Type.FLOAT) { + if (oldSchema.getType() == HoodieSchemaType.FLOAT) { // java float cannot convert to double directly, deal with float precision change return new DoubleWritable(Double.parseDouble(((FloatWritable) writable).get() + "")); - } else if (oldSchema.getType() == Schema.Type.INT) { + } else if (oldSchema.getType() == HoodieSchemaType.INT) { return new DoubleWritable(((IntWritable) writable).get()); - } else if (oldSchema.getType() == Schema.Type.LONG) { + } else if (oldSchema.getType() == HoodieSchemaType.LONG) { return new DoubleWritable(((LongWritable) writable).get()); } break; case BYTES: - if (oldSchema.getType() == Schema.Type.STRING) { + if (oldSchema.getType() == HoodieSchemaType.STRING) { return new BytesWritable(getUTF8Bytes(writable.toString())); } break; case STRING: - if (oldSchema.getType() == Schema.Type.ENUM) { + if (oldSchema.getType() == HoodieSchemaType.ENUM) { return writable; } - if (oldSchema.getType() == Schema.Type.BYTES) { + if (oldSchema.getType() == HoodieSchemaType.BYTES) { return new Text(StringUtils.fromUTF8Bytes(((BytesWritable) writable).getBytes())); } - if (oldSchema.getLogicalType() == LogicalTypes.date()) { + if (oldSchema.getType() == HoodieSchemaType.DATE) { return new Text(toJavaDate(((IntWritable) writable).get()).toString()); } - if (oldSchema.getType() == Schema.Type.INT - || oldSchema.getType() == Schema.Type.LONG - || oldSchema.getType() == Schema.Type.FLOAT - || oldSchema.getType() == Schema.Type.DOUBLE) { + if (oldSchema.getType() == HoodieSchemaType.INT + || oldSchema.getType() == HoodieSchemaType.LONG + || oldSchema.getType() == HoodieSchemaType.FLOAT + || oldSchema.getType() == HoodieSchemaType.DOUBLE) { return new Text(writable.toString()); } - if (oldSchema.getType() == Schema.Type.FIXED && oldSchema.getLogicalType() instanceof LogicalTypes.Decimal) { + if (oldSchema instanceof HoodieSchema.Decimal) { HiveDecimalWritable hdw = (HiveDecimalWritable) writable; return new Text(hdw.getHiveDecimal().bigDecimalValue().toPlainString()); } break; case FIXED: - if (newSchema.getLogicalType() instanceof LogicalTypes.Decimal) { - LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) newSchema.getLogicalType(); + case DECIMAL: + if (newSchema instanceof HoodieSchema.Decimal) { + HoodieSchema.Decimal decimal = (HoodieSchema.Decimal) newSchema; DecimalTypeInfo decimalTypeInfo = new DecimalTypeInfo(decimal.getPrecision(), decimal.getScale()); - if (oldSchema.getType() == Schema.Type.STRING - || oldSchema.getType() == Schema.Type.INT - || oldSchema.getType() == Schema.Type.LONG - || oldSchema.getType() == Schema.Type.FLOAT - || oldSchema.getType() == Schema.Type.DOUBLE) { + if (oldSchema.getType() == HoodieSchemaType.STRING + || oldSchema.getType() == HoodieSchemaType.INT + || oldSchema.getType() == HoodieSchemaType.LONG + || oldSchema.getType() == HoodieSchemaType.FLOAT + || oldSchema.getType() == HoodieSchemaType.DOUBLE) { // loses trailing zeros due to hive issue. Since we only read with hive, I think this is fine HiveDecimalWritable converted = new HiveDecimalWritable(HiveDecimal.create(new BigDecimal(writable.toString()))); return HiveDecimalUtils.enforcePrecisionScale(converted, decimalTypeInfo); } - if (oldSchema.getType() == Schema.Type.BYTES) { + if (oldSchema.getType() == HoodieSchemaType.BYTES) { ByteBuffer buffer = ByteBuffer.wrap(((BytesWritable) writable).getBytes()); BigDecimal bd = new BigDecimal(new BigInteger(buffer.array()), decimal.getScale()); HiveDecimalWritable converted = new HiveDecimalWritable(HiveDecimal.create(bd)); @@ -305,7 +307,7 @@ private static Writable rewritePrimaryTypeWithDiffSchemaType(Writable writable, break; default: } - throw new HoodieAvroSchemaException(String.format("cannot support rewrite value for schema type: %s since the old schema type is: %s", newSchema, oldSchema)); + throw new HoodieSchemaException(String.format("cannot support rewrite value for schema type: %s since the old schema type is: %s", newSchema, oldSchema)); } private static int[] getReverseProjectionMapping(HoodieSchema from, HoodieSchema to) { @@ -334,4 +336,3 @@ public static UnaryOperator getReverseProjection(HoodieSchema fro }; } } - diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeRecordReaderUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeRecordReaderUtils.java index 2c8f6e28849a9..a6a1fc2eb1346 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeRecordReaderUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeRecordReaderUtils.java @@ -21,6 +21,10 @@ import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.config.HoodieMemoryConfig; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaField; +import org.apache.hudi.common.schema.HoodieSchemaType; +import org.apache.hudi.common.schema.HoodieSchemaUtils; import org.apache.hudi.common.util.HadoopConfigUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; @@ -30,7 +34,6 @@ import org.apache.hudi.storage.HoodieStorageUtils; import org.apache.hudi.storage.StorageConfiguration; -import org.apache.avro.JsonProperties; import org.apache.avro.LogicalType; import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; @@ -66,9 +69,6 @@ import java.util.TreeMap; import java.util.stream.Collectors; -import static org.apache.hudi.avro.AvroSchemaUtils.appendFieldsToSchema; -import static org.apache.hudi.avro.AvroSchemaUtils.createNullableSchema; -import static org.apache.hudi.avro.HoodieAvroUtils.createNewSchemaField; import static org.apache.hudi.common.util.ConfigUtils.getReaderConfigs; import static org.apache.hudi.hadoop.fs.HadoopFSUtils.convertToStoragePath; @@ -121,7 +121,7 @@ public static String arrayWritableToString(ArrayWritable writable) { /** * Generate a reader schema off the provided writeSchema, to just project out the provided columns. */ - public static Schema generateProjectionSchema(Schema writeSchema, Map schemaFieldsMap, + public static HoodieSchema generateProjectionSchema(HoodieSchema writeSchema, Map schemaFieldsMap, List fieldNames) { /** * Avro & Presto field names seems to be case sensitive (support fields differing only in case) whereas @@ -133,24 +133,23 @@ public static Schema generateProjectionSchema(Schema writeSchema, Map projectedFields = new ArrayList<>(); + List projectedFields = new ArrayList<>(); for (String fn : fieldNames) { - Schema.Field field = schemaFieldsMap.get(fn.toLowerCase()); + HoodieSchemaField field = schemaFieldsMap.get(fn.toLowerCase()); if (field == null) { throw new HoodieException("Field " + fn + " not found in log schema. Query cannot proceed! " + "Derived Schema Fields: " + new ArrayList<>(schemaFieldsMap.keySet())); } else { - projectedFields.add(createNewSchemaField(field)); + projectedFields.add(HoodieSchemaUtils.createNewSchemaField(field)); } } - Schema projectedSchema = Schema.createRecord(writeSchema.getName(), writeSchema.getDoc(), - writeSchema.getNamespace(), writeSchema.isError()); - projectedSchema.setFields(projectedFields); + HoodieSchema projectedSchema = HoodieSchema.createRecord(writeSchema.getName(), writeSchema.getDoc().orElse(null), + writeSchema.getNamespace().orElse(null), writeSchema.isError(), projectedFields); return projectedSchema; } - public static Map getNameToFieldMap(Schema schema) { + public static Map getNameToFieldMap(HoodieSchema schema) { return schema.getFields().stream().map(r -> Pair.of(r.name().toLowerCase(), r)) .collect(Collectors.toMap(Pair::getLeft, Pair::getRight)); } @@ -300,9 +299,9 @@ public static List orderFields(String fieldNameCsv, String fieldOrderCsv * * @param schema Schema to be changed */ - public static Schema addPartitionFields(Schema schema, List partitioningFields) { + public static HoodieSchema addPartitionFields(HoodieSchema schema, List partitioningFields) { final Set firstLevelFieldNames = - schema.getFields().stream().map(Schema.Field::name).map(String::toLowerCase).collect(Collectors.toSet()); + schema.getFields().stream().map(HoodieSchemaField::name).map(String::toLowerCase).collect(Collectors.toSet()); List fieldsToAdd = partitioningFields.stream().map(String::toLowerCase) .filter(x -> !firstLevelFieldNames.contains(x)).collect(Collectors.toList()); @@ -317,12 +316,12 @@ public static HoodieFileReader getBaseFileReader(Path path, JobConf conf) throws .getFileReader(hoodieConfig, convertToStoragePath(path)); } - private static Schema appendNullSchemaFields(Schema schema, List newFieldNames) { - List newFields = new ArrayList<>(); + private static HoodieSchema appendNullSchemaFields(HoodieSchema schema, List newFieldNames) { + List newFields = new ArrayList<>(); for (String newField : newFieldNames) { - newFields.add(new Schema.Field(newField, createNullableSchema(Schema.Type.STRING), "", JsonProperties.NULL_VALUE)); + newFields.add(HoodieSchemaField.of(newField, HoodieSchema.createNullable(HoodieSchemaType.STRING), "", HoodieSchema.NULL_VALUE)); } - return appendFieldsToSchema(schema, newFields); + return HoodieSchemaUtils.appendFieldsToSchema(schema, newFields); } private static HiveDecimalWritable toHiveDecimalWritable(byte[] bytes, Schema schema) { diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/utils/TestHoodieArrayWritableAvroUtils.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/utils/TestHoodieArrayWritableSchemaUtils.java similarity index 69% rename from hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/utils/TestHoodieArrayWritableAvroUtils.java rename to hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/utils/TestHoodieArrayWritableSchemaUtils.java index e05a89abe9c50..1c5a9575ecd56 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/utils/TestHoodieArrayWritableAvroUtils.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/utils/TestHoodieArrayWritableSchemaUtils.java @@ -22,14 +22,13 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.engine.RecordContext; import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaType; import org.apache.hudi.common.schema.HoodieSchemaUtils; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; -import org.apache.hudi.exception.HoodieAvroSchemaException; +import org.apache.hudi.internal.schema.HoodieSchemaException; import org.apache.hudi.hadoop.HiveHoodieReaderContext; import org.apache.hudi.hadoop.HiveRecordContext; -import org.apache.avro.LogicalTypes; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.hive.common.type.HiveDecimal; @@ -69,7 +68,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -public class TestHoodieArrayWritableAvroUtils { +public class TestHoodieArrayWritableSchemaUtils { private final HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); @@ -77,7 +76,7 @@ public class TestHoodieArrayWritableAvroUtils { public void testProjection() { HoodieSchema from = HoodieTestDataGenerator.HOODIE_SCHEMA; HoodieSchema to = HoodieSchemaUtils.generateProjectionSchema(from, Arrays.asList("trip_type", "current_ts", "weight")); - UnaryOperator reverseProjection = HoodieArrayWritableAvroUtils.getReverseProjection(to, from); + UnaryOperator reverseProjection = HoodieArrayWritableSchemaUtils.getReverseProjection(to, from); //We reuse the ArrayWritable, so we need to get the values before projecting ArrayWritable record = convertArrayWritable(dataGen.generateGenericRecord()); @@ -87,7 +86,7 @@ public void testProjection() { Object weight = fromSerializer.getValue(record, "weight"); //Make sure the projected fields can be read - ArrayWritable projectedRecord = HoodieArrayWritableAvroUtils.rewriteRecordWithNewSchema(record, from.toAvroSchema(), to.toAvroSchema(), Collections.emptyMap()); + ArrayWritable projectedRecord = HoodieArrayWritableSchemaUtils.rewriteRecordWithNewSchema(record, from, to, Collections.emptyMap()); HiveAvroSerializer toSerializer = new HiveAvroSerializer(to.toAvroSchema()); assertEquals(tripType, toSerializer.getValue(projectedRecord, "trip_type")); assertEquals(currentTs, toSerializer.getValue(projectedRecord, "current_ts")); @@ -128,10 +127,10 @@ public void testCastOrderingField() { @Test void testRewriteStringToDateInt() throws AvroSerdeException { - Schema oldSchema = Schema.create(Schema.Type.STRING); - Schema newSchema = LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT)); + HoodieSchema oldSchema = HoodieSchema.create(HoodieSchemaType.STRING); + HoodieSchema newSchema = HoodieSchema.create(HoodieSchemaType.DATE); Writable oldWritable = new Text("2023-01-01"); - Writable result = HoodieArrayWritableAvroUtils.rewritePrimaryType(oldWritable, oldSchema, newSchema); + Writable result = HoodieArrayWritableSchemaUtils.rewritePrimaryType(oldWritable, oldSchema, newSchema); Writable expected = HoodieHiveUtils.getDateWriteable(HoodieAvroUtils.fromJavaDate(Date.valueOf("2023-01-01"))); assertEquals(expected, result); validateRewriteWithAvro(oldWritable, oldSchema, result, newSchema); @@ -140,9 +139,9 @@ void testRewriteStringToDateInt() throws AvroSerdeException { @Test void testRewriteIntToLong() throws AvroSerdeException { Writable oldWritable = new IntWritable(42); - Schema oldSchema = Schema.create(Schema.Type.INT); - Schema newSchema = Schema.create(Schema.Type.LONG); - Writable result = HoodieArrayWritableAvroUtils.rewritePrimaryType(oldWritable, oldSchema, newSchema); + HoodieSchema oldSchema = HoodieSchema.create(HoodieSchemaType.INT); + HoodieSchema newSchema = HoodieSchema.create(HoodieSchemaType.LONG); + Writable result = HoodieArrayWritableSchemaUtils.rewritePrimaryType(oldWritable, oldSchema, newSchema); Writable expected = new LongWritable(42); assertEquals(expected, result); validateRewriteWithAvro(oldWritable, oldSchema, result, newSchema); @@ -151,9 +150,9 @@ void testRewriteIntToLong() throws AvroSerdeException { @Test void testRewriteLongToFloat() throws AvroSerdeException { Writable oldWritable = new LongWritable(123); - Schema oldSchema = Schema.create(Schema.Type.LONG); - Schema newSchema = Schema.create(Schema.Type.FLOAT); - Writable result = HoodieArrayWritableAvroUtils.rewritePrimaryType(oldWritable, oldSchema, newSchema); + HoodieSchema oldSchema = HoodieSchema.create(HoodieSchemaType.LONG); + HoodieSchema newSchema = HoodieSchema.create(HoodieSchemaType.FLOAT); + Writable result = HoodieArrayWritableSchemaUtils.rewritePrimaryType(oldWritable, oldSchema, newSchema); Writable expected = new FloatWritable(123.0f); assertEquals(expected, result); validateRewriteWithAvro(oldWritable, oldSchema, result, newSchema); @@ -162,9 +161,9 @@ void testRewriteLongToFloat() throws AvroSerdeException { @Test void testRewriteFloatToDouble() throws AvroSerdeException { Writable oldWritable = new FloatWritable(3.14f); - Schema oldSchema = Schema.create(Schema.Type.FLOAT); - Schema newSchema = Schema.create(Schema.Type.DOUBLE); - Writable result = HoodieArrayWritableAvroUtils.rewritePrimaryType(oldWritable, oldSchema, newSchema); + HoodieSchema oldSchema = HoodieSchema.create(HoodieSchemaType.FLOAT); + HoodieSchema newSchema = HoodieSchema.create(HoodieSchemaType.DOUBLE); + Writable result = HoodieArrayWritableSchemaUtils.rewritePrimaryType(oldWritable, oldSchema, newSchema); Writable expected = new DoubleWritable(3.14d); assertEquals(expected, result); validateRewriteWithAvro(oldWritable, oldSchema, result, newSchema); @@ -173,9 +172,9 @@ void testRewriteFloatToDouble() throws AvroSerdeException { @Test void testRewriteBytesToString() throws AvroSerdeException { BytesWritable oldWritable = new BytesWritable("hello".getBytes()); - Schema oldSchema = Schema.create(Schema.Type.BYTES); - Schema newSchema = Schema.create(Schema.Type.STRING); - Writable result = HoodieArrayWritableAvroUtils.rewritePrimaryType(oldWritable, oldSchema, newSchema); + HoodieSchema oldSchema = HoodieSchema.create(HoodieSchemaType.BYTES); + HoodieSchema newSchema = HoodieSchema.create(HoodieSchemaType.STRING); + Writable result = HoodieArrayWritableSchemaUtils.rewritePrimaryType(oldWritable, oldSchema, newSchema); Writable expected = new Text("hello"); assertEquals(expected, result); validateRewriteWithAvro(oldWritable, oldSchema, result, newSchema); @@ -184,9 +183,9 @@ void testRewriteBytesToString() throws AvroSerdeException { @Test void testRewriteIntToString() throws AvroSerdeException { Writable oldWritable = new IntWritable(123); - Schema oldSchema = Schema.create(Schema.Type.INT); - Schema newSchema = Schema.create(Schema.Type.STRING); - Writable result = HoodieArrayWritableAvroUtils.rewritePrimaryType(oldWritable, oldSchema, newSchema); + HoodieSchema oldSchema = HoodieSchema.create(HoodieSchemaType.INT); + HoodieSchema newSchema = HoodieSchema.create(HoodieSchemaType.STRING); + Writable result = HoodieArrayWritableSchemaUtils.rewritePrimaryType(oldWritable, oldSchema, newSchema); Writable expected = new Text("123"); assertEquals(expected, result); validateRewriteWithAvro(oldWritable, oldSchema, result, newSchema); @@ -194,10 +193,10 @@ void testRewriteIntToString() throws AvroSerdeException { @Test void testRewriteFixedDecimalToString() throws AvroSerdeException { - Schema decimalSchema = LogicalTypes.decimal(10, 2).addToSchema(Schema.createFixed("decimal", null, null, 5)); + HoodieSchema decimalSchema = HoodieSchema.createDecimal("decimal", null, null, 10, 2, 5); HiveDecimalWritable oldWritable = new HiveDecimalWritable(HiveDecimal.create(new BigDecimal("123.45"))); - Schema newSchema = Schema.create(Schema.Type.STRING); - Writable result = HoodieArrayWritableAvroUtils.rewritePrimaryType(oldWritable, decimalSchema, newSchema); + HoodieSchema newSchema = HoodieSchema.create(HoodieSchemaType.STRING); + Writable result = HoodieArrayWritableSchemaUtils.rewritePrimaryType(oldWritable, decimalSchema, newSchema); Writable expected = new Text("123.45"); assertEquals(expected, result); validateRewriteWithAvro(oldWritable, decimalSchema, result, newSchema); @@ -205,12 +204,13 @@ void testRewriteFixedDecimalToString() throws AvroSerdeException { @Test void testRewriteStringToFixedDecimal() throws AvroSerdeException { - Schema decimalSchema = LogicalTypes.decimal(10, 2).addToSchema(Schema.createFixed("decimal", null, null, 5)); + HoodieSchema decimalSchema = HoodieSchema.createDecimal("decimal", null, null, 10, 2, 5); Writable oldWritable = new Text("123.45"); - Writable result = HoodieArrayWritableAvroUtils.rewritePrimaryType(oldWritable, Schema.create(Schema.Type.STRING), decimalSchema); + HoodieSchema oldSchema = HoodieSchema.create(HoodieSchemaType.STRING); + Writable result = HoodieArrayWritableSchemaUtils.rewritePrimaryType(oldWritable, oldSchema, decimalSchema); assertInstanceOf(HiveDecimalWritable.class, result); assertEquals(new BigDecimal("123.45"), ((HiveDecimalWritable) result).getHiveDecimal().bigDecimalValue()); - validateRewriteWithAvro(oldWritable, Schema.create(Schema.Type.STRING), result, decimalSchema); + validateRewriteWithAvro(oldWritable, oldSchema, result, decimalSchema); } @Test @@ -218,26 +218,27 @@ void testRewriteBytesToFixedDecimal() throws AvroSerdeException { BigDecimal input = new BigDecimal("123.45"); byte[] bytes = input.unscaledValue().toByteArray(); BytesWritable oldWritable = new BytesWritable(bytes); - Schema decimalSchema = LogicalTypes.decimal(5, 2).addToSchema(Schema.createFixed("decimal", null, null, 5)); - Writable result = HoodieArrayWritableAvroUtils.rewritePrimaryType(oldWritable, Schema.create(Schema.Type.BYTES), decimalSchema); + HoodieSchema decimalSchema = HoodieSchema.createDecimal("decimal", null, null, 5, 2, 5); + HoodieSchema oldSchema = HoodieSchema.create(HoodieSchemaType.BYTES); + Writable result = HoodieArrayWritableSchemaUtils.rewritePrimaryType(oldWritable, oldSchema, decimalSchema); assertEquals(input, ((HiveDecimalWritable) result).getHiveDecimal().bigDecimalValue()); - validateRewriteWithAvro(oldWritable, Schema.create(Schema.Type.BYTES), result, decimalSchema); + validateRewriteWithAvro(oldWritable, oldSchema, result, decimalSchema); } @Test void testUnsupportedTypeConversionThrows() { - Schema oldSchema = Schema.createMap(Schema.create(Schema.Type.INT)); - Schema newSchema = Schema.create(Schema.Type.STRING); - assertThrows(HoodieAvroSchemaException.class, () -> - HoodieArrayWritableAvroUtils.rewritePrimaryType(null, oldSchema, newSchema)); + HoodieSchema oldSchema = HoodieSchema.createMap(HoodieSchema.create(HoodieSchemaType.INT)); + HoodieSchema newSchema = HoodieSchema.create(HoodieSchemaType.STRING); + assertThrows(HoodieSchemaException.class, () -> + HoodieArrayWritableSchemaUtils.rewritePrimaryType(null, oldSchema, newSchema)); } @Test void testRewriteEnumToString() throws AvroSerdeException { - Schema enumSchema = Schema.createEnum("TestEnum", null, null, Arrays.asList("A", "B", "C")); + HoodieSchema enumSchema = HoodieSchema.createEnum("TestEnum", null, null, Arrays.asList("A", "B", "C")); Writable oldWritable = new Text("B"); - Schema newSchema = Schema.create(Schema.Type.STRING); - Writable result = HoodieArrayWritableAvroUtils.rewritePrimaryType(oldWritable, enumSchema, newSchema); + HoodieSchema newSchema = HoodieSchema.create(HoodieSchemaType.STRING); + Writable result = HoodieArrayWritableSchemaUtils.rewritePrimaryType(oldWritable, enumSchema, newSchema); Writable expected = new Text("B"); assertEquals(expected, result); validateRewriteWithAvro(oldWritable, enumSchema, result, newSchema); @@ -245,19 +246,19 @@ void testRewriteEnumToString() throws AvroSerdeException { @Test void testRewriteFixedWithSameSizeAndFullName() { - Schema oldFixed = Schema.createFixed("decimal", null, "ns", 5); - Schema newFixed = Schema.createFixed("decimal", null, "ns", 5); + HoodieSchema oldFixed = HoodieSchema.createFixed("decimal", null, "ns", 5); + HoodieSchema newFixed = HoodieSchema.createFixed("decimal", null, "ns", 5); HiveDecimalWritable hdw = new HiveDecimalWritable(HiveDecimal.create("123.45")); - Writable result = HoodieArrayWritableAvroUtils.rewritePrimaryType(hdw, oldFixed, newFixed); + Writable result = HoodieArrayWritableSchemaUtils.rewritePrimaryType(hdw, oldFixed, newFixed); assertSame(hdw, result); } @Test void testRewriteFixedWithSameSizeButDifferentNameUsesDecimalFallback() throws AvroSerdeException { - Schema oldFixed = LogicalTypes.decimal(5, 2).addToSchema(Schema.createFixed("decA", null, "ns1", 5)); - Schema newFixed = LogicalTypes.decimal(5, 2).addToSchema(Schema.createFixed("decB", null, "ns2", 5)); + HoodieSchema oldFixed = HoodieSchema.createDecimal("decA", "ns1", null, 5, 2, 5); + HoodieSchema newFixed = HoodieSchema.createDecimal("decB", "ns2", null, 5, 2, 5); HiveDecimalWritable oldWritable = new HiveDecimalWritable(HiveDecimal.create("123.45")); - Writable result = HoodieArrayWritableAvroUtils.rewritePrimaryType(oldWritable, oldFixed, newFixed); + Writable result = HoodieArrayWritableSchemaUtils.rewritePrimaryType(oldWritable, oldFixed, newFixed); assertInstanceOf(HiveDecimalWritable.class, result); assertEquals(new BigDecimal("123.45"), ((HiveDecimalWritable) result).getHiveDecimal().bigDecimalValue()); validateRewriteWithAvro(oldWritable, oldFixed, result, newFixed); @@ -265,59 +266,62 @@ void testRewriteFixedWithSameSizeButDifferentNameUsesDecimalFallback() throws Av @Test void testRewriteBooleanPassthrough() { - Schema boolSchema = Schema.create(Schema.Type.BOOLEAN); + HoodieSchema boolSchema = HoodieSchema.create(HoodieSchemaType.BOOLEAN); BooleanWritable bool = new BooleanWritable(true); - Writable result = HoodieArrayWritableAvroUtils.rewritePrimaryType(bool, boolSchema, boolSchema); + Writable result = HoodieArrayWritableSchemaUtils.rewritePrimaryType(bool, boolSchema, boolSchema); assertSame(bool, result); } @Test void testUnsupportedRewriteMapToIntThrows() { - Schema oldSchema = Schema.createMap(Schema.create(Schema.Type.STRING)); - Schema newSchema = Schema.create(Schema.Type.INT); - assertThrows(HoodieAvroSchemaException.class, () -> - HoodieArrayWritableAvroUtils.rewritePrimaryType(new Text("foo"), oldSchema, newSchema)); + HoodieSchema oldSchema = HoodieSchema.createMap(HoodieSchema.create(HoodieSchemaType.STRING)); + HoodieSchema newSchema = HoodieSchema.create(HoodieSchemaType.INT); + assertThrows(HoodieSchemaException.class, () -> + HoodieArrayWritableSchemaUtils.rewritePrimaryType(new Text("foo"), oldSchema, newSchema)); } @Test void testRewriteIntToDecimalFixed() throws AvroSerdeException { - Schema fixedDecimal = LogicalTypes.decimal(8, 2).addToSchema(Schema.createFixed("dec", null, null, 6)); + HoodieSchema fixedDecimalSchema = HoodieSchema.createDecimal("dec", null, null, 8, 2, 5); + HoodieSchema oldSchema = HoodieSchema.create(HoodieSchemaType.INT); IntWritable oldWritable = new IntWritable(12345); - Writable result = HoodieArrayWritableAvroUtils.rewritePrimaryType(oldWritable, Schema.create(Schema.Type.INT), fixedDecimal); + Writable result = HoodieArrayWritableSchemaUtils.rewritePrimaryType(oldWritable, oldSchema, fixedDecimalSchema); assertInstanceOf(HiveDecimalWritable.class, result); assertEquals(new BigDecimal("12345"), ((HiveDecimalWritable) result).getHiveDecimal().bigDecimalValue()); - validateRewriteWithAvro(oldWritable, Schema.create(Schema.Type.INT), result, fixedDecimal); + validateRewriteWithAvro(oldWritable, oldSchema, result, fixedDecimalSchema); } @Test void testRewriteDoubleToDecimalFixed() throws AvroSerdeException { - Schema fixedDecimal = LogicalTypes.decimal(10, 3).addToSchema(Schema.createFixed("dec", null, null, 8)); + HoodieSchema fixedDecimal = HoodieSchema.createDecimal("dec", null, null, 10, 3, 8); + HoodieSchema oldSchema = HoodieSchema.create(HoodieSchemaType.DOUBLE); DoubleWritable oldWritable = new DoubleWritable(987.654); - Writable result = HoodieArrayWritableAvroUtils.rewritePrimaryType(oldWritable, Schema.create(Schema.Type.DOUBLE), fixedDecimal); + Writable result = HoodieArrayWritableSchemaUtils.rewritePrimaryType(oldWritable, oldSchema, fixedDecimal); assertInstanceOf(HiveDecimalWritable.class, result); assertEquals(new BigDecimal("987.654"), ((HiveDecimalWritable) result).getHiveDecimal().bigDecimalValue()); - validateRewriteWithAvro(oldWritable, Schema.create(Schema.Type.DOUBLE), result, fixedDecimal); + validateRewriteWithAvro(oldWritable, oldSchema, result, fixedDecimal); } @Test void testRewriteDecimalBytesToFixed() throws AvroSerdeException { - Schema decimalSchema = LogicalTypes.decimal(6, 2).addToSchema(Schema.createFixed("dec", null, null, 6)); + HoodieSchema decimalSchema = HoodieSchema.createDecimal("dec", null, null, 6, 2, 6); + HoodieSchema oldSchema = HoodieSchema.create(HoodieSchemaType.BYTES); BigDecimal value = new BigDecimal("999.99"); byte[] unscaledBytes = value.unscaledValue().toByteArray(); BytesWritable oldWritable = new BytesWritable(unscaledBytes); - Writable result = HoodieArrayWritableAvroUtils.rewritePrimaryType(oldWritable, Schema.create(Schema.Type.BYTES), decimalSchema); + Writable result = HoodieArrayWritableSchemaUtils.rewritePrimaryType(oldWritable, oldSchema, decimalSchema); assertEquals(value, ((HiveDecimalWritable) result).getHiveDecimal().bigDecimalValue()); - validateRewriteWithAvro(oldWritable, Schema.create(Schema.Type.BYTES), result, decimalSchema); + validateRewriteWithAvro(oldWritable, oldSchema, result, decimalSchema); } private void validateRewriteWithAvro( Writable oldWritable, - Schema oldSchema, + HoodieSchema oldSchema, Writable newWritable, - Schema newSchema + HoodieSchema newSchema ) throws AvroSerdeException { - TypeInfo oldTypeInfo = HiveTypeUtils.generateTypeInfo(oldSchema, Collections.emptySet()); - TypeInfo newTypeInfo = HiveTypeUtils.generateTypeInfo(newSchema, Collections.emptySet()); + TypeInfo oldTypeInfo = HiveTypeUtils.generateTypeInfo(oldSchema.toAvroSchema(), Collections.emptySet()); + TypeInfo newTypeInfo = HiveTypeUtils.generateTypeInfo(newSchema.toAvroSchema(), Collections.emptySet()); ObjectInspector oldObjectInspector = TypeInfoUtils.getStandardJavaObjectInspectorFromTypeInfo(oldTypeInfo); ObjectInspector newObjectInspector = TypeInfoUtils.getStandardJavaObjectInspectorFromTypeInfo(newTypeInfo); @@ -326,30 +330,26 @@ private void validateRewriteWithAvro( ObjectInspector writableOINew = getWritableOIForType(newTypeInfo); Object javaInput = ObjectInspectorConverters.getConverter(writableOIOld, oldObjectInspector).convert(oldWritable); - if (isDecimalSchema(oldSchema)) { - javaInput = HoodieAvroUtils.DECIMAL_CONVERSION.toFixed(getDecimalValue(javaInput, oldSchema), oldSchema, oldSchema.getLogicalType()); + if (oldSchema.getType() == HoodieSchemaType.DECIMAL) { + javaInput = HoodieAvroUtils.DECIMAL_CONVERSION.toFixed(getDecimalValue(javaInput, (HoodieSchema.Decimal) oldSchema), oldSchema.toAvroSchema(), oldSchema.toAvroSchema().getLogicalType()); } else if (javaInput instanceof byte[]) { javaInput = ByteBuffer.wrap((byte[]) javaInput); } - Object javaOutput = HoodieAvroUtils.rewritePrimaryType(javaInput, oldSchema, newSchema); + Object javaOutput = HoodieAvroUtils.rewritePrimaryType(javaInput, oldSchema.toAvroSchema(), newSchema.toAvroSchema()); Object javaExpected = ObjectInspectorConverters.getConverter(writableOINew, newObjectInspector).convert(newWritable); - if (isDecimalSchema(newSchema)) { - BigDecimal outputDecimal = getDecimalValue(javaOutput, newSchema); - BigDecimal expectedDecimal = getDecimalValue(javaExpected, newSchema); + if (newSchema.getType() == HoodieSchemaType.DECIMAL) { + BigDecimal outputDecimal = getDecimalValue(javaOutput, (HoodieSchema.Decimal) newSchema); + BigDecimal expectedDecimal = getDecimalValue(javaExpected, (HoodieSchema.Decimal) newSchema); assertEquals(0, outputDecimal.compareTo(expectedDecimal)); - } else if (newSchema.getLogicalType() instanceof LogicalTypes.Date) { + } else if (newSchema.getType() == HoodieSchemaType.DATE) { assertEquals(HoodieAvroUtils.toJavaDate((int) javaOutput), javaExpected); } else { assertEquals(javaOutput, javaExpected); } } - private boolean isDecimalSchema(Schema schema) { - return schema.getLogicalType() instanceof LogicalTypes.Decimal; - } - - private BigDecimal getDecimalValue(Object value, Schema decimalSchema) { + private BigDecimal getDecimalValue(Object value, HoodieSchema.Decimal decimalSchema) { if (value instanceof HiveDecimal) { return ((HiveDecimal) value).bigDecimalValue(); } else if (value instanceof HiveDecimalWritable) { @@ -357,12 +357,10 @@ private BigDecimal getDecimalValue(Object value, Schema decimalSchema) { } else if (value instanceof BigDecimal) { return (BigDecimal) value; } else if (value instanceof byte[]) { - int scale = ((LogicalTypes.Decimal) decimalSchema.getLogicalType()).getScale(); - return new BigDecimal(new BigInteger((byte[]) value), scale); + return new BigDecimal(new BigInteger((byte[]) value), decimalSchema.getScale()); } else if (value instanceof GenericData.Fixed) { - int scale = ((LogicalTypes.Decimal) decimalSchema.getLogicalType()).getScale(); byte[] bytes = ((GenericData.Fixed) value).bytes(); - return new BigDecimal(new BigInteger(bytes), scale); + return new BigDecimal(new BigInteger(bytes), decimalSchema.getScale()); } throw new IllegalArgumentException("Unsupported decimal object: " + value.getClass() + " -> " + value); } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java index dba9a31047a14..081b962b44a8b 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java @@ -18,13 +18,14 @@ package org.apache.hudi.client; -import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.model.HoodieAvroIndexedRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.TableServiceType; +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaCompatibility; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; @@ -39,7 +40,6 @@ import org.apache.hudi.testutils.HoodieClientTestBase; import org.apache.hudi.testutils.HoodieClientTestUtils; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -380,11 +380,11 @@ private List generateUpdatesWithSchema(String commitTime, int numR } private List convertToSchema(List records, String schemaStr) { - Schema newSchema = new Schema.Parser().parse(schemaStr); + HoodieSchema newSchema = HoodieSchema.parse(schemaStr); return records.stream().map(r -> { HoodieKey key = r.getKey(); GenericRecord payload = (GenericRecord) ((HoodieAvroIndexedRecord) r).getData(); - GenericRecord newPayload = HoodieAvroUtils.rewriteRecord(payload, newSchema); + GenericRecord newPayload = HoodieAvroUtils.rewriteRecord(payload, newSchema.toAvroSchema()); return new HoodieAvroIndexedRecord(key, newPayload); }).collect(Collectors.toList()); } @@ -401,7 +401,7 @@ private HoodieWriteConfig.Builder getWriteConfigBuilder(String schema) { } private static boolean isSchemaCompatible(String oldSchema, String newSchema, boolean shouldAllowDroppedColumns) { - return AvroSchemaUtils.isSchemaCompatible(new Schema.Parser().parse(oldSchema), new Schema.Parser().parse(newSchema), shouldAllowDroppedColumns); + return HoodieSchemaCompatibility.isSchemaCompatible(HoodieSchema.parse(oldSchema), HoodieSchema.parse(newSchema), shouldAllowDroppedColumns); } @Override diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkSortAndSizeClustering.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkSortAndSizeClustering.java index 43ef5afc5fec8..d554386e4520c 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkSortAndSizeClustering.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkSortAndSizeClustering.java @@ -19,8 +19,6 @@ package org.apache.hudi.functional; import org.apache.hudi.HoodieSparkUtils; -import org.apache.hudi.avro.AvroSchemaUtils; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieClusteringGroup; import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.client.WriteStatus; @@ -30,6 +28,8 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestUtils; @@ -50,7 +50,6 @@ import org.apache.avro.Conversions; import org.apache.avro.LogicalTypes; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericFixed; import org.apache.avro.generic.GenericRecord; @@ -167,16 +166,16 @@ public void writeAndClustering(boolean isRow) throws IOException { } private void validateDateAndTimestampFields(List rows, long ts) { - Schema schema = HoodieAvroUtils.addMetadataFields(getSchema(), false); + HoodieSchema schema = HoodieSchemaUtils.addMetadataFields(getSchema()); Timestamp timestamp = new Timestamp(ts); // sanity check date field is within expected range Date startDate = Date.valueOf(LocalDate.now().minusDays(3)); Date endDate = Date.valueOf(LocalDate.now().plusDays(1)); - int dateFieldIndex = schema.getField("date_nullable_field").pos(); - int tsMillisFieldIndex = schema.getField("timestamp_millis_field").pos(); - int tsMicrosNullableFieldIndex = schema.getField("timestamp_micros_nullable_field").pos(); - int tsLocalMillisFieldIndex = schema.getField("timestamp_local_millis_nullable_field").pos(); - int tsLocalMicrosFieldIndex = schema.getField("timestamp_local_micros_field").pos(); + int dateFieldIndex = schema.getField("date_nullable_field").get().pos(); + int tsMillisFieldIndex = schema.getField("timestamp_millis_field").get().pos(); + int tsMicrosNullableFieldIndex = schema.getField("timestamp_micros_nullable_field").get().pos(); + int tsLocalMillisFieldIndex = schema.getField("timestamp_local_millis_nullable_field").get().pos(); + int tsLocalMicrosFieldIndex = schema.getField("timestamp_local_micros_field").get().pos(); for (Row row : rows) { assertEquals(timestamp, row.get(tsMillisFieldIndex)); if (!row.isNullAt(tsMicrosNullableFieldIndex)) { @@ -269,14 +268,14 @@ public HoodieWriteConfig.Builder getConfigBuilder() { } private List generateInserts(String instant, long ts, int count) { - Schema schema = getSchema(); - Schema decimalSchema = schema.getField("decimal_field").schema(); - Schema nestedSchema = AvroSchemaUtils.getNonNullTypeFromUnion(schema.getField("nested_record").schema()); - Schema enumSchema = AvroSchemaUtils.getNonNullTypeFromUnion(schema.getField("enum_field").schema()); + HoodieSchema schema = getSchema(); + HoodieSchema decimalSchema = schema.getField("decimal_field").get().schema(); + HoodieSchema nestedSchema = HoodieSchemaUtils.getNonNullTypeFromUnion(schema.getField("nested_record").get().schema()); + HoodieSchema enumSchema = HoodieSchemaUtils.getNonNullTypeFromUnion(schema.getField("enum_field").get().schema()); Random random = new Random(0); return IntStream.range(0, count) .mapToObj(i -> { - GenericRecord record = new GenericData.Record(schema); + GenericRecord record = new GenericData.Record(schema.toAvroSchema()); String key = "key_" + i; String partition = "partition_" + (i % 3); record.put("_row_key", key); @@ -289,7 +288,7 @@ private List generateInserts(String instant, long ts, int count) { record.put("long_field", random.nextLong()); record.put("string_field", instant); record.put("bytes_field", ByteBuffer.wrap(instant.getBytes(StandardCharsets.UTF_8))); - GenericRecord nestedRecord = new GenericData.Record(nestedSchema); + GenericRecord nestedRecord = new GenericData.Record(nestedSchema.toAvroSchema()); nestedRecord.put("nested_int", random.nextInt()); nestedRecord.put("nested_string", "nested_" + instant); nestedRecord.put("nested_timestamp_millis_field", ts); @@ -299,7 +298,7 @@ private List generateInserts(String instant, long ts, int count) { // logical types BigDecimal bigDecimal = new BigDecimal(String.format(Locale.ENGLISH, "%5f", random.nextFloat())); Conversions.DecimalConversion decimalConversions = new Conversions.DecimalConversion(); - GenericFixed genericFixed = decimalConversions.toFixed(bigDecimal, decimalSchema, LogicalTypes.decimal(10, 6)); + GenericFixed genericFixed = decimalConversions.toFixed(bigDecimal, decimalSchema.toAvroSchema(), LogicalTypes.decimal(10, 6)); record.put("decimal_field", genericFixed); record.put("date_nullable_field", random.nextBoolean() ? null : LocalDate.now().minusDays(random.nextInt(3))); record.put("timestamp_millis_field", ts); @@ -307,7 +306,7 @@ private List generateInserts(String instant, long ts, int count) { record.put("timestamp_local_millis_nullable_field", random.nextBoolean() ? null : ts); record.put("timestamp_local_micros_field", ts * 1000); record.put("enum_field", new GenericData.EnumSymbol( - enumSchema, + enumSchema.toAvroSchema(), enumSchema .getEnumSymbols() .get(random.nextInt(enumSchema.getEnumSymbols().size())))); @@ -316,12 +315,13 @@ private List generateInserts(String instant, long ts, int count) { .collect(Collectors.toList()); } - private Schema getSchema() { + private HoodieSchema getSchema() { try { String schema = FileIOUtils.readAsUTFString(this.getClass().getClassLoader().getResourceAsStream("schema_with_logical_types.avsc")); - return new Schema.Parser().parse(schema); + return HoodieSchema.parse(schema); } catch (IOException e) { throw new UncheckedIOException(e); } + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/io/TestMergeHandle.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/io/TestMergeHandle.java index 0279bbe44d889..947a3ba3213dc 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/io/TestMergeHandle.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/io/TestMergeHandle.java @@ -18,7 +18,6 @@ package org.apache.hudi.io; -import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.avro.JoinedGenericRecord; import org.apache.hudi.client.SecondaryIndexStats; import org.apache.hudi.client.SparkRDDWriteClient; @@ -42,6 +41,8 @@ import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; import org.apache.hudi.common.model.SerializableIndexedRecord; import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaField; +import org.apache.hudi.common.schema.HoodieSchemaUtils; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.cdc.HoodieCDCSupplementalLoggingMode; @@ -66,7 +67,6 @@ import org.apache.hudi.table.HoodieSparkCopyOnWriteTable; import org.apache.hudi.table.HoodieSparkTable; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.apache.spark.api.java.JavaRDD; @@ -384,7 +384,7 @@ private void testFGReaderBasedMergeHandleInsertUpsertDeleteInternal(String merge // validate event time metadata if enabled if (validateEventTimeMetadata) { List records = new ArrayList<>(inputAndExpectedDataSet.getExpectedRecordsMap().values()); - validateEventTimeMetadata(writeStatus, writerProps.get("hoodie.payload.event.time.field").toString(), AVRO_SCHEMA, config, properties, records); + validateEventTimeMetadata(writeStatus, writerProps.get("hoodie.payload.event.time.field").toString(), HOODIE_SCHEMA, config, properties, records); } else { validateEventTimeMetadataNotSet(writeStatus); } @@ -428,7 +428,7 @@ private void validateEventTimeMetadataNotSet(WriteStatus writeStatus) { assertNull(writeStatus.getStat().getMaxEventTime()); } - private void validateEventTimeMetadata(WriteStatus writeStatus, String eventTimeFieldName, Schema schema, HoodieWriteConfig config, + private void validateEventTimeMetadata(WriteStatus writeStatus, String eventTimeFieldName, HoodieSchema schema, HoodieWriteConfig config, TypedProperties props, List records) { long actualMinEventTime = writeStatus.getStat().getMinEventTime(); long actualMaxEventTime = writeStatus.getStat().getMaxEventTime(); @@ -439,13 +439,13 @@ private void validateEventTimeMetadata(WriteStatus writeStatus, String eventTime // Append event_time. records.forEach(record -> { - Object eventTimeValue = record.getColumnValueAsJava(schema, eventTimeFieldName, props); + Object eventTimeValue = record.getColumnValueAsJava(schema.toAvroSchema(), eventTimeFieldName, props); if (eventTimeValue != null) { // Append event_time. - Option field = AvroSchemaUtils.findNestedField(schema, eventTimeFieldName); + Option field = HoodieSchemaUtils.findNestedField(schema, eventTimeFieldName); // Field should definitely exist. eventTimeValue = record.convertColumnValueForLogicalType( - field.get().schema(), eventTimeValue, keepConsistentLogicalTimestamp); + field.get().schema().toAvroSchema(), eventTimeValue, keepConsistentLogicalTimestamp); int length = eventTimeValue.toString().length(); Long millisEventTime = null; if (length == 10) { diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala index 748ea5b7e6b9a..eeceb209f0fc2 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala @@ -119,7 +119,7 @@ class TestHoodieFileGroupReaderOnSpark extends TestHoodieFileGroupReaderBase[Int firstCommit: Boolean, options: util.Map[String, String], schemaStr: String): Unit = { - val schema = new Schema.Parser().parse(schemaStr) + val schema = HoodieSchema.parse(schemaStr) val genericRecords = spark.sparkContext.parallelize((hoodieRecordsToIndexedRecords(recordList, schema) .stream().map[GenericRecord](entry => entry.getValue.asInstanceOf[GenericRecord]) .collect(Collectors.toList[GenericRecord])).asScala.toSeq, 2) diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/KafkaOffsetPostProcessor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/KafkaOffsetPostProcessor.java index b7baf8efea62d..b93f73ad40051 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/KafkaOffsetPostProcessor.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/KafkaOffsetPostProcessor.java @@ -18,6 +18,7 @@ package org.apache.hudi.utilities.schema; +import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.config.TypedProperties; @@ -33,7 +34,6 @@ import java.util.Set; import java.util.stream.Collectors; -import static org.apache.hudi.avro.AvroSchemaUtils.createNullableSchema; import static org.apache.hudi.common.util.ConfigUtils.getBooleanWithAltKeys; /** @@ -62,6 +62,7 @@ public KafkaOffsetPostProcessor(TypedProperties props, JavaSparkContext jssc) { } @Override + @Deprecated public Schema processSchema(Schema schema) { // this method adds kafka offset fields namely source offset, partition, timestamp and kafka message key to the schema of the batch. List fieldList = schema.getFields(); @@ -84,12 +85,11 @@ public Schema processSchema(Schema schema) { newFieldList.add(new Schema.Field(KAFKA_SOURCE_TIMESTAMP_COLUMN, Schema.create(Schema.Type.LONG), "timestamp column", 0)); } if (!fieldNames.contains(KAFKA_SOURCE_KEY_COLUMN)) { - newFieldList.add(new Schema.Field(KAFKA_SOURCE_KEY_COLUMN, createNullableSchema(Schema.Type.STRING), "kafka key column", JsonProperties.NULL_VALUE)); + newFieldList.add(new Schema.Field(KAFKA_SOURCE_KEY_COLUMN, AvroSchemaUtils.createNullableSchema(Schema.Type.STRING), "kafka key column", JsonProperties.NULL_VALUE)); } return Schema.createRecord(schema.getName() + "_processed", schema.getDoc(), schema.getNamespace(), false, newFieldList); } catch (Exception e) { throw new HoodieSchemaException("Kafka offset post processor failed with schema: " + schema, e); } - } } \ No newline at end of file diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java index d3f597143249e..dc5513d1fdaba 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java @@ -26,6 +26,7 @@ import org.apache.hudi.HoodieSchemaUtils; import org.apache.hudi.HoodieSparkSqlWriter; import org.apache.hudi.HoodieSparkUtils; +import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.callback.common.WriteStatusValidator; import org.apache.hudi.client.HoodieWriteResult; @@ -147,7 +148,6 @@ import scala.Tuple2; import static org.apache.hudi.DataSourceUtils.createUserDefinedBulkInsertPartitioner; -import static org.apache.hudi.avro.AvroSchemaUtils.getAvroRecordQualifiedName; import static org.apache.hudi.common.table.HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE; import static org.apache.hudi.common.table.HoodieTableConfig.TIMELINE_HISTORY_PATH; import static org.apache.hudi.common.table.HoodieTableConfig.URL_ENCODE_PARTITIONING; @@ -729,7 +729,7 @@ Pair fetchNextBatchFromSource(Option resumeChec // Deduce proper target (writer's) schema for the input dataset, reconciling its // schema w/ the table's one HoodieSchema incomingSchema = transformed.map(df -> - HoodieSchema.fromAvroSchema(AvroConversionUtils.convertStructTypeToAvroSchema(df.schema(), getAvroRecordQualifiedName(cfg.targetTableName)))) + HoodieSchema.fromAvroSchema(AvroConversionUtils.convertStructTypeToAvroSchema(df.schema(), AvroSchemaUtils.getAvroRecordQualifiedName(cfg.targetTableName)))) .orElseGet(dataAndCheckpoint.getSchemaProvider()::getTargetHoodieSchema); schemaProvider = getDeducedSchemaProvider(incomingSchema, dataAndCheckpoint.getSchemaProvider(), metaClient);