1919package org .apache .hudi .index ;
2020
2121import org .apache .hudi .avro .AvroSchemaCache ;
22- import org .apache .hudi .avro .HoodieAvroUtils ;
2322import org .apache .hudi .common .config .RecordMergeMode ;
24- import org .apache .hudi .common .config .SerializableSchema ;
2523import org .apache .hudi .common .config .TypedProperties ;
2624import org .apache .hudi .common .data .HoodieData ;
2725import org .apache .hudi .common .data .HoodiePairData ;
4240import org .apache .hudi .common .model .HoodieTableType ;
4341import org .apache .hudi .common .model .MetadataValues ;
4442import org .apache .hudi .common .schema .HoodieSchema ;
43+ import org .apache .hudi .common .schema .HoodieSchemaUtils ;
4544import org .apache .hudi .common .table .HoodieTableConfig ;
4645import org .apache .hudi .common .table .HoodieTableMetaClient ;
4746import org .apache .hudi .common .table .HoodieTableVersion ;
@@ -341,7 +340,7 @@ public static HoodieIndex createUserDefinedIndex(HoodieWriteConfig config) {
341340 * @return {@link HoodieRecord}s that have the current location being set.
342341 */
343342 private static <R > HoodieData <HoodieRecord <R >> getExistingRecords (
344- HoodieData <Pair <String , String >> partitionLocations , HoodieWriteConfig config , HoodieTable hoodieTable , ReaderContextFactory <R > readerContextFactory , Schema dataSchema ) {
343+ HoodieData <Pair <String , String >> partitionLocations , HoodieWriteConfig config , HoodieTable hoodieTable , ReaderContextFactory <R > readerContextFactory , HoodieSchema dataSchema ) {
345344 HoodieTableMetaClient metaClient = hoodieTable .getMetaClient ();
346345 final Option <String > instantTime = metaClient
347346 .getActiveTimeline () // we need to include all actions and completed
@@ -368,8 +367,8 @@ private static <R> HoodieData<HoodieRecord<R>> getExistingRecords(
368367 .withHoodieTableMetaClient (metaClient )
369368 .withLatestCommitTime (instantTime .get ())
370369 .withFileSlice (fileSlice )
371- .withDataSchema (dataSchema )
372- .withRequestedSchema (dataSchema )
370+ .withDataSchema (dataSchema . toAvroSchema () )
371+ .withRequestedSchema (dataSchema . toAvroSchema () )
373372 .withInternalSchema (internalSchemaOption )
374373 .withProps (metaClient .getTableConfig ().getProps ())
375374 .withEnableOptimizedLogBlockScan (config .enableOptimizedLogBlocksScan ())
@@ -516,7 +515,7 @@ public static <R> HoodieData<HoodieRecord<R>> mergeForPartitionUpdatesAndDeletio
516515 HoodieWriteConfig config ,
517516 HoodieTable hoodieTable ,
518517 HoodieReaderContext <R > readerContext ,
519- SerializableSchema writerSchema ) {
518+ HoodieSchema writerSchema ) {
520519 boolean isExpressionPayload = config .getPayloadClass ().equals ("org.apache.spark.sql.hudi.command.payload.ExpressionPayload" );
521520 Pair <HoodieWriteConfig , BaseKeyGenerator > keyGeneratorWriteConfigOpt =
522521 getKeygenAndUpdatedWriteConfig (config , hoodieTable .getMetaClient ().getTableConfig (), isExpressionPayload );
@@ -543,25 +542,27 @@ public static <R> HoodieData<HoodieRecord<R>> mergeForPartitionUpdatesAndDeletio
543542 .getReaderContextFactoryForWrite (hoodieTable .getMetaClient (), config .getRecordMerger ().getRecordType (), hoodieTable .getMetaClient ().getTableConfig ().getProps ());
544543 RecordContext <R > existingRecordContext = readerContextFactoryForExistingRecords .getContext ().getRecordContext ();
545544 // merged existing records with current locations being set
546- SerializableSchema writerSchemaWithMetaFields = new SerializableSchema (HoodieAvroUtils .addMetadataFields (writerSchema .get (), updatedConfig .allowOperationMetadataField ()));
547- AvroSchemaCache .intern (writerSchema .get ());
548- AvroSchemaCache .intern (writerSchemaWithMetaFields .get ());
545+ HoodieSchema writerSchemaWithMetaFields = HoodieSchemaUtils .addMetadataFields (writerSchema , updatedConfig .allowOperationMetadataField ());
546+ // TODO: Add HoodieSchemaCache#intern after #14374 is merged
547+ AvroSchemaCache .intern (writerSchema .toAvroSchema ());
548+ // TODO: Add HoodieSchemaCache#intern after #14374 is merged
549+ AvroSchemaCache .intern (writerSchemaWithMetaFields .toAvroSchema ());
549550 // Read the existing records with the meta fields and current writer schema as the output schema
550551 HoodieData <HoodieRecord <R >> existingRecords =
551- getExistingRecords (globalLocations , keyGeneratorWriteConfigOpt .getLeft (), hoodieTable , readerContextFactoryForExistingRecords , writerSchemaWithMetaFields . get () );
552+ getExistingRecords (globalLocations , keyGeneratorWriteConfigOpt .getLeft (), hoodieTable , readerContextFactoryForExistingRecords , writerSchemaWithMetaFields );
552553 List <String > orderingFieldNames = getOrderingFieldNames (
553554 readerContext .getMergeMode (), hoodieTable .getMetaClient ());
554555 BufferedRecordMerger <R > recordMerger = BufferedRecordMergerFactory .create (
555556 readerContext ,
556557 readerContext .getMergeMode (),
557558 false ,
558559 readerContext .getRecordMerger (),
559- writerSchema .get (),
560+ writerSchema .toAvroSchema (),
560561 Option .ofNullable (Pair .of (hoodieTable .getMetaClient ().getTableConfig ().getPayloadClass (), hoodieTable .getConfig ().getPayloadClass ())),
561562 properties ,
562563 hoodieTable .getMetaClient ().getTableConfig ().getPartialUpdateMode ());
563564 String [] orderingFieldsArray = orderingFieldNames .toArray (new String [0 ]);
564- DeleteContext deleteContext = DeleteContext .fromRecordSchema (properties , HoodieSchema . fromAvroSchema ( writerSchema . get ()) );
565+ DeleteContext deleteContext = DeleteContext .fromRecordSchema (properties , writerSchema );
565566 HoodieData <HoodieRecord <R >> taggedUpdatingRecords = untaggedUpdatingRecords .mapToPair (r -> Pair .of (r .getRecordKey (), r ))
566567 .leftOuterJoin (existingRecords .mapToPair (r -> Pair .of (r .getRecordKey (), r )))
567568 .values ().flatMap (entry -> {
@@ -572,10 +573,9 @@ public static <R> HoodieData<HoodieRecord<R>> mergeForPartitionUpdatesAndDeletio
572573 return Collections .singletonList (incoming ).iterator ();
573574 }
574575 HoodieRecord <R > existing = existingOpt .get ();
575- HoodieSchema writeSchema = HoodieSchema .fromAvroSchema (writerSchema .get ());
576576
577577 Option <HoodieRecord <R >> mergedOpt = mergeIncomingWithExistingRecord (
578- incoming , existing , writeSchema , HoodieSchema . fromAvroSchema ( writerSchemaWithMetaFields . get ()) , updatedConfig ,
578+ incoming , existing , writerSchema , writerSchemaWithMetaFields , updatedConfig ,
579579 recordMerger , keyGenerator , incomingRecordContext , existingRecordContext , orderingFieldsArray , properties , isExpressionPayload , deleteContext );
580580 if (!mergedOpt .isPresent ()) {
581581 // merge resulted in delete: force tag the incoming to the old partition
@@ -632,12 +632,12 @@ public static <R> HoodieData<HoodieRecord<R>> tagGlobalLocationBackToRecords(
632632 HoodieReaderContext <R > readerContext = readerContextFactory .getContext ();
633633 readerContext .initRecordMergerForIngestion (config .getProps ());
634634 TypedProperties properties = readerContext .getMergeProps (config .getProps ());
635- SerializableSchema writerSchema = new SerializableSchema (config .getWriteSchema ());
635+ HoodieSchema writerSchema = HoodieSchema . parse (config .getWriteSchema ());
636636 boolean isCommitTimeOrdered = readerContext .getMergeMode () == RecordMergeMode .COMMIT_TIME_ORDERING ;
637637 // if the index is not updating the partition of the record, and the table is COW, then we do not need to do merging at
638638 // this phase since the writer path will merge when rewriting the files as part of the upsert operation.
639639 boolean requiresMergingWithOlderRecordVersion = shouldUpdatePartitionPath || table .getMetaClient ().getTableConfig ().getTableType () == HoodieTableType .MERGE_ON_READ ;
640- DeleteContext deleteContext = DeleteContext .fromRecordSchema (properties , HoodieSchema . fromAvroSchema ( writerSchema . get ()) );
640+ DeleteContext deleteContext = DeleteContext .fromRecordSchema (properties , writerSchema );
641641
642642 // Pair of incoming record and the global location if meant for merged lookup in later stage
643643 HoodieData <Pair <HoodieRecord <R >, Option <HoodieRecordGlobalLocation >>> incomingRecordsAndLocations
0 commit comments