diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java index 4549050d9a3e7..a469ce23d00f0 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java @@ -31,6 +31,7 @@ import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; +import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.log.HoodieLogFormat; @@ -217,13 +218,13 @@ storage, new StoragePath(logFilePathPattern)).stream() checkArgument(logFilePaths.size() > 0, "There is no log file"); // TODO : readerSchema can change across blocks/log files, fix this inside Scanner - Schema readerSchema = null; + HoodieSchema readerSchema = null; // get schema from last log file for (int i = logFilePaths.size() - 1; i >= 0; i--) { Schema schema = TableSchemaResolver.readSchemaFromLogFile( storage, new StoragePath(logFilePaths.get(i))); if (schema != null) { - readerSchema = schema; + readerSchema = HoodieSchema.fromAvroSchema(schema); break; } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java index d95b6a451719e..edbe4f6aa8c23 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java @@ -367,8 +367,8 @@ private static HoodieData> getExistingRecords( .withHoodieTableMetaClient(metaClient) .withLatestCommitTime(instantTime.get()) .withFileSlice(fileSlice) - .withDataSchema(dataSchema.toAvroSchema()) - .withRequestedSchema(dataSchema.toAvroSchema()) + .withDataSchema(dataSchema) + .withRequestedSchema(dataSchema) .withInternalSchema(internalSchemaOption) .withProps(metaClient.getTableConfig().getProps()) .withEnableOptimizedLogBlockScan(config.enableOptimizedLogBlocksScan()) @@ -555,7 +555,7 @@ public static HoodieData> mergeForPartitionUpdatesAndDeletio readerContext.getMergeMode(), false, readerContext.getRecordMerger(), - writerSchema.toAvroSchema(), + writerSchema, Option.ofNullable(Pair.of(hoodieTable.getMetaClient().getTableConfig().getPayloadClass(), hoodieTable.getConfig().getPayloadClass())), properties, hoodieTable.getMetaClient().getTableConfig().getPartialUpdateMode()); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/FileGroupReaderBasedAppendHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/FileGroupReaderBasedAppendHandle.java index 66758d71c3629..ac8cff79e18cf 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/FileGroupReaderBasedAppendHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/FileGroupReaderBasedAppendHandle.java @@ -84,8 +84,8 @@ public void doAppend() { config.getBasePath(), operation.getPartitionPath()), logFileName))); // Initializes the record iterator, log compaction requires writing the deletes into the delete block of the resulting log file. try (HoodieFileGroupReader fileGroupReader = HoodieFileGroupReader.newBuilder().withReaderContext(readerContext).withHoodieTableMetaClient(hoodieTable.getMetaClient()) - .withLatestCommitTime(instantTime).withPartitionPath(partitionPath).withLogFiles(logFiles).withBaseFileOption(Option.empty()).withDataSchema(writeSchemaWithMetaFields.toAvroSchema()) - .withRequestedSchema(writeSchemaWithMetaFields.toAvroSchema()).withInternalSchema(internalSchemaOption).withProps(props).withEmitDelete(true) + .withLatestCommitTime(instantTime).withPartitionPath(partitionPath).withLogFiles(logFiles).withBaseFileOption(Option.empty()).withDataSchema(writeSchemaWithMetaFields) + .withRequestedSchema(writeSchemaWithMetaFields).withInternalSchema(internalSchemaOption).withProps(props).withEmitDelete(true) .withShouldUseRecordPosition(usePosition).withSortOutput(hoodieTable.requireSortedRecords()) // instead of using config.enableOptimizedLogBlocksScan(), we set to true as log compaction blocks only supported in scanV2 .withEnableOptimizedLogBlockScan(true).build()) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/FileGroupReaderBasedMergeHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/FileGroupReaderBasedMergeHandle.java index 22587965adb85..37d4b7a73a543 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/FileGroupReaderBasedMergeHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/FileGroupReaderBasedMergeHandle.java @@ -317,7 +317,7 @@ private HoodieFileGroupReader getFileGroupReader(boolean usePosition, Option< Option> logFileStreamOpt, Iterator> incomingRecordsItr) { HoodieFileGroupReader.Builder fileGroupBuilder = HoodieFileGroupReader.newBuilder().withReaderContext(readerContext).withHoodieTableMetaClient(hoodieTable.getMetaClient()) .withLatestCommitTime(maxInstantTime).withPartitionPath(partitionPath).withBaseFileOption(Option.ofNullable(baseFileToMerge)) - .withDataSchema(writeSchemaWithMetaFields.toAvroSchema()).withRequestedSchema(writeSchemaWithMetaFields.toAvroSchema()) + .withDataSchema(writeSchemaWithMetaFields).withRequestedSchema(writeSchemaWithMetaFields) .withInternalSchema(internalSchemaOption).withProps(props) .withShouldUseRecordPosition(usePosition).withSortOutput(hoodieTable.requireSortedRecords()) .withFileGroupUpdateCallback(createCallback()); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index ea7bfb162b1ca..d7bb9117e9d97 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -18,8 +18,6 @@ package org.apache.hudi.metadata; -import org.apache.hudi.avro.AvroSchemaCache; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieIndexPartitionInfo; import org.apache.hudi.avro.model.HoodieIndexPlan; @@ -53,6 +51,8 @@ import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaCache; +import org.apache.hudi.common.schema.HoodieSchemaUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; @@ -89,7 +89,6 @@ import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.util.Lazy; -import org.apache.avro.Schema; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -115,8 +114,8 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; -import static org.apache.hudi.avro.HoodieAvroUtils.getRecordKeySchema; import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_POPULATE_META_FIELDS; +import static org.apache.hudi.common.schema.HoodieSchemaUtils.getRecordKeySchema; import static org.apache.hudi.common.table.HoodieTableConfig.TIMELINE_HISTORY_PATH; import static org.apache.hudi.common.table.timeline.HoodieInstant.State.REQUESTED; import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION; @@ -872,9 +871,9 @@ private static HoodieData readRecordKeysFromFileSliceSnapshot( final FileSlice fileSlice = partitionAndFileSlice.getValue(); final String fileId = fileSlice.getFileId(); HoodieReaderContext readerContext = readerContextFactory.getContext(); - Schema dataSchema = AvroSchemaCache.intern(HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(dataWriteConfig.getWriteSchema()), dataWriteConfig.allowOperationMetadataField())); - Schema requestedSchema = metaClient.getTableConfig().populateMetaFields() ? getRecordKeySchema() - : HoodieAvroUtils.projectSchema(dataSchema, Arrays.asList(metaClient.getTableConfig().getRecordKeyFields().orElse(new String[0]))); + HoodieSchema dataSchema = HoodieSchemaCache.intern(HoodieSchemaUtils.addMetadataFields(HoodieSchema.parse(dataWriteConfig.getWriteSchema()), dataWriteConfig.allowOperationMetadataField())); + HoodieSchema requestedSchema = metaClient.getTableConfig().populateMetaFields() ? getRecordKeySchema() + : HoodieSchemaUtils.projectSchema(dataSchema, Arrays.asList(metaClient.getTableConfig().getRecordKeyFields().orElse(new String[0]))); Option internalSchemaOption = SerDeHelper.fromJson(dataWriteConfig.getInternalSchema()); HoodieFileGroupReader fileGroupReader = HoodieFileGroupReader.newBuilder() .withReaderContext(readerContext) @@ -890,7 +889,7 @@ private static HoodieData readRecordKeysFromFileSliceSnapshot( .build(); String baseFileInstantTime = fileSlice.getBaseInstantTime(); return new CloseableMappingIterator<>(fileGroupReader.getClosableIterator(), record -> { - String recordKey = readerContext.getRecordContext().getRecordKey(record, HoodieSchema.fromAvroSchema(requestedSchema)); + String recordKey = readerContext.getRecordContext().getRecordKey(record, requestedSchema); return HoodieMetadataPayload.createRecordIndexUpdate(recordKey, partition, fileId, baseFileInstantTime, 0); }); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/SecondaryIndexRecordGenerationUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/SecondaryIndexRecordGenerationUtils.java index ea0a904cf237d..df52b99a90b66 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/SecondaryIndexRecordGenerationUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/SecondaryIndexRecordGenerationUtils.java @@ -18,7 +18,6 @@ package org.apache.hudi.metadata; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.data.HoodieData; @@ -35,6 +34,7 @@ import org.apache.hudi.common.model.HoodieRecord; 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.table.TableSchemaResolver; import org.apache.hudi.common.table.read.HoodieFileGroupReader; @@ -288,14 +288,14 @@ private static ClosableIterator> createSecondaryIndexRe TypedProperties props, boolean allowInflightInstants) throws IOException { String secondaryKeyField = indexDefinition.getSourceFieldsKey(); - Schema requestedSchema = getRequestedSchemaForSecondaryIndex(metaClient, tableSchema, secondaryKeyField); + HoodieSchema requestedSchema = getRequestedSchemaForSecondaryIndex(metaClient, tableSchema, secondaryKeyField); HoodieFileGroupReader fileGroupReader = HoodieFileGroupReader.newBuilder() .withReaderContext(readerContext) .withFileSlice(fileSlice) .withHoodieTableMetaClient(metaClient) .withProps(props) .withLatestCommitTime(instantTime) - .withDataSchema(tableSchema.toAvroSchema()) + .withDataSchema(tableSchema) .withRequestedSchema(requestedSchema) .withAllowInflightInstants(allowInflightInstants) .build(); @@ -318,9 +318,9 @@ public boolean hasNext() { while (recordIterator.hasNext()) { T record = recordIterator.next(); - Object secondaryKey = readerContext.getRecordContext().getValue(record, HoodieSchema.fromAvroSchema(requestedSchema), secondaryKeyField); + Object secondaryKey = readerContext.getRecordContext().getValue(record, requestedSchema, secondaryKeyField); nextValidRecord = Pair.of( - readerContext.getRecordContext().getRecordKey(record, HoodieSchema.fromAvroSchema(requestedSchema)), + readerContext.getRecordContext().getRecordKey(record, requestedSchema), secondaryKey == null ? null : secondaryKey.toString() ); return true; @@ -342,15 +342,15 @@ public Pair next() { }; } - private static Schema getRequestedSchemaForSecondaryIndex(HoodieTableMetaClient metaClient, HoodieSchema tableSchema, String secondaryKeyField) { + private static HoodieSchema getRequestedSchemaForSecondaryIndex(HoodieTableMetaClient metaClient, HoodieSchema tableSchema, String secondaryKeyField) { String[] recordKeyFields; - if (tableSchema.getField(RECORD_KEY_METADATA_FIELD) != null) { + if (tableSchema.getField(RECORD_KEY_METADATA_FIELD).isPresent()) { recordKeyFields = new String[] {RECORD_KEY_METADATA_FIELD}; } else { recordKeyFields = metaClient.getTableConfig().getRecordKeyFields().orElse(new String[0]); } String[] projectionFields = Arrays.copyOf(recordKeyFields, recordKeyFields.length + 1); projectionFields[recordKeyFields.length] = secondaryKeyField; - return HoodieAvroUtils.projectSchema(tableSchema.toAvroSchema(), Arrays.asList(projectionFields)); + return HoodieSchemaUtils.projectSchema(tableSchema, Arrays.asList(projectionFields)); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringExecutionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringExecutionStrategy.java index 3a98d8b3843ed..46b30bcf80868 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringExecutionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringExecutionStrategy.java @@ -18,7 +18,6 @@ package org.apache.hudi.table.action.cluster.strategy; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.HoodieEngineContext; @@ -31,6 +30,8 @@ import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; +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.read.HoodieFileGroupReader; import org.apache.hudi.common.util.Option; @@ -67,14 +68,14 @@ public abstract class ClusteringExecutionStrategy implements Seriali private final transient HoodieEngineContext engineContext; protected HoodieWriteConfig writeConfig; protected final HoodieRecordType recordType; - protected final Schema readerSchemaWithMetaFields; + protected final HoodieSchema readerSchemaWithMetaFields; public ClusteringExecutionStrategy(HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig writeConfig) { this.writeConfig = writeConfig; this.hoodieTable = table; this.engineContext = engineContext; this.recordType = table.getConfig().getRecordMerger().getRecordType(); - this.readerSchemaWithMetaFields = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(writeConfig.getSchema())); + this.readerSchemaWithMetaFields = HoodieSchemaUtils.addMetadataFields(HoodieSchema.parse(writeConfig.getSchema())); } /** @@ -146,7 +147,7 @@ protected FileSlice clusteringOperationToFileSlice(String basePath, ClusteringOp return fileSlice; } - protected static HoodieFileGroupReader getFileGroupReader(HoodieTableMetaClient metaClient, FileSlice fileSlice, Schema readerSchema, Option internalSchemaOption, + protected static HoodieFileGroupReader getFileGroupReader(HoodieTableMetaClient metaClient, FileSlice fileSlice, HoodieSchema readerSchema, Option internalSchemaOption, ReaderContextFactory readerContextFactory, String instantTime, TypedProperties properties, boolean usePosition, boolean enableLogBlocksScan) { HoodieReaderContext readerContext = readerContextFactory.getContext(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseWriteHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseWriteHelper.java index 6b92250267956..295e8b282ab02 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseWriteHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseWriteHelper.java @@ -18,7 +18,6 @@ package org.apache.hudi.table.action.commit; -import org.apache.hudi.avro.AvroSchemaCache; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.engine.HoodieReaderContext; @@ -28,6 +27,7 @@ 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.HoodieSchemaCache; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.read.BufferedRecord; import org.apache.hudi.common.table.read.BufferedRecordMerger; @@ -44,8 +44,6 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.avro.Schema; - import java.io.IOException; import java.util.List; @@ -109,13 +107,13 @@ public I deduplicateRecords(I records, HoodieTable table, int parall HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig(); readerContext.initRecordMergerForIngestion(table.getConfig().getProps()); List orderingFieldNames = HoodieRecordUtils.getOrderingFieldNames(readerContext.getMergeMode(), table.getMetaClient()); - Schema recordSchema; + HoodieSchema recordSchema; if (StringUtils.nonEmpty(table.getConfig().getPartialUpdateSchema())) { - recordSchema = new Schema.Parser().parse(table.getConfig().getPartialUpdateSchema()); + recordSchema = HoodieSchema.parse(table.getConfig().getPartialUpdateSchema()); } else { - recordSchema = new Schema.Parser().parse(table.getConfig().getWriteSchema()); + recordSchema = HoodieSchema.parse(table.getConfig().getWriteSchema()); } - recordSchema = AvroSchemaCache.intern(recordSchema); + recordSchema = HoodieSchemaCache.intern(recordSchema); TypedProperties mergedProperties = readerContext.getMergeProps(table.getConfig().getProps()); BufferedRecordMerger bufferedRecordMerger = BufferedRecordMergerFactory.create( readerContext, diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/HoodieWriterClientTestHarness.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/HoodieWriterClientTestHarness.java index 9d0684c574f8a..6e41eec667057 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/HoodieWriterClientTestHarness.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/HoodieWriterClientTestHarness.java @@ -46,6 +46,7 @@ import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.model.WriteConcurrencyMode; import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableVersion; @@ -94,7 +95,6 @@ import org.apache.hudi.table.upgrade.UpgradeDowngrade; import org.apache.hudi.testutils.MetadataMergeWriteStatus; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.jetbrains.annotations.NotNull; @@ -573,7 +573,7 @@ protected List> dedupForCopyOnWriteStorage(HoodieWri false, Option.ofNullable(writeClient.getConfig().getRecordMerger()), Option.ofNullable(writeClient.getConfig().getPayloadClass()), - new Schema.Parser().parse(writeClient.getConfig().getSchema()), + HoodieSchema.parse(writeClient.getConfig().getSchema()), writeClient.getConfig().getProps(), metaClient.getTableConfig().getPartialUpdateMode()); HoodieData> dedupedRecsRdd = diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/format/FlinkRecordContext.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/format/FlinkRecordContext.java index 1ba5ef32d4f72..927ff5093b860 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/format/FlinkRecordContext.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/format/FlinkRecordContext.java @@ -203,7 +203,7 @@ public UnaryOperator projectRecord(HoodieSchema from, HoodieSchema to, return rowProjection::project; } - public void initOrderingValueConverter(Schema dataSchema, List orderingFieldNames) { + public void initOrderingValueConverter(HoodieSchema dataSchema, List orderingFieldNames) { this.orderingValueConverter = OrderingValueEngineTypeConverter.create(dataSchema, orderingFieldNames, utcTimezone); } } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/util/OrderingValueEngineTypeConverter.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/util/OrderingValueEngineTypeConverter.java index 8c35adaea1e9c..0d0bbd1fd5a46 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/util/OrderingValueEngineTypeConverter.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/util/OrderingValueEngineTypeConverter.java @@ -18,12 +18,14 @@ package org.apache.hudi.util; -import org.apache.hudi.avro.AvroSchemaUtils; +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.util.Option; import org.apache.hudi.common.util.VisibleForTesting; import org.apache.hudi.common.util.collection.ArrayComparable; +import org.apache.hudi.common.util.collection.Pair; -import org.apache.avro.Schema; import org.apache.flink.table.types.DataType; import java.util.Collections; @@ -36,7 +38,7 @@ */ public class OrderingValueEngineTypeConverter { private final List> converters; - private OrderingValueEngineTypeConverter(Schema dataSchema, List orderingFieldNames, boolean utcTimezone) { + private OrderingValueEngineTypeConverter(HoodieSchema dataSchema, List orderingFieldNames, boolean utcTimezone) { this.converters = createConverters(dataSchema, orderingFieldNames, utcTimezone); } @@ -46,16 +48,16 @@ public Comparable convert(Comparable value) { : this.converters.get(0).apply(value); } - public static List> createConverters(Schema dataSchema, List orderingFieldNames, boolean utcTimezone) { + public static List> createConverters(HoodieSchema dataSchema, List orderingFieldNames, boolean utcTimezone) { if (orderingFieldNames.isEmpty()) { return Collections.singletonList(Function.identity()); } return orderingFieldNames.stream().map(f -> { - Option fieldSchemaOpt = AvroSchemaUtils.findNestedFieldSchema(dataSchema, f, true); + Option fieldSchemaOpt = HoodieSchemaUtils.getNestedField(dataSchema, f).map(Pair::getValue).map(HoodieSchemaField::getNonNullSchema); if (fieldSchemaOpt.isEmpty()) { return Function.identity(); } else { - DataType fieldType = AvroSchemaConverter.convertToDataType(fieldSchemaOpt.get()); + DataType fieldType = HoodieSchemaConverter.convertToDataType(fieldSchemaOpt.get()); return RowDataUtils.flinkValFunc(fieldType.getLogicalType(), utcTimezone); } }).collect(Collectors.toList()); @@ -66,7 +68,7 @@ public List> getConverters() { return converters; } - public static OrderingValueEngineTypeConverter create(Schema dataSchema, List orderingFieldNames, boolean utcTimezone) { + public static OrderingValueEngineTypeConverter create(HoodieSchema dataSchema, List orderingFieldNames, boolean utcTimezone) { return new OrderingValueEngineTypeConverter(dataSchema, orderingFieldNames, utcTimezone); } } diff --git a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/util/TestOrderingValueEngineTypeConverter.java b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/util/TestOrderingValueEngineTypeConverter.java index 9cfd458bf74ef..a1ad3d70f37a0 100644 --- a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/util/TestOrderingValueEngineTypeConverter.java +++ b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/util/TestOrderingValueEngineTypeConverter.java @@ -18,11 +18,11 @@ package org.apache.hudi.util; +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.util.collection.ArrayComparable; -import org.apache.avro.LogicalTypes; -import org.apache.avro.Schema; -import org.apache.avro.SchemaBuilder; import org.apache.flink.table.data.DecimalData; import org.apache.flink.table.data.TimestampData; import org.apache.flink.table.data.binary.BinaryStringData; @@ -45,12 +45,11 @@ public class TestOrderingValueEngineTypeConverter { @Test public void testCreateWithMultipleOrderingFields() { - Schema schema = SchemaBuilder - .record("test").fields() - .name("id").type().stringType().noDefault() - .name("timestamp").type().longType().noDefault() - .name("amount").type().doubleType().noDefault() - .endRecord(); + HoodieSchema schema = HoodieSchema.createRecord("test", null, null, Arrays.asList( + HoodieSchemaField.of("id", HoodieSchema.create(HoodieSchemaType.STRING), null, null), + HoodieSchemaField.of("timestamp", HoodieSchema.create(HoodieSchemaType.LONG), null, null), + HoodieSchemaField.of("amount", HoodieSchema.create(HoodieSchemaType.DOUBLE), null, null) + )); List orderingFieldNames = Arrays.asList("id", "timestamp"); boolean utcTimezone = true; @@ -63,10 +62,8 @@ public void testCreateWithMultipleOrderingFields() { @Test public void testConvertWithSingleValue() { - Schema schema = SchemaBuilder - .record("test").fields() - .name("name").type().stringType().noDefault() - .endRecord(); + HoodieSchema schema = HoodieSchema.createRecord("test", null, null, Collections.singletonList( + HoodieSchemaField.of("name", HoodieSchema.create(HoodieSchemaType.STRING), null, null))); List orderingFieldNames = Collections.singletonList("name"); boolean utcTimezone = true; @@ -83,13 +80,8 @@ public void testConvertWithSingleValue() { @ParameterizedTest @ValueSource(booleans = {true, false}) public void testConvertWithTimestampValue(boolean utcTimezone) { - Schema tsSchema = LogicalTypes.timestampMillis() - .addToSchema(Schema.create(Schema.Type.LONG)); - - Schema schema = SchemaBuilder - .record("test").fields() - .name("timestamp").type(tsSchema).noDefault() - .endRecord(); + HoodieSchema schema = HoodieSchema.createRecord("test", null, null, Collections.singletonList( + HoodieSchemaField.of("timestamp", HoodieSchema.createTimestampMillis(), null, null))); List orderingFieldNames = Collections.singletonList("timestamp"); @@ -105,14 +97,9 @@ public void testConvertWithTimestampValue(boolean utcTimezone) { @Test public void testConvertWithArrayComparable() { - Schema tsSchema = LogicalTypes.timestampMillis() - .addToSchema(Schema.create(Schema.Type.LONG)); - - Schema schema = SchemaBuilder - .record("test").fields() - .name("name").type().stringType().noDefault() - .name("timestamp").type(tsSchema).noDefault() - .endRecord(); + HoodieSchema schema = HoodieSchema.createRecord("test", null, null, Arrays.asList( + HoodieSchemaField.of("name", HoodieSchema.create(HoodieSchemaType.STRING), null, null), + HoodieSchemaField.of("timestamp", HoodieSchema.createTimestampMillis(), null, null))); List orderingFieldNames = Arrays.asList("name", "timestamp"); boolean utcTimezone = true; @@ -135,11 +122,9 @@ public void testConvertWithArrayComparable() { @Test public void testCreateConverters() { - Schema schema = SchemaBuilder - .record("test").fields() - .name("id").type().stringType().noDefault() - .name("age").type().intType().noDefault() - .endRecord(); + HoodieSchema schema = HoodieSchema.createRecord("test", null, null, Arrays.asList( + HoodieSchemaField.of("id", HoodieSchema.create(HoodieSchemaType.STRING), null, null), + HoodieSchemaField.of("age", HoodieSchema.create(HoodieSchemaType.INT), null, null))); List orderingFieldNames = Arrays.asList("id", "age"); boolean utcTimezone = true; @@ -161,13 +146,8 @@ public void testCreateConverters() { @Test public void testConvertWithDecimalValue() { - Schema decimalSchema = LogicalTypes.decimal(10, 2) - .addToSchema(Schema.create(Schema.Type.BYTES)); - - Schema schema = SchemaBuilder - .record("test").fields() - .name("price").type(decimalSchema).noDefault() - .endRecord(); + HoodieSchema schema = HoodieSchema.createRecord("test", null, null, Collections.singletonList( + HoodieSchemaField.of("price", HoodieSchema.createDecimal(10, 2), null, null))); List orderingFieldNames = Collections.singletonList("price"); boolean utcTimezone = true; @@ -183,13 +163,8 @@ public void testConvertWithDecimalValue() { @Test public void testConvertWithDateValue() { - Schema dateSchema = LogicalTypes.date() - .addToSchema(Schema.create(Schema.Type.INT)); - - Schema schema = SchemaBuilder - .record("test").fields() - .name("birth_date").type(dateSchema).noDefault() - .endRecord(); + HoodieSchema schema = HoodieSchema.createRecord("test", null, null, Collections.singletonList( + HoodieSchemaField.of("birth_date", HoodieSchema.createDate()))); List orderingFieldNames = Collections.singletonList("birth_date"); @@ -205,12 +180,8 @@ public void testConvertWithDateValue() { @Test public void testConvertWithNullValue() { - Schema schema = SchemaBuilder - .record("test").fields() - .name("optional_field").type( - Schema.createUnion(Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.STRING)) - ).noDefault() - .endRecord(); + HoodieSchema schema = HoodieSchema.createRecord("test", null, null, Collections.singletonList( + HoodieSchemaField.of("optional_field", HoodieSchema.createNullable(HoodieSchemaType.STRING)))); List orderingFieldNames = Collections.singletonList("optional_field"); boolean utcTimezone = true; diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestJavaHoodieBackedMetadata.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestJavaHoodieBackedMetadata.java index d439189a0a63c..6fdb5b3b4757d 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestJavaHoodieBackedMetadata.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestJavaHoodieBackedMetadata.java @@ -19,7 +19,6 @@ package org.apache.hudi.client; import org.apache.hudi.avro.HoodieAvroReaderContext; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieMetadataColumnStats; import org.apache.hudi.avro.model.HoodieMetadataRecord; @@ -50,6 +49,8 @@ import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.TableServiceType; import org.apache.hudi.common.model.WriteConcurrencyMode; +import org.apache.hudi.common.schema.HoodieSchema; +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; @@ -878,7 +879,7 @@ private void verifyMetadataRecordKeyExcludeFromPayloadLogFiles(HoodieTable table verifyMetadataRawRecords(table, logFiles, enableMetaFields); // Verify the in-memory materialized and merged records - verifyMetadataMergedRecords(metadataMetaClient, logFiles, latestCommitTimestamp, enableMetaFields); + verifyMetadataMergedRecords(metadataMetaClient, logFiles, latestCommitTimestamp); } /** @@ -941,14 +942,10 @@ private void verifyMetadataRawRecords(HoodieTable table, List log * @param metadataMetaClient - Metadata table meta client * @param logFiles - Metadata table log files * @param latestCommitTimestamp - * @param enableMetaFields - Enable meta fields */ private void verifyMetadataMergedRecords(HoodieTableMetaClient metadataMetaClient, List logFiles, - String latestCommitTimestamp, boolean enableMetaFields) { - Schema schema = HoodieAvroUtils.addMetadataFields(HoodieMetadataRecord.getClassSchema()); - if (enableMetaFields) { - schema = HoodieAvroUtils.addMetadataFields(schema); - } + String latestCommitTimestamp) { + HoodieSchema schema = HoodieSchemaUtils.addMetadataFields(HoodieSchema.fromAvroSchema(HoodieMetadataRecord.getClassSchema())); HoodieAvroReaderContext readerContext = new HoodieAvroReaderContext(metadataMetaClient.getStorageConf(), metadataMetaClient.getTableConfig(), Option.empty(), Option.empty(), new TypedProperties()); HoodieFileGroupReader fileGroupReader = HoodieFileGroupReader.newBuilder() diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnJava.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnJava.java index bdcf4f54009f4..ee6b2d823a333 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnJava.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnJava.java @@ -22,13 +22,13 @@ import org.apache.hudi.avro.ConvertingGenericData; import org.apache.hudi.avro.HoodieAvroReaderContext; import org.apache.hudi.common.engine.HoodieReaderContext; +import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; import org.apache.hudi.storage.StorageConfiguration; import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration; -import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -44,18 +44,18 @@ public StorageConfiguration getStorageConf() { @Override public HoodieReaderContext getHoodieReaderContext( - String tablePath, Schema avroSchema, StorageConfiguration storageConf, HoodieTableMetaClient metaClient) { + String tablePath, HoodieSchema schema, StorageConfiguration storageConf, HoodieTableMetaClient metaClient) { return new HoodieAvroReaderContext(storageConf, metaClient.getTableConfig(), Option.empty(), Option.empty()); } @Override - public void assertRecordsEqual(Schema schema, IndexedRecord expected, IndexedRecord actual) { + public void assertRecordsEqual(HoodieSchema schema, IndexedRecord expected, IndexedRecord actual) { assertEquals(expected, actual); } @Override - public void assertRecordMatchesSchema(Schema schema, IndexedRecord record) { - assertTrue(ConvertingGenericData.INSTANCE.validate(schema, record)); + public void assertRecordMatchesSchema(HoodieSchema schema, IndexedRecord record) { + assertTrue(ConvertingGenericData.INSTANCE.validate(schema.toAvroSchema(), record)); } @Override diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/hadoop/TestHoodieFileGroupReaderOnHive.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/hadoop/TestHoodieFileGroupReaderOnHive.java index ec14e7dc6707a..a7e5ca3217f88 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/hadoop/TestHoodieFileGroupReaderOnHive.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/hadoop/TestHoodieFileGroupReaderOnHive.java @@ -22,6 +22,8 @@ import org.apache.hudi.common.config.HoodieMemoryConfig; import org.apache.hudi.common.engine.HoodieReaderContext; import org.apache.hudi.common.model.HoodieFileFormat; +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.read.HoodieFileGroupReaderOnJavaTestBase; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; @@ -32,7 +34,6 @@ import org.apache.hudi.storage.hadoop.HoodieHadoopStorage; import org.apache.hudi.testutils.ArrayWritableTestUtil; -import org.apache.avro.Schema; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; @@ -93,22 +94,22 @@ public StorageConfiguration getStorageConf() { } @Override - public HoodieReaderContext getHoodieReaderContext(String tablePath, Schema avroSchema, StorageConfiguration storageConf, HoodieTableMetaClient metaClient) { + public HoodieReaderContext getHoodieReaderContext(String tablePath, HoodieSchema schema, StorageConfiguration storageConf, HoodieTableMetaClient metaClient) { HoodieFileGroupReaderBasedRecordReader.HiveReaderCreator readerCreator = (inputSplit, jobConf, dataSchema) -> new MapredParquetInputFormat().getRecordReader(inputSplit, jobConf, null); JobConf jobConf = new JobConf(storageConf.unwrapAs(Configuration.class)); - setupJobconf(jobConf, avroSchema); + setupJobconf(jobConf, schema); return new HiveHoodieReaderContext(readerCreator, - getStoredPartitionFieldNames(new JobConf(storageConf.unwrapAs(Configuration.class)), avroSchema), storageConf, metaClient.getTableConfig()); + getStoredPartitionFieldNames(new JobConf(storageConf.unwrapAs(Configuration.class)), schema), storageConf, metaClient.getTableConfig()); } @Override - public void assertRecordsEqual(Schema schema, ArrayWritable expected, ArrayWritable actual) { - ArrayWritableTestUtil.assertArrayWritableEqual(schema, expected, actual, false); + public void assertRecordsEqual(HoodieSchema schema, ArrayWritable expected, ArrayWritable actual) { + ArrayWritableTestUtil.assertArrayWritableEqual(schema.toAvroSchema(), expected, actual, false); } @Override - public void assertRecordMatchesSchema(Schema schema, ArrayWritable record) { - ArrayWritableTestUtil.assertArrayWritableMatchesSchema(schema, record); + public void assertRecordMatchesSchema(HoodieSchema schema, ArrayWritable record) { + ArrayWritableTestUtil.assertArrayWritableMatchesSchema(schema.toAvroSchema(), record); } @Override @@ -116,24 +117,24 @@ public HoodieTestDataGenerator.SchemaEvolutionConfigs getSchemaEvolutionConfigs( return new HoodieTestDataGenerator.SchemaEvolutionConfigs(); } - private void setupJobconf(JobConf jobConf, Schema schema) { - List fields = schema.getFields(); + private void setupJobconf(JobConf jobConf, HoodieSchema schema) { + List fields = schema.getFields(); setHiveColumnNameProps(fields, jobConf, USE_FAKE_PARTITION); try { - String columnTypes = HiveTypeUtils.generateColumnTypes(schema).stream().map(TypeInfo::getTypeName).collect(Collectors.joining(",")); + String columnTypes = HiveTypeUtils.generateColumnTypes(schema.toAvroSchema()).stream().map(TypeInfo::getTypeName).collect(Collectors.joining(",")); jobConf.set("columns.types", columnTypes + ",string"); } catch (SerDeException e) { throw new RuntimeException(e); } } - private void setHiveColumnNameProps(List fields, JobConf jobConf, boolean isPartitioned) { - String names = fields.stream().map(Schema.Field::name).map(s -> s.toLowerCase(Locale.ROOT)).collect(Collectors.joining(",")); + private void setHiveColumnNameProps(List fields, JobConf jobConf, boolean isPartitioned) { + String names = fields.stream().map(HoodieSchemaField::name).map(s -> s.toLowerCase(Locale.ROOT)).collect(Collectors.joining(",")); String positions = fields.stream().map(f -> String.valueOf(f.pos())).collect(Collectors.joining(",")); jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names); jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, positions); - String hiveOrderedColumnNames = fields.stream().map(Schema.Field::name) + String hiveOrderedColumnNames = fields.stream().map(HoodieSchemaField::name) .filter(name -> !name.equalsIgnoreCase(PARTITION_COLUMN)) .map(s -> s.toLowerCase(Locale.ROOT)).collect(Collectors.joining(",")); if (isPartitioned) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/DefaultSparkRecordMerger.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/DefaultSparkRecordMerger.java index 6d37a7fe8656a..dace7cb546589 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/DefaultSparkRecordMerger.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/DefaultSparkRecordMerger.java @@ -26,7 +26,6 @@ import org.apache.hudi.common.table.read.BufferedRecord; import org.apache.hudi.merge.SparkRecordMergingUtils; -import org.apache.avro.Schema; import org.apache.spark.sql.catalyst.InternalRow; import java.io.IOException; @@ -54,7 +53,7 @@ public BufferedRecord merge(BufferedRecord older, BufferedRecord ne } @Override - public BufferedRecord partialMerge(BufferedRecord older, BufferedRecord newer, Schema readerSchema, RecordContext recordContext, TypedProperties props) throws IOException { + public BufferedRecord partialMerge(BufferedRecord older, BufferedRecord newer, HoodieSchema readerSchema, RecordContext recordContext, TypedProperties props) throws IOException { if (HoodieRecordMerger.isCommitTimeOrderingDelete(older, newer)) { return newer; } @@ -64,16 +63,16 @@ public BufferedRecord partialMerge(BufferedRecord older, BufferedRecor } HoodieSchema oldSchema = recordContext.getSchemaFromBufferRecord(older); HoodieSchema newSchema = recordContext.getSchemaFromBufferRecord(newer); - return (BufferedRecord) SparkRecordMergingUtils.mergePartialRecords((BufferedRecord) newer, newSchema.toAvroSchema(), - (BufferedRecord) older, oldSchema.toAvroSchema(), readerSchema, (RecordContext) recordContext); + return (BufferedRecord) SparkRecordMergingUtils.mergePartialRecords((BufferedRecord) newer, newSchema, + (BufferedRecord) older, oldSchema, readerSchema, (RecordContext) recordContext); } else { if (newer.isDelete() || older.isDelete()) { return newer; } HoodieSchema oldSchema = recordContext.getSchemaFromBufferRecord(older); HoodieSchema newSchema = recordContext.getSchemaFromBufferRecord(newer); - return (BufferedRecord) SparkRecordMergingUtils.mergePartialRecords((BufferedRecord) older, oldSchema.toAvroSchema(), - (BufferedRecord) newer, newSchema.toAvroSchema(), readerSchema, (RecordContext) recordContext); + return (BufferedRecord) SparkRecordMergingUtils.mergePartialRecords((BufferedRecord) older, oldSchema, + (BufferedRecord) newer, newSchema, readerSchema, (RecordContext) recordContext); } } } \ No newline at end of file diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/OverwriteWithLatestSparkRecordMerger.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/OverwriteWithLatestSparkRecordMerger.java index 9f1c3123c0356..46a403464f851 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/OverwriteWithLatestSparkRecordMerger.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/OverwriteWithLatestSparkRecordMerger.java @@ -25,7 +25,6 @@ import org.apache.hudi.common.table.read.BufferedRecord; import org.apache.hudi.merge.SparkRecordMergingUtils; -import org.apache.avro.Schema; import org.apache.spark.sql.catalyst.InternalRow; import java.io.IOException; @@ -46,13 +45,13 @@ public BufferedRecord merge(BufferedRecord older, BufferedRecord ne } @Override - public BufferedRecord partialMerge(BufferedRecord older, BufferedRecord newer, Schema readerSchema, RecordContext recordContext, TypedProperties props) throws IOException { + public BufferedRecord partialMerge(BufferedRecord older, BufferedRecord newer, HoodieSchema readerSchema, RecordContext recordContext, TypedProperties props) throws IOException { if (newer.isDelete()) { return newer; } HoodieSchema oldSchema = recordContext.getSchemaFromBufferRecord(older); HoodieSchema newSchema = recordContext.getSchemaFromBufferRecord(newer); - return (BufferedRecord) SparkRecordMergingUtils.mergePartialRecords((BufferedRecord) older, oldSchema.toAvroSchema(), - (BufferedRecord) newer, newSchema.toAvroSchema(), readerSchema, (RecordContext) recordContext); + return (BufferedRecord) SparkRecordMergingUtils.mergePartialRecords((BufferedRecord) older, oldSchema, + (BufferedRecord) newer, newSchema, readerSchema, (RecordContext) recordContext); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java index 0b7cf68b019b9..d34dd06bbeb14 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java @@ -27,7 +27,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.client.utils.LazyConcatenatingIterator; -import org.apache.hudi.common.config.SerializableSchema; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; @@ -37,6 +36,8 @@ import org.apache.hudi.common.model.HoodieFileGroupId; 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.HoodieSchemaUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.read.HoodieFileGroupReader; @@ -149,7 +150,7 @@ public abstract HoodieData performClusteringWithRecordsAsRow(final final int numOutputGroups, final String instantTime, final Map strategyParams, - final Schema schema, + final HoodieSchema schema, final List fileGroupIdList, final boolean shouldPreserveHoodieMetadata, final Map extraMetadata); @@ -252,9 +253,9 @@ protected CompletableFuture> runClusteringForGroupAsyncA ExecutorService clusteringExecutorService) { return CompletableFuture.supplyAsync(() -> { JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(getEngineContext()); // incase of MIT, config.getSchema may not contain the full table schema - Schema tableSchemaWithMetaFields = null; + HoodieSchema tableSchemaWithMetaFields = null; try { - tableSchemaWithMetaFields = HoodieAvroUtils.addMetadataFields(new TableSchemaResolver(getHoodieTable().getMetaClient()).getTableAvroSchema(false), + tableSchemaWithMetaFields = HoodieSchemaUtils.addMetadataFields(new TableSchemaResolver(getHoodieTable().getMetaClient()).getTableSchema(false), getWriteConfig().allowOperationMetadataField()); } catch (Exception e) { throw new HoodieException("Failed to get table schema during clustering", e); @@ -294,7 +295,7 @@ private HoodieData> readRecordsForGroup(JavaSparkContext jsc, Ho private Dataset readRecordsForGroupAsRow(JavaSparkContext jsc, HoodieClusteringGroup clusteringGroup, String instantTime, - Schema tableSchemaWithMetaFields) { + HoodieSchema tableSchemaWithMetaFields) { List clusteringOps = clusteringGroup.getSlices().stream() .map(ClusteringOperation::create).collect(Collectors.toList()); String basePath = getWriteConfig().getBasePath(); @@ -304,23 +305,21 @@ private Dataset readRecordsForGroupAsRow(JavaSparkContext jsc, final boolean usePosition = getWriteConfig().getBooleanOrDefault(MERGE_USE_RECORD_POSITIONS); final boolean enableLogBlocksScan = getWriteConfig().enableOptimizedLogBlocksScan(); String internalSchemaStr = getWriteConfig().getInternalSchema(); - SerializableSchema serializableTableSchemaWithMetaFields = new SerializableSchema(tableSchemaWithMetaFields); // broadcast reader context. HoodieTableMetaClient metaClient = getHoodieTable().getMetaClient(); ReaderContextFactory readerContextFactory = getEngineContext().getReaderContextFactory(metaClient); - StructType sparkSchemaWithMetaFields = AvroConversionUtils.convertAvroSchemaToStructType(tableSchemaWithMetaFields); + StructType sparkSchemaWithMetaFields = AvroConversionUtils.convertAvroSchemaToStructType(tableSchemaWithMetaFields.toAvroSchema()); RDD internalRowRDD = jsc.parallelize(clusteringOps, clusteringOps.size()).flatMap(new FlatMapFunction() { @Override public Iterator call(ClusteringOperation clusteringOperation) throws Exception { FileSlice fileSlice = clusteringOperationToFileSlice(basePath, clusteringOperation); // instantiate other supporting cast - Schema readerSchema = serializableTableSchemaWithMetaFields.get(); Option internalSchemaOption = SerDeHelper.fromJson(internalSchemaStr); // instantiate FG reader - HoodieFileGroupReader fileGroupReader = getFileGroupReader(metaClient, fileSlice, readerSchema, internalSchemaOption, + HoodieFileGroupReader fileGroupReader = getFileGroupReader(metaClient, fileSlice, tableSchemaWithMetaFields, internalSchemaOption, readerContextFactory, instantTime, readerProperties, usePosition, enableLogBlocksScan); // read records from the FG reader return CloseableIteratorListener.addListener(fileGroupReader.getClosableIterator()); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkConsistentBucketClusteringExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkConsistentBucketClusteringExecutionStrategy.java index d61fc85724525..da6a6b05b20a0 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkConsistentBucketClusteringExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkConsistentBucketClusteringExecutionStrategy.java @@ -26,6 +26,7 @@ import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieClusteringException; @@ -64,7 +65,7 @@ public HoodieData performClusteringWithRecordsAsRow(Dataset in int numOutputGroups, String instantTime, Map strategyParams, - Schema schema, + HoodieSchema schema, List fileGroupIdList, boolean shouldPreserveHoodieMetadata, Map extraMetadata) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java index bffe3afb70f53..b3f640c646ba4 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java @@ -26,6 +26,7 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieClusteringException; import org.apache.hudi.io.SingleFileHandleCreateFactory; @@ -62,7 +63,7 @@ public HoodieData performClusteringWithRecordsAsRow(Dataset in int numOutputGroups, String instantTime, Map strategyParams, - Schema schema, + HoodieSchema schema, List fileGroupIdList, boolean shouldPreserveHoodieMetadata, Map extraMetadata) { @@ -78,7 +79,7 @@ public HoodieData performClusteringWithRecordsAsRow(Dataset in // Since clustering will write to single file group using HoodieUnboundedCreateHandle, set max file size to a large value. newConfig.setValue(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE, String.valueOf(Long.MAX_VALUE)); - BulkInsertPartitioner> partitioner = getRowPartitioner(strategyParams, schema); + BulkInsertPartitioner> partitioner = getRowPartitioner(strategyParams, schema.toAvroSchema()); Dataset repartitionedRecords = partitioner.repartitionRecords(inputRecords, numOutputGroups); return HoodieDatasetBulkInsertHelper.bulkInsert(repartitionedRecords, instantTime, getHoodieTable(), newConfig, diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java index 499c309967b51..6023c36f0f5c8 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java @@ -25,6 +25,7 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.io.CreateHandleFactory; import org.apache.hudi.table.BulkInsertPartitioner; @@ -59,7 +60,7 @@ public SparkSortAndSizeExecutionStrategy(HoodieTable table, public HoodieData performClusteringWithRecordsAsRow(Dataset inputRecords, int numOutputGroups, String instantTime, Map strategyParams, - Schema schema, + HoodieSchema schema, List fileGroupIdList, boolean shouldPreserveHoodieMetadata, Map extraMetadata) { @@ -70,7 +71,7 @@ public HoodieData performClusteringWithRecordsAsRow(Dataset in newConfig.setValue(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE, String.valueOf(getWriteConfig().getClusteringTargetFileMaxBytes())); - BulkInsertPartitioner> partitioner = getRowPartitioner(strategyParams, schema); + BulkInsertPartitioner> partitioner = getRowPartitioner(strategyParams, schema.toAvroSchema()); Dataset repartitionedRecords = partitioner.repartitionRecords(inputRecords, numOutputGroups); return HoodieDatasetBulkInsertHelper.bulkInsert(repartitionedRecords, instantTime, getHoodieTable(), newConfig, diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMetadataWriterUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMetadataWriterUtils.java index 9d9437942bc54..339cb06910edf 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMetadataWriterUtils.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMetadataWriterUtils.java @@ -341,8 +341,8 @@ private static Iterator getExpressionIndexRecordsIterator(HoodieReaderConte HoodieFileGroupReader fileGroupReader = HoodieFileGroupReader.newBuilder() .withReaderContext(readerContext) .withHoodieTableMetaClient(metaClient) - .withDataSchema(tableSchema.toAvroSchema()) - .withRequestedSchema(readerSchema.toAvroSchema()) + .withDataSchema(tableSchema) + .withRequestedSchema(readerSchema) .withProps(dataWriteConfig.getProps()) .withLatestCommitTime(metaClient.getActiveTimeline().lastInstant().map(HoodieInstant::requestedTime).orElse("")) .withAllowInflightInstants(true) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/merge/SparkRecordMergingUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/merge/SparkRecordMergingUtils.java index 4b08fe3c27fea..9dd1f5c71fc01 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/merge/SparkRecordMergingUtils.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/merge/SparkRecordMergingUtils.java @@ -20,16 +20,15 @@ package org.apache.hudi.merge; import org.apache.hudi.AvroConversionUtils; -import org.apache.hudi.avro.AvroSchemaCache; import org.apache.hudi.common.engine.RecordContext; import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.model.HoodieSparkRecord; import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaCache; import org.apache.hudi.common.table.read.BufferedRecord; import org.apache.hudi.common.table.read.BufferedRecords; import org.apache.hudi.common.util.collection.Pair; -import org.apache.avro.Schema; import org.apache.spark.sql.HoodieInternalRowUtils; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; @@ -49,10 +48,10 @@ * This can be used by any Spark {@link HoodieRecordMerger} implementation. */ public class SparkRecordMergingUtils { - private static final Map> FIELD_ID_TO_FIELD_MAPPING_CACHE = new ConcurrentHashMap<>(); - private static final Map> FIELD_NAME_TO_ID_MAPPING_CACHE = new ConcurrentHashMap<>(); - private static final Map, Schema>, - Pair, Pair>> MERGED_SCHEMA_CACHE = new ConcurrentHashMap<>(); + private static final Map> FIELD_ID_TO_FIELD_MAPPING_CACHE = new ConcurrentHashMap<>(); + private static final Map> FIELD_NAME_TO_ID_MAPPING_CACHE = new ConcurrentHashMap<>(); + private static final Map, HoodieSchema>, + Pair, Pair>> MERGED_SCHEMA_CACHE = new ConcurrentHashMap<>(); /** * Merges records which can contain partial updates. @@ -104,13 +103,13 @@ public class SparkRecordMergingUtils { * @return The merged record and schema. */ public static BufferedRecord mergePartialRecords(BufferedRecord older, - Schema oldSchema, + HoodieSchema oldSchema, BufferedRecord newer, - Schema newSchema, - Schema readerSchema, + HoodieSchema newSchema, + HoodieSchema readerSchema, RecordContext recordContext) { // The merged schema contains fields that only appear in either older and/or newer record - Pair, Pair> mergedSchemaPair = + Pair, Pair> mergedSchemaPair = getCachedMergedSchema(oldSchema, newSchema, readerSchema); boolean isNewerPartial = isPartial(newSchema, mergedSchemaPair.getRight().getRight()); if (isNewerPartial) { @@ -133,7 +132,7 @@ public static BufferedRecord mergePartialRecords(BufferedRecord mergePartialRecords(BufferedRecord getCachedFieldIdToFieldMapping(Schema avroSchema) { - return FIELD_ID_TO_FIELD_MAPPING_CACHE.computeIfAbsent(avroSchema, schema -> { - StructType structType = HoodieInternalRowUtils.getCachedSchema(schema); + public static Map getCachedFieldIdToFieldMapping(HoodieSchema providedSchema) { + return FIELD_ID_TO_FIELD_MAPPING_CACHE.computeIfAbsent(providedSchema, schema -> { + StructType structType = HoodieInternalRowUtils.getCachedSchema(schema.toAvroSchema()); Map schemaFieldIdMapping = new HashMap<>(); int fieldId = 0; @@ -160,12 +159,12 @@ public static Map getCachedFieldIdToFieldMapping(Schema av } /** - * @param avroSchema Avro schema. + * @param providedSchema provided schema. * @return The field name to ID mapping. */ - public static Map getCachedFieldNameToIdMapping(Schema avroSchema) { - return FIELD_NAME_TO_ID_MAPPING_CACHE.computeIfAbsent(avroSchema, schema -> { - StructType structType = HoodieInternalRowUtils.getCachedSchema(schema); + public static Map getCachedFieldNameToIdMapping(HoodieSchema providedSchema) { + return FIELD_NAME_TO_ID_MAPPING_CACHE.computeIfAbsent(providedSchema, schema -> { + StructType structType = HoodieInternalRowUtils.getCachedSchema(schema.toAvroSchema()); Map schemaFieldIdMapping = new HashMap<>(); int fieldId = 0; @@ -186,16 +185,16 @@ public static Map getCachedFieldNameToIdMapping(Schema avroSche * @param newSchema New schema. * @param readerSchema Reader schema containing all the fields to read. * @return The ID to {@link StructField} instance mapping of the merged schema, and the - * {@link StructType} and Avro schema of the merged schema. + * {@link StructType} and {@link HoodieSchema} of the merged schema. */ - public static Pair, Pair> getCachedMergedSchema(Schema oldSchema, - Schema newSchema, - Schema readerSchema) { + public static Pair, Pair> getCachedMergedSchema(HoodieSchema oldSchema, + HoodieSchema newSchema, + HoodieSchema readerSchema) { return MERGED_SCHEMA_CACHE.computeIfAbsent( Pair.of(Pair.of(oldSchema, newSchema), readerSchema), schemaPair -> { - Schema schema1 = schemaPair.getLeft().getLeft(); - Schema schema2 = schemaPair.getLeft().getRight(); - Schema refSchema = schemaPair.getRight(); + HoodieSchema schema1 = schemaPair.getLeft().getLeft(); + HoodieSchema schema2 = schemaPair.getLeft().getRight(); + HoodieSchema refSchema = schemaPair.getRight(); Map nameToIdMapping1 = getCachedFieldNameToIdMapping(schema1); Map nameToIdMapping2 = getCachedFieldNameToIdMapping(schema2); // Mapping of field ID/position to the StructField instance of the readerSchema @@ -219,18 +218,18 @@ public static Pair, Pair> getCache } } StructType mergedStructType = new StructType(mergedFieldList.toArray(new StructField[0])); - Schema mergedSchema = AvroSchemaCache.intern(AvroConversionUtils.convertStructTypeToAvroSchema( - mergedStructType, readerSchema.getName(), readerSchema.getNamespace())); + HoodieSchema mergedSchema = HoodieSchemaCache.intern(HoodieSchema.fromAvroSchema(AvroConversionUtils.convertStructTypeToAvroSchema( + mergedStructType, readerSchema.getName(), readerSchema.getNamespace().orElse(null)))); return Pair.of(mergedMapping, Pair.of(mergedStructType, mergedSchema)); }); } /** - * @param schema Avro schema to check. + * @param schema Schema to check. * @param mergedSchema The merged schema for the merged record. - * @return whether the Avro schema is partial compared to the merged schema. + * @return whether the schema is partial compared to the merged schema. */ - public static boolean isPartial(Schema schema, Schema mergedSchema) { + public static boolean isPartial(HoodieSchema schema, HoodieSchema mergedSchema) { return !schema.equals(mergedSchema); } } diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRecordContext.java b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRecordContext.java index 3044bc816d961..449e8f289a016 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRecordContext.java +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRecordContext.java @@ -227,7 +227,7 @@ public UnaryOperator projectRecord(HoodieSchema from, HoodieSchema return row -> (InternalRow) unsafeRowWriter.apply(row); } - void initOrderingValueConverter(Schema dataSchema, List orderingFieldNames) { + void initOrderingValueConverter(HoodieSchema dataSchema, List orderingFieldNames) { this.orderingValueConverter = OrderingValueEngineTypeConverter.create(dataSchema, orderingFieldNames); } } diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java index 24a276f0d1deb..a7b3dd9a74219 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.engine.HoodieReaderContext; import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.read.FileGroupReaderSchemaHandler; import org.apache.hudi.common.util.HoodieRecordUtils; @@ -30,7 +31,6 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.storage.StorageConfiguration; -import org.apache.avro.Schema; import org.apache.spark.sql.HoodieInternalRowUtils; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.UnsafeRow; @@ -86,10 +86,11 @@ public Option getRecordMerger(RecordMergeMode mergeMode, Str * @param partitionFieldAndValues the partition fields and their values, if any are required by the reader * @return a function for transforming the row */ - protected UnaryOperator getBootstrapProjection(Schema from, Schema to, List> partitionFieldAndValues) { - Map partitionValuesByIndex = partitionFieldAndValues.stream().collect(Collectors.toMap(pair -> to.getField(pair.getKey()).pos(), Pair::getRight)); + protected UnaryOperator getBootstrapProjection(HoodieSchema from, HoodieSchema to, List> partitionFieldAndValues) { + Map partitionValuesByIndex = partitionFieldAndValues.stream() + .collect(Collectors.toMap(pair -> to.getField(pair.getKey()).orElseThrow(() -> new IllegalArgumentException("Missing field: " + pair.getKey())).pos(), Pair::getRight)); Function1 unsafeRowWriter = - HoodieInternalRowUtils.getCachedUnsafeRowWriter(getCachedSchema(from), getCachedSchema(to), Collections.emptyMap(), partitionValuesByIndex); + HoodieInternalRowUtils.getCachedUnsafeRowWriter(getCachedSchema(from.toAvroSchema()), getCachedSchema(to.toAvroSchema()), Collections.emptyMap(), partitionValuesByIndex); return row -> (InternalRow) unsafeRowWriter.apply(row); } @@ -98,7 +99,7 @@ public void setSchemaHandler(FileGroupReaderSchemaHandler schemaHan super.setSchemaHandler(schemaHandler); // init ordering value converter: java -> engine type List orderingFieldNames = HoodieRecordUtils.getOrderingFieldNames(getMergeMode(), tableConfig); - Schema schema = schemaHandler.getRequiredSchema(); + HoodieSchema schema = schemaHandler.getRequiredSchema(); ((BaseSparkInternalRecordContext) recordContext).initOrderingValueConverter(schema, orderingFieldNames); } } 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 7199e94c126dd..3a644682e0e39 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 @@ -152,10 +152,10 @@ class SparkFileFormatInternalRowReaderContext(baseFileReader: SparkColumnarFileR //If we need to do position based merging with log files we will leave the row index column at the end val dataProjection = if (getShouldMergeUseRecordPosition) { - getBootstrapProjection(dataRequiredSchema.toAvroSchema, dataRequiredSchema.toAvroSchema, partitionFieldAndValues) + getBootstrapProjection(dataRequiredSchema, dataRequiredSchema, partitionFieldAndValues) } else { - getBootstrapProjection(dataRequiredSchema.toAvroSchema, - HoodieAvroUtils.removeFields(dataRequiredSchema.toAvroSchema, rowIndexColumn), partitionFieldAndValues) + getBootstrapProjection(dataRequiredSchema, + HoodieSchemaUtils.removeFields(dataRequiredSchema, rowIndexColumn), partitionFieldAndValues) } //row index will always be the last column @@ -209,7 +209,7 @@ class SparkFileFormatInternalRowReaderContext(baseFileReader: SparkColumnarFileR } } } else { - val dataProjection = getBootstrapProjection(dataRequiredSchema.toAvroSchema, dataRequiredSchema.toAvroSchema, partitionFieldAndValues) + val dataProjection = getBootstrapProjection(dataRequiredSchema, dataRequiredSchema, partitionFieldAndValues) new ClosableIterator[Any] { val combinedRow = new JoinedRow() diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/util/OrderingValueEngineTypeConverter.java b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/util/OrderingValueEngineTypeConverter.java index 7e99a5c148211..aebf8740d46b9 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/util/OrderingValueEngineTypeConverter.java +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/util/OrderingValueEngineTypeConverter.java @@ -20,13 +20,13 @@ import org.apache.hudi.AvroConversionUtils; import org.apache.hudi.SparkAdapterSupport$; -import org.apache.hudi.avro.AvroSchemaUtils; +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.util.Option; import org.apache.hudi.common.util.collection.ArrayComparable; +import org.apache.hudi.common.util.collection.Pair; -import org.apache.avro.LogicalType; -import org.apache.avro.LogicalTypes; -import org.apache.avro.Schema; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.Decimal; import org.apache.spark.sql.types.DecimalType; @@ -45,7 +45,7 @@ */ public class OrderingValueEngineTypeConverter { private final List> converters; - private OrderingValueEngineTypeConverter(Schema dataSchema, List orderingFieldNames) { + private OrderingValueEngineTypeConverter(HoodieSchema dataSchema, List orderingFieldNames) { this.converters = createConverters(dataSchema, orderingFieldNames); } @@ -55,29 +55,29 @@ public Comparable convert(Comparable value) { : this.converters.get(0).apply(value); } - public static OrderingValueEngineTypeConverter create(Schema dataSchema, List orderingFieldNames) { + public static OrderingValueEngineTypeConverter create(HoodieSchema dataSchema, List orderingFieldNames) { return new OrderingValueEngineTypeConverter(dataSchema, orderingFieldNames); } - private static List> createConverters(Schema dataSchema, List orderingFieldNames) { + private static List> createConverters(HoodieSchema dataSchema, List orderingFieldNames) { if (orderingFieldNames.isEmpty()) { return Collections.singletonList(Function.identity()); } return orderingFieldNames.stream().map(f -> { - Option fieldSchemaOpt = AvroSchemaUtils.findNestedFieldSchema(dataSchema, f, true); + Option fieldSchemaOpt = HoodieSchemaUtils.getNestedField(dataSchema, f).map(Pair::getRight).map(HoodieSchemaField::getNonNullSchema); if (fieldSchemaOpt.isEmpty()) { return Function.identity(); } else { - DataType fieldType = AvroConversionUtils.convertAvroSchemaToDataType(fieldSchemaOpt.get()); + DataType fieldType = AvroConversionUtils.convertAvroSchemaToDataType(fieldSchemaOpt.get().toAvroSchema()); return createConverter(fieldType, fieldSchemaOpt.get()); } }).collect(Collectors.toList()); } - public static Function createConverter(DataType fieldType, Schema fieldSchema) { + public static Function createConverter(DataType fieldType, HoodieSchema fieldSchema) { if (fieldType instanceof TimestampType) { - LogicalType logicalType = fieldSchema.getLogicalType(); - if (logicalType == null || logicalType instanceof LogicalTypes.TimestampMillis) { + HoodieSchema.Timestamp timestampSchema = (HoodieSchema.Timestamp) fieldSchema; + if (timestampSchema.getPrecision() == HoodieSchema.TimePrecision.MILLIS) { return comparable -> formatAsMicros((long) comparable); } } else if (fieldType instanceof StringType) { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestSparkRDDMetadataWriteClient.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestSparkRDDMetadataWriteClient.java index e1b99a9092945..e2cb75524f588 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestSparkRDDMetadataWriteClient.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestSparkRDDMetadataWriteClient.java @@ -19,7 +19,6 @@ package org.apache.hudi.client; import org.apache.hudi.avro.HoodieAvroReaderContext; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieMetadataRecord; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.TypedProperties; @@ -30,6 +29,8 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; 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.table.HoodieTableVersion; import org.apache.hudi.common.table.log.InstantRange; @@ -50,7 +51,6 @@ import org.apache.hudi.metadata.MetadataPartitionType; import org.apache.hudi.testutils.HoodieClientTestBase; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.apache.spark.api.java.JavaRDD; @@ -214,7 +214,8 @@ private void readFromMDTFileSliceAndValidate(HoodieTableMetaClient metadataMetaC InstantRange instantRange = InstantRange.builder() .rangeType(InstantRange.RangeType.EXACT_MATCH) .explicitInstants(validInstantTimestamps).build(); - Schema schema = HoodieAvroUtils.addMetadataFields(HoodieMetadataRecord.getClassSchema()); + HoodieSchema metadataSchema = HoodieSchema.fromAvroSchema(HoodieMetadataRecord.getClassSchema()); + HoodieSchema schema = HoodieSchemaUtils.addMetadataFields(metadataSchema); HoodieAvroReaderContext readerContext = new HoodieAvroReaderContext(metadataMetaClient.getStorageConf(), metadataMetaClient.getTableConfig(), Option.of(instantRange), Option.of(predicate)); HoodieFileGroupReader fileGroupReader = HoodieFileGroupReader.newBuilder() @@ -222,7 +223,7 @@ private void readFromMDTFileSliceAndValidate(HoodieTableMetaClient metadataMetaC .withHoodieTableMetaClient(metadataMetaClient) .withFileSlice(fileSlice) .withLatestCommitTime(validMetadataInstant) - .withRequestedSchema(HoodieMetadataRecord.getClassSchema()) + .withRequestedSchema(metadataSchema) .withDataSchema(schema) .withProps(new TypedProperties()) .withEnableOptimizedLogBlockScan(hoodieWriteConfig.getMetadataConfig().isOptimizedLogBlocksScanEnabled()) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java index 4684ebebf5d6b..8ae72764c122e 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java @@ -19,7 +19,6 @@ package org.apache.hudi.client.functional; import org.apache.hudi.avro.HoodieAvroReaderContext; -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.HoodieMetadataRecord; @@ -35,6 +34,8 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.model.HoodieTableType; +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.HoodieTableVersion; import org.apache.hudi.common.table.TableSchemaResolver; @@ -560,7 +561,7 @@ private void verifyMetadataRawRecords(HoodieTable table, List log * @param latestCommitTimestamp - Latest commit timestamp */ private void verifyMetadataMergedRecords(HoodieTableMetaClient metadataMetaClient, List logFiles, String latestCommitTimestamp, HoodieWriteConfig metadataTableWriteConfig) { - Schema schema = HoodieAvroUtils.addMetadataFields(HoodieMetadataRecord.getClassSchema()); + HoodieSchema schema = HoodieSchemaUtils.addMetadataFields(HoodieSchema.fromAvroSchema(HoodieMetadataRecord.getClassSchema())); HoodieAvroReaderContext readerContext = new HoodieAvroReaderContext(metadataMetaClient.getStorageConf(), metadataMetaClient.getTableConfig(), Option.empty(), Option.empty()); HoodieFileGroupReader fileGroupReader = HoodieFileGroupReader.newBuilder() .withReaderContext(readerContext) diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroReaderContext.java b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroReaderContext.java index a0a8b065272e6..8b84999ac27e6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroReaderContext.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroReaderContext.java @@ -31,6 +31,8 @@ import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.model.OverwriteWithLatestMerger; 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.serialization.CustomSerializer; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.log.InstantRange; @@ -212,8 +214,8 @@ public ClosableIterator getFileRecordIterator( fileOutputSchema = requiredSchema; renamedColumns = Collections.emptyMap(); } else { - Pair> requiredSchemaForFileAndRenamedColumns = getSchemaHandler().getRequiredSchemaForFileAndRenamedColumns(filePath); - fileOutputSchema = HoodieSchema.fromAvroSchema(requiredSchemaForFileAndRenamedColumns.getLeft()); + Pair> requiredSchemaForFileAndRenamedColumns = getSchemaHandler().getRequiredSchemaForFileAndRenamedColumns(filePath); + fileOutputSchema = requiredSchemaForFileAndRenamedColumns.getLeft(); renamedColumns = requiredSchemaForFileAndRenamedColumns.getRight(); } if (keyFilterOpt.isEmpty()) { @@ -254,7 +256,7 @@ public Option getRecordMerger(RecordMergeMode mergeMode, Str @Override public SizeEstimator> getRecordSizeEstimator() { - return new AvroRecordSizeEstimator(getSchemaHandler().getSchemaForUpdates()); + return new AvroRecordSizeEstimator(getSchemaHandler().getSchemaForUpdates().toAvroSchema()); } @Override @@ -268,7 +270,7 @@ public ClosableIterator mergeBootstrapReaders(ClosableIterator dataFileIterator, HoodieSchema dataRequiredSchema, List> partitionFieldAndValues) { - return new BootstrapIterator(skeletonFileIterator, skeletonRequiredSchema.toAvroSchema(), dataFileIterator, dataRequiredSchema.toAvroSchema(), partitionFieldAndValues); + return new BootstrapIterator(skeletonFileIterator, skeletonRequiredSchema, dataFileIterator, dataRequiredSchema, partitionFieldAndValues); } /** @@ -277,24 +279,28 @@ public ClosableIterator mergeBootstrapReaders(ClosableIterator { private final ClosableIterator skeletonFileIterator; - private final Schema skeletonRequiredSchema; + private final HoodieSchema skeletonRequiredSchema; private final ClosableIterator dataFileIterator; - private final Schema dataRequiredSchema; - private final Schema mergedSchema; + private final HoodieSchema dataRequiredSchema; + private final HoodieSchema mergedSchema; private final int skeletonFields; private final int[] partitionFieldPositions; private final Object[] partitionValues; - public BootstrapIterator(ClosableIterator skeletonFileIterator, Schema skeletonRequiredSchema, - ClosableIterator dataFileIterator, Schema dataRequiredSchema, + public BootstrapIterator(ClosableIterator skeletonFileIterator, HoodieSchema skeletonRequiredSchema, + ClosableIterator dataFileIterator, HoodieSchema dataRequiredSchema, List> partitionFieldAndValues) { this.skeletonFileIterator = skeletonFileIterator; this.skeletonRequiredSchema = skeletonRequiredSchema; this.dataFileIterator = dataFileIterator; this.dataRequiredSchema = dataRequiredSchema; - this.mergedSchema = AvroSchemaUtils.mergeSchemas(skeletonRequiredSchema, dataRequiredSchema); + this.mergedSchema = HoodieSchemaUtils.mergeSchemas(skeletonRequiredSchema, dataRequiredSchema); this.skeletonFields = skeletonRequiredSchema.getFields().size(); - this.partitionFieldPositions = partitionFieldAndValues.stream().map(Pair::getLeft).map(field -> mergedSchema.getField(field).pos()).mapToInt(Integer::intValue).toArray(); + this.partitionFieldPositions = partitionFieldAndValues.stream() + .map(Pair::getLeft) + .map(field -> mergedSchema.getField(field).orElseThrow(() -> new IllegalArgumentException("Field not found: " + field)).pos()) + .mapToInt(Integer::intValue) + .toArray(); this.partitionValues = partitionFieldAndValues.stream().map(Pair::getValue).toArray(); } @@ -315,13 +321,13 @@ public boolean hasNext() { public IndexedRecord next() { IndexedRecord skeletonRecord = skeletonFileIterator.next(); IndexedRecord dataRecord = dataFileIterator.next(); - GenericRecord mergedRecord = new GenericData.Record(mergedSchema); + GenericRecord mergedRecord = new GenericData.Record(mergedSchema.toAvroSchema()); - for (Schema.Field skeletonField : skeletonRequiredSchema.getFields()) { + for (HoodieSchemaField skeletonField : skeletonRequiredSchema.getFields()) { Schema.Field sourceField = skeletonRecord.getSchema().getField(skeletonField.name()); mergedRecord.put(skeletonField.pos(), skeletonRecord.get(sourceField.pos())); } - for (Schema.Field dataField : dataRequiredSchema.getFields()) { + for (HoodieSchemaField dataField : dataRequiredSchema.getFields()) { Schema.Field sourceField = dataRecord.getSchema().getField(dataField.name()); mergedRecord.put(dataField.pos() + skeletonFields, dataRecord.get(sourceField.pos())); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java index 574c21b426f67..153a264cf6b67 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java @@ -224,7 +224,7 @@ public Option getKeyFilterOpt() { } public SizeEstimator> getRecordSizeEstimator() { - return new HoodieRecordSizeEstimator<>(getSchemaHandler().getSchemaForUpdates()); + return new HoodieRecordSizeEstimator<>(getSchemaHandler().getSchemaForUpdates().toAvroSchema()); } public CustomSerializer> getRecordSerializer() { @@ -358,7 +358,7 @@ public ClosableIterator applyInstantRangeFilter(ClosableIterator fileRecor return fileRecordIterator; } InstantRange instantRange = getInstantRange().get(); - final Option commitTimeFieldOpt = HoodieSchema.fromAvroSchema(getSchemaHandler().getRequiredSchema()).getField(HoodieRecord.COMMIT_TIME_METADATA_FIELD); + final Option commitTimeFieldOpt = getSchemaHandler().getRequiredSchema().getField(HoodieRecord.COMMIT_TIME_METADATA_FIELD); final int commitTimePos = commitTimeFieldOpt.orElseThrow(() -> new HoodieSchemaException("Commit time metadata field '" + HoodieRecord.COMMIT_TIME_METADATA_FIELD + "' not found in required schema")).pos(); java.util.function.Predicate instantFilter = diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java index b8bf948a22e9c..3bf52f6909101 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java @@ -24,13 +24,12 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.RecordContext; import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; +import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.read.BufferedRecord; import org.apache.hudi.common.util.Option; -import org.apache.avro.Schema; - import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; @@ -127,7 +126,7 @@ public interface HoodieRecordMerger extends Serializable { * @return The merged record and schema. The record is expected to be non-null. If the record represents a deletion, the operation must be set as {@link HoodieOperation#DELETE}. * @throws IOException upon merging error. */ - default BufferedRecord partialMerge(BufferedRecord older, BufferedRecord newer, Schema readerSchema, RecordContext recordContext, TypedProperties props) throws IOException { + default BufferedRecord partialMerge(BufferedRecord older, BufferedRecord newer, HoodieSchema readerSchema, RecordContext recordContext, TypedProperties props) throws IOException { throw new UnsupportedOperationException("Partial merging logic is not implemented by " + this.getClass().getName()); } @@ -144,7 +143,7 @@ default boolean isProjectionCompatible() { /** * Returns a list of fields required for mor merging. The default implementation will return the recordKey field and the ordering fields. */ - default String[] getMandatoryFieldsForMerging(Schema dataSchema, HoodieTableConfig cfg, TypedProperties properties) { + default String[] getMandatoryFieldsForMerging(HoodieSchema dataSchema, HoodieTableConfig cfg, TypedProperties properties) { ArrayList requiredFields = new ArrayList<>(); if (cfg.populateMetaFields()) { 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 8334f0408270b..4f30a1665e29e 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 @@ -18,7 +18,6 @@ package org.apache.hudi.common.schema; -import org.apache.avro.JsonProperties; import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.model.HoodieRecord; @@ -26,6 +25,7 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; +import org.apache.avro.JsonProperties; import org.apache.avro.Schema; import java.math.BigDecimal; @@ -573,4 +573,16 @@ public static HoodieSchema getRecordKeySchema() { return RECORD_KEY_SCHEMA; } + /** + * Fetches projected schema given list of fields to project. The field can be nested in format `a.b.c` where a is + * the top level field, b is at second level and so on. + * This is equivalent to {@link HoodieAvroUtils#projectSchema(Schema, List)} but operates on HoodieSchema. + * + * @param fileSchema the original schema + * @param fields list of fields to project + * @return projected schema containing only specified fields + */ + public static HoodieSchema projectSchema(HoodieSchema fileSchema, List fields) { + return HoodieSchema.fromAvroSchema(HoodieAvroUtils.projectSchema(fileSchema.toAvroSchema(), fields)); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/PartitionPathParser.java b/hudi-common/src/main/java/org/apache/hudi/common/table/PartitionPathParser.java index 88ddcea7d39bc..3ba13f870c427 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/PartitionPathParser.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/PartitionPathParser.java @@ -19,22 +19,20 @@ package org.apache.hudi.common.table; +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.util.Option; import org.apache.hudi.common.util.PartitionPathEncodeUtils; import org.apache.hudi.common.util.VisibleForTesting; -import org.apache.avro.LogicalType; -import org.apache.avro.LogicalTypes; -import org.apache.avro.Schema; - +import java.math.BigDecimal; import java.nio.charset.StandardCharsets; import java.sql.Date; import java.sql.Timestamp; import java.time.LocalDateTime; import java.time.ZoneOffset; -import static org.apache.hudi.avro.AvroSchemaUtils.getNonNullTypeFromUnion; - public class PartitionPathParser { public static final String DEPRECATED_DEFAULT_PARTITION_PATH = "default"; public static final String DEFAULT_PARTITION_PATH = "__HIVE_DEFAULT_PARTITION__"; @@ -44,7 +42,7 @@ public class PartitionPathParser { public Object[] getPartitionFieldVals(Option partitionFields, String partitionPath, - Schema writerSchema) { + HoodieSchema writerSchema) { if (!partitionFields.isPresent()) { return new Object[0]; } @@ -53,18 +51,17 @@ public Object[] getPartitionFieldVals(Option partitionFields, private static Object[] getPartitionValues(String[] partitionFields, String partitionPath, - Schema schema) { + HoodieSchema schema) { String[] parts = partitionPath.split("/"); int pathSegment = 0; boolean hasDateField = false; Object[] partitionValues = new Object[partitionFields.length]; for (int i = 0; i < partitionFields.length; i++) { String partitionField = partitionFields[i]; - Schema.Field field = schema.getField(partitionField); + Option field = schema.getField(partitionField); // if the field is not present in the schema, we assume it is a string - Schema fieldSchema = field == null ? Schema.create(Schema.Type.STRING) : getNonNullTypeFromUnion(field.schema()); - LogicalType logicalType = fieldSchema.getLogicalType(); - if (isTimeBasedLogicalType(logicalType)) { + HoodieSchema fieldSchema = field.map(f -> f.schema().getNonNullType()).orElseGet(() -> HoodieSchema.create(HoodieSchemaType.STRING)); + if (isTimeBasedType(fieldSchema.getType())) { if (hasDateField) { throw new IllegalArgumentException("Only one date field based partition is supported"); } @@ -83,13 +80,15 @@ private static Object[] getPartitionValues(String[] partitionFields, } @VisibleForTesting - static Object parseValue(String partitionValue, Schema fieldSchema) { + static Object parseValue(String partitionValue, HoodieSchema fieldSchema) { if (partitionValue.equals(DEFAULT_PARTITION_PATH) || partitionValue.equals(DEPRECATED_DEFAULT_PARTITION_PATH)) { return null; } switch (fieldSchema.getType()) { case STRING: + case ENUM: + case UUID: return PartitionPathEncodeUtils.unescapePathName(partitionValue); case INT: return Integer.parseInt(partitionValue); @@ -103,11 +102,9 @@ static Object parseValue(String partitionValue, Schema fieldSchema) { return Boolean.parseBoolean(partitionValue); case BYTES: case FIXED: - if (fieldSchema.getLogicalType() instanceof LogicalTypes.Decimal) { - return new java.math.BigDecimal(partitionValue); - } else { - return partitionValue.getBytes(StandardCharsets.UTF_8); - } + return partitionValue.getBytes(StandardCharsets.UTF_8); + case DECIMAL: + return new BigDecimal(partitionValue); default: throw new IllegalArgumentException("Unexpected type " + fieldSchema.getType()); } @@ -118,7 +115,7 @@ private static Object inferDateValue( String[] parts, int pathSegment, int numDateDirs, - Schema fieldSchema) { + HoodieSchema fieldSchema) { StringBuilder condensedPartitionValue = new StringBuilder(); for (int i = 0; i < numDateDirs; i++) { String partitionValue = parts[pathSegment + i]; @@ -158,19 +155,13 @@ private static Object inferDateValue( throw new IllegalArgumentException( "Unknown date format for partition path: " + partitionPath); } - if (fieldSchema.getLogicalType() instanceof LogicalTypes.Date) { + if (fieldSchema.getType() == HoodieSchemaType.DATE) { return Date.valueOf(time.toLocalDate()); } return Timestamp.from(time.toInstant(ZoneOffset.UTC)); } - private static boolean isTimeBasedLogicalType(LogicalType logicalType) { - return logicalType instanceof LogicalTypes.Date - || logicalType instanceof LogicalTypes.TimestampMillis - || logicalType instanceof LogicalTypes.TimestampMicros - || logicalType instanceof LogicalTypes.TimeMillis - || logicalType instanceof LogicalTypes.TimeMicros - || logicalType instanceof LogicalTypes.LocalTimestampMicros - || logicalType instanceof LogicalTypes.LocalTimestampMillis; + private static boolean isTimeBasedType(HoodieSchemaType type) { + return type == HoodieSchemaType.DATE || type == HoodieSchemaType.TIMESTAMP || type == HoodieSchemaType.TIME; } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/BaseHoodieLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/BaseHoodieLogRecordReader.java index 0b5302d9b1499..69a293810e3d9 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/BaseHoodieLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/BaseHoodieLogRecordReader.java @@ -21,9 +21,11 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.HoodieReaderContext; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodiePayloadProps; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableVersion; @@ -33,7 +35,6 @@ import org.apache.hudi.common.table.log.block.HoodieLogBlock; import org.apache.hudi.common.table.read.buffer.HoodieFileGroupRecordBuffer; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; @@ -41,7 +42,6 @@ import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; -import org.apache.avro.Schema; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -78,7 +78,7 @@ public abstract class BaseHoodieLogRecordReader { private static final Logger LOG = LoggerFactory.getLogger(BaseHoodieLogRecordReader.class); // Reader schema for the records - protected final Schema readerSchema; + protected final HoodieSchema readerSchema; // Latest valid instant time // Log-Blocks belonging to inflight delta-instants are filtered-out using this high-watermark. private final String latestInstantTime; @@ -215,7 +215,7 @@ private void scanInternalV1(Option keySpecOpt) { try { // Iterate over the paths logFormatReaderWrapper = new HoodieLogFormatReader(storage, logFiles, - readerSchema, reverseReader, bufferSize, shouldLookupRecords(), recordKeyField, internalSchema); + readerSchema.toAvroSchema(), reverseReader, bufferSize, shouldLookupRecords(), recordKeyField, internalSchema); Set scannedLogFiles = new HashSet<>(); while (logFormatReaderWrapper.hasNext()) { @@ -377,7 +377,7 @@ private void scanInternalV2(Option keySpecOption, boolean skipProcessin try { // Iterate over the paths logFormatReaderWrapper = new HoodieLogFormatReader(storage, logFiles, - readerSchema, reverseReader, bufferSize, shouldLookupRecords(), recordKeyField, internalSchema); + readerSchema == null ? null : readerSchema.toAvroSchema(), reverseReader, bufferSize, shouldLookupRecords(), recordKeyField, internalSchema); /** * Scanning log blocks and placing the compacted blocks at the right place require two traversals. diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/BufferedRecordConverter.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/BufferedRecordConverter.java index 0ffc86cfcda66..38f0fcfab3d5e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/BufferedRecordConverter.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/BufferedRecordConverter.java @@ -21,8 +21,6 @@ import org.apache.hudi.common.engine.RecordContext; import org.apache.hudi.common.schema.HoodieSchema; -import org.apache.avro.Schema; - import java.util.List; /** @@ -33,7 +31,7 @@ public interface BufferedRecordConverter { BufferedRecord convert(T record); static BufferedRecordConverter createConverter( - IteratorMode iteratorMode, Schema readerSchema, RecordContext recordContext, List orderingFieldNames) { + IteratorMode iteratorMode, HoodieSchema readerSchema, RecordContext recordContext, List orderingFieldNames) { switch (iteratorMode) { case ENGINE_RECORD: return new BufferedRecordConverter() { @@ -54,7 +52,7 @@ public BufferedRecord convert(T record) { @Override public BufferedRecord convert(T record) { - String recordKey = recordContext.getRecordKey(record, HoodieSchema.fromAvroSchema(readerSchema)); + String recordKey = recordContext.getRecordKey(record, readerSchema); return reusedBufferedRecord.replaceRecordKey(recordKey); } }; @@ -62,7 +60,7 @@ public BufferedRecord convert(T record) { return new BufferedRecordConverter() { @Override public BufferedRecord convert(T record) { - return BufferedRecords.fromEngineRecord(record, HoodieSchema.fromAvroSchema(readerSchema), recordContext, orderingFieldNames, false); + return BufferedRecords.fromEngineRecord(record, readerSchema, recordContext, orderingFieldNames, false); } }; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/BufferedRecordMergerFactory.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/BufferedRecordMergerFactory.java index 1a025aed09add..81d2de0ac0a23 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/BufferedRecordMergerFactory.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/BufferedRecordMergerFactory.java @@ -33,8 +33,6 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieIOException; -import org.apache.avro.Schema; - import java.io.IOException; /** @@ -50,7 +48,7 @@ public static BufferedRecordMerger create(HoodieReaderContext readerCo boolean enablePartialMerging, Option recordMerger, Option payloadClass, - Schema readerSchema, + HoodieSchema readerSchema, TypedProperties props, Option partialUpdateModeOpt) { return create(readerContext, recordMergeMode, enablePartialMerging, recordMerger, @@ -61,7 +59,7 @@ public static BufferedRecordMerger create(HoodieReaderContext readerCo RecordMergeMode recordMergeMode, boolean enablePartialMerging, Option recordMerger, - Schema readerSchema, + HoodieSchema readerSchema, Option> payloadClasses, TypedProperties props, Option partialUpdateModeOpt) { @@ -280,14 +278,14 @@ private static class PartialUpdateBufferedRecordMerger implements BufferedRec private final RecordContext recordContext; private final Option recordMerger; private final BufferedRecordMerger deleteRecordMerger; - private final Schema readerSchema; + private final HoodieSchema readerSchema; private final TypedProperties props; public PartialUpdateBufferedRecordMerger( RecordContext recordContext, Option recordMerger, BufferedRecordMerger deleteRecordMerger, - Schema readerSchema, + HoodieSchema readerSchema, TypedProperties props) { this.recordContext = recordContext; this.recordMerger = recordMerger; @@ -340,7 +338,7 @@ private static class CustomRecordMerger extends BaseCustomMerger { public CustomRecordMerger( RecordContext recordContext, Option recordMerger, - Schema readerSchema, + HoodieSchema readerSchema, TypedProperties props) { super(recordContext, recordMerger, readerSchema, props); } @@ -366,7 +364,7 @@ private static class ExpressionPayloadRecordMerger extends CustomPayloadRecor private final HoodieRecordMerger deltaMerger; public ExpressionPayloadRecordMerger(RecordContext recordContext, Option recordMerger, String incomingPayloadClass, - Schema readerSchema, TypedProperties props) { + HoodieSchema readerSchema, TypedProperties props) { super(recordContext, recordMerger, incomingPayloadClass, readerSchema, props); this.deltaMerger = HoodieRecordUtils.mergerToPreCombineMode(recordMerger.get()); } @@ -393,7 +391,7 @@ public CustomPayloadRecordMerger( RecordContext recordContext, Option recordMerger, String payloadClass, - Schema readerSchema, + HoodieSchema readerSchema, TypedProperties props) { super(recordContext, recordMerger, readerSchema, props); this.payloadClass = payloadClass; @@ -418,13 +416,13 @@ public Option> deltaMergeRecords(BufferedRecord newRecord, private abstract static class BaseCustomMerger implements BufferedRecordMerger { protected final RecordContext recordContext; protected final HoodieRecordMerger recordMerger; - protected final Schema readerSchema; + protected final HoodieSchema readerSchema; protected final TypedProperties props; public BaseCustomMerger( RecordContext recordContext, Option recordMerger, - Schema readerSchema, + HoodieSchema readerSchema, TypedProperties props) { this.recordContext = recordContext; this.recordMerger = recordMerger.orElseThrow(() -> new IllegalArgumentException("RecordMerger must be present for custom merging")); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/FileGroupReaderSchemaHandler.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/FileGroupReaderSchemaHandler.java index 714108d413ef1..09ad49be147f0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/FileGroupReaderSchemaHandler.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/FileGroupReaderSchemaHandler.java @@ -19,8 +19,6 @@ package org.apache.hudi.common.table.read; -import org.apache.hudi.avro.AvroSchemaCache; -import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.common.config.RecordMergeMode; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.HoodieReaderContext; @@ -28,9 +26,11 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordMerger; 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.table.HoodieTableConfig; -import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.read.buffer.PositionBasedFileGroupRecordBuffer; import org.apache.hudi.common.util.InternalSchemaCache; import org.apache.hudi.common.util.Option; @@ -42,6 +42,8 @@ import org.apache.hudi.internal.schema.convert.InternalSchemaConverter; import org.apache.hudi.storage.StoragePath; +import lombok.Getter; +import lombok.Setter; import org.apache.avro.Schema; import java.util.ArrayList; @@ -56,10 +58,11 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.hudi.avro.AvroSchemaUtils.appendFieldsToSchemaDedupNested; -import static org.apache.hudi.avro.AvroSchemaUtils.createNewSchemaFromFieldsWithReference; -import static org.apache.hudi.avro.AvroSchemaUtils.findNestedField; -import static org.apache.hudi.avro.HoodieAvroUtils.createNewSchemaField; +import static org.apache.hudi.common.schema.HoodieSchemaCompatibility.areSchemasProjectionEquivalent; +import static org.apache.hudi.common.schema.HoodieSchemaUtils.appendFieldsToSchemaDedupNested; +import static org.apache.hudi.common.schema.HoodieSchemaUtils.createNewSchemaField; +import static org.apache.hudi.common.schema.HoodieSchemaUtils.createNewSchemaFromFieldsWithReference; +import static org.apache.hudi.common.schema.HoodieSchemaUtils.findNestedField; import static org.apache.hudi.common.table.HoodieTableConfig.inferMergingConfigsForPreV9Table; /** @@ -67,20 +70,31 @@ */ public class FileGroupReaderSchemaHandler { - protected final Schema tableSchema; + @Getter + protected final HoodieSchema tableSchema; + @Getter // requestedSchema: the schema that the caller requests - protected final Schema requestedSchema; + protected final HoodieSchema requestedSchema; + @Getter // requiredSchema: the requestedSchema with any additional columns required for merging etc - protected final Schema requiredSchema; + protected final HoodieSchema requiredSchema; + /** + * -- SETTER -- + * This is a special case for incoming records, which do not have metadata fields in schema. + */ // the schema for updates, usually it equals with the requiredSchema, // the only exception is for incoming records, which do not include the metadata fields. - protected Schema schemaForUpdates; + @Setter + @Getter + protected HoodieSchema schemaForUpdates; + @Getter protected final InternalSchema internalSchema; + @Getter protected final Option internalSchemaOpt; protected final HoodieTableConfig hoodieTableConfig; @@ -88,71 +102,37 @@ public class FileGroupReaderSchemaHandler { protected final HoodieReaderContext readerContext; protected final TypedProperties properties; + @Getter private final DeleteContext deleteContext; private final HoodieTableMetaClient metaClient; public FileGroupReaderSchemaHandler(HoodieReaderContext readerContext, - Schema tableSchema, - Schema requestedSchema, + HoodieSchema tableSchema, + HoodieSchema requestedSchema, Option internalSchemaOpt, TypedProperties properties, HoodieTableMetaClient metaClient) { this.properties = properties; this.readerContext = readerContext; this.tableSchema = tableSchema; - this.requestedSchema = AvroSchemaCache.intern(requestedSchema); + this.requestedSchema = HoodieSchemaCache.intern(requestedSchema); this.hoodieTableConfig = metaClient.getTableConfig(); - this.deleteContext = new DeleteContext(properties, HoodieSchema.fromAvroSchema(tableSchema)); - this.requiredSchema = AvroSchemaCache.intern(prepareRequiredSchema(this.deleteContext)); + this.deleteContext = new DeleteContext(properties, tableSchema); + this.requiredSchema = HoodieSchemaCache.intern(prepareRequiredSchema(this.deleteContext)); this.schemaForUpdates = requiredSchema; this.internalSchema = pruneInternalSchema(requiredSchema, internalSchemaOpt); this.internalSchemaOpt = getInternalSchemaOpt(internalSchemaOpt); this.metaClient = metaClient; } - public Schema getTableSchema() { - return this.tableSchema; - } - - public Schema getRequestedSchema() { - return this.requestedSchema; - } - - public Schema getRequiredSchema() { - return this.requiredSchema; - } - - public Schema getSchemaForUpdates() { - return this.schemaForUpdates; - } - - /** - * This is a special case for incoming records, which do not have metadata fields in schema. - */ - public void setSchemaForUpdates(Schema schema) { - this.schemaForUpdates = schema; - } - - public InternalSchema getInternalSchema() { - return this.internalSchema; - } - - public Option getInternalSchemaOpt() { - return this.internalSchemaOpt; - } - public Option> getOutputConverter() { - if (!AvroSchemaUtils.areSchemasProjectionEquivalent(requiredSchema, requestedSchema)) { - return Option.of(readerContext.getRecordContext().projectRecord(HoodieSchema.fromAvroSchema(requiredSchema), HoodieSchema.fromAvroSchema(requestedSchema))); + if (!areSchemasProjectionEquivalent(requiredSchema, requestedSchema)) { + return Option.of(readerContext.getRecordContext().projectRecord(requiredSchema, requestedSchema)); } return Option.empty(); } - public DeleteContext getDeleteContext() { - return deleteContext; - } - - public Pair> getRequiredSchemaForFileAndRenamedColumns(StoragePath path) { + public Pair> getRequiredSchemaForFileAndRenamedColumns(StoragePath path) { if (internalSchema.isEmptySchema()) { return Pair.of(requiredSchema, Collections.emptyMap()); } @@ -160,11 +140,11 @@ public Pair> getRequiredSchemaForFileAndRenamedColum InternalSchema fileSchema = InternalSchemaCache.searchSchemaAndCache(commitInstantTime, metaClient); Pair> mergedInternalSchema = new InternalSchemaMerger(fileSchema, internalSchema, true, false, false).mergeSchemaGetRenamed(); - Schema mergedAvroSchema = AvroSchemaCache.intern(InternalSchemaConverter.convert(mergedInternalSchema.getLeft(), requiredSchema.getFullName()).toAvroSchema()); + HoodieSchema mergedAvroSchema = HoodieSchemaCache.intern(InternalSchemaConverter.convert(mergedInternalSchema.getLeft(), requiredSchema.getFullName())); return Pair.of(mergedAvroSchema, mergedInternalSchema.getRight()); } - private InternalSchema pruneInternalSchema(Schema requiredSchema, Option internalSchemaOption) { + private InternalSchema pruneInternalSchema(HoodieSchema requiredSchema, Option internalSchemaOption) { if (!internalSchemaOption.isPresent()) { return InternalSchema.getEmptyInternalSchema(); } @@ -180,18 +160,17 @@ protected Option getInternalSchemaOpt(Option int return internalSchemaOpt; } - protected InternalSchema doPruneInternalSchema(Schema requiredSchema, InternalSchema internalSchema) { - return InternalSchemaConverter.pruneHoodieSchemaToInternalSchema(HoodieSchema.fromAvroSchema(requiredSchema), internalSchema); + protected InternalSchema doPruneInternalSchema(HoodieSchema requiredSchema, InternalSchema internalSchema) { + return InternalSchemaConverter.pruneHoodieSchemaToInternalSchema(requiredSchema, internalSchema); } @VisibleForTesting - Schema generateRequiredSchema(DeleteContext deleteContext) { + HoodieSchema generateRequiredSchema(DeleteContext deleteContext) { boolean hasInstantRange = readerContext.getInstantRange().isPresent(); //might need to change this if other queries than mor have mandatory fields if (!readerContext.getHasLogFiles()) { if (hasInstantRange && !findNestedField(requestedSchema, HoodieRecord.COMMIT_TIME_METADATA_FIELD).isPresent()) { - List addedFields = new ArrayList<>(); - addedFields.add(getField(this.tableSchema, HoodieRecord.COMMIT_TIME_METADATA_FIELD)); + List addedFields = Collections.singletonList(getField(this.tableSchema, HoodieRecord.COMMIT_TIME_METADATA_FIELD)); return appendFieldsToSchemaDedupNested(requestedSchema, addedFields); } return requestedSchema; @@ -203,7 +182,7 @@ Schema generateRequiredSchema(DeleteContext deleteContext) { } } - List addedFields = new ArrayList<>(); + List addedFields = new ArrayList<>(); for (String field : getMandatoryFieldsForMerging( hoodieTableConfig, this.properties, this.tableSchema, readerContext.getRecordMerger(), deleteContext.hasBuiltInDeleteField(), deleteContext.getCustomDeleteMarkerKeyValue(), hasInstantRange)) { @@ -221,7 +200,7 @@ Schema generateRequiredSchema(DeleteContext deleteContext) { private static String[] getMandatoryFieldsForMerging(HoodieTableConfig cfg, TypedProperties props, - Schema tableSchema, + HoodieSchema tableSchema, Option recordMerger, boolean hasBuiltInDelete, Option> customDeleteMarkerKeyAndValue, @@ -271,16 +250,16 @@ private static String[] getMandatoryFieldsForMerging(HoodieTableConfig cfg, requiredFields.add(customDeleteMarkerKeyAndValue.get().getLeft()); } // Add _hoodie_operation if it exists in table schema - if (tableSchema.getField(HoodieRecord.OPERATION_METADATA_FIELD) != null) { + if (tableSchema.getField(HoodieRecord.OPERATION_METADATA_FIELD).isPresent()) { requiredFields.add(HoodieRecord.OPERATION_METADATA_FIELD); } return requiredFields.toArray(new String[0]); } - protected Schema prepareRequiredSchema(DeleteContext deleteContext) { - Schema preReorderRequiredSchema = generateRequiredSchema(deleteContext); - Pair, List> requiredFields = getDataAndMetaCols(preReorderRequiredSchema); + protected HoodieSchema prepareRequiredSchema(DeleteContext deleteContext) { + HoodieSchema preReorderRequiredSchema = generateRequiredSchema(deleteContext); + Pair, List> requiredFields = getDataAndMetaCols(preReorderRequiredSchema); readerContext.setNeedsBootstrapMerge(readerContext.getHasBootstrapBaseFile() && !requiredFields.getLeft().isEmpty() && !requiredFields.getRight().isEmpty()); return readerContext.getNeedsBootstrapMerge() @@ -288,17 +267,17 @@ protected Schema prepareRequiredSchema(DeleteContext deleteContext) { : preReorderRequiredSchema; } - public Pair,List> getBootstrapRequiredFields() { + public Pair, List> getBootstrapRequiredFields() { return getDataAndMetaCols(requiredSchema); } - public Pair,List> getBootstrapDataFields() { + public Pair, List> getBootstrapDataFields() { return getDataAndMetaCols(tableSchema); } @VisibleForTesting - static Pair, List> getDataAndMetaCols(Schema schema) { - Map> fieldsByMeta = schema.getFields().stream() + static Pair, List> getDataAndMetaCols(HoodieSchema schema) { + Map> fieldsByMeta = schema.getFields().stream() //if there are no data fields, then we don't want to think the temp col is a data col .filter(f -> !Objects.equals(f.name(), PositionBasedFileGroupRecordBuffer.ROW_INDEX_TEMPORARY_COLUMN_NAME)) .collect(Collectors.partitioningBy(f -> HoodieRecord.HOODIE_META_COLUMNS_WITH_OPERATION.contains(f.name()))); @@ -306,23 +285,18 @@ static Pair, List> getDataAndMetaCols(Schema sc fieldsByMeta.getOrDefault(false, Collections.emptyList())); } - public Schema createSchemaFromFields(List fields) { + public HoodieSchema createSchemaFromFields(List fields) { //fields have positions set, so we need to remove them due to avro setFields implementation - for (int i = 0; i < fields.size(); i++) { - Schema.Field curr = fields.get(i); - fields.set(i, createNewSchemaField(curr)); - } - return createNewSchemaFromFieldsWithReference(tableSchema, fields); + List newFields = new ArrayList<>(fields.size()); + fields.forEach(f -> newFields.add(createNewSchemaField(f))); + return createNewSchemaFromFieldsWithReference(tableSchema, newFields); } /** * Get {@link Schema.Field} from {@link Schema} by field name. */ - private static Schema.Field getField(Schema schema, String fieldName) { - Option foundFieldOpt = findNestedField(schema, fieldName); - if (!foundFieldOpt.isPresent()) { - throw new IllegalArgumentException("Field: " + fieldName + " does not exist in the table schema"); - } - return foundFieldOpt.get(); + private static HoodieSchemaField getField(HoodieSchema schema, String fieldName) { + Option foundFieldOpt = findNestedField(schema, fieldName); + return foundFieldOpt.orElseThrow(() -> new IllegalArgumentException("Field: " + fieldName + " does not exist in the table schema")); } } 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 8d5274827f0bd..7ea9fac74cd30 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 @@ -28,6 +28,7 @@ import org.apache.hudi.common.model.HoodieLogFile; 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.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.PartitionPathParser; @@ -48,8 +49,6 @@ import org.apache.hudi.storage.StoragePath; import org.apache.hudi.storage.StoragePathInfo; -import org.apache.avro.Schema; - import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; @@ -90,7 +89,7 @@ public final class HoodieFileGroupReader implements Closeable { private BufferedRecordConverter bufferedRecordConverter; private HoodieFileGroupReader(HoodieReaderContext readerContext, HoodieStorage storage, String tablePath, - String latestCommitTime, Schema dataSchema, Schema requestedSchema, + String latestCommitTime, HoodieSchema dataSchema, HoodieSchema requestedSchema, Option internalSchemaOpt, HoodieTableMetaClient hoodieTableMetaClient, TypedProperties props, ReaderParameters readerParameters, InputSplit inputSplit, Option> updateCallback, FileGroupRecordBufferLoader recordBufferLoader) { @@ -157,13 +156,13 @@ private ClosableIterator makeBaseFileIterator() throws IOException { if (baseFileStoragePathInfo != null) { recordIterator = readerContext.getFileRecordIterator( baseFileStoragePathInfo, inputSplit.getStart(), inputSplit.getLength(), - HoodieSchema.fromAvroSchema(readerContext.getSchemaHandler().getTableSchema()), - HoodieSchema.fromAvroSchema(readerContext.getSchemaHandler().getRequiredSchema()), storage); + readerContext.getSchemaHandler().getTableSchema(), + readerContext.getSchemaHandler().getRequiredSchema(), storage); } else { recordIterator = readerContext.getFileRecordIterator( baseFile.getStoragePath(), inputSplit.getStart(), inputSplit.getLength(), - HoodieSchema.fromAvroSchema(readerContext.getSchemaHandler().getTableSchema()), - HoodieSchema.fromAvroSchema(readerContext.getSchemaHandler().getRequiredSchema()), storage); + readerContext.getSchemaHandler().getTableSchema(), + readerContext.getSchemaHandler().getRequiredSchema(), storage); } return readerContext.getInstantRange().isPresent() ? readerContext.applyInstantRangeFilter(recordIterator) @@ -172,11 +171,11 @@ private ClosableIterator makeBaseFileIterator() throws IOException { private ClosableIterator makeBootstrapBaseFileIterator(HoodieBaseFile baseFile) throws IOException { BaseFile dataFile = baseFile.getBootstrapBaseFile().get(); - Pair, List> requiredFields = readerContext.getSchemaHandler().getBootstrapRequiredFields(); - Pair, List> allFields = readerContext.getSchemaHandler().getBootstrapDataFields(); - Option, Schema>> dataFileIterator = + Pair, List> requiredFields = readerContext.getSchemaHandler().getBootstrapRequiredFields(); + Pair, List> allFields = readerContext.getSchemaHandler().getBootstrapDataFields(); + Option, HoodieSchema>> dataFileIterator = makeBootstrapBaseFileIteratorHelper(requiredFields.getRight(), allFields.getRight(), dataFile); - Option, Schema>> skeletonFileIterator = + Option, HoodieSchema>> skeletonFileIterator = makeBootstrapBaseFileIteratorHelper(requiredFields.getLeft(), allFields.getLeft(), baseFile); if (!dataFileIterator.isPresent() && !skeletonFileIterator.isPresent()) { throw new IllegalStateException("should not be here if only partition cols are required"); @@ -192,18 +191,18 @@ private ClosableIterator makeBootstrapBaseFileIterator(HoodieBaseFile baseFil 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 -> { - Schema dataSchema = dataFileIterator.get().getRight(); + HoodieSchema dataSchema = dataFileIterator.get().getRight(); List> filterFieldsAndValues = new ArrayList<>(partitionFields.length); for (int i = 0; i < partitionFields.length; i++) { String field = partitionFields[i]; - if (dataSchema.getField(field) != null) { + if (dataSchema.getField(field).isPresent()) { filterFieldsAndValues.add(Pair.of(field, readerContext.getRecordContext().convertPartitionValueToEngineType((Comparable) partitionValues[i]))); } } return filterFieldsAndValues; }).orElseGet(Collections::emptyList); - return readerContext.mergeBootstrapReaders(skeletonFileIterator.get().getLeft(), HoodieSchema.fromAvroSchema(skeletonFileIterator.get().getRight()), - dataFileIterator.get().getLeft(), HoodieSchema.fromAvroSchema(dataFileIterator.get().getRight()), partitionPathFieldsAndValues); + return readerContext.mergeBootstrapReaders(skeletonFileIterator.get().getLeft(), skeletonFileIterator.get().getRight(), + dataFileIterator.get().getLeft(), dataFileIterator.get().getRight(), partitionPathFieldsAndValues); } } @@ -215,24 +214,24 @@ private ClosableIterator makeBootstrapBaseFileIterator(HoodieBaseFile baseFil * @param file file to be read * @return pair of the record iterator of the file, and the schema of the data being read */ - private Option, Schema>> makeBootstrapBaseFileIteratorHelper(List requiredFields, - List allFields, - BaseFile file) throws IOException { + private Option, HoodieSchema>> makeBootstrapBaseFileIteratorHelper(List requiredFields, + List allFields, + BaseFile file) throws IOException { if (requiredFields.isEmpty()) { return Option.empty(); } - Schema requiredSchema = readerContext.getSchemaHandler().createSchemaFromFields(requiredFields); + HoodieSchema requiredSchema = readerContext.getSchemaHandler().createSchemaFromFields(requiredFields); StoragePathInfo fileStoragePathInfo = file.getPathInfo(); if (fileStoragePathInfo != null) { return Option.of(Pair.of(readerContext.getFileRecordIterator(fileStoragePathInfo, 0, file.getFileLen(), - HoodieSchema.fromAvroSchema(readerContext.getSchemaHandler().createSchemaFromFields(allFields)), HoodieSchema.fromAvroSchema(requiredSchema), storage), requiredSchema)); + readerContext.getSchemaHandler().createSchemaFromFields(allFields), requiredSchema, storage), requiredSchema)); } else { // If the base file length passed in is invalid, i.e., -1, // the file group reader fetches the length from the file system long fileLength = file.getFileLen() >= 0 ? file.getFileLen() : storage.getPathInfo(file.getStoragePath()).getLength(); return Option.of(Pair.of(readerContext.getFileRecordIterator(file.getStoragePath(), 0, fileLength, - HoodieSchema.fromAvroSchema(readerContext.getSchemaHandler().createSchemaFromFields(allFields)), HoodieSchema.fromAvroSchema(requiredSchema), storage), requiredSchema)); + readerContext.getSchemaHandler().createSchemaFromFields(allFields), requiredSchema, storage), requiredSchema)); } } @@ -365,8 +364,8 @@ public static class Builder { private HoodieStorage storage; private String tablePath; private String latestCommitTime; - private Schema dataSchema; - private Schema requestedSchema; + private HoodieSchema dataSchema; + private HoodieSchema requestedSchema; private Option internalSchemaOpt = Option.empty(); private HoodieTableMetaClient hoodieTableMetaClient; private TypedProperties props; @@ -422,12 +421,12 @@ public Builder withPartitionPath(String partitionPath) { return this; } - public Builder withDataSchema(Schema dataSchema) { + public Builder withDataSchema(HoodieSchema dataSchema) { this.dataSchema = dataSchema; return this; } - public Builder withRequestedSchema(Schema requestedSchema) { + public Builder withRequestedSchema(HoodieSchema requestedSchema) { this.requestedSchema = requestedSchema; return this; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/ParquetRowIndexBasedSchemaHandler.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/ParquetRowIndexBasedSchemaHandler.java index d0be7534884e0..04fb62f1e197f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/ParquetRowIndexBasedSchemaHandler.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/ParquetRowIndexBasedSchemaHandler.java @@ -21,6 +21,9 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.HoodieReaderContext; +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.HoodieTableMetaClient; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.VisibleForTesting; @@ -30,12 +33,10 @@ import org.apache.hudi.internal.schema.action.TableChanges; import org.apache.hudi.internal.schema.utils.SchemaChangeUtils; -import org.apache.avro.Schema; - import java.util.Collections; import java.util.List; -import static org.apache.hudi.avro.AvroSchemaUtils.appendFieldsToSchemaDedupNested; +import static org.apache.hudi.common.schema.HoodieSchemaUtils.appendFieldsToSchemaDedupNested; import static org.apache.hudi.common.table.read.buffer.PositionBasedFileGroupRecordBuffer.ROW_INDEX_TEMPORARY_COLUMN_NAME; /** @@ -43,8 +44,8 @@ */ public class ParquetRowIndexBasedSchemaHandler extends FileGroupReaderSchemaHandler { public ParquetRowIndexBasedSchemaHandler(HoodieReaderContext readerContext, - Schema dataSchema, - Schema requestedSchema, + HoodieSchema dataSchema, + HoodieSchema requestedSchema, Option internalSchemaOpt, TypedProperties properties, HoodieTableMetaClient metaClient) { @@ -55,8 +56,8 @@ public ParquetRowIndexBasedSchemaHandler(HoodieReaderContext readerContext, } @Override - protected Schema prepareRequiredSchema(DeleteContext deleteContext) { - Schema preMergeSchema = super.prepareRequiredSchema(deleteContext); + protected HoodieSchema prepareRequiredSchema(DeleteContext deleteContext) { + HoodieSchema preMergeSchema = super.prepareRequiredSchema(deleteContext); return readerContext.getShouldMergeUseRecordPosition() ? addPositionalMergeCol(preMergeSchema) : preMergeSchema; @@ -68,7 +69,7 @@ protected Option getInternalSchemaOpt(Option int } @Override - protected InternalSchema doPruneInternalSchema(Schema requiredSchema, InternalSchema internalSchema) { + protected InternalSchema doPruneInternalSchema(HoodieSchema requiredSchema, InternalSchema internalSchema) { if (!readerContext.getShouldMergeUseRecordPosition()) { return super.doPruneInternalSchema(requiredSchema, internalSchema); } @@ -84,8 +85,8 @@ private static InternalSchema addPositionalMergeCol(InternalSchema internalSchem } @Override - public Pair,List> getBootstrapRequiredFields() { - Pair,List> dataAndMetaCols = super.getBootstrapRequiredFields(); + public Pair, List> getBootstrapRequiredFields() { + Pair, List> dataAndMetaCols = super.getBootstrapRequiredFields(); if (readerContext.getNeedsBootstrapMerge() || readerContext.getShouldMergeUseRecordPosition()) { if (!dataAndMetaCols.getLeft().isEmpty()) { dataAndMetaCols.getLeft().add(getPositionalMergeField()); @@ -98,13 +99,13 @@ public Pair,List> getBootstrapRequiredFields() } @VisibleForTesting - static Schema addPositionalMergeCol(Schema input) { + static HoodieSchema addPositionalMergeCol(HoodieSchema input) { return appendFieldsToSchemaDedupNested(input, Collections.singletonList(getPositionalMergeField())); } @VisibleForTesting - static Schema.Field getPositionalMergeField() { - return new Schema.Field(ROW_INDEX_TEMPORARY_COLUMN_NAME, - Schema.create(Schema.Type.LONG), "", -1L); + static HoodieSchemaField getPositionalMergeField() { + return HoodieSchemaField.of(ROW_INDEX_TEMPORARY_COLUMN_NAME, + HoodieSchema.create(HoodieSchemaType.LONG), "", -1L); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/UpdateProcessor.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/UpdateProcessor.java index d0549062a1678..0f6b647c868d3 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/UpdateProcessor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/UpdateProcessor.java @@ -30,7 +30,6 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.metadata.HoodieMetadataPayload; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import java.io.IOException; @@ -142,9 +141,9 @@ protected BufferedRecord handleNonDeletes(BufferedRecord previousRecord, B if (hoodieRecord.shouldIgnore(recordSchema.toAvroSchema(), properties)) { return null; } else { - Schema readerSchema = readerContext.getSchemaHandler().getRequestedSchema(); + HoodieSchema readerSchema = readerContext.getSchemaHandler().getRequestedSchema(); // If the record schema is different from the reader schema, rewrite the record using the payload methods to ensure consistency with legacy writer paths - hoodieRecord.rewriteRecordWithNewSchema(recordSchema.toAvroSchema(), properties, readerSchema).toIndexedRecord(readerSchema, properties) + hoodieRecord.rewriteRecordWithNewSchema(recordSchema.toAvroSchema(), properties, readerSchema.toAvroSchema()).toIndexedRecord(readerSchema.toAvroSchema(), properties) .ifPresent(rewrittenRecord -> mergedRecord.replaceRecord(readerContext.getRecordContext().convertAvroRecord(rewrittenRecord.getData()))); } } catch (IOException e) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/buffer/FileGroupRecordBuffer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/buffer/FileGroupRecordBuffer.java index db325c6e68c9e..783983cad34e6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/buffer/FileGroupRecordBuffer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/buffer/FileGroupRecordBuffer.java @@ -98,7 +98,7 @@ protected FileGroupRecordBuffer(HoodieReaderContext readerContext, UpdateProcessor updateProcessor) { this.readerContext = readerContext; this.updateProcessor = updateProcessor; - this.readerSchema = HoodieSchemaCache.intern(HoodieSchema.fromAvroSchema(readerContext.getSchemaHandler().getRequiredSchema())); + this.readerSchema = HoodieSchemaCache.intern(readerContext.getSchemaHandler().getRequiredSchema()); this.recordMergeMode = recordMergeMode; this.partialUpdateModeOpt = partialUpdateModeOpt; this.recordMerger = readerContext.getRecordMerger(); @@ -116,9 +116,9 @@ protected FileGroupRecordBuffer(HoodieReaderContext readerContext, throw new HoodieIOException("IOException when creating ExternalSpillableMap at " + spillableMapBasePath, e); } this.bufferedRecordMerger = BufferedRecordMergerFactory.create( - readerContext, recordMergeMode, enablePartialMerging, recordMerger, readerSchema.toAvroSchema(), payloadClasses, props, partialUpdateModeOpt); + readerContext, recordMergeMode, enablePartialMerging, recordMerger, readerSchema, payloadClasses, props, partialUpdateModeOpt); this.deleteContext = readerContext.getSchemaHandler().getDeleteContext().withReaderSchema(this.readerSchema); - this.bufferedRecordConverter = BufferedRecordConverter.createConverter(readerContext.getIteratorMode(), readerSchema.toAvroSchema(), readerContext.getRecordContext(), orderingFieldNames); + this.bufferedRecordConverter = BufferedRecordConverter.createConverter(readerContext.getIteratorMode(), readerSchema, readerContext.getRecordContext(), orderingFieldNames); } protected ExternalSpillableMap> initializeRecordsMap(String spillableMapBasePath) throws IOException { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/buffer/KeyBasedFileGroupRecordBuffer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/buffer/KeyBasedFileGroupRecordBuffer.java index 0d6e406472da7..f052f5cee844f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/buffer/KeyBasedFileGroupRecordBuffer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/buffer/KeyBasedFileGroupRecordBuffer.java @@ -82,7 +82,7 @@ public void processDataBlock(HoodieDataBlock dataBlock, Option keySpecO recordMergeMode, true, recordMerger, - readerSchema.toAvroSchema(), + readerSchema, payloadClasses, props, partialUpdateModeOpt); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/buffer/PositionBasedFileGroupRecordBuffer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/buffer/PositionBasedFileGroupRecordBuffer.java index 1e879eb8a31a9..18f9acc435111 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/buffer/PositionBasedFileGroupRecordBuffer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/buffer/PositionBasedFileGroupRecordBuffer.java @@ -120,7 +120,7 @@ public void processDataBlock(HoodieDataBlock dataBlock, Option keySpecO recordMergeMode, true, recordMerger, - readerSchema.toAvroSchema(), + readerSchema, payloadClasses, props, partialUpdateModeOpt); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/buffer/StreamingFileGroupRecordBufferLoader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/buffer/StreamingFileGroupRecordBufferLoader.java index 85b7bc67cb81c..02c1d5b4cb9c1 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/buffer/StreamingFileGroupRecordBufferLoader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/buffer/StreamingFileGroupRecordBufferLoader.java @@ -64,7 +64,7 @@ public Pair, List> getRecordBuffer(Hoodie TypedProperties props, ReaderParameters readerParameters, HoodieReadStats readStats, Option> fileGroupUpdateCallback) { HoodieSchema recordSchema = HoodieSchemaCache.intern(getRecordSchema(readerContext, props)); - readerContext.getSchemaHandler().setSchemaForUpdates(recordSchema.toAvroSchema()); + readerContext.getSchemaHandler().setSchemaForUpdates(recordSchema); HoodieTableConfig tableConfig = hoodieTableMetaClient.getTableConfig(); Option partialUpdateModeOpt = tableConfig.getPartialUpdateMode(); UpdateProcessor updateProcessor = UpdateProcessor.create(readStats, readerContext, readerParameters.emitDeletes(), fileGroupUpdateCallback, props); @@ -100,7 +100,7 @@ private static HoodieSchema getRecordSchema(HoodieReaderContext readerCon String schemaStr = props.getString("hoodie.payload.record.schema"); return HoodieSchema.parse(schemaStr); } else { - return HoodieSchemaUtils.removeMetadataFields(HoodieSchema.fromAvroSchema(readerContext.getSchemaHandler().getRequestedSchema())); + return HoodieSchemaUtils.removeMetadataFields(readerContext.getSchemaHandler().getRequestedSchema()); } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieBootstrapFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieBootstrapFileReader.java index 3920f3499a12e..66aa763e5f40d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieBootstrapFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieBootstrapFileReader.java @@ -18,7 +18,6 @@ package org.apache.hudi.io.storage; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.schema.HoodieSchema; @@ -85,8 +84,7 @@ protected void setPartitionPathField(int position, Object fieldValue, T row) { @Override public ClosableIterator getRecordKeyIterator() throws IOException { - // TODO boundary for now to revisit HoodieAvroUtils in later pr to use HoodieSchema - HoodieSchema schema = HoodieSchema.fromAvroSchema(HoodieAvroUtils.getRecordKeySchema()); + HoodieSchema schema = HoodieSchemaUtils.getRecordKeySchema(); ClosableIterator> skeletonIterator = skeletonFileReader.getRecordIterator(schema, schema); return new ClosableIterator() { @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java index 4ec41d412a6e2..5f83a5a6cc426 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java @@ -19,7 +19,6 @@ package org.apache.hudi.metadata; import org.apache.hudi.avro.HoodieAvroReaderContext; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieMetadataRecord; import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.config.HoodieMetadataConfig; @@ -39,6 +38,8 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordGlobalLocation; +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.log.InstantRange; import org.apache.hudi.common.table.read.FileGroupReaderSchemaHandler; @@ -72,7 +73,6 @@ import org.apache.hudi.storage.StoragePath; import org.apache.hudi.storage.StoragePathInfo; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.slf4j.Logger; @@ -110,7 +110,7 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata { private static final Logger LOG = LoggerFactory.getLogger(HoodieBackedTableMetadata.class); - private static final Schema SCHEMA = HoodieAvroUtils.addMetadataFields(HoodieMetadataRecord.getClassSchema()); + private static final HoodieSchema SCHEMA = HoodieSchemaUtils.addMetadataFields(HoodieSchema.fromAvroSchema(HoodieMetadataRecord.getClassSchema())); private final String metadataBasePath; private final HoodieDataCleanupManager dataCleanupManager = new HoodieDataCleanupManager(); diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java index ef95791335d84..7fa98d0c6ebb0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -1086,7 +1086,7 @@ private static ClosableIterator> getLogRecords(List(readerContext, writerSchemaOpt.get().toAvroSchema(), writerSchemaOpt.get().toAvroSchema(), Option.empty(), properties, datasetMetaClient)); + new FileGroupReaderSchemaHandler<>(readerContext, writerSchemaOpt.get(), writerSchemaOpt.get(), Option.empty(), properties, datasetMetaClient)); HoodieReadStats readStats = new HoodieReadStats(); KeyBasedFileGroupRecordBuffer recordBuffer = new KeyBasedFileGroupRecordBuffer<>(readerContext, datasetMetaClient, readerContext.getMergeMode(), Option.empty(), properties, tableConfig.getOrderingFields(), @@ -1823,8 +1823,8 @@ public static List> getLogFileColumnRangeM .withLogFiles(Stream.of(logFile)) .withPartitionPath(partitionPath) .withBaseFileOption(Option.empty()) - .withDataSchema(writerSchemaOpt.get().toAvroSchema()) - .withRequestedSchema(writerSchemaOpt.get().toAvroSchema()) + .withDataSchema(writerSchemaOpt.get()) + .withRequestedSchema(writerSchemaOpt.get()) .withLatestCommitTime(datasetMetaClient.getActiveTimeline().getCommitsTimeline().lastInstant().get().requestedTime()) .withProps(properties) .build(); @@ -2538,9 +2538,9 @@ public static HoodieData readRecordKeysFromFileSlices(HoodieEn final int parallelism = Math.min(partitionFileSlicePairs.size(), recordIndexMaxParallelism); final StoragePath basePath = metaClient.getBasePath(); final StorageConfiguration storageConf = metaClient.getStorageConf(); - final Schema tableSchema; + final HoodieSchema tableSchema; try { - tableSchema = new TableSchemaResolver(metaClient).getTableAvroSchema(); + tableSchema = new TableSchemaResolver(metaClient).getTableSchema(); } catch (Exception e) { throw new HoodieException("Unable to resolve table schema for table", e); } @@ -2555,7 +2555,7 @@ public static HoodieData readRecordKeysFromFileSlices(HoodieEn .withHoodieTableMetaClient(metaClient) .withFileSlice(fileSlice) .withDataSchema(tableSchema) - .withRequestedSchema(HoodieAvroUtils.getRecordKeySchema()) + .withRequestedSchema(HoodieSchemaUtils.getRecordKeySchema()) .withLatestCommitTime(latestCommitTime) .withProps(getFileGroupReaderPropertiesFromStorageConf(storageConf)) .build(); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/schema/TestHoodieSchemaUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/schema/TestHoodieSchemaUtils.java index d0f295cf1c05a..b170ff30613f7 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/schema/TestHoodieSchemaUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/schema/TestHoodieSchemaUtils.java @@ -25,6 +25,9 @@ import org.apache.avro.Schema; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import java.util.Arrays; import java.util.Collections; @@ -32,6 +35,7 @@ import java.util.List; import java.util.Set; import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -67,6 +71,38 @@ public class TestHoodieSchemaUtils { + "{\"name\": \"non_pii_col\", \"type\": \"string\"}," + "{\"name\": \"pii_col\", \"type\": \"string\", \"column_category\": \"user_profile\"}]}"; + private static final String SCHEMA_WITH_AVRO_TYPES_STR = "{\"name\":\"TestRecordAvroTypes\",\"type\":\"record\",\"fields\":[" + // Primitive types + + "{\"name\":\"booleanField\",\"type\":\"boolean\"}," + + "{\"name\":\"intField\",\"type\":\"int\"}," + + "{\"name\":\"longField\",\"type\":\"long\"}," + + "{\"name\":\"floatField\",\"type\":\"float\"}," + + "{\"name\":\"doubleField\",\"type\":\"double\"}," + + "{\"name\":\"bytesField\",\"type\":\"bytes\"}," + + "{\"name\":\"stringField\",\"type\":\"string\"}," + + "{\"name\":\"secondLevelField\",\"type\":[\"null\", {\"name\":\"secondLevelField\",\"type\":\"record\",\"fields\":[" + + "{\"name\":\"firstname\",\"type\":[\"null\",\"string\"],\"default\":null}," + + "{\"name\":\"lastname\",\"type\":[\"null\",\"string\"],\"default\":null}" + + "]}],\"default\":null}," + // Logical types + + "{\"name\":\"decimalField\",\"type\":\"bytes\",\"logicalType\":\"decimal\",\"precision\":20,\"scale\":5}," + + "{\"name\":\"timeMillisField\",\"type\":\"int\",\"logicalType\":\"time-millis\"}," + + "{\"name\":\"timeMicrosField\",\"type\":\"long\",\"logicalType\":\"time-micros\"}," + + "{\"name\":\"timestampMillisField\",\"type\":\"long\",\"logicalType\":\"timestamp-millis\"}," + + "{\"name\":\"timestampMicrosField\",\"type\":\"long\",\"logicalType\":\"timestamp-micros\"}," + + "{\"name\":\"localTimestampMillisField\",\"type\":\"long\",\"logicalType\":\"local-timestamp-millis\"}," + + "{\"name\":\"localTimestampMicrosField\",\"type\":\"long\",\"logicalType\":\"local-timestamp-micros\"}" + + "]}"; + + private static String SCHEMA_WITH_NESTED_FIELD_LARGE_STR = "{\"name\":\"MyClass\",\"type\":\"record\",\"namespace\":\"com.acme.avro\",\"fields\":[" + + "{\"name\":\"firstname\",\"type\":\"string\"}," + + "{\"name\":\"lastname\",\"type\":\"string\"}," + + "{\"name\":\"nested_field\",\"type\":[\"null\"," + SCHEMA_WITH_AVRO_TYPES_STR + "],\"default\":null}," + + "{\"name\":\"student\",\"type\":{\"name\":\"student\",\"type\":\"record\",\"fields\":[" + + "{\"name\":\"firstname\",\"type\":[\"null\" ,\"string\"],\"default\": null},{\"name\":\"lastname\",\"type\":[\"null\" ,\"string\"],\"default\": null}]}}]}"; + + private static HoodieSchema SCHEMA_WITH_NESTED_FIELD_LARGE = HoodieSchema.parse(SCHEMA_WITH_NESTED_FIELD_LARGE_STR); + @Test public void testCreateHoodieWriteSchema() { // Test with operation field @@ -1245,4 +1281,148 @@ public void testCreateNewSchemaFromFieldsWithReference_WithObjectProps() { assertEquals("value1", newSchema.getProp("prop1")); assertEquals("newField", newSchema.getFields().get(0).name()); } + + static Stream getExpectedSchemaForFields() { + // Projection of two nested fields. secondLevelField is entirely projected since both its fields are included + List fields1 = Arrays.asList("nested_field.secondLevelField.firstname", "nested_field.secondLevelField.lastname"); + // Expected schema - top level field and one nested field + String expectedSchema1 = + "{\n" + + " \"type\": \"record\",\n" + + " \"name\": \"MyClass\",\n" + + " \"doc\": \"\",\n" + + " \"namespace\": \"com.acme.avro\",\n" + + " \"fields\": [\n" + + " { \"name\": \"nested_field\", \"type\": [\"null\", {\n" + + " \"type\": \"record\",\n" + + " \"name\": \"TestRecordAvroTypes\",\n" + + " \"fields\": [\n" + + " { \"name\": \"secondLevelField\", \"type\": [\"null\", {\n" + + " \"type\": \"record\",\n" + + " \"name\": \"secondLevelField\",\n" + + " \"fields\": [\n" + + " { \"name\": \"firstname\", \"type\": [\"null\", \"string\"], \"default\": null },\n" + + " { \"name\": \"lastname\", \"type\": [\"null\", \"string\"], \"default\": null }\n" + + " ]\n" + + " }], \"default\": null }\n" + + " ]\n" + + " }], \"default\": null }\n" + + " ]\n" + + "}"; + + // Projection of first level nested field and top level field which contains the nested field + // Also include the nested field twice + // Expected schema - top level field + List fields2 = Arrays.asList("nested_field.secondLevelField.lastname", "nested_field", + "nested_field.secondLevelField.lastname"); + String expectedSchema2 = + "{\n" + + " \"type\": \"record\",\n" + + " \"name\": \"MyClass\",\n" + + " \"doc\": \"\",\n" + + " \"namespace\": \"com.acme.avro\",\n" + + " \"fields\": [\n" + + " { \"name\": \"nested_field\", \"type\": [\"null\", " + SCHEMA_WITH_AVRO_TYPES_STR + "], \"default\": null }\n" + + " ]\n" + + "}"; + + // Projection of non overlapping nested field and top level field with nested fields + // Expected schema - top level field and one nested field + List fields3 = Arrays.asList("student.lastname", "nested_field"); + String expectedSchema3 = + "{\n" + + " \"type\": \"record\",\n" + + " \"name\": \"MyClass\",\n" + + " \"doc\": \"\",\n" + + " \"namespace\": \"com.acme.avro\",\n" + + " \"fields\": [\n" + + " { \"name\": \"nested_field\", \"type\": [\"null\", " + SCHEMA_WITH_AVRO_TYPES_STR + "], \"default\": null },\n" + + " { \"name\": \"student\", \"type\": {\n" + + " \"type\": \"record\",\n" + + " \"name\": \"student\",\n" + + " \"fields\": [\n" + + " { \"name\": \"lastname\", \"type\": [\"null\", \"string\"], \"default\": null }\n" + + " ]\n" + + " }}\n" + + " ]\n" + + "}"; + + // Projection of two nested fields + // Expected schema - two nested fields + List fields4 = Arrays.asList("student.lastname", "nested_field.secondLevelField.lastname"); + String expectedSchema4 = + "{\n" + + " \"type\": \"record\",\n" + + " \"name\": \"MyClass\",\n" + + " \"doc\": \"\",\n" + + " \"namespace\": \"com.acme.avro\",\n" + + " \"fields\": [\n" + + " { \"name\": \"nested_field\", \"type\": [\"null\", {\n" + + " \"type\": \"record\",\n" + + " \"name\": \"TestRecordAvroTypes\",\n" + + " \"fields\": [\n" + + " { \"name\": \"secondLevelField\", \"type\": [\"null\", {\n" + + " \"type\": \"record\",\n" + + " \"name\": \"secondLevelField\",\n" + + " \"fields\": [\n" + + " { \"name\": \"lastname\", \"type\": [\"null\", \"string\"], \"default\": null }\n" + + " ]\n" + + " }], \"default\": null }\n" + + " ]\n" + + " }], \"default\": null },\n" + + " { \"name\": \"student\", \"type\": {\n" + + " \"type\": \"record\",\n" + + " \"name\": \"student\",\n" + + " \"namespace\": \"com.acme.avro\"," + + " \"fields\": [\n" + + " { \"name\": \"lastname\", \"type\": [\"null\", \"string\"], \"default\": null }\n" + + " ]\n" + + " }}\n" + + " ]\n" + + "}"; + + // Projection of top level field and nested field column + List fields5 = Arrays.asList("firstname", "nested_field.secondLevelField.lastname", "nested_field.longField"); + // Expected schema - top level field and one nested field + String expectedSchema5 = + "{\n" + + " \"type\": \"record\",\n" + + " \"name\": \"MyClass\",\n" + + " \"doc\": \"\",\n" + + " \"namespace\": \"com.acme.avro\",\n" + + " \"fields\": [\n" + + " { \"name\": \"firstname\", \"type\": \"string\" },\n" + + " { \"name\": \"nested_field\", \"type\": [\"null\", {\n" + + " \"type\": \"record\",\n" + + " \"name\": \"TestRecordAvroTypes\",\n" + + " \"fields\": [\n" + + " { \"name\": \"longField\", \"type\": \"long\" },\n" + + " { \"name\": \"secondLevelField\", \"type\": [\"null\", {\n" + + " \"type\": \"record\",\n" + + " \"name\": \"secondLevelField\",\n" + + " \"fields\": [\n" + + " { \"name\": \"lastname\", \"type\": [\"null\", \"string\"], \"default\": null }\n" + + " ]\n" + + " }], \"default\": null }\n" + + " ]\n" + + " }], \"default\": null }\n" + + " ]\n" + + "}"; + + return Stream.of( + Arguments.of(fields1, expectedSchema1), + Arguments.of(fields2, expectedSchema2), + Arguments.of(fields3, expectedSchema3), + Arguments.of(fields4, expectedSchema4), + Arguments.of(fields5, expectedSchema5)); + } + + @ParameterizedTest + @MethodSource("getExpectedSchemaForFields") + void testProjectSchemaWithNullableAndNestedFields(List projectedFields, String expectedSchemaStr) { + HoodieSchema expectedSchema = HoodieSchema.parse(expectedSchemaStr); + HoodieSchema projectedSchema = HoodieSchemaUtils.projectSchema(SCHEMA_WITH_NESTED_FIELD_LARGE, projectedFields); + assertEquals(expectedSchema, projectedSchema); + assertTrue(HoodieSchemaCompatibility.isSchemaCompatible(projectedSchema, expectedSchema, 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 e65f449929195..1476c1ecd2c9a 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 @@ -19,9 +19,10 @@ package org.apache.hudi.common.table; +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaType; import org.apache.hudi.common.util.Option; -import org.apache.avro.Schema; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; @@ -55,7 +56,7 @@ private static Stream partitionPathCases() { @MethodSource("partitionPathCases") void testGetPartitionFieldVals(String partitionPath, String[] partitionFields, Object[] expectedValues) { PartitionPathParser parser = new PartitionPathParser(); - Schema schema = new Schema.Parser().parse("{\"type\":\"record\",\"name\":\"TestRecord\",\"fields\":[{\"name\":\"string_field\",\"type\":[\"null\", \"string\"]}," + HoodieSchema schema = HoodieSchema.parse("{\"type\":\"record\",\"name\":\"TestRecord\",\"fields\":[{\"name\":\"string_field\",\"type\":[\"null\", \"string\"]}," + "{\"name\":\"date_field\",\"type\": {\"type\":\"int\",\"logicalType\": \"date\"}},{\"name\":\"timestamp_field\",\"type\": {\"type\":\"long\",\"logicalType\": \"timestamp-millis\"}}]}"); Object[] result = parser.getPartitionFieldVals(Option.ofNullable(partitionFields), partitionPath, schema); @@ -67,22 +68,22 @@ void testGetPartitionFieldVals(String partitionPath, String[] partitionFields, O private static Stream fieldCases() { return Stream.of( - Arguments.of("123", Schema.create(Schema.Type.LONG), 123L), - Arguments.of("123", Schema.create(Schema.Type.INT), 123), - Arguments.of("123.45", Schema.create(Schema.Type.DOUBLE), 123.45), - Arguments.of("123.45", Schema.create(Schema.Type.FLOAT), 123.45f), - Arguments.of("false", Schema.create(Schema.Type.BOOLEAN), false), - Arguments.of("__HIVE_DEFAULT_PARTITION__", Schema.create(Schema.Type.INT), null), - Arguments.of("default", Schema.create(Schema.Type.INT), null), - Arguments.of("2025-01-03", Schema.create(Schema.Type.STRING), "2025-01-03"), - Arguments.of("value1", Schema.create(Schema.Type.BYTES), "value1".getBytes(StandardCharsets.UTF_8)), - Arguments.of("value1", Schema.createFixed("fixed", "docs", null, 50), "value1".getBytes(StandardCharsets.UTF_8)) + Arguments.of("123", HoodieSchema.create(HoodieSchemaType.LONG), 123L), + Arguments.of("123", HoodieSchema.create(HoodieSchemaType.INT), 123), + Arguments.of("123.45", HoodieSchema.create(HoodieSchemaType.DOUBLE), 123.45), + Arguments.of("123.45", HoodieSchema.create(HoodieSchemaType.FLOAT), 123.45f), + Arguments.of("false", HoodieSchema.create(HoodieSchemaType.BOOLEAN), false), + Arguments.of("__HIVE_DEFAULT_PARTITION__", HoodieSchema.create(HoodieSchemaType.INT), null), + 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)) ); } @ParameterizedTest @MethodSource("fieldCases") - void testValueParsing(String value, Schema fieldSchema, Object expected) { + void testValueParsing(String value, HoodieSchema fieldSchema, Object expected) { if (expected instanceof byte[]) { String expectedString = new String((byte[]) expected, StandardCharsets.UTF_8); String actualString = new String((byte[]) PartitionPathParser.parseValue(value, fieldSchema)); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/read/SchemaHandlerTestBase.java b/hudi-common/src/test/java/org/apache/hudi/common/table/read/SchemaHandlerTestBase.java index 6661fc89c2c66..c59923cfa232e 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/read/SchemaHandlerTestBase.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/read/SchemaHandlerTestBase.java @@ -19,13 +19,14 @@ package org.apache.hudi.common.table.read; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.config.RecordMergeMode; import org.apache.hudi.common.engine.HoodieReaderContext; import org.apache.hudi.common.engine.RecordContext; import org.apache.hudi.common.model.HoodieRecord; 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.HoodieTableVersion; @@ -37,7 +38,6 @@ import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.junit.jupiter.api.BeforeEach; @@ -65,11 +65,11 @@ public abstract class SchemaHandlerTestBase { - protected static final Schema DATA_SCHEMA = HoodieAvroUtils.addMetadataFields(HoodieTestDataGenerator.AVRO_SCHEMA); - protected static final Schema DATA_SCHEMA_NO_DELETE = generateProjectionSchema(DATA_SCHEMA.getFields().stream() - .map(Schema.Field::name).filter(f -> !f.equals("_hoodie_is_deleted")).toArray(String[]::new)); - protected static final Schema DATA_COLS_ONLY_SCHEMA = generateProjectionSchema("begin_lat", "tip_history", "rider"); - protected static final Schema META_COLS_ONLY_SCHEMA = generateProjectionSchema("_hoodie_commit_seqno", "_hoodie_record_key"); + protected static final HoodieSchema DATA_SCHEMA = HoodieSchemaUtils.addMetadataFields(HoodieTestDataGenerator.HOODIE_SCHEMA); + protected static final HoodieSchema DATA_SCHEMA_NO_DELETE = generateProjectionSchema(DATA_SCHEMA.getFields().stream() + .map(HoodieSchemaField::name).filter(f -> !f.equals("_hoodie_is_deleted")).toArray(String[]::new)); + protected static final HoodieSchema DATA_COLS_ONLY_SCHEMA = generateProjectionSchema("begin_lat", "tip_history", "rider"); + protected static final HoodieSchema META_COLS_ONLY_SCHEMA = generateProjectionSchema("_hoodie_commit_seqno", "_hoodie_record_key"); protected final HoodieTableMetaClient metaClient = mock(HoodieTableMetaClient.class); protected final HoodieTableConfig hoodieTableConfig = mock(HoodieTableConfig.class); @@ -98,15 +98,15 @@ public void testMor(RecordMergeMode mergeMode, boolean mergeUseRecordPosition, boolean supportsParquetRowIndex, boolean hasBuiltInDelete) throws IOException { - Schema dataSchema = hasBuiltInDelete ? DATA_SCHEMA : DATA_SCHEMA_NO_DELETE; + HoodieSchema dataSchema = hasBuiltInDelete ? DATA_SCHEMA : DATA_SCHEMA_NO_DELETE; setupMORTable(mergeMode, hasPrecombine, hoodieTableConfig); HoodieRecordMerger merger = mockRecordMerger(isProjectionCompatible, isProjectionCompatible ? new String[] {"begin_lat", "begin_lon", "_hoodie_record_key", "timestamp"} : new String[] {"begin_lat", "begin_lon", "timestamp"}); HoodieReaderContext readerContext = createReaderContext(hoodieTableConfig, supportsParquetRowIndex, true, false, mergeUseRecordPosition, merger); readerContext.setRecordMerger(Option.of(merger)); - Schema requestedSchema = dataSchema; + HoodieSchema requestedSchema = dataSchema; FileGroupReaderSchemaHandler schemaHandler = createSchemaHandler(readerContext, dataSchema, requestedSchema, supportsParquetRowIndex); - Schema expectedRequiredFullSchema = supportsParquetRowIndex && mergeUseRecordPosition + HoodieSchema expectedRequiredFullSchema = supportsParquetRowIndex && mergeUseRecordPosition ? ParquetRowIndexBasedSchemaHandler.addPositionalMergeCol(requestedSchema) : requestedSchema; assertEquals(expectedRequiredFullSchema, schemaHandler.getRequiredSchema()); @@ -115,7 +115,7 @@ public void testMor(RecordMergeMode mergeMode, //read subset of columns requestedSchema = DATA_COLS_ONLY_SCHEMA; schemaHandler = createSchemaHandler(readerContext, dataSchema, requestedSchema, supportsParquetRowIndex); - Schema expectedRequiredSchema; + HoodieSchema expectedRequiredSchema; if (mergeMode == EVENT_TIME_ORDERING && hasPrecombine) { expectedRequiredSchema = generateProjectionSchema(hasBuiltInDelete, "begin_lat", "tip_history", "rider", "_hoodie_record_key", "timestamp"); } else if (mergeMode == EVENT_TIME_ORDERING || mergeMode == COMMIT_TIME_ORDERING) { @@ -138,20 +138,20 @@ public void testMorBootstrap(RecordMergeMode mergeMode, boolean mergeUseRecordPosition, boolean supportsParquetRowIndex, boolean hasBuiltInDelete) throws IOException { - Schema dataSchema = hasBuiltInDelete ? DATA_SCHEMA : DATA_SCHEMA_NO_DELETE; + HoodieSchema dataSchema = hasBuiltInDelete ? DATA_SCHEMA : DATA_SCHEMA_NO_DELETE; setupMORTable(mergeMode, hasPrecombine, hoodieTableConfig); HoodieRecordMerger merger = mockRecordMerger(isProjectionCompatible, new String[] {"begin_lat", "begin_lon", "timestamp"}); HoodieReaderContext readerContext = createReaderContext(hoodieTableConfig, supportsParquetRowIndex, true, true, mergeUseRecordPosition, merger); readerContext.setRecordMerger(Option.of(merger)); - Schema requestedSchema = dataSchema; + HoodieSchema requestedSchema = dataSchema; FileGroupReaderSchemaHandler schemaHandler = createSchemaHandler(readerContext, dataSchema, requestedSchema, supportsParquetRowIndex); - Schema expectedRequiredFullSchema = supportsParquetRowIndex && mergeUseRecordPosition + HoodieSchema expectedRequiredFullSchema = supportsParquetRowIndex && mergeUseRecordPosition ? ParquetRowIndexBasedSchemaHandler.addPositionalMergeCol(requestedSchema) : requestedSchema; assertEquals(expectedRequiredFullSchema, schemaHandler.getRequiredSchema()); assertTrue(readerContext.getNeedsBootstrapMerge()); - Pair, List> bootstrapFields = schemaHandler.getBootstrapRequiredFields(); - Pair, List> expectedBootstrapFields = FileGroupReaderSchemaHandler.getDataAndMetaCols(expectedRequiredFullSchema); + Pair, List> bootstrapFields = schemaHandler.getBootstrapRequiredFields(); + Pair, List> expectedBootstrapFields = FileGroupReaderSchemaHandler.getDataAndMetaCols(expectedRequiredFullSchema); if (supportsParquetRowIndex) { expectedBootstrapFields.getLeft().add(ParquetRowIndexBasedSchemaHandler.getPositionalMergeField()); expectedBootstrapFields.getRight().add(ParquetRowIndexBasedSchemaHandler.getPositionalMergeField()); @@ -162,7 +162,7 @@ public void testMorBootstrap(RecordMergeMode mergeMode, //read subset of columns requestedSchema = generateProjectionSchema("begin_lat", "tip_history", "_hoodie_record_key", "rider"); schemaHandler = createSchemaHandler(readerContext, dataSchema, requestedSchema, supportsParquetRowIndex); - Schema expectedRequiredSchema; + HoodieSchema expectedRequiredSchema; if (mergeMode == EVENT_TIME_ORDERING && hasPrecombine) { expectedRequiredSchema = generateProjectionSchema(hasBuiltInDelete, "_hoodie_record_key", "begin_lat", "tip_history", "rider", "timestamp"); } else if (mergeMode == EVENT_TIME_ORDERING || mergeMode == COMMIT_TIME_ORDERING) { @@ -289,24 +289,24 @@ static HoodieReaderContext createReaderContext(HoodieTableConfig hoodieT return readerContext; } - abstract FileGroupReaderSchemaHandler createSchemaHandler(HoodieReaderContext readerContext, Schema dataSchema, - Schema requestedSchema, boolean supportsParquetRowIndex); + abstract FileGroupReaderSchemaHandler createSchemaHandler(HoodieReaderContext readerContext, HoodieSchema dataSchema, + HoodieSchema requestedSchema, boolean supportsParquetRowIndex); - static Schema generateProjectionSchema(String... fields) { - return HoodieAvroUtils.generateProjectionSchema(DATA_SCHEMA, Arrays.asList(fields)); + static HoodieSchema generateProjectionSchema(String... fields) { + return HoodieSchemaUtils.generateProjectionSchema(DATA_SCHEMA, Arrays.asList(fields)); } - private static Schema generateProjectionSchema(boolean hasBuiltInDelete, String... fields) { + private static HoodieSchema generateProjectionSchema(boolean hasBuiltInDelete, String... fields) { List fieldList = Arrays.asList(fields); if (hasBuiltInDelete) { fieldList = new ArrayList<>(fieldList); fieldList.add("_hoodie_is_deleted"); } - return HoodieAvroUtils.generateProjectionSchema(DATA_SCHEMA, fieldList); + return HoodieSchemaUtils.generateProjectionSchema(DATA_SCHEMA, fieldList); } - Schema.Field getField(String fieldName) { - return DATA_SCHEMA.getField(fieldName); + HoodieSchemaField getField(String fieldName) { + return DATA_SCHEMA.getField(fieldName).get(); } static class StubbedReaderContext extends HoodieReaderContext { diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestFileGroupReaderSchemaHandler.java b/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestFileGroupReaderSchemaHandler.java index 9672e20caabb9..0f6c274c0f7d0 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestFileGroupReaderSchemaHandler.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestFileGroupReaderSchemaHandler.java @@ -74,7 +74,7 @@ public class TestFileGroupReaderSchemaHandler extends SchemaHandlerTestBase { public void testCow() { when(hoodieTableConfig.populateMetaFields()).thenReturn(true); HoodieReaderContext readerContext = createReaderContext(hoodieTableConfig, false, false, false, false, null); - Schema requestedSchema = DATA_SCHEMA; + HoodieSchema requestedSchema = DATA_SCHEMA; FileGroupReaderSchemaHandler schemaHandler = createSchemaHandler(readerContext, DATA_SCHEMA, requestedSchema, false); assertEquals(requestedSchema, schemaHandler.getRequiredSchema()); @@ -89,12 +89,12 @@ public void testCow() { public void testCowBootstrap() { when(hoodieTableConfig.populateMetaFields()).thenReturn(true); HoodieReaderContext readerContext = createReaderContext(hoodieTableConfig, false, false, true, false, null); - Schema requestedSchema = generateProjectionSchema("begin_lat", "tip_history", "_hoodie_record_key", "rider"); + HoodieSchema requestedSchema = generateProjectionSchema("begin_lat", "tip_history", "_hoodie_record_key", "rider"); //meta cols must go first in the required schema FileGroupReaderSchemaHandler schemaHandler = createSchemaHandler(readerContext, DATA_SCHEMA, requestedSchema, false); assertTrue(readerContext.getNeedsBootstrapMerge()); - Schema expectedRequiredSchema = generateProjectionSchema("_hoodie_record_key", "begin_lat", "tip_history", "rider"); + HoodieSchema expectedRequiredSchema = generateProjectionSchema("_hoodie_record_key", "begin_lat", "tip_history", "rider"); assertEquals(expectedRequiredSchema, schemaHandler.getRequiredSchema()); Pair, List> bootstrapFields = schemaHandler.getBootstrapRequiredFields(); assertEquals(Collections.singletonList(getField("_hoodie_record_key")), bootstrapFields.getLeft()); @@ -105,9 +105,9 @@ public void testCowBootstrap() { void testGetRequiredSchemaForFileAndRenameColumns() { when(hoodieTableConfig.populateMetaFields()).thenReturn(true); HoodieReaderContext readerContext = createReaderContext(hoodieTableConfig, false, false, true, false, null); - Schema requestedSchema = generateProjectionSchema("_hoodie_record_key", "timestamp", "rider"); + HoodieSchema requestedSchema = generateProjectionSchema("_hoodie_record_key", "timestamp", "rider"); - InternalSchema internalSchema = InternalSchemaConverter.convert(HoodieSchema.fromAvroSchema(DATA_SCHEMA)); + InternalSchema internalSchema = InternalSchemaConverter.convert(DATA_SCHEMA); InternalSchema originalSchema = new InternalSchema(Types.RecordType.get(internalSchema.columns().stream().map(field -> { if (field.name().equals("timestamp")) { // rename timestamp to ts in file schema and change type to int, output schema names and types must match the requested schema @@ -123,7 +123,7 @@ void testGetRequiredSchemaForFileAndRenameColumns() { mockedStatic.when(() -> InternalSchemaCache.searchSchemaAndCache(Long.parseLong(instantTime), metaClient)) .thenReturn(originalSchema); StoragePath filePath = new StoragePath("/2023-01-01/" + FSUtils.makeBaseFileName(instantTime, "1-0-1", UUID.randomUUID().toString(), HoodieFileFormat.PARQUET.getFileExtension())); - Pair> requiredSchemaAndRenamedFields = schemaHandler.getRequiredSchemaForFileAndRenamedColumns(filePath); + Pair> requiredSchemaAndRenamedFields = schemaHandler.getRequiredSchemaForFileAndRenamedColumns(filePath); assertEquals(Collections.singletonMap("timestamp", "ts"), requiredSchemaAndRenamedFields.getRight()); assertEquals(requestedSchema, requiredSchemaAndRenamedFields.getLeft()); } @@ -156,7 +156,7 @@ public void testMorBootstrap(RecordMergeMode mergeMode, } @Override - FileGroupReaderSchemaHandler createSchemaHandler(HoodieReaderContext readerContext, Schema dataSchema, Schema requestedSchema, + FileGroupReaderSchemaHandler createSchemaHandler(HoodieReaderContext readerContext, HoodieSchema dataSchema, HoodieSchema requestedSchema, boolean supportsParquetRowIndex) { return new FileGroupReaderSchemaHandler(readerContext, dataSchema, requestedSchema, Option.empty(), new TypedProperties(), metaClient); @@ -264,7 +264,7 @@ public void testSchemaForMandatoryFields(boolean setPrecombine, expectedFields.add(HoodieRecord.HOODIE_IS_DELETED_FIELD); } HoodieSchema expectedSchema = ((mergeMode == RecordMergeMode.CUSTOM) && !isProjectionCompatible) ? dataSchema : SchemaTestUtil.getSchemaFromFields(expectedFields); - when(recordMerger.getMandatoryFieldsForMerging(dataSchema.toAvroSchema(), hoodieTableConfig, props)).thenReturn(expectedFields.toArray(new String[0])); + when(recordMerger.getMandatoryFieldsForMerging(dataSchema, hoodieTableConfig, props)).thenReturn(expectedFields.toArray(new String[0])); DeleteContext deleteContext = new DeleteContext(props, dataSchema); assertEquals(addHoodieIsDeleted, deleteContext.hasBuiltInDeleteField()); @@ -272,8 +272,8 @@ public void testSchemaForMandatoryFields(boolean setPrecombine, ? Option.of(Pair.of(customDeleteKey, customDeleteValue)) : Option.empty(), deleteContext.getCustomDeleteMarkerKeyValue()); FileGroupReaderSchemaHandler fileGroupReaderSchemaHandler = new FileGroupReaderSchemaHandler(readerContext, - dataSchema.toAvroSchema(), requestedSchema.toAvroSchema(), Option.empty(), props, metaClient); - Schema actualSchema = fileGroupReaderSchemaHandler.generateRequiredSchema(deleteContext); - assertEquals(expectedSchema.toAvroSchema(), actualSchema); + dataSchema, requestedSchema, Option.empty(), props, metaClient); + HoodieSchema actualSchema = fileGroupReaderSchemaHandler.generateRequiredSchema(deleteContext); + assertEquals(expectedSchema, actualSchema); } } 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 0afcf75a82b08..0bc955dfa293b 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 @@ -40,6 +40,7 @@ import org.apache.hudi.common.model.HoodieTableType; 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.serialization.DefaultSerializer; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -123,7 +124,7 @@ public abstract class TestHoodieFileGroupReaderBase { public abstract String getBasePath(); - public abstract HoodieReaderContext getHoodieReaderContext(String tablePath, Schema avroSchema, StorageConfiguration storageConf, HoodieTableMetaClient metaClient); + public abstract HoodieReaderContext getHoodieReaderContext(String tablePath, HoodieSchema schema, StorageConfiguration storageConf, HoodieTableMetaClient metaClient); public abstract String getCustomPayload(); @@ -140,9 +141,9 @@ public void commitToTable(List recordList, commitToTable(recordList, operation, firstCommit, writeConfigs, TRIP_EXAMPLE_SCHEMA); } - public abstract void assertRecordsEqual(Schema schema, T expected, T actual); + public abstract void assertRecordsEqual(HoodieSchema schema, T expected, T actual); - public abstract void assertRecordMatchesSchema(Schema schema, T record); + public abstract void assertRecordMatchesSchema(HoodieSchema schema, T record); public abstract HoodieTestDataGenerator.SchemaEvolutionConfigs getSchemaEvolutionConfigs(); @@ -593,20 +594,20 @@ public void testSpillableMapUsage(ExternalSpillableMap.DiskMapType diskMapType) commitToTable(recordList, INSERT.value(), true, writeConfigs); String baseMapPath = Files.createTempDirectory(null).toString(); HoodieTableMetaClient metaClient = HoodieTestUtils.createMetaClient(getStorageConf(), getBasePath()); - Schema avroSchema = new TableSchemaResolver(metaClient).getTableAvroSchema(); + HoodieSchema schema = new TableSchemaResolver(metaClient).getTableSchema(); List fileSlices = getFileSlicesToRead(getStorageConf(), getBasePath(), metaClient, true, 0); List records = readRecordsFromFileGroup(getStorageConf(), getBasePath(), metaClient, fileSlices, - avroSchema, RecordMergeMode.COMMIT_TIME_ORDERING, false, false); - HoodieReaderContext readerContext = getHoodieReaderContext(getBasePath(), avroSchema, getStorageConf(), metaClient); + schema, RecordMergeMode.COMMIT_TIME_ORDERING, false, false); + HoodieReaderContext readerContext = getHoodieReaderContext(getBasePath(), schema, getStorageConf(), metaClient); for (Boolean isCompressionEnabled : new boolean[] {true, false}) { try (ExternalSpillableMap> spillableMap = new ExternalSpillableMap<>(16L, baseMapPath, new DefaultSizeEstimator(), - new HoodieRecordSizeEstimator(avroSchema), diskMapType, new DefaultSerializer<>(), isCompressionEnabled, getClass().getSimpleName())) { + new HoodieRecordSizeEstimator(schema.toAvroSchema()), diskMapType, new DefaultSerializer<>(), isCompressionEnabled, getClass().getSimpleName())) { Long position = 0L; for (T record : records) { - String recordKey = readerContext.getRecordContext().getRecordKey(record, HoodieSchema.fromAvroSchema(avroSchema)); + String recordKey = readerContext.getRecordContext().getRecordKey(record, schema); //test key based - BufferedRecord bufferedRecord = BufferedRecords.fromEngineRecord(record, HoodieSchema.fromAvroSchema(avroSchema), + BufferedRecord bufferedRecord = BufferedRecords.fromEngineRecord(record, schema, readerContext.getRecordContext(), Collections.singletonList("timestamp"), false); spillableMap.put(recordKey, bufferedRecord.toBinary(readerContext.getRecordContext())); @@ -618,16 +619,16 @@ public void testSpillableMapUsage(ExternalSpillableMap.DiskMapType diskMapType) //Validate that everything is correct position = 0L; for (T record : records) { - String recordKey = readerContext.getRecordContext().getRecordKey(record, HoodieSchema.fromAvroSchema(avroSchema)); + String recordKey = readerContext.getRecordContext().getRecordKey(record, schema); BufferedRecord keyBased = spillableMap.get(recordKey); assertNotNull(keyBased); BufferedRecord positionBased = spillableMap.get(position++); assertNotNull(positionBased); - assertRecordsEqual(avroSchema, record, keyBased.getRecord()); - assertRecordsEqual(avroSchema, record, positionBased.getRecord()); + assertRecordsEqual(schema, record, keyBased.getRecord()); + assertRecordsEqual(schema, record, positionBased.getRecord()); assertEquals(keyBased.getRecordKey(), recordKey); assertEquals(positionBased.getRecordKey(), recordKey); - assertEquals(avroSchema, readerContext.getRecordContext().getSchemaFromBufferRecord(keyBased).toAvroSchema()); + assertEquals(schema, readerContext.getRecordContext().getSchemaFromBufferRecord(keyBased)); assertEquals(readerContext.getRecordContext().convertValueToEngineType(timestamp), positionBased.getOrderingValue()); } } @@ -665,22 +666,22 @@ private void validateOutputFromFileGroupReaderWithNativeRecords(StorageConfigura Set metaCols = new HashSet<>(HoodieRecord.HOODIE_META_COLUMNS); HoodieTableMetaClient metaClient = HoodieTestUtils.createMetaClient(storageConf, tablePath); TableSchemaResolver resolver = new TableSchemaResolver(metaClient); - Schema avroSchema = resolver.getTableAvroSchema(); - Schema avroSchemaWithoutMeta = resolver.getTableAvroSchema(false); + HoodieSchema schema = resolver.getTableSchema(); + HoodieSchema schemaWithoutMeta = resolver.getTableSchema(false); // use reader context for conversion to engine specific objects - HoodieReaderContext readerContext = getHoodieReaderContext(tablePath, avroSchema, getStorageConf(), metaClient); + HoodieReaderContext readerContext = getHoodieReaderContext(tablePath, schema, getStorageConf(), metaClient); List fileSlices = getFileSlicesToRead(storageConf, tablePath, metaClient, containsBaseFile, expectedLogFileNum); boolean sortOutput = !containsBaseFile; List actualRecordList = - readRecordsFromFileGroup(storageConf, tablePath, metaClient, fileSlices, avroSchema, recordMergeMode, false, sortOutput); + readRecordsFromFileGroup(storageConf, tablePath, metaClient, fileSlices, schema, recordMergeMode, false, sortOutput); assertEquals(expectedRecords.size(), actualRecordList.size()); - actualRecordList.forEach(r -> assertRecordMatchesSchema(avroSchema, r)); - Set actualRecordSet = actualRecordList.stream().map(r -> readerContext.getRecordContext().convertToAvroRecord(r, HoodieSchema.fromAvroSchema(avroSchema))) + actualRecordList.forEach(r -> assertRecordMatchesSchema(schema, r)); + Set actualRecordSet = actualRecordList.stream().map(r -> readerContext.getRecordContext().convertToAvroRecord(r, schema)) .map(r -> HoodieAvroUtils.removeFields(r, metaCols)) .collect(Collectors.toSet()); Set expectedRecordSet = expectedRecords.stream() .map(r -> resetByteBufferPosition((GenericRecord) r.getRight())) - .map(r -> HoodieAvroUtils.rewriteRecordWithNewSchema(r, avroSchemaWithoutMeta)) + .map(r -> HoodieAvroUtils.rewriteRecordWithNewSchema(r, schemaWithoutMeta.toAvroSchema())) .collect(Collectors.toSet()); compareRecordSets(expectedRecordSet, actualRecordSet); } @@ -738,27 +739,27 @@ private void validateOutputFromFileGroupReaderWithExistingRecords(StorageConfigu List expectedRecords, List expectedUnmergedRecords) throws Exception { HoodieTableMetaClient metaClient = HoodieTestUtils.createMetaClient(storageConf, tablePath); - Schema avroSchema = new TableSchemaResolver(metaClient).getTableAvroSchema(); + HoodieSchema schema = new TableSchemaResolver(metaClient).getTableSchema(); // use reader context for conversion to engine specific objects - HoodieReaderContext readerContext = getHoodieReaderContext(tablePath, avroSchema, getStorageConf(), metaClient); + HoodieReaderContext readerContext = getHoodieReaderContext(tablePath, schema, getStorageConf(), metaClient); List fileSlices = getFileSlicesToRead(storageConf, tablePath, metaClient, containsBaseFile, expectedLogFileNum); boolean sortOutput = !containsBaseFile; List actualRecordList = convertEngineRecords( - readRecordsFromFileGroup(storageConf, tablePath, metaClient, fileSlices, avroSchema, recordMergeMode, false, sortOutput), - avroSchema, readerContext, metaClient.getTableConfig().getOrderingFields()); + readRecordsFromFileGroup(storageConf, tablePath, metaClient, fileSlices, schema, recordMergeMode, false, sortOutput), + schema, readerContext, metaClient.getTableConfig().getOrderingFields()); // validate size is equivalent to ensure no duplicates are returned assertEquals(expectedRecords.size(), actualRecordList.size()); assertEquals(new HashSet<>(expectedRecords), new HashSet<>(actualRecordList)); // validate records can be read from file group as HoodieRecords actualRecordList = convertHoodieRecords( - readHoodieRecordsFromFileGroup(storageConf, tablePath, metaClient, fileSlices, avroSchema, recordMergeMode), - avroSchema, readerContext, metaClient.getTableConfig().getOrderingFields()); + readHoodieRecordsFromFileGroup(storageConf, tablePath, metaClient, fileSlices, schema, recordMergeMode), + schema, readerContext, metaClient.getTableConfig().getOrderingFields()); assertEquals(expectedRecords.size(), actualRecordList.size()); assertEquals(new HashSet<>(expectedRecords), new HashSet<>(actualRecordList)); // validate unmerged records actualRecordList = convertEngineRecords( - readRecordsFromFileGroup(storageConf, tablePath, metaClient, fileSlices, avroSchema, recordMergeMode, true, false), - avroSchema, readerContext, metaClient.getTableConfig().getOrderingFields()); + readRecordsFromFileGroup(storageConf, tablePath, metaClient, fileSlices, schema, recordMergeMode, true, false), + schema, readerContext, metaClient.getTableConfig().getOrderingFields()); assertEquals(expectedUnmergedRecords.size(), actualRecordList.size()); assertEquals(new HashSet<>(expectedUnmergedRecords), new HashSet<>(actualRecordList)); } @@ -805,7 +806,7 @@ private List readRecordsFromFileGroup(StorageConfiguration storageConf, String tablePath, HoodieTableMetaClient metaClient, List fileSlices, - Schema avroSchema, + HoodieSchema schema, RecordMergeMode recordMergeMode, boolean isSkipMerge, boolean sortOutput) { @@ -816,11 +817,11 @@ private List readRecordsFromFileGroup(StorageConfiguration storageConf, props.setProperty(HoodieReaderConfig.MERGE_TYPE.key(), HoodieReaderConfig.REALTIME_SKIP_MERGE); } fileSlices.forEach(fileSlice -> { - if (shouldValidatePartialRead(fileSlice, avroSchema)) { - assertThrows(IllegalArgumentException.class, () -> getHoodieFileGroupReader(storageConf, tablePath, metaClient, avroSchema, fileSlice, 1, props, sortOutput)); + if (shouldValidatePartialRead(fileSlice, schema)) { + assertThrows(IllegalArgumentException.class, () -> getHoodieFileGroupReader(storageConf, tablePath, metaClient, schema, fileSlice, 1, props, sortOutput)); } - try (HoodieFileGroupReader fileGroupReader = getHoodieFileGroupReader(storageConf, tablePath, metaClient, avroSchema, fileSlice, 0, props, sortOutput)) { - readWithFileGroupReader(fileGroupReader, actualRecordList, avroSchema, getHoodieReaderContext(tablePath, avroSchema, storageConf, metaClient), sortOutput); + try (HoodieFileGroupReader fileGroupReader = getHoodieFileGroupReader(storageConf, tablePath, metaClient, schema, fileSlice, 0, props, sortOutput)) { + readWithFileGroupReader(fileGroupReader, actualRecordList, schema, getHoodieReaderContext(tablePath, schema, storageConf, metaClient), sortOutput); } catch (Exception ex) { throw new RuntimeException(ex); } @@ -831,16 +832,16 @@ private List readRecordsFromFileGroup(StorageConfiguration storageConf, private HoodieFileGroupReader getHoodieFileGroupReader(StorageConfiguration storageConf, String tablePath, HoodieTableMetaClient metaClient, - Schema avroSchema, + HoodieSchema schema, FileSlice fileSlice, int start, TypedProperties props, boolean sortOutput) { return HoodieFileGroupReader.newBuilder() - .withReaderContext(getHoodieReaderContext(tablePath, avroSchema, storageConf, metaClient)) + .withReaderContext(getHoodieReaderContext(tablePath, schema, storageConf, metaClient)) .withHoodieTableMetaClient(metaClient) .withLatestCommitTime(metaClient.getActiveTimeline().lastInstant().get().requestedTime()) .withFileSlice(fileSlice) - .withDataSchema(avroSchema) - .withRequestedSchema(avroSchema) + .withDataSchema(schema) + .withRequestedSchema(schema) .withProps(props) .withStart(start) .withLength(fileSlice.getTotalFileSize()) @@ -853,7 +854,7 @@ private HoodieFileGroupReader getHoodieFileGroupReader(StorageConfiguration fileGroupReader, List recordList, - Schema avroSchema, + HoodieSchema schema, HoodieReaderContext readerContext, boolean sortOutput) throws IOException { String lastKey = null; @@ -861,7 +862,7 @@ protected void readWithFileGroupReader( while (fileGroupReaderIterator.hasNext()) { T next = fileGroupReaderIterator.next(); if (sortOutput) { - String currentKey = readerContext.getRecordContext().getRecordKey(next, HoodieSchema.fromAvroSchema(avroSchema)); + String currentKey = readerContext.getRecordContext().getRecordKey(next, schema); assertTrue(lastKey == null || lastKey.compareTo(currentKey) < 0, "Record keys should be sorted within the file group"); lastKey = currentKey; } @@ -874,13 +875,13 @@ private List> readHoodieRecordsFromFileGroup(StorageConfiguratio String tablePath, HoodieTableMetaClient metaClient, List fileSlices, - Schema avroSchema, + HoodieSchema schema, RecordMergeMode recordMergeMode) { List> actualRecordList = new ArrayList<>(); TypedProperties props = buildProperties(metaClient, recordMergeMode); fileSlices.forEach(fileSlice -> { - try (HoodieFileGroupReader fileGroupReader = getHoodieFileGroupReader(storageConf, tablePath, metaClient, avroSchema, fileSlice, 0, props, false); + try (HoodieFileGroupReader fileGroupReader = getHoodieFileGroupReader(storageConf, tablePath, metaClient, schema, fileSlice, 0, props, false); ClosableIterator> iter = fileGroupReader.getClosableHoodieRecordIterator()) { while (iter.hasNext()) { actualRecordList.add(iter.next()); @@ -911,13 +912,13 @@ private TypedProperties buildProperties(HoodieTableMetaClient metaClient, Record return props; } - private boolean shouldValidatePartialRead(FileSlice fileSlice, Schema requestedSchema) { + private boolean shouldValidatePartialRead(FileSlice fileSlice, HoodieSchema requestedSchema) { if (fileSlice.getLogFiles().findAny().isPresent()) { return true; } if (fileSlice.getBaseFile().get().getBootstrapBaseFile().isPresent()) { //TODO: [HUDI-8169] this code path will not hit until we implement bootstrap tests - Pair, List> dataAndMetaCols = FileGroupReaderSchemaHandler.getDataAndMetaCols(requestedSchema); + Pair, List> dataAndMetaCols = FileGroupReaderSchemaHandler.getDataAndMetaCols(requestedSchema); return !dataAndMetaCols.getLeft().isEmpty() && !dataAndMetaCols.getRight().isEmpty(); } return false; @@ -939,30 +940,30 @@ private List convertHoodieRecords(List return records.stream().map(record -> HoodieTestDataGenerator.RecordIdentifier.fromTripTestPayload((HoodieAvroIndexedRecord) record, orderingFields)).collect(Collectors.toList()); } - private List convertEngineRecords(List records, Schema schema, HoodieReaderContext readerContext, List preCombineFields) { + private List convertEngineRecords(List records, HoodieSchema schema, HoodieReaderContext readerContext, List preCombineFields) { return records.stream() .map(record -> new HoodieTestDataGenerator.RecordIdentifier( - readerContext.getRecordContext().getValue(record, HoodieSchema.fromAvroSchema(schema), KEY_FIELD_NAME).toString(), - readerContext.getRecordContext().getValue(record, HoodieSchema.fromAvroSchema(schema), PARTITION_FIELD_NAME).toString(), + readerContext.getRecordContext().getValue(record, schema, KEY_FIELD_NAME).toString(), + readerContext.getRecordContext().getValue(record, schema, PARTITION_FIELD_NAME).toString(), OrderingValues.create(preCombineFields, - field -> (Comparable) readerContext.getRecordContext().getValue(record, HoodieSchema.fromAvroSchema(schema), field)) + field -> (Comparable) readerContext.getRecordContext().getValue(record, schema, field)) .toString(), - readerContext.getRecordContext().getValue(record, HoodieSchema.fromAvroSchema(schema), RIDER_FIELD_NAME).toString())) + readerContext.getRecordContext().getValue(record, schema, RIDER_FIELD_NAME).toString())) .collect(Collectors.toList()); } - private List convertHoodieRecords(List> records, Schema schema, HoodieReaderContext readerContext, + private List convertHoodieRecords(List> records, HoodieSchema schema, HoodieReaderContext readerContext, List orderingFields) { TypedProperties props = new TypedProperties(); props.setProperty(HoodieTableConfig.ORDERING_FIELDS.key(), String.join(",", orderingFields)); return records.stream() .map(record -> { - T data = readerContext.getRecordContext().extractDataFromRecord(record, HoodieSchema.fromAvroSchema(schema), props); + T data = readerContext.getRecordContext().extractDataFromRecord(record, schema, props); return new HoodieTestDataGenerator.RecordIdentifier( record.getRecordKey(), removeHiveStylePartition(record.getPartitionPath()), - record.getOrderingValue(schema, props, orderingFields.toArray(new String[0])).toString(), - readerContext.getRecordContext().getValue(data, HoodieSchema.fromAvroSchema(schema), RIDER_FIELD_NAME).toString()); + record.getOrderingValue(schema.toAvroSchema(), props, orderingFields.toArray(new String[0])).toString(), + readerContext.getRecordContext().getValue(data, schema, RIDER_FIELD_NAME).toString()); }) .collect(Collectors.toList()); } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestParquetRowIndexBasedSchemaHandler.java b/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestParquetRowIndexBasedSchemaHandler.java index 865026f1a1603..a005465313644 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestParquetRowIndexBasedSchemaHandler.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestParquetRowIndexBasedSchemaHandler.java @@ -22,6 +22,7 @@ import org.apache.hudi.common.config.RecordMergeMode; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.HoodieReaderContext; +import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; @@ -48,11 +49,11 @@ public class TestParquetRowIndexBasedSchemaHandler extends SchemaHandlerTestBase public void testCowBootstrapWithPositionMerge() { when(hoodieTableConfig.populateMetaFields()).thenReturn(true); HoodieReaderContext readerContext = createReaderContext(hoodieTableConfig, true, false, true, false, null); - Schema requestedSchema = generateProjectionSchema("begin_lat", "tip_history", "_hoodie_record_key", "rider"); + HoodieSchema requestedSchema = generateProjectionSchema("begin_lat", "tip_history", "_hoodie_record_key", "rider"); FileGroupReaderSchemaHandler schemaHandler = createSchemaHandler(readerContext, DATA_SCHEMA, requestedSchema, true); assertTrue(readerContext.getNeedsBootstrapMerge()); //meta cols must go first in the required schema - Schema expectedRequiredSchema = generateProjectionSchema("_hoodie_record_key", "begin_lat", "tip_history", "rider"); + HoodieSchema expectedRequiredSchema = generateProjectionSchema("_hoodie_record_key", "begin_lat", "tip_history", "rider"); assertEquals(expectedRequiredSchema, schemaHandler.getRequiredSchema()); Pair, List> bootstrapFields = schemaHandler.getBootstrapRequiredFields(); assertEquals(Arrays.asList(getField("_hoodie_record_key"), getPositionalMergeField()), bootstrapFields.getLeft()); @@ -100,7 +101,7 @@ public void testMorBootstrap(RecordMergeMode mergeMode, } @Override - FileGroupReaderSchemaHandler createSchemaHandler(HoodieReaderContext readerContext, Schema dataSchema, Schema requestedSchema, + FileGroupReaderSchemaHandler createSchemaHandler(HoodieReaderContext readerContext, HoodieSchema dataSchema, HoodieSchema requestedSchema, boolean supportsParquetRowIndex) { return new ParquetRowIndexBasedSchemaHandler(readerContext, dataSchema, requestedSchema, Option.empty(), new TypedProperties(), metaClient); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestUpdateProcessor.java b/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestUpdateProcessor.java index 8cf40b1a8ad25..ab9ba09cd4982 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestUpdateProcessor.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestUpdateProcessor.java @@ -164,7 +164,7 @@ void testHandleInsertWithPayload(boolean shouldIgnore) { } else { when(recordContext.convertToAvroRecord(merged.getRecord(), SCHEMA)) .thenReturn((GenericRecord) ((SerializableIndexedRecord) merged.getRecord()).getData()); - when(readerContext.getSchemaHandler().getRequestedSchema()).thenReturn(SCHEMA.toAvroSchema()); + when(readerContext.getSchemaHandler().getRequestedSchema()).thenReturn(SCHEMA); when(recordContext.convertAvroRecord(any())).thenAnswer(invocationOnMock -> invocationOnMock.getArgument(0)); } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/BaseTestFileGroupRecordBuffer.java b/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/BaseTestFileGroupRecordBuffer.java index 4dc566c70871b..e6ee09b3e6add 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/BaseTestFileGroupRecordBuffer.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/BaseTestFileGroupRecordBuffer.java @@ -117,7 +117,8 @@ protected static KeyBasedFileGroupRecordBuffer buildKeyBasedFileG props.setProperty(DELETE_MARKER, markerKeyValue.getRight()); }); FileGroupReaderSchemaHandler fileGroupReaderSchemaHandler = mock(FileGroupReaderSchemaHandler.class); - when(fileGroupReaderSchemaHandler.getRequiredSchema()).thenReturn(SCHEMA.toAvroSchema()); + when(fileGroupReaderSchemaHandler.getRequiredSchema()).thenReturn(SCHEMA); + when(fileGroupReaderSchemaHandler.getSchemaForUpdates()).thenReturn(SCHEMA); when(fileGroupReaderSchemaHandler.getInternalSchema()).thenReturn(InternalSchema.getEmptyInternalSchema()); when(fileGroupReaderSchemaHandler.getDeleteContext()).thenReturn(new DeleteContext(props, SCHEMA)); readerContext.setSchemaHandler(fileGroupReaderSchemaHandler); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestFileGroupRecordBuffer.java b/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestFileGroupRecordBuffer.java index 897bb2361fa90..c8455c702948d 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestFileGroupRecordBuffer.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestFileGroupRecordBuffer.java @@ -80,7 +80,7 @@ class TestFileGroupRecordBuffer { + "{\"name\": \"_hoodie_is_deleted\", \"type\": \"boolean\"}" + "]" + "}"; - private HoodieSchema schema = HoodieSchema.parse(schemaString); + private final HoodieSchema schema = HoodieSchema.parse(schemaString); private final HoodieReaderContext readerContext = mock(HoodieReaderContext.class); private final RecordContext recordContext = mock(RecordContext.class); private final FileGroupReaderSchemaHandler schemaHandler = @@ -94,7 +94,7 @@ void setUp() { props = new TypedProperties(); when(readerContext.getRecordContext()).thenReturn(recordContext); when(readerContext.getSchemaHandler()).thenReturn(schemaHandler); - when(schemaHandler.getRequiredSchema()).thenReturn(schema.toAvroSchema()); + when(schemaHandler.getRequiredSchema()).thenReturn(schema); when(schemaHandler.getDeleteContext()).thenReturn(new DeleteContext(props, schema)); when(readerContext.getRecordMerger()).thenReturn(Option.empty()); when(readerContext.getRecordSerializer()).thenReturn(new DefaultSerializer<>()); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestFileGroupRecordBufferLoader.java b/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestFileGroupRecordBufferLoader.java index 1a607f1c81d9c..51dad1c752af6 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestFileGroupRecordBufferLoader.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestFileGroupRecordBufferLoader.java @@ -70,8 +70,9 @@ public void testDefaultFileGroupBufferRecordLoader(String fileGroupRecordBufferT HoodieReaderContext readerContext = new HoodieAvroReaderContext(storageConfiguration, tableConfig, Option.empty(), Option.empty()); readerContext.initRecordMerger(new TypedProperties()); FileGroupReaderSchemaHandler fileGroupReaderSchemaHandler = mock(FileGroupReaderSchemaHandler.class); - when(fileGroupReaderSchemaHandler.getRequiredSchema()).thenReturn(SCHEMA.toAvroSchema()); - when(fileGroupReaderSchemaHandler.getRequestedSchema()).thenReturn(SCHEMA.toAvroSchema()); + when(fileGroupReaderSchemaHandler.getRequiredSchema()).thenReturn(SCHEMA); + when(fileGroupReaderSchemaHandler.getRequestedSchema()).thenReturn(SCHEMA); + when(fileGroupReaderSchemaHandler.getSchemaForUpdates()).thenReturn(SCHEMA); when(fileGroupReaderSchemaHandler.getInternalSchema()).thenReturn(InternalSchema.getEmptyInternalSchema()); DeleteContext deleteContext = mock(DeleteContext.class); when(deleteContext.getCustomDeleteMarkerKeyValue()).thenReturn(Option.empty()); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestKeyBasedFileGroupRecordBuffer.java b/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestKeyBasedFileGroupRecordBuffer.java index cbefbc18749de..36149f65e97d7 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestKeyBasedFileGroupRecordBuffer.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestKeyBasedFileGroupRecordBuffer.java @@ -154,7 +154,7 @@ void readWithEventTimeOrderingWithRecords() throws IOException { readerContext.setHasLogFiles(false); readerContext.setHasBootstrapBaseFile(false); readerContext.initRecordMerger(properties); - FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA.toAvroSchema(), SCHEMA.toAvroSchema(), Option.empty(), + FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA, SCHEMA, Option.empty(), properties, mock(HoodieTableMetaClient.class)); readerContext.setSchemaHandler(schemaHandler); List inputRecords = convertToHoodieRecordsList(Arrays.asList(testRecord1UpdateWithSameTime, testRecord2Update, testRecord3Update, testRecord4EarlierUpdate, testRecord7)); @@ -222,7 +222,7 @@ void readWithCommitTimeOrderingWithRecords() throws IOException { readerContext.setHasLogFiles(false); readerContext.setHasBootstrapBaseFile(false); readerContext.initRecordMerger(properties); - FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA.toAvroSchema(), SCHEMA.toAvroSchema(), Option.empty(), + FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA, SCHEMA, Option.empty(), properties, mock(HoodieTableMetaClient.class)); readerContext.setSchemaHandler(schemaHandler); List inputRecords = convertToHoodieRecordsList(Arrays.asList(testRecord1UpdateWithSameTime, testRecord2Update, testRecord3Update, @@ -302,7 +302,7 @@ void readWithCustomPayloadWithRecords() throws IOException { readerContext.setHasLogFiles(false); readerContext.setHasBootstrapBaseFile(false); readerContext.initRecordMerger(properties); - FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA.toAvroSchema(), SCHEMA.toAvroSchema(), Option.empty(), + FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA, SCHEMA, Option.empty(), properties, mock(HoodieTableMetaClient.class)); readerContext.setSchemaHandler(schemaHandler); List inputRecords = convertToHoodieRecordsList(Arrays.asList(testRecord1UpdateWithSameTime, testRecord2Update, testRecord3Update, testRecord4EarlierUpdate)); @@ -377,7 +377,7 @@ void readWithCustomMergerWithRecords() throws IOException { readerContext.setHasLogFiles(false); readerContext.setHasBootstrapBaseFile(false); readerContext.initRecordMerger(properties); - FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA.toAvroSchema(), SCHEMA.toAvroSchema(), Option.empty(), + FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA, SCHEMA, Option.empty(), properties, mock(HoodieTableMetaClient.class)); readerContext.setSchemaHandler(schemaHandler); List inputRecords = convertToHoodieRecordsList(Arrays.asList(testRecord1UpdateWithSameTime, testRecord2Update, testRecord3Update, testRecord4EarlierUpdate)); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestReusableKeyBasedRecordBuffer.java b/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestReusableKeyBasedRecordBuffer.java index 50a8f1b43dd08..6d0531ce2b46e 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestReusableKeyBasedRecordBuffer.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestReusableKeyBasedRecordBuffer.java @@ -70,7 +70,7 @@ void testRemainingLogEntryHandling() throws IOException { // Filter excludes key "4", so it should not be returned. It also includes key "5" which is not in the base file or log file. Predicate keyFilter = Predicates.in(null, Arrays.asList(Literal.from("1"), Literal.from("2"), Literal.from("3"), Literal.from("5"))); when(mockReaderContext.getKeyFilterOpt()).thenReturn(Option.of(keyFilter)); - when(mockReaderContext.getSchemaHandler().getRequiredSchema()).thenReturn(HoodieTestDataGenerator.AVRO_SCHEMA); + when(mockReaderContext.getSchemaHandler().getRequiredSchema()).thenReturn(HoodieTestDataGenerator.HOODIE_SCHEMA); when(mockReaderContext.getSchemaHandler().getInternalSchema()).thenReturn(InternalSchema.getEmptyInternalSchema()); when(mockReaderContext.getRecordContext().getDeleteRow(any())).thenAnswer(invocation -> { String recordKey = invocation.getArgument(0); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestSortedKeyBasedFileGroupRecordBuffer.java b/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestSortedKeyBasedFileGroupRecordBuffer.java index ca780829f97f8..0d43c1d463ca7 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestSortedKeyBasedFileGroupRecordBuffer.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestSortedKeyBasedFileGroupRecordBuffer.java @@ -124,7 +124,7 @@ void readWithStreamingRecordBufferLoaderAndEventTimeOrdering() throws IOExceptio HoodieReaderContext readerContext = new HoodieAvroReaderContext(storageConfiguration, tableConfig, Option.empty(), Option.empty()); readerContext.setHasLogFiles(false); readerContext.setHasBootstrapBaseFile(false); - FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA.toAvroSchema(), SCHEMA.toAvroSchema(), Option.empty(), + FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA, SCHEMA, Option.empty(), properties, mock(HoodieTableMetaClient.class)); readerContext.setSchemaHandler(schemaHandler); readerContext.initRecordMerger(properties); @@ -189,7 +189,7 @@ void readLogFiles() throws IOException { } private SortedKeyBasedFileGroupRecordBuffer buildSortedKeyBasedFileGroupRecordBuffer(HoodieReaderContext mockReaderContext, HoodieReadStats readStats) { - when(mockReaderContext.getSchemaHandler().getRequiredSchema()).thenReturn(HoodieTestDataGenerator.AVRO_SCHEMA); + when(mockReaderContext.getSchemaHandler().getRequiredSchema()).thenReturn(HoodieTestDataGenerator.HOODIE_SCHEMA); when(mockReaderContext.getSchemaHandler().getInternalSchema()).thenReturn(InternalSchema.getEmptyInternalSchema()); when(mockReaderContext.getRecordContext().getDeleteRow(any())).thenAnswer(invocation -> { String recordKey = invocation.getArgument(0); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestStreamingKeyBasedFileGroupRecordBuffer.java b/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestStreamingKeyBasedFileGroupRecordBuffer.java index 28a5a0fb78de1..d2997f7f53098 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestStreamingKeyBasedFileGroupRecordBuffer.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestStreamingKeyBasedFileGroupRecordBuffer.java @@ -84,7 +84,7 @@ void readWithEventTimeOrdering() throws IOException { readerContext.setHasLogFiles(false); readerContext.setHasBootstrapBaseFile(false); readerContext.initRecordMerger(properties); - FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA.toAvroSchema(), SCHEMA.toAvroSchema(), Option.empty(), + FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA, SCHEMA, Option.empty(), properties, mock(HoodieTableMetaClient.class)); readerContext.setSchemaHandler(schemaHandler); List inputRecords = convertToHoodieRecordsList(Arrays.asList(testRecord1UpdateWithSameTime, testRecord2Update, testRecord3Update, testRecord4EarlierUpdate, testRecord7)); @@ -122,7 +122,7 @@ void readWithCommitTimeOrdering() throws IOException { readerContext.setHasLogFiles(false); readerContext.setHasBootstrapBaseFile(false); readerContext.initRecordMerger(properties); - FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA.toAvroSchema(), SCHEMA.toAvroSchema(), Option.empty(), + FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA, SCHEMA, Option.empty(), properties, mock(HoodieTableMetaClient.class)); readerContext.setSchemaHandler(schemaHandler); List inputRecords = convertToHoodieRecordsList(Arrays.asList(testRecord1UpdateWithSameTime, testRecord2Update, testRecord3Update, testRecord4EarlierUpdate, testRecord7)); @@ -162,7 +162,7 @@ void readWithCustomPayload() throws IOException { readerContext.setHasLogFiles(false); readerContext.setHasBootstrapBaseFile(false); readerContext.initRecordMerger(properties); - FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA.toAvroSchema(), SCHEMA.toAvroSchema(), Option.empty(), + FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA, SCHEMA, Option.empty(), properties, mock(HoodieTableMetaClient.class)); readerContext.setSchemaHandler(schemaHandler); List inputRecords = convertToHoodieRecordsList(Arrays.asList(testRecord1UpdateWithSameTime, testRecord2Update, testRecord3Update, testRecord4EarlierUpdate)); @@ -200,7 +200,7 @@ void readWithCustomMergerWithRecords() throws IOException { readerContext.setHasLogFiles(false); readerContext.setHasBootstrapBaseFile(false); readerContext.initRecordMerger(properties); - FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA.toAvroSchema(), SCHEMA.toAvroSchema(), Option.empty(), + FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA, SCHEMA, Option.empty(), properties, mock(HoodieTableMetaClient.class)); readerContext.setSchemaHandler(schemaHandler); List inputRecords = convertToHoodieRecordsList(Arrays.asList(testRecord1UpdateWithSameTime, testRecord2Update, testRecord3Update, testRecord4EarlierUpdate)); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java index 85b85007056b7..5d9be1d2eb534 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.model.HoodieOperation; 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.table.read.BufferedRecordMerger; import org.apache.hudi.common.table.read.BufferedRecordMergerFactory; import org.apache.hudi.common.util.ValidationUtils; @@ -48,7 +49,6 @@ import org.apache.hudi.util.MutableIteratorWrapperIterator; import org.apache.hudi.util.StreamerUtil; -import org.apache.avro.Schema; import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.streaming.api.functions.ProcessFunction; @@ -242,7 +242,7 @@ private void initMergeClass() { readerContext.getMergeMode(), false, readerContext.getRecordMerger(), - new Schema.Parser().parse(writeClient.getConfig().getSchema()), + HoodieSchema.parse(writeClient.getConfig().getSchema()), readerContext.getPayloadClasses(writeClient.getConfig().getProps()), writeClient.getConfig().getProps(), metaClient.getTableConfig().getPartialUpdateMode()); 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 7df7987070aed..eda1e3c4864c3 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 @@ -31,6 +31,7 @@ import org.apache.hudi.common.model.HoodieRecord; 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.table.HoodieTableConfig; import org.apache.hudi.common.table.log.InstantRange; import org.apache.hudi.common.table.read.FileGroupReaderSchemaHandler; @@ -46,8 +47,8 @@ import org.apache.hudi.storage.StorageConfiguration; import org.apache.hudi.storage.StoragePath; import org.apache.hudi.util.Lazy; -import org.apache.hudi.util.RowDataAvroQueryContexts; import org.apache.hudi.util.RecordKeyToRowDataConverter; +import org.apache.hudi.util.RowDataAvroQueryContexts; import org.apache.avro.Schema; import org.apache.flink.table.data.RowData; @@ -189,14 +190,14 @@ public void setSchemaHandler(FileGroupReaderSchemaHandler schemaHandler return; } // primary key semantic is lost if not all primary key fields are included in the request schema. - boolean pkSemanticLost = Arrays.stream(recordKeysOpt.get()).anyMatch(k -> schemaHandler.getRequestedSchema().getField(k) == null); + boolean pkSemanticLost = Arrays.stream(recordKeysOpt.get()).anyMatch(k -> schemaHandler.getRequestedSchema().getField(k).isEmpty()); if (pkSemanticLost) { return; } - Schema requiredSchema = schemaHandler.getRequiredSchema(); + HoodieSchema requiredSchema = schemaHandler.getRequiredSchema(); // get primary key field position in required schema. int[] pkFieldsPos = Arrays.stream(recordKeysOpt.get()) - .map(k -> Option.ofNullable(requiredSchema.getField(k)).map(Schema.Field::pos).orElse(-1)) + .map(k -> requiredSchema.getField(k).map(HoodieSchemaField::pos).orElse(-1)) .mapToInt(Integer::intValue) .toArray(); // the converter is used to create a RowData contains primary key fields only @@ -204,7 +205,7 @@ public void setSchemaHandler(FileGroupReaderSchemaHandler schemaHandler // For e.g, if the pk fields are [a, b] but user only select a, then the pk // semantics is lost. RecordKeyToRowDataConverter recordKeyRowConverter = new RecordKeyToRowDataConverter( - pkFieldsPos, (RowType) RowDataAvroQueryContexts.fromAvroSchema(requiredSchema).getRowType().getLogicalType()); + pkFieldsPos, (RowType) RowDataAvroQueryContexts.fromAvroSchema(requiredSchema.toAvroSchema()).getRowType().getLogicalType()); ((FlinkRecordContext) recordContext).setRecordKeyRowConverter(recordKeyRowConverter); } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java index 8741e66eeac92..b2a6212751ab3 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java @@ -135,8 +135,8 @@ public static HoodieFileGroupReader createFileGroupReader( .withHoodieTableMetaClient(metaClient) .withLatestCommitTime(latestInstant) .withFileSlice(fileSlice) - .withDataSchema(tableSchema.getAvroSchema()) - .withRequestedSchema(requiredSchema.getAvroSchema()) + .withDataSchema(tableSchema) + .withRequestedSchema(requiredSchema) .withInternalSchema(Option.ofNullable(internalSchemaManager.getQuerySchema())) .withProps(typedProps) .withShouldUseRecordPosition(false) diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cdc/CdcInputFormat.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cdc/CdcInputFormat.java index 17d6f93773cac..17ea108514a48 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cdc/CdcInputFormat.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cdc/CdcInputFormat.java @@ -379,7 +379,7 @@ static class DataLogFileIterator implements ClosableIterator { readerContext.getMergeMode(), false, Option.of(imageManager.writeConfig.getRecordMerger()), - tableSchema.toAvroSchema(), + tableSchema, Option.ofNullable(Pair.of(metaClient.getTableConfig().getPayloadClass(), writeConfig.getPayloadClass())), props, metaClient.getTableConfig().getPartialUpdateMode()); diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieFileGroupReaderOnFlink.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieFileGroupReaderOnFlink.java index a17264707ba72..09df34c8a547c 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieFileGroupReaderOnFlink.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieFileGroupReaderOnFlink.java @@ -19,7 +19,6 @@ package org.apache.hudi.table; -import org.apache.avro.Schema; import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.config.RecordMergeMode; import org.apache.hudi.common.engine.HoodieReaderContext; @@ -109,7 +108,7 @@ public String getBasePath() { @Override public HoodieReaderContext getHoodieReaderContext( String tablePath, - Schema avroSchema, + HoodieSchema schema, StorageConfiguration storageConf, HoodieTableMetaClient metaClient) { return new FlinkRowDataReaderContext( @@ -129,10 +128,10 @@ public String getCustomPayload() { protected void readWithFileGroupReader( HoodieFileGroupReader fileGroupReader, List recordList, - Schema recordSchema, + HoodieSchema recordSchema, HoodieReaderContext readerContext, boolean sortOutput) throws IOException { - RowDataSerializer rowDataSerializer = RowDataAvroQueryContexts.getRowDataSerializer(recordSchema); + RowDataSerializer rowDataSerializer = RowDataAvroQueryContexts.getRowDataSerializer(recordSchema.toAvroSchema()); try (ClosableIterator iterator = fileGroupReader.getClosableIterator()) { while (iterator.hasNext()) { RowData rowData = rowDataSerializer.copy(iterator.next()); @@ -166,15 +165,15 @@ public void commitToTable(List recordList, String operation, boole } @Override - public void assertRecordsEqual(Schema schema, RowData expected, RowData actual) { + public void assertRecordsEqual(HoodieSchema schema, RowData expected, RowData actual) { TestData.assertRowDataEquals( Collections.singletonList(actual), Collections.singletonList(expected), - RowDataAvroQueryContexts.fromAvroSchema(schema).getRowType()); + RowDataAvroQueryContexts.fromAvroSchema(schema.toAvroSchema()).getRowType()); } @Override - public void assertRecordMatchesSchema(Schema schema, RowData record) { + public void assertRecordMatchesSchema(HoodieSchema schema, RowData record) { // TODO: Add support for RowData } diff --git a/hudi-hadoop-common/src/main/java/org/apache/hudi/io/hadoop/HoodieAvroParquetReader.java b/hudi-hadoop-common/src/main/java/org/apache/hudi/io/hadoop/HoodieAvroParquetReader.java index 4bcf52ef6239e..902a15bd03ef8 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/hudi/io/hadoop/HoodieAvroParquetReader.java +++ b/hudi-hadoop-common/src/main/java/org/apache/hudi/io/hadoop/HoodieAvroParquetReader.java @@ -25,6 +25,7 @@ import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieRecord; 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; @@ -211,8 +212,7 @@ private ClosableIterator getIndexedRecordIteratorInternal(Schema @Override public ClosableIterator getRecordKeyIterator() throws IOException { - //TODO boundary for now to revisit in later pr to use HoodieSchema - ClosableIterator recordKeyIterator = getIndexedRecordIterator(HoodieSchema.fromAvroSchema(HoodieAvroUtils.getRecordKeySchema())); + ClosableIterator recordKeyIterator = getIndexedRecordIterator(HoodieSchemaUtils.getRecordKeySchema()); return new ClosableIterator() { @Override public boolean hasNext() { diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestHarness.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestHarness.java index c82ee9ba7b662..8a3863d769c85 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestHarness.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestHarness.java @@ -47,7 +47,7 @@ import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS; import static org.apache.hudi.common.table.HoodieTableConfig.RECORDKEY_FIELDS; -import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.AVRO_SCHEMA; +import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.HOODIE_SCHEMA; import static org.apache.hudi.common.testutils.HoodieTestUtils.getDefaultStorageConf; import static org.apache.hudi.common.testutils.reader.HoodieFileSliceTestUtils.ROW_KEY; @@ -142,8 +142,8 @@ protected ClosableIterator getFileGroupIterator(int numFiles, boo .withHoodieTableMetaClient(metaClient) .withLatestCommitTime("1000") // Not used internally. .withFileSlice(fileSliceOpt.orElseThrow(() -> new IllegalArgumentException("FileSlice is not present"))) - .withDataSchema(AVRO_SCHEMA) - .withRequestedSchema(AVRO_SCHEMA) + .withDataSchema(HOODIE_SCHEMA) + .withRequestedSchema(HOODIE_SCHEMA) .withProps(properties) .withShouldUseRecordPosition(shouldReadPositions) .withAllowInflightInstants(allowInflightCommits) 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 8d06728860f37..bf60b21401619 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 @@ -19,12 +19,13 @@ package org.apache.hudi.hadoop; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.BaseFile; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileGroupId; +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.read.HoodieFileGroupReader; @@ -122,8 +123,8 @@ public HoodieFileGroupReaderBasedRecordReader(HiveReaderCreator readerCreator, .setBasePath(tableBasePath) .build(); String latestCommitTime = getLatestCommitTime(split, metaClient); - Schema tableSchema = getLatestTableSchema(metaClient, jobConfCopy, latestCommitTime); - Schema requestedSchema = createRequestedSchema(tableSchema, jobConfCopy); + HoodieSchema tableSchema = getLatestTableSchema(metaClient, jobConfCopy, latestCommitTime); + HoodieSchema requestedSchema = createRequestedSchema(tableSchema, jobConfCopy); this.readerContext = new HiveHoodieReaderContext(readerCreator, getStoredPartitionFieldNames(jobConfCopy, tableSchema), new HadoopStorageConfiguration(jobConfCopy), metaClient.getTableConfig()); @@ -159,7 +160,7 @@ public HoodieFileGroupReaderBasedRecordReader(HiveReaderCreator readerCreator, .build() .getClosableIterator(); // it expects the partition columns to be at the end - Schema outputSchema = HoodieAvroUtils.generateProjectionSchema(tableSchema, + 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); @@ -233,8 +234,8 @@ public float getProgress() throws IOException { * List of partition fields that are actually written to the file */ @VisibleForTesting - static List getStoredPartitionFieldNames(JobConf jobConf, Schema writerSchema) { - return getPartitionFieldNames(jobConf).stream().filter(n -> writerSchema.getField(n) != null).collect(Collectors.toList()); + static List getStoredPartitionFieldNames(JobConf jobConf, HoodieSchema writerSchema) { + return getPartitionFieldNames(jobConf).stream().filter(n -> writerSchema.getField(n).isPresent()).collect(Collectors.toList()); } public RealtimeSplit getSplit() { @@ -245,12 +246,12 @@ public JobConf getJobConf() { return jobConfCopy; } - private static Schema getLatestTableSchema(HoodieTableMetaClient metaClient, JobConf jobConf, String latestCommitTime) { + private static HoodieSchema getLatestTableSchema(HoodieTableMetaClient metaClient, JobConf jobConf, String latestCommitTime) { TableSchemaResolver tableSchemaResolver = new TableSchemaResolver(metaClient); try { Schema schema = tableSchemaResolver.getTableAvroSchema(latestCommitTime); // Add partitioning fields to writer schema for resulting row to contain null values for these fields - return HoodieRealtimeRecordReaderUtils.addPartitionFields(schema, getPartitionFieldNames(jobConf)); + return HoodieSchema.fromAvroSchema(HoodieRealtimeRecordReaderUtils.addPartitionFields(schema, getPartitionFieldNames(jobConf))); } catch (Exception e) { throw new RuntimeException("Unable to get table schema", e); } @@ -312,12 +313,11 @@ private static BaseFile createBootstrapBaseFile(FileSplit split, FileSystem fs) } @VisibleForTesting - public static Schema createRequestedSchema(Schema tableSchema, JobConf jobConf) { + public static HoodieSchema createRequestedSchema(HoodieSchema tableSchema, JobConf jobConf) { String readCols = jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR); if (StringUtils.isNullOrEmpty(readCols)) { - Schema emptySchema = Schema.createRecord(tableSchema.getName(), tableSchema.getDoc(), - tableSchema.getNamespace(), tableSchema.isError()); - emptySchema.setFields(Collections.emptyList()); + HoodieSchema emptySchema = HoodieSchema.createRecord(tableSchema.getName(), tableSchema.getDoc().orElse(null), + tableSchema.getNamespace().orElse(null), tableSchema.isError(), Collections.emptyList()); return emptySchema; } // hive will handle the partition cols @@ -330,7 +330,7 @@ public static Schema createRequestedSchema(Schema tableSchema, JobConf jobConf) } // if they are actually written to the file, then it is ok to read them from the file tableSchema.getFields().forEach(f -> partitionColumns.remove(f.name().toLowerCase(Locale.ROOT))); - return HoodieAvroUtils.generateProjectionSchema(tableSchema, + return HoodieSchemaUtils.generateProjectionSchema(tableSchema, // 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. 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/HoodieArrayWritableAvroUtils.java index 31f564dbce5d2..e65ca612ca5ca 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/HoodieArrayWritableAvroUtils.java @@ -21,6 +21,8 @@ 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.HoodieSchemaField; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.exception.HoodieAvroSchemaException; import org.apache.hudi.exception.SchemaCompatibilityException; @@ -306,11 +308,12 @@ private static Writable rewritePrimaryTypeWithDiffSchemaType(Writable writable, throw new HoodieAvroSchemaException(String.format("cannot support rewrite value for schema type: %s since the old schema type is: %s", newSchema, oldSchema)); } - private static int[] getReverseProjectionMapping(Schema from, Schema to) { - List fromFields = from.getFields(); + private static int[] getReverseProjectionMapping(HoodieSchema from, HoodieSchema to) { + List fromFields = from.getFields(); int[] newProjection = new int[fromFields.size()]; for (int i = 0; i < newProjection.length; i++) { - newProjection[i] = to.getField(fromFields.get(i).name()).pos(); + String fieldName = fromFields.get(i).name(); + newProjection[i] = to.getField(fieldName).orElseThrow(() -> new IllegalArgumentException("Schema missing field with name: " + fieldName)).pos(); } return newProjection; } @@ -319,7 +322,7 @@ private static int[] getReverseProjectionMapping(Schema from, Schema to) { * After the reading and merging etc is done, we need to put the records * into the positions of the original schema */ - public static UnaryOperator getReverseProjection(Schema from, Schema to) { + public static UnaryOperator getReverseProjection(HoodieSchema from, HoodieSchema to) { int[] projection = getReverseProjectionMapping(from, to); return arrayWritable -> { Writable[] values = new Writable[to.getFields().size()]; diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieFileGroupReaderBasedRecordReader.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieFileGroupReaderBasedRecordReader.java index 0d16dabc3fc39..a103c15d8df5b 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieFileGroupReaderBasedRecordReader.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieFileGroupReaderBasedRecordReader.java @@ -19,9 +19,9 @@ package org.apache.hudi.hadoop; +import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.util.collection.ClosableIterator; -import org.apache.avro.Schema; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; import org.apache.hadoop.io.ArrayWritable; @@ -76,8 +76,8 @@ void testDuplicateFieldHandlingInHiveQueryWithWhereClause() { + " {\"name\": \"field3\", \"type\": \"int\"}\n" + " ]\n" + "}"; - Schema tableSchema = new Schema.Parser().parse(schemaStr); - Schema requestedSchema = HoodieFileGroupReaderBasedRecordReader.createRequestedSchema(tableSchema, jobConf); + HoodieSchema tableSchema = HoodieSchema.parse(schemaStr); + HoodieSchema requestedSchema = HoodieFileGroupReaderBasedRecordReader.createRequestedSchema(tableSchema, jobConf); assertEquals(2, requestedSchema.getFields().size()); assertEquals("field1", requestedSchema.getFields().get(0).name()); assertEquals("field2", requestedSchema.getFields().get(1).name()); 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/TestHoodieArrayWritableAvroUtils.java index 5febe8f600837..a24434768879c 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/TestHoodieArrayWritableAvroUtils.java @@ -21,6 +21,8 @@ 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.HoodieSchemaUtils; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.exception.HoodieAvroSchemaException; import org.apache.hudi.hadoop.HiveHoodieReaderContext; @@ -70,25 +72,24 @@ public class TestHoodieArrayWritableAvroUtils { - HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); - Schema tableSchema = HoodieTestDataGenerator.AVRO_SCHEMA; + private final HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); @Test public void testProjection() { - Schema from = tableSchema; - Schema to = HoodieAvroUtils.generateProjectionSchema(from, Arrays.asList("trip_type", "current_ts", "weight")); + HoodieSchema from = HoodieTestDataGenerator.HOODIE_SCHEMA; + HoodieSchema to = HoodieSchemaUtils.generateProjectionSchema(from, Arrays.asList("trip_type", "current_ts", "weight")); UnaryOperator reverseProjection = HoodieArrayWritableAvroUtils.getReverseProjection(to, from); //We reuse the ArrayWritable, so we need to get the values before projecting ArrayWritable record = convertArrayWritable(dataGen.generateGenericRecord()); - HiveAvroSerializer fromSerializer = new HiveAvroSerializer(from); + HiveAvroSerializer fromSerializer = new HiveAvroSerializer(from.toAvroSchema()); Object tripType = fromSerializer.getValue(record, "trip_type"); Object currentTs = fromSerializer.getValue(record, "current_ts"); Object weight = fromSerializer.getValue(record, "weight"); //Make sure the projected fields can be read - ArrayWritable projectedRecord = HoodieArrayWritableAvroUtils.rewriteRecordWithNewSchema(record, from, to, Collections.emptyMap()); - HiveAvroSerializer toSerializer = new HiveAvroSerializer(to); + ArrayWritable projectedRecord = HoodieArrayWritableAvroUtils.rewriteRecordWithNewSchema(record, from.toAvroSchema(), to.toAvroSchema(), Collections.emptyMap()); + HiveAvroSerializer toSerializer = new HiveAvroSerializer(to.toAvroSchema()); assertEquals(tripType, toSerializer.getValue(projectedRecord, "trip_type")); assertEquals(currentTs, toSerializer.getValue(projectedRecord, "current_ts")); assertEquals(weight, toSerializer.getValue(projectedRecord, "weight")); diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDDV2.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDDV2.scala index a1523a4f84e9f..47079e7a03b41 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDDV2.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDDV2.scala @@ -25,6 +25,7 @@ import org.apache.hudi.common.config.{HoodieReaderConfig, TypedProperties} import org.apache.hudi.common.config.HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.model.{HoodieBaseFile, HoodieFileFormat, HoodieRecord} +import org.apache.hudi.common.schema.HoodieSchema import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.table.log.InstantRange import org.apache.hudi.common.table.log.InstantRange.RangeType @@ -165,7 +166,7 @@ class HoodieMergeOnReadRDDV2(@transient sc: SparkContext, val partitionPath = FSUtils.getRelativePartitionPath(metaClient.getBasePath, fullPartitionPath) if (metaClient.isMetadataTable) { - val requestedSchema = new Schema.Parser().parse(requiredSchema.avroSchemaStr) + val requestedSchema = HoodieSchema.parse(requiredSchema.avroSchemaStr) val instantRange = InstantRange.builder().rangeType(RangeType.EXACT_MATCH).explicitInstants(validInstants.value).build() val readerContext = new HoodieAvroReaderContext(storageConf, metaClient.getTableConfig, HOption.of(instantRange), HOption.empty().asInstanceOf[HOption[HPredicate]]) val fileGroupReader: HoodieFileGroupReader[IndexedRecord] = HoodieFileGroupReader.newBuilder() @@ -176,11 +177,11 @@ class HoodieMergeOnReadRDDV2(@transient sc: SparkContext, .withBaseFileOption(baseFileOption) .withPartitionPath(partitionPath) .withProps(properties) - .withDataSchema(new Schema.Parser().parse(tableSchema.avroSchemaStr)) + .withDataSchema(HoodieSchema.parse(tableSchema.avroSchemaStr)) .withRequestedSchema(requestedSchema) .withInternalSchema(HOption.ofNullable(tableSchema.internalSchema.orNull)) .build() - convertAvroToRowIterator(fileGroupReader.getClosableIterator, requestedSchema) + convertAvroToRowIterator(fileGroupReader.getClosableIterator, requestedSchema.toAvroSchema) } else { val readerContext = new SparkFileFormatInternalRowReaderContext(fileGroupBaseFileReader.value, optionalFilters, Seq.empty, storageConf, metaClient.getTableConfig) @@ -192,8 +193,8 @@ class HoodieMergeOnReadRDDV2(@transient sc: SparkContext, .withBaseFileOption(baseFileOption) .withPartitionPath(partitionPath) .withProps(properties) - .withDataSchema(new Schema.Parser().parse(tableSchema.avroSchemaStr)) - .withRequestedSchema(new Schema.Parser().parse(requiredSchema.avroSchemaStr)) + .withDataSchema(HoodieSchema.parse(tableSchema.avroSchemaStr)) + .withRequestedSchema(HoodieSchema.parse(requiredSchema.avroSchemaStr)) .withInternalSchema(HOption.ofNullable(tableSchema.internalSchema.orNull)) .build() convertCloseableIterator(fileGroupReader.getClosableIterator) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCFileGroupIterator.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCFileGroupIterator.scala index 9bfde280419c7..31a982e0910b2 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCFileGroupIterator.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCFileGroupIterator.scala @@ -104,7 +104,7 @@ class CDCFileGroupIterator(split: HoodieCDCFileGroupSplit, private var bufferedRecordMerger = getBufferedRecordMerger private def getBufferedRecordMerger: BufferedRecordMerger[InternalRow] = BufferedRecordMergerFactory.create(readerContext, readerContext.getMergeMode, isPartialMergeEnabled, Option.of(recordMerger), - payloadClass, avroSchema, props, partialUpdateModeOpt) + payloadClass, schema, props, partialUpdateModeOpt) private lazy val storage = metaClient.getStorage @@ -524,8 +524,8 @@ class CDCFileGroupIterator(split: HoodieCDCFileGroupSplit, .withReaderContext(readerContext) .withHoodieTableMetaClient(metaClient) .withFileSlice(fileSlice) - .withDataSchema(avroSchema) - .withRequestedSchema(avroSchema) + .withDataSchema(schema) + .withRequestedSchema(schema) .withInternalSchema(toJavaOption(originTableSchema.internalSchema)) .withProps(readerProperties) .withLatestCommitTime(split.changes.last.getInstant) @@ -539,7 +539,7 @@ class CDCFileGroupIterator(split: HoodieCDCFileGroupSplit, readerContext.setHasBootstrapBaseFile(false) readerContext.setHasLogFiles(true) readerContext.setSchemaHandler( - new FileGroupReaderSchemaHandler[InternalRow](readerContext, avroSchema, avroSchema, Option.empty(), readerProperties, metaClient)) + new FileGroupReaderSchemaHandler[InternalRow](readerContext, schema, schema, Option.empty(), readerProperties, metaClient)) val stats = new HoodieReadStats keyBasedFileGroupRecordBuffer.ifPresent(k => k.close()) keyBasedFileGroupRecordBuffer = Option.of(new KeyBasedFileGroupRecordBuffer[InternalRow](readerContext, metaClient, readerContext.getMergeMode, diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala index 718dc827df530..01eb729c252cf 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala @@ -25,6 +25,7 @@ import org.apache.hudi.client.utils.SparkInternalSchemaConverter import org.apache.hudi.common.config.{HoodieMemoryConfig, TypedProperties} import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.model.HoodieFileFormat +import org.apache.hudi.common.schema.{HoodieSchema, HoodieSchemaUtils} import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, ParquetTableSchemaResolver} import org.apache.hudi.common.table.read.HoodieFileGroupReader import org.apache.hudi.common.util.{Option => HOption} @@ -260,8 +261,8 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, .withHoodieTableMetaClient(metaClient) .withLatestCommitTime(queryTimestamp) .withFileSlice(fileSlice) - .withDataSchema(dataAvroSchema) - .withRequestedSchema(requestedAvroSchema) + .withDataSchema(HoodieSchema.fromAvroSchema(dataAvroSchema)) + .withRequestedSchema(HoodieSchema.fromAvroSchema(requestedAvroSchema)) .withInternalSchema(internalSchemaOpt) .withProps(props) .withStart(file.start) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/PartitionBucketIndexManager.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/PartitionBucketIndexManager.scala index cf97caba5bb92..3ff435055987a 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/PartitionBucketIndexManager.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/PartitionBucketIndexManager.scala @@ -25,6 +25,7 @@ import org.apache.hudi.common.config.{HoodieMetadataConfig, HoodieReaderConfig, import org.apache.hudi.common.engine.{HoodieEngineContext, ReaderContextFactory} import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.model.{PartitionBucketIndexHashingConfig, WriteOperationType} +import org.apache.hudi.common.schema.{HoodieSchema, HoodieSchemaUtils} import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.common.table.read.HoodieFileGroupReader import org.apache.hudi.common.table.view.HoodieTableFileSystemView @@ -208,25 +209,23 @@ class PartitionBucketIndexManager extends BaseProcedure }).toList // read all fileSlice para and get DF - var tableSchemaWithMetaFields: Schema = null - try tableSchemaWithMetaFields = HoodieAvroUtils.addMetadataFields(new TableSchemaResolver(metaClient).getTableAvroSchema(false), false) + var tableSchemaWithMetaFields: HoodieSchema = null + try tableSchemaWithMetaFields = HoodieSchemaUtils.addMetadataFields(new TableSchemaResolver(metaClient).getTableSchema(false), false) catch { case e: Exception => throw new HoodieException("Failed to get table schema during clustering", e) } val readerContextFactory: ReaderContextFactory[InternalRow] = context.getReaderContextFactory(metaClient) - val sparkSchemaWithMetaFields = AvroConversionUtils.convertAvroSchemaToStructType(tableSchemaWithMetaFields) + val sparkSchemaWithMetaFields = AvroConversionUtils.convertAvroSchemaToStructType(tableSchemaWithMetaFields.toAvroSchema) val res: RDD[InternalRow] = if (allFileSlice.isEmpty) { spark.sparkContext.emptyRDD } else { - val serializableTableSchemaWithMetaFields = new SerializableSchema(tableSchemaWithMetaFields) val latestInstantTime = metaClient.getActiveTimeline.getCommitsTimeline.filterCompletedInstants().lastInstant().get() spark.sparkContext.parallelize(allFileSlice, allFileSlice.size).flatMap(fileSlice => { // instantiate other supporting cast - val readerSchema = serializableTableSchemaWithMetaFields.get val internalSchemaOption: Option[InternalSchema] = Option.empty() // get this value from config, which has obtained this from write client val enableOptimizedLogBlockScan = config.getOrElse(HoodieReaderConfig.ENABLE_OPTIMIZED_LOG_BLOCKS_SCAN.key(), @@ -237,8 +236,8 @@ class PartitionBucketIndexManager extends BaseProcedure .withHoodieTableMetaClient(metaClient) .withLatestCommitTime(latestInstantTime.requestedTime()) .withFileSlice(fileSlice) - .withDataSchema(readerSchema) - .withRequestedSchema(readerSchema) + .withDataSchema(tableSchemaWithMetaFields) + .withRequestedSchema(tableSchemaWithMetaFields) .withInternalSchema(internalSchemaOption) // not support evolution of schema for now .withProps(metaClient.getTableConfig.getProps) .withShouldUseRecordPosition(false) diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestPositionBasedFileGroupRecordBuffer.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestPositionBasedFileGroupRecordBuffer.java index f3c7dd7e1aa92..6ca050c9542ba 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestPositionBasedFileGroupRecordBuffer.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestPositionBasedFileGroupRecordBuffer.java @@ -31,6 +31,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; @@ -51,7 +52,6 @@ import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.testutils.SparkClientFunctionalTestHarness; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.conf.Configuration; @@ -88,7 +88,7 @@ public class TestPositionBasedFileGroupRecordBuffer extends SparkClientFunctionalTestHarness { private final HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(0xDEEF); private final UpdateProcessor updateProcessor = mock(UpdateProcessor.class); - private Schema avroSchema; + private HoodieSchema schema; private PositionBasedFileGroupRecordBuffer buffer; private void prepareBuffer(RecordMergeMode mergeMode, String baseFileInstantTime) throws Exception { @@ -122,7 +122,7 @@ private void prepareBuffer(RecordMergeMode mergeMode, String baseFileInstantTime .setBasePath(basePath()) .setConf(storageConf()) .build(); - avroSchema = new TableSchemaResolver(metaClient).getTableAvroSchema(); + schema = new TableSchemaResolver(metaClient).getTableSchema(); SparkColumnarFileReader reader = SparkAdapterSupport$.MODULE$.sparkAdapter().createParquetFileReader(false, spark().sessionState().conf(), Map$.MODULE$.empty(), storageConf().unwrapAs(Configuration.class)); @@ -140,8 +140,8 @@ private void prepareBuffer(RecordMergeMode mergeMode, String baseFileInstantTime ctx.setRecordMerger(Option.empty()); } ctx.setSchemaHandler(HoodieSparkUtils.gteqSpark3_5() - ? new ParquetRowIndexBasedSchemaHandler<>(ctx, avroSchema, avroSchema, Option.empty(), new TypedProperties(), metaClient) - : new FileGroupReaderSchemaHandler<>(ctx, avroSchema, avroSchema, Option.empty(), new TypedProperties(), metaClient)); + ? new ParquetRowIndexBasedSchemaHandler<>(ctx, schema, schema, Option.empty(), new TypedProperties(), metaClient) + : new FileGroupReaderSchemaHandler<>(ctx, schema, schema, Option.empty(), new TypedProperties(), metaClient)); TypedProperties props = new TypedProperties(); props.put("hoodie.write.record.merge.mode", mergeMode.name()); props.setProperty(HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE.key(),String.valueOf(HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE.defaultValue())); @@ -179,7 +179,7 @@ private void commitToTable(List recordList, String operation, Map< private Map getHeader(boolean shouldWriteRecordPositions, String baseFileInstantTime) { Map header = new HashMap<>(); - header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, avroSchema.toString()); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); if (shouldWriteRecordPositions) { header.put(BASE_FILE_INSTANT_TIME_OF_RECORD_POSITIONS, baseFileInstantTime); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestMetadataUtilRLIandSIRecordGeneration.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestMetadataUtilRLIandSIRecordGeneration.java index 309169555ef90..90720d57d8a1f 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestMetadataUtilRLIandSIRecordGeneration.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestMetadataUtilRLIandSIRecordGeneration.java @@ -714,8 +714,8 @@ Set getRecordKeys(String partition, String baseInstantTime, String fileI HoodieReaderContext readerContext = context.getReaderContextFactory(metaClient).getContext(); HoodieFileGroupReader reader = HoodieFileGroupReader.newBuilder() .withReaderContext(readerContext) - .withDataSchema(writerSchemaOpt.get().toAvroSchema()) - .withRequestedSchema(writerSchemaOpt.get().toAvroSchema()) + .withDataSchema(writerSchemaOpt.get()) + .withRequestedSchema(writerSchemaOpt.get()) .withEmitDelete(true) .withPartitionPath(partition) .withLogFiles(logFilePaths.stream().map(HoodieLogFile::new)) diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBufferedRecordMerger.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBufferedRecordMerger.java index 9ece944807917..69267e9d85625 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBufferedRecordMerger.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBufferedRecordMerger.java @@ -106,7 +106,7 @@ class TestBufferedRecordMerger extends SparkClientFunctionalTestHarness { getSchema4(), getSchema5(), getSchema6()); - private static final Schema READER_SCHEMA = getSchema6().toAvroSchema(); + private static final HoodieSchema READER_SCHEMA = getSchema6(); private HoodieTableConfig tableConfig; private StorageConfiguration storageConfig; private TypedProperties props; @@ -622,7 +622,7 @@ void testCustomMerging(boolean usePayload) throws IOException { avroReaderContext.setHasBootstrapBaseFile(false); HoodieTableMetaClient metaClient = mock(HoodieTableMetaClient.class); when(metaClient.getTableConfig()).thenReturn(tableConfig); - avroReaderContext.setSchemaHandler(new FileGroupReaderSchemaHandler<>(avroReaderContext, customSchema.toAvroSchema(), customSchema.toAvroSchema(), Option.empty(), props, metaClient)); + avroReaderContext.setSchemaHandler(new FileGroupReaderSchemaHandler<>(avroReaderContext, customSchema, customSchema, Option.empty(), props, metaClient)); avroReaderContext.getRecordContext().encodeSchema(customSchema); BufferedRecordMerger merger = BufferedRecordMergerFactory.create( @@ -631,7 +631,7 @@ void testCustomMerging(boolean usePayload) throws IOException { false, recordMerger, payloadClassName, - customSchema.toAvroSchema(), + customSchema, props, Option.empty()); @@ -1039,15 +1039,15 @@ public ClosableIterator mergeBootstrapReaders( } } - public static void assertRowEqual(InternalRow expected, InternalRow actual, Schema schema) { + public static void assertRowEqual(InternalRow expected, InternalRow actual, HoodieSchema schema) { assertRowEqualsRecursive(expected, actual, schema.getFields(), ""); } private static void assertRowEqualsRecursive(InternalRow expected, InternalRow actual, - List fields, String pathPrefix) { + List fields, String pathPrefix) { for (int i = 0; i < fields.size(); i++) { - Schema.Field field = fields.get(i); - Schema fieldSchema = getNonNullSchema(field.schema()); + HoodieSchemaField field = fields.get(i); + HoodieSchema fieldSchema = field.schema().getNonNullType(); String path = pathPrefix + field.name(); if (expected.isNullAt(i) || actual.isNullAt(i)) { diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieBackedMetadata.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieBackedMetadata.java index e4f74aba6521b..44703b04d6ca0 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieBackedMetadata.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieBackedMetadata.java @@ -19,7 +19,6 @@ package org.apache.hudi.functional; import org.apache.hudi.avro.HoodieAvroReaderContext; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieMetadataColumnStats; import org.apache.hudi.avro.model.HoodieMetadataRecord; @@ -57,6 +56,8 @@ import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.TableServiceType; import org.apache.hudi.common.model.WriteConcurrencyMode; +import org.apache.hudi.common.schema.HoodieSchema; +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; @@ -1380,7 +1381,7 @@ private void verifyMetadataRecordKeyExcludeFromPayloadLogFiles(HoodieTable table verifyMetadataRawRecords(table, logFiles, enableMetaFields); // Verify the in-memory materialized and merged records - verifyMetadataMergedRecords(metadataMetaClient, logFiles, latestCommitTimestamp, enableMetaFields); + verifyMetadataMergedRecords(metadataMetaClient, logFiles, latestCommitTimestamp); } /** @@ -1444,14 +1445,10 @@ private static void verifyMetadataRawRecords(HoodieTable table, List logFiles, - String latestCommitTimestamp, boolean enableMetaFields) { - Schema schema = HoodieAvroUtils.addMetadataFields(HoodieMetadataRecord.getClassSchema()); - if (enableMetaFields) { - schema = HoodieAvroUtils.addMetadataFields(schema); - } + String latestCommitTimestamp) { + HoodieSchema schema = HoodieSchemaUtils.addMetadataFields(HoodieSchema.fromAvroSchema(HoodieMetadataRecord.getClassSchema())); HoodieAvroReaderContext readerContext = new HoodieAvroReaderContext(metadataMetaClient.getStorageConf(), metadataMetaClient.getTableConfig(), Option.empty(), Option.empty()); HoodieFileGroupReader fileGroupReader = HoodieFileGroupReader.newBuilder() .withReaderContext(readerContext) 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 e82e9d9aa5613..748ea5b7e6b9a 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 @@ -106,9 +106,9 @@ class TestHoodieFileGroupReaderOnSpark extends TestHoodieFileGroupReaderBase[Int tempDir.toAbsolutePath.toUri.toString } - override def getHoodieReaderContext(tablePath: String, avroSchema: Schema, storageConf: StorageConfiguration[_], metaClient: HoodieTableMetaClient): HoodieReaderContext[InternalRow] = { + override def getHoodieReaderContext(tablePath: String, schema: HoodieSchema, storageConf: StorageConfiguration[_], metaClient: HoodieTableMetaClient): HoodieReaderContext[InternalRow] = { val parquetReader = sparkAdapter.createParquetFileReader(vectorized = false, spark.sessionState.conf, Map.empty, storageConf.unwrapAs(classOf[Configuration])) - val dataSchema = AvroConversionUtils.convertAvroSchemaToStructType(avroSchema); + val dataSchema = AvroConversionUtils.convertAvroSchemaToStructType(schema.toAvroSchema) val orcReader = sparkAdapter.createOrcFileReader(vectorized = false, spark.sessionState.conf, Map.empty, storageConf.unwrapAs(classOf[Configuration]), dataSchema) val multiFormatReader = new MultipleColumnarFileFormatReader(parquetReader, orcReader) new SparkFileFormatInternalRowReaderContext(multiFormatReader, Seq.empty, Seq.empty, getStorageConf, metaClient.getTableConfig) @@ -136,9 +136,9 @@ class TestHoodieFileGroupReaderOnSpark extends TestHoodieFileGroupReaderBase[Int override def getCustomPayload: String = classOf[CustomPayloadForTesting].getName - override def assertRecordsEqual(schema: Schema, expected: InternalRow, actual: InternalRow): Unit = { + override def assertRecordsEqual(schema: HoodieSchema, expected: InternalRow, actual: InternalRow): Unit = { assertEquals(expected.numFields, actual.numFields) - val expectedStruct = HoodieSparkAvroSchemaConverters.toSqlType(schema)._1.asInstanceOf[StructType] + val expectedStruct = HoodieSparkAvroSchemaConverters.toSqlType(schema.toAvroSchema)._1.asInstanceOf[StructType] expected.toSeq(expectedStruct).zip(actual.toSeq(expectedStruct)).zipWithIndex.foreach { case ((v1, v2), i) => @@ -398,8 +398,8 @@ class TestHoodieFileGroupReaderOnSpark extends TestHoodieFileGroupReaderBase[Int schema } - override def assertRecordMatchesSchema(schema: Schema, record: InternalRow): Unit = { - val structType = HoodieInternalRowUtils.getCachedSchema(schema) + override def assertRecordMatchesSchema(schema: HoodieSchema, record: InternalRow): Unit = { + val structType = HoodieInternalRowUtils.getCachedSchema(schema.toAvroSchema) assertRecordMatchesSchema(structType, record) }