diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/WriteStatus.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/WriteStatus.java index 71d3308e616e0..e90f24148bbb3 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/WriteStatus.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/WriteStatus.java @@ -73,7 +73,7 @@ public class WriteStatus implements Serializable { private long totalErrorRecords = 0; private final double failureFraction; - private final boolean trackSuccessRecords; + private boolean trackSuccessRecords; private final transient Random random; private IndexStats indexStats = new IndexStats(); @@ -109,6 +109,17 @@ public void markSuccess(HoodieRecord record, Option> optiona updateStatsForSuccess(optionalRecordMetadata); } + /** + * Allows the writer to manually add record delegates to the index stats. + */ + public void manuallyTrackSuccess() { + this.trackSuccessRecords = false; + } + + public void addRecordDelegate(HoodieRecordDelegate recordDelegate) { + indexStats.addHoodieRecordDelegate(recordDelegate); + } + /** * Used by native write handles like HoodieRowCreateHandle and HoodieRowDataCreateHandle. * diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 84c0e20a475c1..1bfc167985c49 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -73,7 +73,6 @@ import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.io.FileGroupReaderBasedMergeHandle; import org.apache.hudi.io.HoodieConcatHandle; -import org.apache.hudi.io.HoodieWriteMergeHandle; import org.apache.hudi.keygen.SimpleAvroKeyGenerator; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.keygen.constant.KeyGeneratorType; @@ -858,7 +857,7 @@ public class HoodieWriteConfig extends HoodieConfig { public static final ConfigProperty MERGE_HANDLE_CLASS_NAME = ConfigProperty .key("hoodie.write.merge.handle.class") - .defaultValue(HoodieWriteMergeHandle.class.getName()) + .defaultValue(FileGroupReaderBasedMergeHandle.class.getName()) .markAdvanced() .sinceVersion("1.1.0") .withDocumentation("The merge handle class that implements interface{@link HoodieMergeHandle} to merge the records " 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 09285392e4e2b..97ed10814bb5c 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 @@ -430,16 +430,11 @@ private static Option> mergeIncomingWithExistingRecordWithEx //the record was deleted return Option.empty(); } - if (mergeResult.getRecord() == null) { - // SENTINEL case: the record did not match and merge case and should not be modified + if (mergeResult.getRecord() == null || mergeResult == existingBufferedRecord) { + // SENTINEL case: the record did not match the merge case and should not be modified or there is no update to the record return Option.of((HoodieRecord) new HoodieAvroIndexedRecord(HoodieRecord.SENTINEL)); } - if (mergeResult.getRecord().equals(HoodieRecord.SENTINEL)) { - //the record did not match and merge case and should not be modified - return Option.of(existingRecordContext.constructHoodieRecord(mergeResult, incoming.getPartitionPath())); - } - //record is inserted or updated String partitionPath = inferPartitionPath(incoming, existing, writeSchemaWithMetaFields, keyGenerator, existingRecordContext, mergeResult); HoodieRecord result = existingRecordContext.constructHoodieRecord(mergeResult, partitionPath); @@ -526,14 +521,14 @@ public static HoodieData> mergeForPartitionUpdatesIfNeeded( .distinct(updatedConfig.getGlobalIndexReconcileParallelism()); // define the buffered record merger. ReaderContextFactory readerContextFactory = (ReaderContextFactory) hoodieTable.getContext() - .getReaderContextFactoryForWrite(hoodieTable.getMetaClient(), config.getRecordMerger().getRecordType(), config.getProps()); + .getReaderContextFactoryForWrite(hoodieTable.getMetaClient(), config.getRecordMerger().getRecordType(), config.getProps(), true); HoodieReaderContext readerContext = readerContextFactory.getContext(); RecordContext incomingRecordContext = readerContext.getRecordContext(); readerContext.initRecordMergerForIngestion(config.getProps()); // Create a reader context for the existing records. In the case of merge-into commands, the incoming records // can be using an expression payload so here we rely on the table's configured payload class if it is required. ReaderContextFactory readerContextFactoryForExistingRecords = (ReaderContextFactory) hoodieTable.getContext() - .getReaderContextFactoryForWrite(hoodieTable.getMetaClient(), config.getRecordMerger().getRecordType(), hoodieTable.getMetaClient().getTableConfig().getProps()); + .getReaderContextFactoryForWrite(hoodieTable.getMetaClient(), config.getRecordMerger().getRecordType(), hoodieTable.getMetaClient().getTableConfig().getProps(), true); RecordContext existingRecordContext = readerContextFactoryForExistingRecords.getContext().getRecordContext(); // merged existing records with current locations being set SerializableSchema writerSchema = new SerializableSchema(hoodieTable.getConfig().getWriteSchema()); @@ -551,7 +546,7 @@ public static HoodieData> mergeForPartitionUpdatesIfNeeded( false, readerContext.getRecordMerger(), orderingFieldNames, - writerSchema.get(), + isExpressionPayload ? writerSchema.get() : writerSchemaWithMetaFields.get(), Option.ofNullable(Pair.of(hoodieTable.getMetaClient().getTableConfig().getPayloadClass(), hoodieTable.getConfig().getPayloadClass())), hoodieTable.getConfig().getProps(), hoodieTable.getMetaClient().getTableConfig().getPartialUpdateMode()); 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 efe7b1c5fe1e1..d28f472e3f910 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 @@ -20,18 +20,25 @@ package org.apache.hudi.io; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.HoodieCommonConfig; import org.apache.hudi.common.config.HoodieMemoryConfig; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.HoodieReaderContext; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.CompactionOperation; +import org.apache.hudi.common.model.HoodieIndexDefinition; +import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieOperation; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordDelegate; +import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.cdc.HoodieCDCUtils; import org.apache.hudi.common.table.read.BaseFileUpdateCallback; +import org.apache.hudi.common.table.read.BufferedRecord; import org.apache.hudi.common.table.read.HoodieFileGroupReader; import org.apache.hudi.common.table.read.HoodieReadStats; import org.apache.hudi.common.util.Option; @@ -39,8 +46,10 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.utils.AvroSchemaEvolutionUtils; import org.apache.hudi.internal.schema.utils.SerDeHelper; import org.apache.hudi.io.storage.HoodieFileWriterFactory; +import org.apache.hudi.keygen.BaseKeyGenerator; import org.apache.hudi.storage.StoragePath; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.compact.strategy.CompactionStrategy; @@ -54,8 +63,10 @@ import javax.annotation.concurrent.NotThreadSafe; import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.function.UnaryOperator; @@ -75,22 +86,71 @@ public class FileGroupReaderBasedMergeHandle extends HoodieWriteMergeHandle { private static final Logger LOG = LoggerFactory.getLogger(FileGroupReaderBasedMergeHandle.class); - private final HoodieReaderContext readerContext; - private final CompactionOperation operation; + private final Option compactionOperation; private final String maxInstantTime; + private HoodieReaderContext readerContext; private HoodieReadStats readStats; - private final HoodieRecord.HoodieRecordType recordType; - private final Option cdcLogger; + private HoodieRecord.HoodieRecordType recordType; + private Option cdcLogger; + private final TypedProperties props; + private final Iterator> incomingRecordsItr; + /** + * Constructor for Copy-On-Write (COW) merge path. + * Takes in a base path and an iterator of records to be merged with that file. + * + * @param config instance of {@link HoodieWriteConfig} to use. + * @param instantTime instant time of the current commit. + * @param hoodieTable instance of {@link HoodieTable} being updated. + * @param recordItr iterator of records to be merged with the file. + * @param partitionPath partition path of the base file. + * @param fileId file ID of the base file. + * @param taskContextSupplier instance of {@link TaskContextSupplier} to use. + * @param keyGeneratorOpt optional instance of {@link BaseKeyGenerator} to use for extracting keys from records. + */ + public FileGroupReaderBasedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, + Iterator> recordItr, String partitionPath, String fileId, + TaskContextSupplier taskContextSupplier, Option keyGeneratorOpt) { + super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, taskContextSupplier, keyGeneratorOpt); + this.compactionOperation = Option.empty(); + this.readerContext = hoodieTable.getReaderContextFactoryForWrite().getContext(); + TypedProperties properties = config.getProps(); + properties.putAll(hoodieTable.getMetaClient().getTableConfig().getProps()); + this.maxInstantTime = instantTime; + initRecordTypeAndCdcLogger(hoodieTable.getConfig().getRecordMerger().getRecordType()); + this.props = TypedProperties.copy(config.getProps()); + this.incomingRecordsItr = recordItr; + } + + /** + * Constructor used for Compaction flows. + * Take in a base path and list of log files, to merge them together to produce a new base file. + * + * @param config instance of {@link HoodieWriteConfig} to use. + * @param instantTime instant time of interest. + * @param hoodieTable instance of {@link HoodieTable} to use. + * @param compactionOperation compaction operation containing info about base and log files. + * @param taskContextSupplier instance of {@link TaskContextSupplier} to use. + * @param maxInstantTime max instant time to use. + * @param enginRecordType engine record type. + */ + @SuppressWarnings("unused") public FileGroupReaderBasedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, - CompactionOperation operation, TaskContextSupplier taskContextSupplier, + CompactionOperation compactionOperation, TaskContextSupplier taskContextSupplier, HoodieReaderContext readerContext, String maxInstantTime, HoodieRecord.HoodieRecordType enginRecordType) { - super(config, instantTime, operation.getPartitionPath(), operation.getFileId(), hoodieTable, taskContextSupplier); + super(config, instantTime, compactionOperation.getPartitionPath(), compactionOperation.getFileId(), hoodieTable, taskContextSupplier); this.maxInstantTime = maxInstantTime; this.keyToNewRecords = Collections.emptyMap(); this.readerContext = readerContext; - this.operation = operation; + this.compactionOperation = Option.of(compactionOperation); + initRecordTypeAndCdcLogger(enginRecordType); + init(compactionOperation, this.partitionPath); + this.props = TypedProperties.copy(config.getProps()); + this.incomingRecordsItr = null; + } + + private void initRecordTypeAndCdcLogger(HoodieRecord.HoodieRecordType enginRecordType) { // If the table is a metadata table or the base file is an HFile, we use AVRO record type, otherwise we use the engine record type. this.recordType = hoodieTable.isMetadataTable() || HFILE.getFileExtension().equals(hoodieTable.getBaseFileExtension()) ? HoodieRecord.HoodieRecordType.AVRO : enginRecordType; if (hoodieTable.getMetaClient().getTableConfig().isCDCEnabled()) { @@ -106,7 +166,6 @@ public FileGroupReaderBasedMergeHandle(HoodieWriteConfig config, String instantT } else { this.cdcLogger = Option.empty(); } - init(operation, this.partitionPath); } private void init(CompactionOperation operation, String partitionPath) { @@ -156,40 +215,53 @@ private void init(CompactionOperation operation, String partitionPath) { fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, newFilePath, hoodieTable.getStorage(), config, writeSchemaWithMetaFields, taskContextSupplier, recordType); } catch (IOException io) { - LOG.error("Error in update task at commit {}", instantTime, io); writeStatus.setGlobalError(io); throw new HoodieUpsertException("Failed to initialize HoodieUpdateHandle for FileId: " + fileId + " on commit " + instantTime + " on path " + hoodieTable.getMetaClient().getBasePath(), io); } } + @Override + protected void populateIncomingRecordsMap(Iterator> newRecordsItr) { + // no op. + } + + /** + * This is only for spark, the engine context fetched from a serialized hoodie table is always local, + * overrides it to spark specific reader context. + */ + public void setReaderContext(HoodieReaderContext readerContext) { + this.readerContext = readerContext; + } + /** * Reads the file slice of a compaction operation using a file group reader, * by getting an iterator of the records; then writes the records to a new base file. */ @Override public void doMerge() { + // For non-compaction operations, the merger needs to be initialized with the writer properties to handle cases like Merge-Into commands + if (compactionOperation.isEmpty()) { + this.readerContext.initRecordMergerForIngestion(config.getProps()); + } boolean usePosition = config.getBooleanOrDefault(MERGE_USE_RECORD_POSITIONS); - Option internalSchemaOption = SerDeHelper.fromJson(config.getInternalSchema()); - TypedProperties props = TypedProperties.copy(config.getProps()); + Option internalSchemaOption = SerDeHelper.fromJson(config.getInternalSchema()) + .map(internalSchema -> AvroSchemaEvolutionUtils.reconcileSchema(writeSchemaWithMetaFields, internalSchema, + config.getBooleanOrDefault(HoodieCommonConfig.SET_NULL_FOR_MISSING_COLUMNS))); long maxMemoryPerCompaction = IOUtils.getMaxMemoryPerCompaction(taskContextSupplier, config); props.put(HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE.key(), String.valueOf(maxMemoryPerCompaction)); - Stream logFiles = operation.getDeltaFileNames().stream().map(logFileName -> + Option> logFilesStreamOpt = compactionOperation.map(op -> op.getDeltaFileNames().stream().map(logFileName -> new HoodieLogFile(new StoragePath(FSUtils.constructAbsolutePath( - config.getBasePath(), operation.getPartitionPath()), logFileName))); + config.getBasePath(), op.getPartitionPath()), logFileName)))); // Initializes file group reader - try (HoodieFileGroupReader fileGroupReader = HoodieFileGroupReader.newBuilder().withReaderContext(readerContext).withHoodieTableMetaClient(hoodieTable.getMetaClient()) - .withLatestCommitTime(maxInstantTime).withPartitionPath(partitionPath).withBaseFileOption(Option.ofNullable(baseFileToMerge)).withLogFiles(logFiles) - .withDataSchema(writeSchemaWithMetaFields).withRequestedSchema(writeSchemaWithMetaFields).withInternalSchema(internalSchemaOption).withProps(props) - .withShouldUseRecordPosition(usePosition).withSortOutput(hoodieTable.requireSortedRecords()) - .withFileGroupUpdateCallback(cdcLogger.map(logger -> new CDCCallback(logger, readerContext))).withEnableOptimizedLogBlockScan(config.enableOptimizedLogBlocksScan()).build()) { + try (HoodieFileGroupReader fileGroupReader = getFileGroupReader(usePosition, internalSchemaOption, props, logFilesStreamOpt, incomingRecordsItr)) { // Reads the records from the file slice try (ClosableIterator> recordIterator = fileGroupReader.getClosableHoodieRecordIterator()) { while (recordIterator.hasNext()) { HoodieRecord record = recordIterator.next(); + Option> recordMetadata = compactionOperation.isEmpty() ? getRecordMetadata(record, writeSchema, props) : Option.empty(); record.setCurrentLocation(newRecordLocation); record.setNewLocation(newRecordLocation); - Option> recordMetadata = record.getMetadata(); if (!partitionPath.equals(record.getPartitionPath())) { HoodieUpsertException failureEx = new HoodieUpsertException("mismatched partition path, record partition: " + record.getPartitionPath() + " but trying to insert into partition: " + partitionPath); @@ -198,8 +270,12 @@ public void doMerge() { } // Writes the record try { - writeToFile(record.getKey(), record, writeSchemaWithMetaFields, - config.getPayloadConfig().getProps(), preserveMetadata); + // For Compaction operations, the preserveMetadata flag is always true as we want to preserve the existing record metadata. + // For other updates, we only want to preserve the metadata if the record is not being modified by this update. If the record already exists in the base file and is not updated, + // the operation will be null. Records that are being updated or records being added to the file group for the first time will have an operation set and must generate new metadata. + boolean shouldPreserveRecordMetadata = preserveMetadata || record.getOperation() == null; + Schema recordSchema = shouldPreserveRecordMetadata ? writeSchemaWithMetaFields : writeSchema; + writeToFile(record.getKey(), record, recordSchema, config.getPayloadConfig().getProps(), shouldPreserveRecordMetadata); writeStatus.markSuccess(record, recordMetadata); recordsWritten++; } catch (Exception e) { @@ -220,6 +296,23 @@ public void doMerge() { } } + private HoodieFileGroupReader getFileGroupReader(boolean usePosition, Option internalSchemaOption, TypedProperties props, + Option> logFileStreamOpt, Iterator> incomingRecordsItr) { + HoodieFileGroupReader.Builder fileGroupBuilder = HoodieFileGroupReader.newBuilder().withReaderContext(readerContext).withHoodieTableMetaClient(hoodieTable.getMetaClient()) + .withLatestCommitTime(maxInstantTime).withPartitionPath(partitionPath).withBaseFileOption(Option.ofNullable(baseFileToMerge)) + .withDataSchema(writeSchemaWithMetaFields).withRequestedSchema(writeSchemaWithMetaFields) + .withInternalSchema(internalSchemaOption).withProps(props) + .withShouldUseRecordPosition(usePosition).withSortOutput(hoodieTable.requireSortedRecords()) + .withFileGroupUpdateCallback(createCallback()); + + if (logFileStreamOpt.isPresent()) { + fileGroupBuilder.withLogFiles(logFileStreamOpt.get()); + } else { + fileGroupBuilder.withRecordIterator(incomingRecordsItr); + } + return fileGroupBuilder.build(); + } + @Override protected void writeIncomingRecords() { // no operation. @@ -240,7 +333,9 @@ public List close() { writeStatus.getStat().setTotalLogBlocks(readStats.getTotalLogBlocks()); writeStatus.getStat().setTotalCorruptLogBlock(readStats.getTotalCorruptLogBlock()); writeStatus.getStat().setTotalRollbackBlocks(readStats.getTotalRollbackBlocks()); - writeStatus.getStat().setTotalLogSizeCompacted(operation.getMetrics().get(CompactionStrategy.TOTAL_LOG_FILE_SIZE).longValue()); + if (compactionOperation.isPresent()) { + writeStatus.getStat().setTotalLogSizeCompacted(compactionOperation.get().getMetrics().get(CompactionStrategy.TOTAL_LOG_FILE_SIZE).longValue()); + } if (writeStatus.getStat().getRuntimeStats() != null) { writeStatus.getStat().getRuntimeStats().setTotalScanTime(readStats.getTotalLogReadTimeMs()); @@ -251,6 +346,31 @@ public List close() { } } + private Option> createCallback() { + List> callbacks = new ArrayList<>(); + // Handle CDC workflow. + if (cdcLogger.isPresent()) { + callbacks.add(new CDCCallback<>(cdcLogger.get(), readerContext)); + } + // Indexes are not updated during compaction + if (compactionOperation.isEmpty()) { + // record index callback + if (this.writeStatus.isTrackingSuccessfulWrites()) { + writeStatus.manuallyTrackSuccess(); + callbacks.add(new RecordLevelIndexCallback<>(writeStatus, newRecordLocation, partitionPath)); + } + // Stream secondary index stats. + if (isSecondaryIndexStatsStreamingWritesEnabled) { + callbacks.add(new SecondaryIndexCallback<>( + partitionPath, + readerContext, + writeStatus, + secondaryIndexDefns)); + } + } + return callbacks.isEmpty() ? Option.empty() : Option.of(CompositeCallback.of(callbacks)); + } + private static class CDCCallback implements BaseFileUpdateCallback { private final HoodieCDCLogger cdcLogger; private final HoodieReaderContext readerContext; @@ -267,26 +387,140 @@ private static class CDCCallback implements BaseFileUpdateCallback { } @Override - public void onUpdate(String recordKey, T previousRecord, T mergedRecord) { + public void onUpdate(String recordKey, BufferedRecord previousRecord, BufferedRecord mergedRecord) { cdcLogger.put(recordKey, convertOutput(previousRecord), Option.of(convertOutput(mergedRecord))); - } @Override - public void onInsert(String recordKey, T newRecord) { + public void onInsert(String recordKey, BufferedRecord newRecord) { cdcLogger.put(recordKey, null, Option.of(convertOutput(newRecord))); - } @Override - public void onDelete(String recordKey, T previousRecord) { + public void onDelete(String recordKey, BufferedRecord previousRecord, HoodieOperation hoodieOperation) { cdcLogger.put(recordKey, convertOutput(previousRecord), Option.empty()); - } - private GenericRecord convertOutput(T record) { - T convertedRecord = outputConverter.get().map(converter -> record == null ? null : converter.apply(record)).orElse(record); + private GenericRecord convertOutput(BufferedRecord record) { + T convertedRecord = outputConverter.get().map(converter -> record == null ? null : converter.apply(record.getRecord())).orElse(record.getRecord()); return convertedRecord == null ? null : readerContext.getRecordContext().convertToAvroRecord(convertedRecord, requestedSchema.get()); } } + + private static class RecordLevelIndexCallback implements BaseFileUpdateCallback { + private final WriteStatus writeStatus; + private final HoodieRecordLocation fileRecordLocation; + private final String partitionPath; + + public RecordLevelIndexCallback(WriteStatus writeStatus, HoodieRecordLocation fileRecordLocation, String partitionPath) { + this.writeStatus = writeStatus; + this.fileRecordLocation = fileRecordLocation; + this.partitionPath = partitionPath; + } + + @Override + public void onUpdate(String recordKey, BufferedRecord previousRecord, BufferedRecord mergedRecord) { + writeStatus.addRecordDelegate(HoodieRecordDelegate.create(recordKey, partitionPath, fileRecordLocation, fileRecordLocation)); + } + + @Override + public void onInsert(String recordKey, BufferedRecord newRecord) { + writeStatus.addRecordDelegate(HoodieRecordDelegate.create(recordKey, partitionPath, null, fileRecordLocation)); + } + + @Override + public void onDelete(String recordKey, BufferedRecord previousRecord, HoodieOperation hoodieOperation) { + // The update before operation is used when a deletion is being sent to the old File Group in a different partition. + // In this case, we do not want to delete the record metadata from the index. + if (hoodieOperation != HoodieOperation.UPDATE_BEFORE) { + writeStatus.addRecordDelegate(HoodieRecordDelegate.create(recordKey, partitionPath, fileRecordLocation, null)); + } + } + } + + private static class SecondaryIndexCallback implements BaseFileUpdateCallback { + private final String partitionPath; + private final HoodieReaderContext readerContext; + private final WriteStatus writeStatus; + private final List secondaryIndexDefns; + + public SecondaryIndexCallback(String partitionPath, + HoodieReaderContext readerContext, + WriteStatus writeStatus, + List secondaryIndexDefns) { + this.partitionPath = partitionPath; + this.readerContext = readerContext; + this.secondaryIndexDefns = secondaryIndexDefns; + this.writeStatus = writeStatus; + } + + @Override + public void onUpdate(String recordKey, BufferedRecord previousRecord, BufferedRecord mergedRecord) { + HoodieKey hoodieKey = new HoodieKey(recordKey, partitionPath); + SecondaryIndexStreamingTracker.trackSecondaryIndexStats( + hoodieKey, + Option.of(mergedRecord), + previousRecord, + false, + writeStatus, + secondaryIndexDefns, + readerContext.getRecordContext()); + } + + @Override + public void onInsert(String recordKey, BufferedRecord newRecord) { + HoodieKey hoodieKey = new HoodieKey(recordKey, partitionPath); + SecondaryIndexStreamingTracker.trackSecondaryIndexStats( + hoodieKey, + Option.of(newRecord), + null, + false, + writeStatus, + secondaryIndexDefns, + readerContext.getRecordContext()); + } + + @Override + public void onDelete(String recordKey, BufferedRecord previousRecord, HoodieOperation hoodieOperation) { + HoodieKey hoodieKey = new HoodieKey(recordKey, partitionPath); + SecondaryIndexStreamingTracker.trackSecondaryIndexStats( + hoodieKey, + Option.empty(), + previousRecord, + true, + writeStatus, + secondaryIndexDefns, + readerContext.getRecordContext()); + } + } + + private static class CompositeCallback implements BaseFileUpdateCallback { + private final List> callbacks; + + static BaseFileUpdateCallback of(List> callbacks) { + if (callbacks.size() == 1) { + return callbacks.get(0); + } + return new CompositeCallback<>(callbacks); + } + + private CompositeCallback(List> callbacks) { + this.callbacks = callbacks; + } + + @Override + public void onUpdate(String recordKey, BufferedRecord previousRecord, BufferedRecord mergedRecord) { + this.callbacks.forEach(callback -> callback.onUpdate(recordKey, previousRecord, mergedRecord)); + } + + @Override + public void onInsert(String recordKey, BufferedRecord newRecord) { + this.callbacks.forEach(callback -> callback.onInsert(recordKey, newRecord)); + } + + @Override + public void onDelete(String recordKey, BufferedRecord previousRecord, HoodieOperation hoodieOperation) { + this.callbacks.forEach(callback -> callback.onDelete(recordKey, previousRecord, hoodieOperation)); + } + } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java index 63b79102fd16d..990a624a9bcff 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java @@ -527,7 +527,7 @@ public List close() { try { if (isClosed()) { // Handle has already been closed - return Collections.emptyList(); + return Collections.singletonList(writeStatus); } markClosed(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java index d51c823e6e8bb..e3e91b6e4edfa 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java @@ -214,7 +214,7 @@ public List close() { try { if (isClosed()) { // Handle has already been closed - return Collections.emptyList(); + return Collections.singletonList(writeStatus); } markClosed(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandleFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandleFactory.java index a0fe1291a9322..0688d55a5e707 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandleFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandleFactory.java @@ -165,6 +165,7 @@ private static HoodieMergeHandle instantiateMergeHandle static Pair getMergeHandleClassesWrite(WriteOperationType operationType, HoodieWriteConfig writeConfig, HoodieTable table) { String mergeHandleClass; String fallbackMergeHandleClass = null; + // Overwrite to a different implementation for {@link HoodieWriteMergeHandle} if sorting or CDC is enabled. if (table.requireSortedRecords()) { if (table.getMetaClient().getTableConfig().isCDCEnabled()) { @@ -178,7 +179,14 @@ static Pair getMergeHandleClassesWrite(WriteOperationType operat fallbackMergeHandleClass = HoodieWriteConfig.CONCAT_HANDLE_CLASS_NAME.defaultValue(); } } else if (table.getMetaClient().getTableConfig().isCDCEnabled()) { - mergeHandleClass = HoodieMergeHandleWithChangeLog.class.getName(); + if (writeConfig.getMergeHandleClassName().equals(FileGroupReaderBasedMergeHandle.class.getName())) { + mergeHandleClass = writeConfig.getMergeHandleClassName(); + if (!mergeHandleClass.equals(HoodieWriteConfig.MERGE_HANDLE_CLASS_NAME.defaultValue())) { + fallbackMergeHandleClass = HoodieWriteConfig.MERGE_HANDLE_CLASS_NAME.defaultValue(); + } + } else { + mergeHandleClass = HoodieMergeHandleWithChangeLog.class.getName(); + } } else { mergeHandleClass = writeConfig.getMergeHandleClassName(); if (!mergeHandleClass.equals(HoodieWriteConfig.MERGE_HANDLE_CLASS_NAME.defaultValue())) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteMergeHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteMergeHandle.java index 7577da82846eb..0347a8de595d9 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteMergeHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteMergeHandle.java @@ -446,7 +446,7 @@ public List close() { try { if (isClosed()) { // Handle has already been closed - return Collections.emptyList(); + return Collections.singletonList(writeStatus); } markClosed(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/IOUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/IOUtils.java index 29fe5246b613e..6e1b8b7a34b76 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/IOUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/IOUtils.java @@ -126,6 +126,6 @@ public static Iterator> runMerge(HoodieMergeHandle LOG.info("Upsert Handle has partition path as null " + mergeHandle.getOldFilePath() + ", " + mergeHandle.getWriteStatuses()); } - return Collections.singletonList(mergeHandle.getWriteStatuses()).iterator(); + return Collections.singletonList(mergeHandle.close()).iterator(); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/SecondaryIndexStreamingTracker.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/SecondaryIndexStreamingTracker.java index db794beeca6ac..7251043234439 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/SecondaryIndexStreamingTracker.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/SecondaryIndexStreamingTracker.java @@ -20,12 +20,14 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.HoodieReaderContext; +import org.apache.hudi.common.engine.RecordContext; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieIndexDefinition; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.read.BufferedRecord; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; @@ -226,6 +228,81 @@ static void trackSecondaryIndexStats(@Nullable HoodieKey hoodieKey, Option void trackSecondaryIndexStats(HoodieKey hoodieKey, Option> combinedRecordOpt, @Nullable BufferedRecord oldRecord, boolean isDelete, + WriteStatus writeStatus, List secondaryIndexDefns, RecordContext recordContext) { + + secondaryIndexDefns.forEach(def -> { + String secondaryIndexSourceField = def.getSourceFieldsKey(); + + // Handle three cases explicitly: + // 1. Old record does not exist (INSERT operation) + // 2. Old record exists with a value (could be null value) + // 3. New record state after operation + + boolean hasOldValue = oldRecord != null; + Object oldSecondaryKey = null; + + if (hasOldValue) { + Schema schema = recordContext.decodeAvroSchema(oldRecord.getSchemaId()); + oldSecondaryKey = recordContext.getTypeConverter().castToString(recordContext.getValue(oldRecord.getRecord(), schema, secondaryIndexSourceField)); + } + + // For new/combined record + boolean hasNewValue = false; + Object newSecondaryKey = null; + + if (combinedRecordOpt.isPresent() && !isDelete) { + Schema schema = recordContext.decodeAvroSchema(combinedRecordOpt.get().getSchemaId()); + newSecondaryKey = recordContext.getTypeConverter().castToString(recordContext.getValue(combinedRecordOpt.get().getRecord(), schema, secondaryIndexSourceField)); + hasNewValue = true; + } + + // Determine if we need to update the secondary index + boolean shouldUpdate; + if (!hasOldValue && !hasNewValue) { + // Case 4: Neither old nor new value exists - do nothing + shouldUpdate = false; + } else if (hasOldValue && hasNewValue) { + // Both old and new values exist - check if they differ + shouldUpdate = !Objects.equals(oldSecondaryKey, newSecondaryKey); + } else { + // One exists but not the other - need to update + shouldUpdate = true; + } + + // All possible cases: + // 1. Old record exists, new record does not exist - delete old secondary index entry + // 2. Old record exists, new record exists - update secondary index entry + // 3. Old record does not exist, new record exists - add new secondary index entry + // 4. Old record does not exist, new record does not exist - do nothing + if (shouldUpdate) { + String recordKey = hoodieKey.getRecordKey(); + + // Delete old secondary index entry if old record exists. + if (hasOldValue) { + addSecondaryIndexStat(writeStatus, def.getIndexName(), recordKey, oldSecondaryKey, true); + } + + // Add new secondary index entry if new value exists (including null values) + if (hasNewValue) { + addSecondaryIndexStat(writeStatus, def.getIndexName(), recordKey, newSecondaryKey, false); + } + } + }); + } + private static void addSecondaryIndexStat(WriteStatus writeStatus, String secondaryIndexPartitionPath, String recordKey, Object secKey, boolean isDeleted) { // Convert null to string representation - null values are valid in secondary indexes String secKeyStr = secKey == null ? null : secKey.toString(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java index 01cd232de66d1..fc87ac6fb5b84 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java @@ -38,6 +38,7 @@ import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.engine.ReaderContextFactory; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.fs.ConsistencyGuard; import org.apache.hudi.common.fs.ConsistencyGuard.FileVisibility; @@ -1280,4 +1281,10 @@ private Set getDropPartitionColNames() { } return new HashSet<>(Arrays.asList(partitionFields.get())); } + + public ReaderContextFactory getReaderContextFactoryForWrite() { + // question: should we just return null when context is serialized as null? the mismatch reader context would throw anyway. + return (ReaderContextFactory) getContext().getReaderContextFactoryForWrite(metaClient, config.getRecordMerger().getRecordType(), + config.getProps(), false); + } } 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 5d7e12217a802..a8efb1d13d849 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 @@ -103,7 +103,7 @@ public I combineOnCondition( */ public I deduplicateRecords(I records, HoodieTable table, int parallelism) { HoodieReaderContext readerContext = - (HoodieReaderContext) table.getContext().getReaderContextFactoryForWrite(table.getMetaClient(), table.getConfig().getRecordMerger().getRecordType(), table.getConfig().getProps()) + (HoodieReaderContext) table.getContext().getReaderContextFactoryForWrite(table.getMetaClient(), table.getConfig().getRecordMerger().getRecordType(), table.getConfig().getProps(), true) .getContext(); HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig(); readerContext.initRecordMergerForIngestion(table.getConfig().getProps()); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/TestHoodieMergeHandleFactory.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/TestHoodieMergeHandleFactory.java index ca59cde89c663..1cc3703df22fe 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/TestHoodieMergeHandleFactory.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/TestHoodieMergeHandleFactory.java @@ -61,7 +61,7 @@ public void validateWriterPathFactoryImpl() { Properties properties = new Properties(); properties.setProperty(MERGE_ALLOW_DUPLICATE_ON_INSERTS_ENABLE.key(), "false"); Pair mergeHandleClasses = HoodieMergeHandleFactory.getMergeHandleClassesWrite(WriteOperationType.UPSERT, getWriterConfig(properties), mockHoodieTable); - validateMergeClasses(mergeHandleClasses, HoodieWriteMergeHandle.class.getName()); + validateMergeClasses(mergeHandleClasses, FileGroupReaderBasedMergeHandle.class.getName()); // sorted case when(mockHoodieTable.requireSortedRecords()).thenReturn(true); @@ -80,21 +80,21 @@ public void validateWriterPathFactoryImpl() { validateMergeClasses(mergeHandleClasses, HoodieConcatHandle.class.getName()); mergeHandleClasses = HoodieMergeHandleFactory.getMergeHandleClassesWrite(WriteOperationType.UPSERT, getWriterConfig(propsWithDups), mockHoodieTable); - validateMergeClasses(mergeHandleClasses, HoodieWriteMergeHandle.class.getName()); + validateMergeClasses(mergeHandleClasses, FileGroupReaderBasedMergeHandle.class.getName()); mergeHandleClasses = HoodieMergeHandleFactory.getMergeHandleClassesWrite(WriteOperationType.INSERT, getWriterConfig(properties), mockHoodieTable); - validateMergeClasses(mergeHandleClasses, HoodieWriteMergeHandle.class.getName()); + validateMergeClasses(mergeHandleClasses, FileGroupReaderBasedMergeHandle.class.getName()); // non-sorted: CDC enabled when(mockHoodieTableConfig.isCDCEnabled()).thenReturn(true); mergeHandleClasses = HoodieMergeHandleFactory.getMergeHandleClassesWrite(WriteOperationType.UPSERT, getWriterConfig(propsWithDups), mockHoodieTable); - validateMergeClasses(mergeHandleClasses, HoodieMergeHandleWithChangeLog.class.getName()); + validateMergeClasses(mergeHandleClasses, FileGroupReaderBasedMergeHandle.class.getName()); mergeHandleClasses = HoodieMergeHandleFactory.getMergeHandleClassesWrite(WriteOperationType.INSERT, getWriterConfig(properties), mockHoodieTable); - validateMergeClasses(mergeHandleClasses, HoodieMergeHandleWithChangeLog.class.getName()); + validateMergeClasses(mergeHandleClasses, FileGroupReaderBasedMergeHandle.class.getName()); mergeHandleClasses = HoodieMergeHandleFactory.getMergeHandleClassesWrite(WriteOperationType.UPSERT, getWriterConfig(properties), mockHoodieTable); - validateMergeClasses(mergeHandleClasses, HoodieMergeHandleWithChangeLog.class.getName()); + validateMergeClasses(mergeHandleClasses, FileGroupReaderBasedMergeHandle.class.getName()); // custom merge handle when(mockHoodieTableConfig.isCDCEnabled()).thenReturn(false); @@ -102,7 +102,7 @@ public void validateWriterPathFactoryImpl() { properties.setProperty(HoodieWriteConfig.CONCAT_HANDLE_CLASS_NAME.key(), CUSTOM_MERGE_HANDLE); propsWithDups.setProperty(HoodieWriteConfig.MERGE_HANDLE_CLASS_NAME.key(), CUSTOM_MERGE_HANDLE); mergeHandleClasses = HoodieMergeHandleFactory.getMergeHandleClassesWrite(WriteOperationType.UPSERT, getWriterConfig(properties), mockHoodieTable); - validateMergeClasses(mergeHandleClasses, CUSTOM_MERGE_HANDLE, HoodieWriteMergeHandle.class.getName()); + validateMergeClasses(mergeHandleClasses, CUSTOM_MERGE_HANDLE, FileGroupReaderBasedMergeHandle.class.getName()); when(mockHoodieTable.requireSortedRecords()).thenReturn(true); mergeHandleClasses = HoodieMergeHandleFactory.getMergeHandleClassesWrite(WriteOperationType.UPSERT, getWriterConfig(properties), mockHoodieTable); @@ -115,6 +115,20 @@ public void validateWriterPathFactoryImpl() { propsWithDups.setProperty(HoodieWriteConfig.CONCAT_HANDLE_CLASS_NAME.key(), CUSTOM_MERGE_HANDLE); mergeHandleClasses = HoodieMergeHandleFactory.getMergeHandleClassesWrite(WriteOperationType.INSERT, getWriterConfig(propsWithDups), mockHoodieTable); validateMergeClasses(mergeHandleClasses, CUSTOM_MERGE_HANDLE, HoodieConcatHandle.class.getName()); + + // Filegroup reader based merge handle class + when(mockHoodieTableConfig.isCDCEnabled()).thenReturn(false); + properties.setProperty(HoodieWriteConfig.MERGE_HANDLE_CLASS_NAME.key(), FileGroupReaderBasedMergeHandle.class.getName()); + propsWithDups.setProperty(HoodieWriteConfig.MERGE_HANDLE_CLASS_NAME.key(), CUSTOM_MERGE_HANDLE); + mergeHandleClasses = HoodieMergeHandleFactory.getMergeHandleClassesWrite(WriteOperationType.UPSERT, getWriterConfig(properties), mockHoodieTable); + validateMergeClasses(mergeHandleClasses, FileGroupReaderBasedMergeHandle.class.getName(), null); + + // even if CDC is enabled, its the same FG reader based merge handle class. + when(mockHoodieTableConfig.isCDCEnabled()).thenReturn(true); + properties.setProperty(HoodieWriteConfig.MERGE_HANDLE_CLASS_NAME.key(), FileGroupReaderBasedMergeHandle.class.getName()); + propsWithDups.setProperty(HoodieWriteConfig.MERGE_HANDLE_CLASS_NAME.key(), CUSTOM_MERGE_HANDLE); + mergeHandleClasses = HoodieMergeHandleFactory.getMergeHandleClassesWrite(WriteOperationType.UPSERT, getWriterConfig(properties), mockHoodieTable); + validateMergeClasses(mergeHandleClasses, FileGroupReaderBasedMergeHandle.class.getName(), null); } @Test @@ -122,7 +136,7 @@ public void validateCompactionPathFactoryImpl() { // default case Properties properties = new Properties(); Pair mergeHandleClasses = HoodieMergeHandleFactory.getMergeHandleClassesCompaction(getWriterConfig(properties), mockHoodieTable); - validateMergeClasses(mergeHandleClasses, HoodieWriteMergeHandle.class.getName()); + validateMergeClasses(mergeHandleClasses, FileGroupReaderBasedMergeHandle.class.getName()); // sorted case when(mockHoodieTable.requireSortedRecords()).thenReturn(true); @@ -133,7 +147,7 @@ public void validateCompactionPathFactoryImpl() { when(mockHoodieTable.requireSortedRecords()).thenReturn(false); properties.setProperty(HoodieWriteConfig.MERGE_HANDLE_CLASS_NAME.key(), CUSTOM_MERGE_HANDLE); mergeHandleClasses = HoodieMergeHandleFactory.getMergeHandleClassesCompaction(getWriterConfig(properties), mockHoodieTable); - validateMergeClasses(mergeHandleClasses, CUSTOM_MERGE_HANDLE, HoodieWriteMergeHandle.class.getName()); + validateMergeClasses(mergeHandleClasses, CUSTOM_MERGE_HANDLE, FileGroupReaderBasedMergeHandle.class.getName()); when(mockHoodieTable.requireSortedRecords()).thenReturn(true); mergeHandleClasses = HoodieMergeHandleFactory.getMergeHandleClassesCompaction(getWriterConfig(properties), mockHoodieTable); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/MetadataMergeWriteStatus.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/MetadataMergeWriteStatus.java index 8ce85a7da59b8..fe1902dd9df59 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/MetadataMergeWriteStatus.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/MetadataMergeWriteStatus.java @@ -60,12 +60,12 @@ private static void mergeMetadataMaps(Map mergeFromMap, Map> dedupForCopyOnWriteStorage(Hood int dedupParallelism = records.getNumPartitions() + additionalParallelism; BaseHoodieWriteClient writeClient = getHoodieWriteClient(writeConfig); HoodieReaderContext readerContext = writeClient.getEngineContext() - .getReaderContextFactoryForWrite(metaClient, HoodieRecord.HoodieRecordType.AVRO, writeConfig.getProps()).getContext(); + .getReaderContextFactoryForWrite(metaClient, HoodieRecord.HoodieRecordType.AVRO, writeConfig.getProps(), true).getContext(); List orderingFieldNames = getOrderingFieldNames( readerContext.getMergeMode(), writeClient.getConfig().getProps(), metaClient); BufferedRecordMerger recordMerger = BufferedRecordMergerFactory.create( diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/common/HoodieFlinkEngineContext.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/common/HoodieFlinkEngineContext.java index ce7f3f2218c04..aee7a2d904cab 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/common/HoodieFlinkEngineContext.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/common/HoodieFlinkEngineContext.java @@ -222,10 +222,10 @@ public ReaderContextFactory getReaderContextFactory(HoodieTableMetaClient met if (metaClient.isMetadataTable()) { return new AvroReaderContextFactory(metaClient); } - return getDefaultContextFactory(metaClient); + return getEngineReaderContextFactory(metaClient); } - public ReaderContextFactory getDefaultContextFactory(HoodieTableMetaClient metaClient) { + public ReaderContextFactory getEngineReaderContextFactory(HoodieTableMetaClient metaClient) { return (ReaderContextFactory) ReflectionUtils.loadClass("org.apache.hudi.table.format.FlinkReaderContextFactory", metaClient); } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeAndReplaceHandle.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeAndReplaceHandle.java index 2888faf0368ca..41733aaeb7306 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeAndReplaceHandle.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeAndReplaceHandle.java @@ -185,6 +185,9 @@ public void finalizeWrite() { @Override public List close() { try { + if (isClosed()) { + return getWriteStatuses(); + } List writeStatuses = super.close(); finalizeWrite(); return writeStatuses; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeHandle.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeHandle.java index ab4862bbb975e..b8f539a3082bd 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeHandle.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeHandle.java @@ -179,6 +179,9 @@ protected void setWriteStatusPath() { @Override public List close() { try { + if (isClosed()) { + return getWriteStatuses(); + } List writeStatus = super.close(); finalizeWrite(); return writeStatus; diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/common/HoodieJavaEngineContext.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/common/HoodieJavaEngineContext.java index 5cb329359ba3e..a96caf311328f 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/common/HoodieJavaEngineContext.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/common/HoodieJavaEngineContext.java @@ -194,11 +194,11 @@ public O aggregate(HoodieData data, O zeroValue, Functions.Function2 getReaderContextFactory(HoodieTableMetaClient metaClient) { - return getDefaultContextFactory(metaClient); + return getEngineReaderContextFactory(metaClient); } @Override - public ReaderContextFactory getDefaultContextFactory(HoodieTableMetaClient metaClient) { + public ReaderContextFactory getEngineReaderContextFactory(HoodieTableMetaClient metaClient) { return new AvroReaderContextFactory(metaClient); } diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java index e2472322122a5..b810561abcf85 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java @@ -442,9 +442,10 @@ public void testInsertUpsertWithHoodieAvroPayload() throws Exception { //TODO : Find race condition that causes the timeline sometime to reflect 000.commit and sometimes not final HoodieJavaCopyOnWriteTable reloadedTable = (HoodieJavaCopyOnWriteTable) HoodieJavaTable.create(config, context, HoodieTableMetaClient.reload(metaClient)); - final List updates = dataGen.generateUpdatesWithHoodieAvroPayload(instantTime, inserts); - String partitionPath = writeStatus.getPartitionPath(); + final List updates = dataGen.generateUpdatesWithHoodieAvroPayload(instantTime, inserts) + .stream().filter(record -> record.getPartitionPath().equals(partitionPath)).collect(Collectors.toList()); + long numRecordsInPartition = updates.stream().filter(u -> u.getPartitionPath().equals(partitionPath)).count(); BaseJavaCommitActionExecutor newActionExecutor = new JavaUpsertCommitActionExecutor(context, config, reloadedTable, instantTime, updates); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/HoodieSparkEngineContext.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/HoodieSparkEngineContext.java index 0d4ef9e9c9446..09d371d63eae1 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/HoodieSparkEngineContext.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/HoodieSparkEngineContext.java @@ -262,11 +262,11 @@ public ReaderContextFactory getReaderContextFactory(HoodieTableMetaClient met if (metaClient.isMetadataTable()) { return new AvroReaderContextFactory(metaClient); } - return getDefaultContextFactory(metaClient); + return getEngineReaderContextFactory(metaClient); } @Override - public ReaderContextFactory getDefaultContextFactory(HoodieTableMetaClient metaClient) { + public ReaderContextFactory getEngineReaderContextFactory(HoodieTableMetaClient metaClient) { return new SparkReaderContextFactory(this, metaClient); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/SparkReaderContextFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/SparkReaderContextFactory.java index 42735e1470265..08f2c2bd6ed40 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/SparkReaderContextFactory.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/SparkReaderContextFactory.java @@ -61,16 +61,16 @@ /** * Factory that provides the {@link InternalRow} based {@link HoodieReaderContext} for reading data into the spark native format. */ -class SparkReaderContextFactory implements ReaderContextFactory { +public class SparkReaderContextFactory implements ReaderContextFactory { private final Broadcast baseFileReaderBroadcast; private final Broadcast configurationBroadcast; private final Broadcast tableConfigBroadcast; - SparkReaderContextFactory(HoodieSparkEngineContext hoodieSparkEngineContext, HoodieTableMetaClient metaClient) { + public SparkReaderContextFactory(HoodieSparkEngineContext hoodieSparkEngineContext, HoodieTableMetaClient metaClient) { this(hoodieSparkEngineContext, metaClient, new TableSchemaResolver(metaClient), SparkAdapterSupport$.MODULE$.sparkAdapter()); } - SparkReaderContextFactory(HoodieSparkEngineContext hoodieSparkEngineContext, HoodieTableMetaClient metaClient, + public SparkReaderContextFactory(HoodieSparkEngineContext hoodieSparkEngineContext, HoodieTableMetaClient metaClient, TableSchemaResolver resolver, SparkAdapter sparkAdapter) { SQLConf sqlConf = hoodieSparkEngineContext.getSqlContext().sessionState().conf(); JavaSparkContext jsc = hoodieSparkEngineContext.jsc(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java index 3ff31990e7fae..4047f6df44daf 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java @@ -27,6 +27,7 @@ import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.data.HoodieData.HoodieDataCacheKey; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.engine.ReaderContextFactory; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -48,6 +49,7 @@ import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieSparkIndexClient; import org.apache.hudi.io.CreateHandleFactory; +import org.apache.hudi.io.FileGroupReaderBasedMergeHandle; import org.apache.hudi.io.HoodieMergeHandle; import org.apache.hudi.io.HoodieMergeHandleFactory; import org.apache.hudi.io.IOUtils; @@ -91,6 +93,7 @@ public abstract class BaseSparkCommitActionExecutor extends private static final Logger LOG = LoggerFactory.getLogger(BaseSparkCommitActionExecutor.class); protected final Option keyGeneratorOpt; + private final ReaderContextFactory readerContextFactory; public BaseSparkCommitActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, @@ -108,6 +111,7 @@ public BaseSparkCommitActionExecutor(HoodieEngineContext context, Option> extraMetadata) { super(context, config, table, instantTime, operationType, extraMetadata); keyGeneratorOpt = HoodieSparkKeyGeneratorFactory.createBaseKeyGenerator(config); + readerContextFactory = WriteOperationType.isChangingRecords(operationType) ? table.getReaderContextFactoryForWrite() : null; } protected HoodieData> clusteringHandleUpdate(HoodieData> inputRecords) { @@ -371,7 +375,7 @@ public Iterator> handleUpdate(String partitionPath, String fil throws IOException { // This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records if (!recordItr.hasNext()) { - LOG.info("Empty partition with fileId => " + fileId); + LOG.info("Empty partition with fileId => {}", fileId); return Collections.emptyIterator(); } @@ -388,7 +392,7 @@ public Iterator> handleUpdate(String partitionPath, String fil protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId, Iterator> recordItr) { HoodieMergeHandle mergeHandle = HoodieMergeHandleFactory.create(operationType, config, instantTime, table, recordItr, partitionPath, fileId, - taskContextSupplier, keyGeneratorOpt); + taskContextSupplier, keyGeneratorOpt); if (mergeHandle.getOldFilePath() != null && mergeHandle.baseFileForMerge().getBootstrapBaseFile().isPresent()) { Option partitionFields = table.getMetaClient().getTableConfig().getPartitionFields(); Object[] partitionValues = SparkPartitionUtils.getPartitionFieldVals(partitionFields, mergeHandle.getPartitionPath(), @@ -397,6 +401,9 @@ protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId, mergeHandle.setPartitionFields(partitionFields); mergeHandle.setPartitionValues(partitionValues); } + if (readerContextFactory != null && mergeHandle instanceof FileGroupReaderBasedMergeHandle) { + ((FileGroupReaderBasedMergeHandle) mergeHandle).setReaderContext(readerContextFactory.getContext()); + } return mergeHandle; } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/BaseTestHandle.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/BaseTestHandle.java index 67a7c2e215712..8492988a7ca05 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/BaseTestHandle.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/BaseTestHandle.java @@ -82,7 +82,12 @@ protected int generateDeleteRecords(List existingRecords, HoodieTe } } } - existingRecords.addAll(deletes); + // Add 5 deletes with ignoreIndexUpdate set to true to validate that this is propagated to the record delegates + deletes.stream().limit(5).forEach(hoodieRecord -> { + hoodieRecord.setIgnoreIndexUpdate(true); + existingRecords.add(hoodieRecord); + }); + existingRecords.addAll(deletes.subList(5, deletes.size())); return deletes.size(); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieDefaultMergeHandle.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieDefaultMergeHandle.java index 2fab2abf24ecf..54e5017fd25d0 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieDefaultMergeHandle.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieDefaultMergeHandle.java @@ -96,6 +96,7 @@ public void testUpsertsForMultipleRecordsInSameFile(ExternalSpillableMap.DiskMap Properties properties = new Properties(); properties.setProperty(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.key(), diskMapType.name()); properties.setProperty(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.key(), String.valueOf(isCompressionEnabled)); + properties.setProperty(HoodieWriteConfig.MERGE_HANDLE_CLASS_NAME.key(), HoodieWriteMergeHandle.class.getName()); // Build a write config with bulkinsertparallelism set HoodieWriteConfig cfg = getConfigBuilder() diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java index 9e003fc92d8be..7423d9f6d0fc2 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java @@ -86,6 +86,7 @@ import java.util.Map; import java.util.Properties; import java.util.UUID; +import java.util.stream.Collectors; import java.util.stream.Stream; import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION; @@ -484,10 +485,11 @@ public void testInsertUpsertWithHoodieAvroPayload() throws Exception { String fileId = writeStatus.getFileId(); metaClient.getStorage().create( new StoragePath(Paths.get(basePath, ".hoodie/timeline", "000.commit").toString())).close(); + String partitionPath = writeStatus.getPartitionPath(); final List updates = - dataGen.generateUpdatesWithHoodieAvroPayload(instantTime, inserts); + dataGen.generateUpdatesWithHoodieAvroPayload(instantTime, inserts) + .stream().filter(record -> record.getPartitionPath().equals(partitionPath)).collect(Collectors.toList()); - String partitionPath = writeStatus.getPartitionPath(); long numRecordsInPartition = updates.stream().filter(u -> u.getPartitionPath().equals(partitionPath)).count(); table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java index ab5458c67309c..552864cf92c65 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java @@ -176,12 +176,16 @@ public HoodieTableMetaClient getHoodieMetaClient(HoodieTableType tableType, Prop return getHoodieMetaClient(storageConf(), basePath(), tableType, props); } - public HoodieTableMetaClient getHoodieMetaClient(StorageConfiguration storageConf, String basePath, HoodieTableType tableType, Properties props) throws IOException { - return HoodieTableMetaClient.newTableBuilder() - .setTableName(RAW_TRIPS_TEST_NAME) - .setTableType(tableType) - .fromProperties(props) - .initTable(storageConf.newInstance(), basePath); + public HoodieTableMetaClient getHoodieMetaClient(StorageConfiguration storageConf, String basePath, HoodieTableType tableType, Properties props) { + try { + return HoodieTableMetaClient.newTableBuilder() + .setTableName(RAW_TRIPS_TEST_NAME) + .setTableType(tableType) + .fromProperties(props) + .initTable(storageConf.newInstance(), basePath); + } catch (IOException e) { + throw new HoodieIOException("Failed to create HoodieTableMetaClient for basePath: " + basePath, e); + } } public HoodieTableMetaClient getHoodieMetaClient(StorageConfiguration storageConf, String basePath) throws IOException { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieEngineContext.java b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieEngineContext.java index 5199c2e93e686..d0a4f7bcaffe1 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieEngineContext.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieEngineContext.java @@ -139,25 +139,26 @@ public abstract List reduceByKey( /** * Returns reader context factory for write operations in the table. * - * @param metaClient Table meta client - * @param recordType Record type - * @param properties Typed properties + * @param metaClient Table meta client + * @param recordType Record type + * @param properties Typed properties + * @param outputsCustomPayloads Whether the reader context factory should output custom payloads. Final merging of records before writes does not require custom payloads. */ public ReaderContextFactory getReaderContextFactoryForWrite(HoodieTableMetaClient metaClient, HoodieRecord.HoodieRecordType recordType, - TypedProperties properties) { + TypedProperties properties, boolean outputsCustomPayloads) { if (recordType == HoodieRecord.HoodieRecordType.AVRO) { String payloadClass = ConfigUtils.getPayloadClass(properties); - return new AvroReaderContextFactory(metaClient, payloadClass, true); + return new AvroReaderContextFactory(metaClient, payloadClass, outputsCustomPayloads); } - return getDefaultContextFactory(metaClient); + return getEngineReaderContextFactory(metaClient); } /** - * Returns default reader context factory for the engine. + * Returns reader context factory specific for the engine. * - * @param metaClient Table metadata client + * @param metaClient Table metadata client */ - public abstract ReaderContextFactory getDefaultContextFactory(HoodieTableMetaClient metaClient); + public abstract ReaderContextFactory getEngineReaderContextFactory(HoodieTableMetaClient metaClient); /** * Groups values by key and applies a processing function to each group. diff --git a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieLocalEngineContext.java b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieLocalEngineContext.java index 8425e14b333cf..469225cf3c87b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieLocalEngineContext.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieLocalEngineContext.java @@ -190,11 +190,11 @@ public O aggregate(HoodieData data, O zeroValue, Functions.Function2 getReaderContextFactory(HoodieTableMetaClient metaClient) { - return getDefaultContextFactory(metaClient); + return getEngineReaderContextFactory(metaClient); } @Override - public ReaderContextFactory getDefaultContextFactory(HoodieTableMetaClient metaClient) { + public ReaderContextFactory getEngineReaderContextFactory(HoodieTableMetaClient metaClient) { return new AvroReaderContextFactory(metaClient); } 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 5c57275496ac8..0b065004ca523 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 @@ -55,6 +55,7 @@ import static org.apache.hudi.common.config.HoodieReaderConfig.RECORD_MERGE_IMPL_CLASSES_DEPRECATED_WRITE_CONFIG_KEY; import static org.apache.hudi.common.config.HoodieReaderConfig.RECORD_MERGE_IMPL_CLASSES_WRITE_CONFIG_KEY; +import static org.apache.hudi.common.model.HoodieRecordMerger.PAYLOAD_BASED_MERGE_STRATEGY_UUID; import static org.apache.hudi.common.table.HoodieTableConfig.inferMergingConfigsForPreV9Table; /** @@ -265,13 +266,17 @@ public void initRecordMergerForIngestion(TypedProperties properties) { * @param isIngestion indicates if the context is used in ingestion path. */ private void initRecordMerger(TypedProperties properties, boolean isIngestion) { - Option providedPayloadClass = HoodieRecordPayload.getPayloadClassNameIfPresent(properties); + if (recordMerger != null && mergeMode != null) { + // already initialized + return; + } + Option writerPayloadClass = HoodieRecordPayload.getWriterPayloadOverride(properties); RecordMergeMode recordMergeMode = tableConfig.getRecordMergeMode(); String mergeStrategyId = tableConfig.getRecordMergeStrategyId(); HoodieTableVersion tableVersion = tableConfig.getTableVersion(); // If the provided payload class differs from the table's payload class, we need to infer the correct merging behavior. - if (isIngestion && providedPayloadClass.map(className -> !className.equals(tableConfig.getPayloadClass())).orElse(false)) { - Triple triple = HoodieTableConfig.inferMergingConfigsForWrites(null, providedPayloadClass.get(), null, + if (isIngestion && writerPayloadClass.map(className -> !className.equals(tableConfig.getPayloadClass())).orElse(false)) { + Triple triple = HoodieTableConfig.inferMergingConfigsForWrites(null, writerPayloadClass.get(), null, tableConfig.getPreCombineFieldsStr().orElse(null), tableVersion); recordMergeMode = triple.getLeft(); mergeStrategyId = triple.getRight(); @@ -335,4 +340,14 @@ public abstract ClosableIterator mergeBootstrapReaders(ClosableIterator sk ClosableIterator dataFileIterator, Schema dataRequiredSchema, List> requiredPartitionFieldAndValues); + + public Option> getPayloadClasses(TypedProperties props) { + return getRecordMerger().map(merger -> { + if (merger.getMergingStrategy().equals(PAYLOAD_BASED_MERGE_STRATEGY_UUID)) { + String incomingPayloadClass = HoodieRecordPayload.getWriterPayloadOverride(props).orElseGet(tableConfig::getPayloadClass); + return Pair.of(tableConfig.getPayloadClass(), incomingPayloadClass); + } + return null; + }); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/engine/RecordContext.java b/hudi-common/src/main/java/org/apache/hudi/common/engine/RecordContext.java index 9c772971f1192..51dd59992d49e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/engine/RecordContext.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/engine/RecordContext.java @@ -20,6 +20,7 @@ package org.apache.hudi.common.engine; import org.apache.hudi.common.function.SerializableBiFunction; +import org.apache.hudi.common.model.DeleteRecord; import org.apache.hudi.common.model.HoodieOperation; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.HoodieTableConfig; @@ -48,6 +49,7 @@ import static org.apache.hudi.common.model.HoodieRecord.HOODIE_IS_DELETED_FIELD; import static org.apache.hudi.common.model.HoodieRecord.RECORD_KEY_METADATA_FIELD; +import static org.apache.hudi.common.util.OrderingValues.isCommitTimeOrderingValue; /** * Record context provides the APIs for record related operations. Record context is associated with @@ -203,7 +205,7 @@ public Comparable convertValueToEngineType(Comparable value) { */ public final Comparable convertOrderingValueToEngineType(Comparable value) { return value instanceof ArrayComparable - ? ((ArrayComparable) value).apply(comparable -> convertValueToEngineType(comparable)) + ? ((ArrayComparable) value).apply(this::convertValueToEngineType) : convertValueToEngineType(value); } @@ -332,6 +334,20 @@ public Comparable getOrderingValue(T record, }); } + /** + * Gets the ordering value from given delete record. + * + * @param deleteRecord The delete record + * + * @return The ordering value. + */ + public Comparable getOrderingValue(DeleteRecord deleteRecord) { + Comparable orderingValue = deleteRecord.getOrderingValue(); + return isCommitTimeOrderingValue(orderingValue) + ? OrderingValues.getDefault() + : convertOrderingValueToEngineType(orderingValue); + } + /** * Gets the ordering value in particular type. * diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordPayload.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordPayload.java index bd11d3a0d3fb3..28c766164548d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordPayload.java @@ -217,4 +217,13 @@ static Option getPayloadClassNameIfPresent(Properties props) { // Need to transparently change to org.apache.hudi. return Option.ofNullable(payloadClassName).map(className -> className.replace("com.uber.hoodie", "org.apache.hudi")); } + + /** + * Returns the writer payload override class if present in the properties. + * @param properties the properties to inspect + * @return an Option containing the writer payload override class name if present, otherwise an empty Option + */ + static Option getWriterPayloadOverride(Properties properties) { + return Option.ofNullable(properties.getProperty("hoodie.datasource.write.payload.class")).map(className -> className.replace("com.uber.hoodie", "org.apache.hudi")); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/BaseFileUpdateCallback.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/BaseFileUpdateCallback.java index 6685cd6cef459..a4e4ca631e49a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/BaseFileUpdateCallback.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/BaseFileUpdateCallback.java @@ -19,6 +19,8 @@ package org.apache.hudi.common.table.read; +import org.apache.hudi.common.model.HoodieOperation; + /** * Callback interface for handling updates to the base file of the file group. */ @@ -29,19 +31,20 @@ public interface BaseFileUpdateCallback { * @param previousRecord the record in the base file before the update * @param mergedRecord the result of merging the previous and new records */ - void onUpdate(String recordKey, T previousRecord, T mergedRecord); + void onUpdate(String recordKey, BufferedRecord previousRecord, BufferedRecord mergedRecord); /** * Callback method to handle insertion of a new record into the base file. * @param recordKey the key of the record being inserted * @param newRecord the new record being added to the base file */ - void onInsert(String recordKey, T newRecord); + void onInsert(String recordKey, BufferedRecord newRecord); /** * Callback method to handle deletion of a record from the base file. * @param recordKey the key of the record being deleted * @param previousRecord the record in the base file before deletion + * @param hoodieOperation the operation type of the incoming record, used to infer type of delete operation */ - void onDelete(String recordKey, T previousRecord); + void onDelete(String recordKey, BufferedRecord previousRecord, HoodieOperation hoodieOperation); } 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 96b67c9aa83dd..8a12a13ec9cc9 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 @@ -413,11 +413,13 @@ public BufferedRecord mergeNonDeleteRecord(BufferedRecord olderRecord, Buf */ private static class ExpressionPayloadRecordMerger extends CustomPayloadRecordMerger { private final String basePayloadClass; + private final HoodieRecordMerger deltaMerger; public ExpressionPayloadRecordMerger(RecordContext recordContext, Option recordMerger, List orderingFieldNames, String basePayloadClass, String incomingPayloadClass, Schema readerSchema, TypedProperties props) { super(recordContext, recordMerger, orderingFieldNames, incomingPayloadClass, readerSchema, props); this.basePayloadClass = basePayloadClass; + this.deltaMerger = HoodieRecordUtils.mergerToPreCombineMode(recordMerger.get()); } @Override @@ -426,6 +428,16 @@ protected Pair getFinalMergeRecords(BufferedRecord> getMergedRecord(BufferedRecord olderRecord, BufferedRecord newerRecord, boolean isFinalMerge) throws IOException { + if (isFinalMerge) { + return super.getMergedRecord(olderRecord, newerRecord, isFinalMerge); + } else { + Pair records = getDeltaMergeRecords(olderRecord, newerRecord); + return deltaMerger.merge(records.getLeft(), getSchemaForAvroPayloadMerge(olderRecord), records.getRight(), getSchemaForAvroPayloadMerge(newerRecord), props); + } + } } /** @@ -481,7 +493,7 @@ public BufferedRecord mergeNonDeleteRecord(BufferedRecord olderRecord, Buf Schema mergeResultSchema = mergedRecordAndSchema.get().getRight(); // Special handling for SENTINEL record in Expression Payload if (mergedRecord.getData() == HoodieRecord.SENTINEL) { - return BufferedRecords.SENTINEL; + return olderRecord; } if (!mergedRecord.isDelete(mergeResultSchema, props)) { IndexedRecord indexedRecord; @@ -506,7 +518,7 @@ protected Pair getFinalMergeRecords(BufferedRecord> getMergedRecord(BufferedRecord olderRecord, BufferedRecord newerRecord, boolean isFinalMerge) throws IOException { + protected Option> getMergedRecord(BufferedRecord olderRecord, BufferedRecord newerRecord, boolean isFinalMerge) throws IOException { Pair records = isFinalMerge ? getFinalMergeRecords(olderRecord, newerRecord) : getDeltaMergeRecords(olderRecord, newerRecord); return recordMerger.merge(records.getLeft(), getSchemaForAvroPayloadMerge(olderRecord), records.getRight(), getSchemaForAvroPayloadMerge(newerRecord), props); } @@ -522,7 +534,7 @@ record = recordContext.convertToAvroRecord(bufferedRecord.getRecord(), recordSch HoodieRecordUtils.loadPayload(payloadClass, record, bufferedRecord.getOrderingValue()), null); } - private Schema getSchemaForAvroPayloadMerge(BufferedRecord bufferedRecord) { + protected Schema getSchemaForAvroPayloadMerge(BufferedRecord bufferedRecord) { if (bufferedRecord.getSchemaId() == null) { return readerSchema; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/BufferedRecords.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/BufferedRecords.java index 0349d5bc518ee..1fd1757333d0b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/BufferedRecords.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/BufferedRecords.java @@ -33,8 +33,6 @@ * Factory to create {@link BufferedRecord}. */ public class BufferedRecords { - // Special handling for SENTINEL record in Expression Payload - public static final BufferedRecord SENTINEL = new BufferedRecord(null, null, null, null, null); public static BufferedRecord fromHoodieRecord(HoodieRecord record, Schema schema, RecordContext recordContext, Properties props, String[] orderingFields) { boolean isDelete = record.isDelete(schema, props); @@ -50,12 +48,12 @@ public static BufferedRecord fromHoodieRecord(HoodieRecord record, Schema } public static BufferedRecord fromEngineRecord(T record, Schema schema, RecordContext recordContext, List orderingFieldNames, boolean isDelete) { - return fromEngineRecord(record, schema, recordContext, orderingFieldNames, isDelete ? HoodieOperation.DELETE : null); + String recordKey = recordContext.getRecordKey(record, schema); + return fromEngineRecord(record, recordKey, schema, recordContext, orderingFieldNames, isDelete ? HoodieOperation.DELETE : null); } public static BufferedRecord fromEngineRecord( - T record, Schema schema, RecordContext recordContext, List orderingFieldNames, HoodieOperation hoodieOperation) { - String recordKey = recordContext.getRecordKey(record, schema); + T record, String recordKey, Schema schema, RecordContext recordContext, List orderingFieldNames, HoodieOperation hoodieOperation) { Integer schemaId = recordContext.encodeAvroSchema(schema); Comparable orderingValue = recordContext.getOrderingValue(record, schema, orderingFieldNames); return new BufferedRecord<>(recordKey, orderingValue, record, schemaId, hoodieOperation); @@ -67,13 +65,13 @@ public static BufferedRecord fromEngineRecord(T record, Schema schema, Re return new BufferedRecord<>(recordKey, orderingValue, record, schemaId, isDelete ? HoodieOperation.DELETE : null); } - public static BufferedRecord fromDeleteRecord(DeleteRecord deleteRecord, Comparable orderingValue) { - return new BufferedRecord<>(deleteRecord.getRecordKey(), orderingValue, null, null, HoodieOperation.DELETE); + public static BufferedRecord fromDeleteRecord(DeleteRecord deleteRecord, RecordContext recordContext) { + return new BufferedRecord<>(deleteRecord.getRecordKey(), recordContext.getOrderingValue(deleteRecord), null, null, HoodieOperation.DELETE); } - public static BufferedRecord fromDeleteRecord(DeleteRecord deleteRecord, Comparable orderingValue, HoodieOperation hoodieOperation) { + public static BufferedRecord fromDeleteRecord(DeleteRecord deleteRecord, RecordContext recordContext, HoodieOperation hoodieOperation) { hoodieOperation = HoodieOperation.isUpdateBefore(hoodieOperation) ? HoodieOperation.UPDATE_BEFORE : HoodieOperation.DELETE; - return new BufferedRecord<>(deleteRecord.getRecordKey(), orderingValue, null, null, hoodieOperation); + return new BufferedRecord<>(deleteRecord.getRecordKey(), recordContext.getOrderingValue(deleteRecord), null, null, hoodieOperation); } public static BufferedRecord createDelete(String recordKey) { 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 1a7294854f188..ab81e21d678cb 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 @@ -100,7 +100,7 @@ private HoodieFileGroupReader(HoodieReaderContext readerContext, HoodieStorag this.storage = storage; this.readerParameters = readerParameters; this.inputSplit = inputSplit; - readerContext.setHasLogFiles(!this.inputSplit.getLogFiles().isEmpty()); + readerContext.setHasLogFiles(this.inputSplit.hasLogFiles()); readerContext.getRecordContext().setPartitionPath(inputSplit.getPartitionPath()); if (readerContext.getHasLogFiles() && inputSplit.getStart() != 0) { throw new IllegalArgumentException("Filegroup reader is doing log file merge but not reading from the start of the base file"); @@ -404,8 +404,8 @@ public Builder withLogFiles(Stream logFiles) { return this; } - public Builder withRecordIterator(Iterator recordIterator) { - this.recordIterator = recordIterator; + public Builder withRecordIterator(Iterator recordIterator) { + this.recordIterator = (Iterator) recordIterator; this.recordBufferLoader = FileGroupRecordBufferLoader.createStreamingRecordsBufferLoader(); return this; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/InputSplit.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/InputSplit.java index c6029129748a0..6d7b578e15b3e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/InputSplit.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/InputSplit.java @@ -75,6 +75,10 @@ public List getLogFiles() { return logFiles; } + public boolean hasLogFiles() { + return !logFiles.isEmpty(); + } + public String getPartitionPath() { return partitionPath; } 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 60a4151a11e08..85e550eb66b59 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 @@ -18,9 +18,22 @@ package org.apache.hudi.common.table.read; +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.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieOperation; +import org.apache.hudi.common.util.HoodieRecordUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +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; +import java.util.Properties; /** * Interface used within the {@link HoodieFileGroupReader} for processing updates to records in Merge-on-Read tables. @@ -39,8 +52,16 @@ public interface UpdateProcessor { BufferedRecord processUpdate(String recordKey, BufferedRecord previousRecord, BufferedRecord mergedRecord, boolean isDelete); static UpdateProcessor create(HoodieReadStats readStats, HoodieReaderContext readerContext, - boolean emitDeletes, Option> updateCallback) { - UpdateProcessor handler = new StandardUpdateProcessor<>(readStats, readerContext, emitDeletes); + boolean emitDeletes, Option> updateCallback, + TypedProperties properties) { + UpdateProcessor handler; + Option payloadClass = readerContext.getPayloadClasses(properties).map(Pair::getRight); + boolean hasNonMetadataPayload = payloadClass.map(className -> !className.equals(HoodieMetadataPayload.class.getName())).orElse(false); + if (readerContext.getMergeMode() == RecordMergeMode.CUSTOM && hasNonMetadataPayload) { + handler = new PayloadUpdateProcessor<>(readStats, readerContext, emitDeletes, properties, payloadClass.get()); + } else { + handler = new StandardUpdateProcessor<>(readStats, readerContext, emitDeletes); + } if (updateCallback.isPresent()) { return new CallbackProcessor<>(updateCallback.get(), handler); } @@ -80,17 +101,56 @@ public BufferedRecord processUpdate(String recordKey, BufferedRecord previ } return null; } else { - T prevRow = previousRecord != null ? previousRecord.getRecord() : null; - T mergedRow = mergedRecord.getRecord(); - if (prevRow != null && prevRow != mergedRow) { - mergedRecord.setHoodieOperation(HoodieOperation.UPDATE_AFTER); - readStats.incrementNumUpdates(); - } else if (prevRow == null) { - mergedRecord.setHoodieOperation(HoodieOperation.INSERT); - readStats.incrementNumInserts(); + return handleNonDeletes(previousRecord, mergedRecord); + } + } + + protected BufferedRecord handleNonDeletes(BufferedRecord previousRecord, BufferedRecord mergedRecord) { + T prevRow = previousRecord != null ? previousRecord.getRecord() : null; + T mergedRow = mergedRecord.getRecord(); + if (prevRow != null && prevRow != mergedRow) { + mergedRecord.setHoodieOperation(HoodieOperation.UPDATE_AFTER); + readStats.incrementNumUpdates(); + } else if (prevRow == null) { + mergedRecord.setHoodieOperation(HoodieOperation.INSERT); + readStats.incrementNumInserts(); + } + return mergedRecord.seal(readerContext.getRecordContext()); + } + } + + class PayloadUpdateProcessor extends StandardUpdateProcessor { + private final String payloadClass; + private final Properties properties; + + public PayloadUpdateProcessor(HoodieReadStats readStats, HoodieReaderContext readerContext, boolean emitDeletes, + Properties properties, String payloadClass) { + super(readStats, readerContext, emitDeletes); + this.payloadClass = payloadClass; + this.properties = properties; + } + + @Override + protected BufferedRecord handleNonDeletes(BufferedRecord previousRecord, BufferedRecord mergedRecord) { + if (previousRecord == null) { + // special case for payloads when there is no previous record + Schema recordSchema = readerContext.getRecordContext().decodeAvroSchema(mergedRecord.getSchemaId()); + GenericRecord record = readerContext.getRecordContext().convertToAvroRecord(mergedRecord.getRecord(), recordSchema); + HoodieAvroRecord hoodieRecord = new HoodieAvroRecord<>(null, HoodieRecordUtils.loadPayload(payloadClass, record, mergedRecord.getOrderingValue())); + try { + if (hoodieRecord.shouldIgnore(recordSchema, properties)) { + return null; + } else { + Schema 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, properties, readerSchema).toIndexedRecord(readerSchema, properties) + .ifPresent(rewrittenRecord -> mergedRecord.replaceRecord(readerContext.getRecordContext().convertAvroRecord(rewrittenRecord.getData()))); + } + } catch (IOException e) { + throw new HoodieIOException("Error processing record with payload class: " + payloadClass, e); } - return mergedRecord.seal(readerContext.getRecordContext()); } + return super.handleNonDeletes(previousRecord, mergedRecord); } } @@ -102,7 +162,7 @@ class CallbackProcessor implements UpdateProcessor { private final BaseFileUpdateCallback callback; private final UpdateProcessor delegate; - public CallbackProcessor(BaseFileUpdateCallback callback, UpdateProcessor delegate) { + public CallbackProcessor(BaseFileUpdateCallback callback, UpdateProcessor delegate) { this.callback = callback; this.delegate = delegate; } @@ -112,11 +172,11 @@ public BufferedRecord processUpdate(String recordKey, BufferedRecord previ BufferedRecord result = delegate.processUpdate(recordKey, previousRecord, mergedRecord, isDelete); if (isDelete) { - callback.onDelete(recordKey, previousRecord == null ? null : previousRecord.getRecord()); - } else if (HoodieOperation.isUpdateAfter(result.getHoodieOperation())) { - callback.onUpdate(recordKey, previousRecord.getRecord(), mergedRecord.getRecord()); - } else if (HoodieOperation.isInsert(result.getHoodieOperation())) { - callback.onInsert(recordKey, mergedRecord.getRecord()); + callback.onDelete(recordKey, previousRecord, mergedRecord.getHoodieOperation()); + } else if (result != null && HoodieOperation.isUpdateAfter(result.getHoodieOperation())) { + callback.onUpdate(recordKey, previousRecord, mergedRecord); + } else if (result != null && HoodieOperation.isInsert(result.getHoodieOperation())) { + callback.onInsert(recordKey, mergedRecord); } return result; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/buffer/DefaultFileGroupRecordBufferLoader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/buffer/DefaultFileGroupRecordBufferLoader.java index 66a5d751529bf..e3fb3bfc44675 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/buffer/DefaultFileGroupRecordBufferLoader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/buffer/DefaultFileGroupRecordBufferLoader.java @@ -62,7 +62,7 @@ public Pair, List> getRecordBuffer(Hoodie Option> fileGroupUpdateCallback) { boolean isSkipMerge = ConfigUtils.getStringWithAltKeys(props, HoodieReaderConfig.MERGE_TYPE, true).equalsIgnoreCase(HoodieReaderConfig.REALTIME_SKIP_MERGE); PartialUpdateMode partialUpdateMode = hoodieTableMetaClient.getTableConfig().getPartialUpdateMode(); - UpdateProcessor updateProcessor = UpdateProcessor.create(readStats, readerContext, readerParameters.emitDeletes(), fileGroupUpdateCallback); + UpdateProcessor updateProcessor = UpdateProcessor.create(readStats, readerContext, readerParameters.emitDeletes(), fileGroupUpdateCallback, props); FileGroupRecordBuffer recordBuffer; if (isSkipMerge) { recordBuffer = new UnmergedFileGroupRecordBuffer<>( 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 27c7be2d68313..7052ba8c59c0d 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 @@ -23,8 +23,6 @@ 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.model.DeleteRecord; -import org.apache.hudi.common.model.HoodieOperation; import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.PartialUpdateMode; @@ -42,7 +40,6 @@ import org.apache.hudi.common.util.FileIOUtils; import org.apache.hudi.common.util.InternalSchemaCache; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.OrderingValues; import org.apache.hudi.common.util.collection.ClosableIterator; import org.apache.hudi.common.util.collection.CloseableMappingIterator; import org.apache.hudi.common.util.collection.ExternalSpillableMap; @@ -66,7 +63,6 @@ import static org.apache.hudi.common.config.HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE; import static org.apache.hudi.common.config.HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE; import static org.apache.hudi.common.config.HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH; -import static org.apache.hudi.common.model.HoodieRecordMerger.PAYLOAD_BASED_MERGE_STRATEGY_UUID; import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType.INSTANT_TIME; abstract class FileGroupRecordBuffer implements HoodieFileGroupRecordBuffer { @@ -76,7 +72,10 @@ abstract class FileGroupRecordBuffer implements HoodieFileGroupRecordBuffer recordMerger; - protected final Option payloadClass; + // The pair of payload classes represents the payload class for the table and the payload class for the incoming records. + // The two classes are only expected to be different when there is a merge-into operation that leverages the ExpressionPayload. + // The option will be empty if the payload merger is not used for this table. + protected final Option> payloadClasses; protected final TypedProperties props; protected final ExternalSpillableMap> records; protected final DeleteContext deleteContext; @@ -104,11 +103,7 @@ protected FileGroupRecordBuffer(HoodieReaderContext readerContext, this.recordMergeMode = recordMergeMode; this.partialUpdateMode = partialUpdateMode; this.recordMerger = readerContext.getRecordMerger(); - if (recordMerger.isPresent() && recordMerger.get().getMergingStrategy().equals(PAYLOAD_BASED_MERGE_STRATEGY_UUID)) { - this.payloadClass = Option.of(hoodieTableMetaClient.getTableConfig().getPayloadClass()); - } else { - this.payloadClass = Option.empty(); - } + this.payloadClasses = readerContext.getPayloadClasses(props); this.orderingFieldNames = orderingFieldNames; // Ensure that ordering field is populated for mergers and legacy payloads this.props = ConfigUtils.supplementOrderingFields(props, orderingFieldNames); @@ -122,7 +117,7 @@ protected FileGroupRecordBuffer(HoodieReaderContext readerContext, throw new HoodieIOException("IOException when creating ExternalSpillableMap at " + spillableMapBasePath, e); } this.bufferedRecordMerger = BufferedRecordMergerFactory.create( - readerContext, recordMergeMode, enablePartialMerging, recordMerger, orderingFieldNames, payloadClass, readerSchema, props, partialUpdateMode); + readerContext, recordMergeMode, enablePartialMerging, recordMerger, orderingFieldNames, readerSchema, payloadClasses, props, partialUpdateMode); this.deleteContext = readerContext.getSchemaHandler().getDeleteContext().withReaderSchema(this.readerSchema); this.bufferedRecordConverter = BufferedRecordConverter.createConverter(readerContext.getIteratorMode(), readerSchema, readerContext.getRecordContext(), orderingFieldNames); } @@ -247,7 +242,6 @@ protected boolean hasNextBaseRecord(T baseRecord, BufferedRecord logRecordInf // Inserts nextRecord = bufferedRecordConverter.convert(readerContext.getRecordContext().seal(baseRecord)); - nextRecord.setHoodieOperation(HoodieOperation.INSERT); return true; } @@ -285,18 +279,6 @@ protected Pair, Schema> getSchemaTransformerWithEvolvedSchema(Hoo return Pair.of(transformer, evolvedSchema); } - static boolean isCommitTimeOrderingValue(Comparable orderingValue) { - return orderingValue == null || OrderingValues.isDefault(orderingValue); - } - - static Comparable getOrderingValue(HoodieReaderContext readerContext, - DeleteRecord deleteRecord) { - Comparable orderingValue = deleteRecord.getOrderingValue(); - return isCommitTimeOrderingValue(orderingValue) - ? OrderingValues.getDefault() - : readerContext.getRecordContext().convertOrderingValueToEngineType(orderingValue); - } - private static class LogRecordIterator implements ClosableIterator> { private final FileGroupRecordBuffer fileGroupRecordBuffer; private final Iterator> logRecordIterator; 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 f66629aaf5193..5f2d35f1186a8 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 @@ -84,8 +84,8 @@ public void processDataBlock(HoodieDataBlock dataBlock, Option keySpecO true, recordMerger, orderingFieldNames, - payloadClass, readerSchema, + payloadClasses, props, partialUpdateMode); } @@ -125,10 +125,8 @@ public void processNextDeletedRecord(DeleteRecord deleteRecord, Serializable rec BufferedRecord existingRecord = records.get(recordIdentifier); totalLogRecords++; Option recordOpt = bufferedRecordMerger.deltaMerge(deleteRecord, existingRecord); - recordOpt.ifPresent(deleteRec -> { - Comparable orderingValue = getOrderingValue(readerContext, deleteRec); - records.put(recordIdentifier, BufferedRecords.fromDeleteRecord(deleteRec, orderingValue)); - }); + recordOpt.ifPresent(deleteRec -> + records.put(recordIdentifier, BufferedRecords.fromDeleteRecord(deleteRec, readerContext.getRecordContext()))); } @Override 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 433c52034c2d0..51a48e34e8a4b 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 @@ -121,8 +121,8 @@ public void processDataBlock(HoodieDataBlock dataBlock, Option keySpecO true, recordMerger, orderingFieldNames, - payloadClass, readerSchema, + payloadClasses, props, partialUpdateMode); } @@ -203,7 +203,7 @@ public void processDeleteBlock(HoodieDeleteBlock deleteBlock) throws IOException // this delete-vector could be kept in the records cache(see the check in #fallbackToKeyBasedBuffer), // and these keys would be deleted no matter whether there are following-up inserts/updates. DeleteRecord deleteRecord = deleteRecords[commitTimeBasedRecordIndex++]; - BufferedRecord record = BufferedRecords.fromDeleteRecord(deleteRecord, deleteRecord.getOrderingValue()); + BufferedRecord record = BufferedRecords.fromDeleteRecord(deleteRecord, readerContext.getRecordContext()); records.put(recordPosition, record); } return; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/buffer/ReusableFileGroupRecordBufferLoader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/buffer/ReusableFileGroupRecordBufferLoader.java index 44ff95cf6e845..deedd4808b0e6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/buffer/ReusableFileGroupRecordBufferLoader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/buffer/ReusableFileGroupRecordBufferLoader.java @@ -58,7 +58,7 @@ public synchronized Pair, List> getRecord ReaderParameters readerParameters, HoodieReadStats readStats, Option> fileGroupUpdateCallback) { - UpdateProcessor updateProcessor = UpdateProcessor.create(readStats, readerContext, readerParameters.emitDeletes(), fileGroupUpdateCallback); + UpdateProcessor updateProcessor = UpdateProcessor.create(readStats, readerContext, readerParameters.emitDeletes(), fileGroupUpdateCallback, props); PartialUpdateMode partialUpdateMode = hoodieTableMetaClient.getTableConfig().getPartialUpdateMode(); if (cachedResults == null) { // Create an initial buffer to process the log files 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 994b954fc4592..ebe09133ab5cc 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 @@ -18,17 +18,20 @@ package org.apache.hudi.common.table.read.buffer; +import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.HoodieReaderContext; import org.apache.hudi.common.engine.RecordContext; import org.apache.hudi.common.model.DeleteRecord; import org.apache.hudi.common.model.HoodieOperation; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.PartialUpdateMode; import org.apache.hudi.common.table.read.BaseFileUpdateCallback; import org.apache.hudi.common.table.read.BufferedRecord; import org.apache.hudi.common.table.read.BufferedRecords; +import org.apache.hudi.common.table.read.DeleteContext; import org.apache.hudi.common.table.read.HoodieReadStats; import org.apache.hudi.common.table.read.InputSplit; import org.apache.hudi.common.table.read.ReaderParameters; @@ -62,8 +65,10 @@ public Pair, List> getRecordBuffer(Hoodie List orderingFieldNames, HoodieTableMetaClient hoodieTableMetaClient, TypedProperties props, ReaderParameters readerParameters, HoodieReadStats readStats, Option> fileGroupUpdateCallback) { - PartialUpdateMode partialUpdateMode = hoodieTableMetaClient.getTableConfig().getPartialUpdateMode(); - UpdateProcessor updateProcessor = UpdateProcessor.create(readStats, readerContext, readerParameters.emitDeletes(), fileGroupUpdateCallback); + Schema recordSchema = HoodieAvroUtils.removeMetadataFields(readerContext.getSchemaHandler().getRequestedSchema()); + HoodieTableConfig tableConfig = hoodieTableMetaClient.getTableConfig(); + PartialUpdateMode partialUpdateMode = tableConfig.getPartialUpdateMode(); + UpdateProcessor updateProcessor = UpdateProcessor.create(readStats, readerContext, readerParameters.emitDeletes(), fileGroupUpdateCallback, props); FileGroupRecordBuffer recordBuffer; if (readerParameters.sortOutputs()) { recordBuffer = new SortedKeyBasedFileGroupRecordBuffer<>( @@ -74,11 +79,12 @@ public Pair, List> getRecordBuffer(Hoodie } RecordContext recordContext = readerContext.getRecordContext(); - Schema recordSchema = readerContext.getSchemaHandler().getTableSchema(); Iterator recordIterator = inputSplit.getRecordIterator(); String[] orderingFieldsArray = orderingFieldNames.toArray(new String[0]); + DeleteContext deleteContext = new DeleteContext(props, recordSchema); + deleteContext.withReaderSchema(recordSchema); while (recordIterator.hasNext()) { - HoodieRecord hoodieRecord = recordIterator.next(); + HoodieRecord hoodieRecord = recordIterator.next(); T data = recordContext.extractDataFromRecord(hoodieRecord, recordSchema, props); try { // we use -U operation to represent the record should be ignored during updating index. @@ -86,12 +92,13 @@ public Pair, List> getRecordBuffer(Hoodie BufferedRecord bufferedRecord; if (data == null) { DeleteRecord deleteRecord = DeleteRecord.create(hoodieRecord.getKey(), hoodieRecord.getOrderingValue(recordSchema, props, orderingFieldsArray)); - bufferedRecord = BufferedRecords.fromDeleteRecord(deleteRecord, deleteRecord.getOrderingValue(), hoodieOperation); + bufferedRecord = BufferedRecords.fromDeleteRecord(deleteRecord, recordContext, hoodieOperation); } else { // HoodieRecord#isDelete does not check if a record is a DELETE marked by a custom delete marker, // so we use recordContext#isDeleteRecord here if the data field is not null. - boolean isDelete = recordContext.isDeleteRecord(data, recordBuffer.getDeleteContext()); - bufferedRecord = BufferedRecords.fromEngineRecord(data, recordSchema, recordContext, orderingFieldNames, BufferedRecords.inferOperation(isDelete, hoodieOperation)); + boolean isDelete = recordContext.isDeleteRecord(data, deleteContext); + bufferedRecord = BufferedRecords.fromEngineRecord(data, hoodieRecord.getRecordKey(), recordSchema, recordContext, orderingFieldNames, + BufferedRecords.inferOperation(isDelete, hoodieOperation)); } recordBuffer.processNextDataRecord(bufferedRecord, bufferedRecord.getRecordKey()); } catch (IOException e) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/InternalSchemaCache.java b/hudi-common/src/main/java/org/apache/hudi/common/util/InternalSchemaCache.java index b62f7742b5a46..907d89eedd0c2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/InternalSchemaCache.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/InternalSchemaCache.java @@ -96,8 +96,7 @@ public static InternalSchema searchSchemaAndCache(long versionID, HoodieTableMet historicalSchemas = getHistoricalSchemas(metaClient); HISTORICAL_SCHEMA_CACHE.put(tablePath, historicalSchemas); } else { - long maxVersionId = historicalSchemas.keySet().stream().max(Long::compareTo).get(); - if (versionID > maxVersionId) { + if (historicalSchemas.keySet().stream().max(Long::compareTo).map(maxVersionId -> versionID > maxVersionId).orElse(false)) { historicalSchemas = getHistoricalSchemas(metaClient); HISTORICAL_SCHEMA_CACHE.put(tablePath, historicalSchemas); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/OrderingValues.java b/hudi-common/src/main/java/org/apache/hudi/common/util/OrderingValues.java index 2799294189150..caf67f7cb87ee 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/OrderingValues.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/OrderingValues.java @@ -113,4 +113,8 @@ public static boolean isSameClass(Comparable val1, Comparable val2) { public static List getValues(ArrayComparable orderingValue) { return orderingValue.getValues(); } + + public static boolean isCommitTimeOrderingValue(Comparable orderingValue) { + return orderingValue == null || OrderingValues.isDefault(orderingValue); + } } 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 230eb954507b6..494be05327c06 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 @@ -1048,7 +1048,7 @@ private static ClosableIterator> getLogRecords(List recordBuffer = new KeyBasedFileGroupRecordBuffer<>(readerContext, datasetMetaClient, readerContext.getMergeMode(), PartialUpdateMode.NONE, properties, tableConfig.getPreCombineFields(), - UpdateProcessor.create(readStats, readerContext, true, Option.empty())); + UpdateProcessor.create(readStats, readerContext, true, Option.empty(), properties)); // CRITICAL: Ensure allowInflightInstants is set to true try (HoodieMergedLogRecordReader mergedLogRecordReader = HoodieMergedLogRecordReader.newBuilder() diff --git a/hudi-common/src/test/java/org/apache/hudi/common/serialization/TestBufferedRecordSerializer.java b/hudi-common/src/test/java/org/apache/hudi/common/serialization/TestBufferedRecordSerializer.java index 09efd9bf5d807..50f84df3eae20 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/serialization/TestBufferedRecordSerializer.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/serialization/TestBufferedRecordSerializer.java @@ -18,6 +18,7 @@ package org.apache.hudi.common.serialization; +import org.apache.hudi.avro.AvroRecordContext; import org.apache.hudi.avro.AvroRecordSerializer; import org.apache.hudi.avro.model.HoodieMetadataRecord; import org.apache.hudi.common.model.DeleteRecord; @@ -101,7 +102,7 @@ void testBufferedRecordSerAndDe() throws IOException { void testDeleteRecordSerAndDe(HoodieOperation hoodieOperation) throws IOException { Schema schema = SchemaTestUtil.getSimpleSchema(); DeleteRecord record = DeleteRecord.create("id", "partition", 100); - BufferedRecord bufferedRecord = BufferedRecords.fromDeleteRecord(record, 100); + BufferedRecord bufferedRecord = BufferedRecords.fromDeleteRecord(record, AvroRecordContext.getFieldAccessorInstance()); bufferedRecord.setHoodieOperation(hoodieOperation); AvroRecordSerializer avroRecordSerializer = new AvroRecordSerializer(integer -> schema); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestBufferedRecords.java b/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestBufferedRecords.java new file mode 100644 index 0000000000000..fdbb9b43b0b60 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestBufferedRecords.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.table.read; + +import org.apache.hudi.common.engine.RecordContext; +import org.apache.hudi.common.model.DeleteRecord; +import org.apache.hudi.common.util.OrderingValues; + +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +class TestBufferedRecords { + + @Test + void testGetOrderingValueFromDeleteRecord() { + RecordContext recordContext = mock(RecordContext.class); + when(recordContext.getOrderingValue(any(DeleteRecord.class))).thenCallRealMethod(); + DeleteRecord deleteRecord = mock(DeleteRecord.class); + mockDeleteRecord(deleteRecord, null); + assertEquals(OrderingValues.getDefault(), recordContext.getOrderingValue(deleteRecord)); + mockDeleteRecord(deleteRecord, OrderingValues.getDefault()); + assertEquals(OrderingValues.getDefault(), recordContext.getOrderingValue(deleteRecord)); + Comparable orderingValue = "xyz"; + Comparable convertedValue = "_xyz"; + mockDeleteRecord(deleteRecord, orderingValue); + when(recordContext.convertOrderingValueToEngineType(orderingValue)).thenReturn(convertedValue); + assertEquals(convertedValue, recordContext.getOrderingValue(deleteRecord)); + } + + private void mockDeleteRecord(DeleteRecord deleteRecord, + Comparable orderingValue) { + when(deleteRecord.getOrderingValue()).thenReturn(orderingValue); + } +} 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 new file mode 100644 index 0000000000000..0e49e4ecb43fd --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestUpdateProcessor.java @@ -0,0 +1,267 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.read; + +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.common.engine.HoodieReaderContext; +import org.apache.hudi.common.engine.RecordContext; +import org.apache.hudi.common.model.BaseAvroPayload; +import org.apache.hudi.common.model.HoodieOperation; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.Option; + +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +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 org.junit.jupiter.params.provider.ValueSource; + +import java.io.IOException; +import java.util.Properties; +import java.util.stream.Stream; + +import static org.apache.hudi.common.model.HoodieRecord.SENTINEL; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.RETURNS_DEEP_STUBS; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; +import static org.mockito.Mockito.when; + +class TestUpdateProcessor { + private final HoodieReaderContext readerContext = mock(HoodieReaderContext.class, RETURNS_DEEP_STUBS); + private final RecordContext recordContext = mock(RecordContext.class); + private static final String KEY = "key"; + private static final Schema SCHEMA = SchemaBuilder.record("TestRecord") + .fields() + .name("key").type().stringType().noDefault() + .name("value").type().stringType().noDefault() + .endRecord(); + + private static Stream handleEmitDeletes() { + BufferedRecord previous = getRecord("value1", null); + BufferedRecord merged = getRecord("value2", null); + BufferedRecord mergedWithUpdateBefore = getRecord("value2", HoodieOperation.UPDATE_BEFORE); + BufferedRecord mergedWithEmpty = new BufferedRecord<>(KEY, 1, null, 0, null); + + BufferedRecord expected = getRecord("value2", HoodieOperation.DELETE); + BufferedRecord expectedWithUpdateBefore = getRecord("value2", HoodieOperation.UPDATE_BEFORE); + + return Stream.of( + Arguments.of(previous, merged, expected, true), + Arguments.of(previous, merged, expected, false), + Arguments.of(previous, mergedWithUpdateBefore, expectedWithUpdateBefore, true), + Arguments.of(previous, mergedWithUpdateBefore, expectedWithUpdateBefore, false), + Arguments.of(previous, mergedWithEmpty, null, true), + Arguments.of(previous, mergedWithEmpty, null, false)); + } + + @ParameterizedTest + @MethodSource("handleEmitDeletes") + void testHandleEmitDeletes(BufferedRecord previous, BufferedRecord merged, BufferedRecord expected, boolean usePayload) { + if (merged.getRecord() == null) { + when(readerContext.getRecordContext()).thenReturn(recordContext); + } + HoodieReadStats readStats = new HoodieReadStats(); + BaseFileUpdateCallback updateCallback = mock(BaseFileUpdateCallback.class); + UpdateProcessor.StandardUpdateProcessor delegate; + if (usePayload) { + delegate = new UpdateProcessor.PayloadUpdateProcessor<>(readStats, readerContext, true, new Properties(), "org.apache.hudi.common.testutils.DummyPayload"); + } else { + delegate = new UpdateProcessor.StandardUpdateProcessor<>(readStats, readerContext, true); + } + UpdateProcessor updateProcessor = new UpdateProcessor.CallbackProcessor<>(updateCallback, delegate); + BufferedRecord result = updateProcessor.processUpdate(KEY, previous, merged, true); + assertEquals(expected, result); + verifyReadStats(readStats, 0, 0, 1); + if (merged.getRecord() == null) { + verify(recordContext).getDeleteRow(KEY); + } + verify(updateCallback).onDelete(KEY, previous, merged.getHoodieOperation()); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testHandleDelete(boolean usePayload) { + BufferedRecord previous = getRecord("value1", null); + BufferedRecord merged = getRecord("value2", null); + HoodieReadStats readStats = new HoodieReadStats(); + BaseFileUpdateCallback updateCallback = mock(BaseFileUpdateCallback.class); + UpdateProcessor.StandardUpdateProcessor delegate; + if (usePayload) { + delegate = new UpdateProcessor.PayloadUpdateProcessor<>(readStats, readerContext, false, new Properties(), "org.apache.hudi.common.testutils.DummyPayload"); + } else { + delegate = new UpdateProcessor.StandardUpdateProcessor<>(readStats, readerContext, false); + } + UpdateProcessor updateProcessor = new UpdateProcessor.CallbackProcessor<>(updateCallback, delegate); + BufferedRecord result = updateProcessor.processUpdate(KEY, previous, merged, true); + assertNull(result); + verifyReadStats(readStats, 0, 0, 1); + verify(updateCallback).onDelete(KEY, previous, merged.getHoodieOperation()); + } + + @Test + void testHandleInsert() { + when(readerContext.getRecordContext()).thenReturn(recordContext); + when(recordContext.seal(any())).thenAnswer(invocationOnMock -> invocationOnMock.getArgument(0)); + BufferedRecord previous = null; + BufferedRecord merged = getRecord("value2", null); + BufferedRecord expected = getRecord("value2", HoodieOperation.INSERT); + HoodieReadStats readStats = new HoodieReadStats(); + BaseFileUpdateCallback updateCallback = mock(BaseFileUpdateCallback.class); + UpdateProcessor updateProcessor = new UpdateProcessor.CallbackProcessor<>(updateCallback, new UpdateProcessor.StandardUpdateProcessor<>(readStats, readerContext, false)); + BufferedRecord result = updateProcessor.processUpdate(KEY, previous, merged, false); + assertEquals(expected, result); + verifyReadStats(readStats, 1, 0, 0); + verify(updateCallback).onInsert(KEY, merged); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testHandleInsertWithPayload(boolean shouldIgnore) { + when(readerContext.getRecordContext()).thenReturn(recordContext); + when(recordContext.seal(any())).thenAnswer(invocationOnMock -> invocationOnMock.getArgument(0)); + BufferedRecord previous = null; + BufferedRecord merged = getRecord("value2", null); + BufferedRecord expected = getRecord("value2", HoodieOperation.INSERT); + HoodieReadStats readStats = new HoodieReadStats(); + BaseFileUpdateCallback updateCallback = mock(BaseFileUpdateCallback.class); + UpdateProcessor updateProcessor = new UpdateProcessor.CallbackProcessor<>(updateCallback, new UpdateProcessor.PayloadUpdateProcessor<>(readStats, readerContext, false, + new Properties(), DummyPayload.class.getName())); + + // mock record creation + when(recordContext.decodeAvroSchema(merged.getSchemaId())).thenReturn(SCHEMA); + when(recordContext.convertToAvroRecord(merged.getRecord(), SCHEMA)).thenReturn((GenericRecord) merged.getRecord()); + if (shouldIgnore) { + when(recordContext.convertToAvroRecord(merged.getRecord(), SCHEMA)).thenReturn(SENTINEL); + } else { + when(recordContext.convertToAvroRecord(merged.getRecord(), SCHEMA)).thenReturn((GenericRecord) merged.getRecord()); + when(readerContext.getSchemaHandler().getRequestedSchema()).thenReturn(SCHEMA); + when(recordContext.convertAvroRecord(any())).thenAnswer(invocationOnMock -> invocationOnMock.getArgument(0)); + } + + BufferedRecord result = updateProcessor.processUpdate(KEY, previous, merged, false); + if (shouldIgnore) { + assertNull(result); + verifyReadStats(readStats, 0, 0, 0); + verifyNoInteractions(updateCallback); + } else { + assertEquals(expected, result); + verifyReadStats(readStats, 1, 0, 0); + verify(updateCallback).onInsert(KEY, merged); + } + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testHandleNoUpdate(boolean usePayload) { + when(readerContext.getRecordContext()).thenReturn(recordContext); + when(recordContext.seal(any())).thenAnswer(invocationOnMock -> invocationOnMock.getArgument(0)); + BufferedRecord previous = getRecord("value2", null); + BufferedRecord merged = previous; + HoodieReadStats readStats = new HoodieReadStats(); + BaseFileUpdateCallback updateCallback = mock(BaseFileUpdateCallback.class); + UpdateProcessor.StandardUpdateProcessor delegate; + if (usePayload) { + delegate = new UpdateProcessor.PayloadUpdateProcessor<>(readStats, readerContext, false, new Properties(), "org.apache.hudi.common.testutils.DummyPayload"); + } else { + delegate = new UpdateProcessor.StandardUpdateProcessor<>(readStats, readerContext, false); + } + UpdateProcessor updateProcessor = new UpdateProcessor.CallbackProcessor<>(updateCallback, delegate); + BufferedRecord result = updateProcessor.processUpdate(KEY, previous, merged, false); + assertEquals(merged, result); + verifyReadStats(readStats, 0, 0, 0); + verifyNoInteractions(updateCallback); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testHandleUpdate(boolean usePayload) { + when(readerContext.getRecordContext()).thenReturn(recordContext); + when(recordContext.seal(any())).thenAnswer(invocationOnMock -> invocationOnMock.getArgument(0)); + BufferedRecord previous = getRecord("value1", null); + BufferedRecord merged = getRecord("value2", null); + BufferedRecord expected = getRecord("value2", HoodieOperation.UPDATE_AFTER); + HoodieReadStats readStats = new HoodieReadStats(); + BaseFileUpdateCallback updateCallback = mock(BaseFileUpdateCallback.class); + UpdateProcessor.StandardUpdateProcessor delegate; + if (usePayload) { + delegate = new UpdateProcessor.PayloadUpdateProcessor<>(readStats, readerContext, false, new Properties(), "org.apache.hudi.common.testutils.DummyPayload"); + } else { + delegate = new UpdateProcessor.StandardUpdateProcessor<>(readStats, readerContext, false); + } + UpdateProcessor updateProcessor = new UpdateProcessor.CallbackProcessor<>(updateCallback, delegate); + BufferedRecord result = updateProcessor.processUpdate(KEY, previous, merged, false); + assertEquals(expected, result); + verifyReadStats(readStats, 0, 1, 0); + verify(updateCallback).onUpdate(KEY, previous, merged); + } + + private void verifyReadStats(HoodieReadStats readStats, int numInserts, int numUpdates, int numDeletes) { + assertEquals(numInserts, readStats.getNumInserts()); + assertEquals(numUpdates, readStats.getNumUpdates()); + assertEquals(numDeletes, readStats.getNumDeletes()); + } + + private static BufferedRecord getRecord(String value, HoodieOperation operation) { + GenericRecord record = new GenericData.Record(SCHEMA); + record.put("key", KEY); + record.put("value", value); + return new BufferedRecord<>(KEY, 1, record, 0, operation); + } + + public static class DummyPayload extends BaseAvroPayload implements HoodieRecordPayload { + private final boolean isSentinel; + + public DummyPayload(GenericRecord record, Comparable orderingVal) { + super(record == SENTINEL ? null : record, orderingVal); + this.isSentinel = record == SENTINEL; + } + + @Override + public boolean canProduceSentinel() { + return true; + } + + @Override + public DummyPayload preCombine(DummyPayload oldValue) { + return null; + } + + @Override + public Option combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException { + return null; + } + + @Override + public Option getInsertValue(Schema schema) throws IOException { + if (isSentinel) { + return Option.of(SENTINEL); + } + return Option.of(HoodieAvroUtils.bytesToAvro(recordBytes, schema)); + } + } +} 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 12645cd118e97..a0c0b27ef5371 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 @@ -121,7 +121,7 @@ protected static KeyBasedFileGroupRecordBuffer buildKeyBasedFileG readerContext.setRecordMerger(Option.ofNullable(recordMerger)); HoodieTableMetaClient mockMetaClient = mock(HoodieTableMetaClient.class, RETURNS_DEEP_STUBS); when(mockMetaClient.getTableConfig()).thenReturn(tableConfig); - UpdateProcessor updateProcessor = UpdateProcessor.create(readStats, readerContext, false, Option.empty()); + UpdateProcessor updateProcessor = UpdateProcessor.create(readStats, readerContext, false, Option.empty(), props); if (fileGroupRecordBufferItrOpt.isEmpty()) { return new KeyBasedFileGroupRecordBuffer<>( 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 d0da78cc37c9b..67ace014bd617 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 @@ -23,7 +23,6 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.HoodieReaderContext; import org.apache.hudi.common.engine.RecordContext; -import org.apache.hudi.common.model.DeleteRecord; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.serialization.DefaultSerializer; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -32,13 +31,11 @@ import org.apache.hudi.common.table.read.BufferedRecords; import org.apache.hudi.common.table.read.DeleteContext; import org.apache.hudi.common.table.read.FileGroupReaderSchemaHandler; -import org.apache.hudi.common.table.read.HoodieReadStats; import org.apache.hudi.common.table.read.IteratorMode; import org.apache.hudi.common.table.read.UpdateProcessor; import org.apache.hudi.common.testutils.SchemaTestUtil; import org.apache.hudi.common.util.DefaultSizeEstimator; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.OrderingValues; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; @@ -58,7 +55,6 @@ import static org.apache.hudi.common.model.DefaultHoodieRecordPayload.DELETE_KEY; import static org.apache.hudi.common.model.DefaultHoodieRecordPayload.DELETE_MARKER; -import static org.apache.hudi.common.table.read.buffer.FileGroupRecordBuffer.getOrderingValue; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNull; @@ -93,7 +89,6 @@ class TestFileGroupRecordBuffer { private final UpdateProcessor updateProcessor = mock(UpdateProcessor.class); private HoodieTableMetaClient hoodieTableMetaClient = mock(HoodieTableMetaClient.class); private TypedProperties props; - private HoodieReadStats readStats = mock(HoodieReadStats.class); @BeforeEach void setUp() { @@ -108,22 +103,6 @@ void setUp() { when(readerContext.getIteratorMode()).thenReturn(IteratorMode.ENGINE_RECORD); } - @Test - void testGetOrderingValueFromDeleteRecord() { - HoodieReaderContext readerContext = mock(HoodieReaderContext.class); - DeleteRecord deleteRecord = mock(DeleteRecord.class); - mockDeleteRecord(deleteRecord, null); - assertEquals(OrderingValues.getDefault(), getOrderingValue(readerContext, deleteRecord)); - mockDeleteRecord(deleteRecord, OrderingValues.getDefault()); - assertEquals(OrderingValues.getDefault(), getOrderingValue(readerContext, deleteRecord)); - Comparable orderingValue = "xyz"; - Comparable convertedValue = "_xyz"; - mockDeleteRecord(deleteRecord, orderingValue); - when(recordContext.convertOrderingValueToEngineType(orderingValue)).thenReturn(convertedValue); - when(readerContext.getRecordContext()).thenReturn(recordContext); - assertEquals(convertedValue, getOrderingValue(readerContext, deleteRecord)); - } - @ParameterizedTest @CsvSource({"true,false", "false,true"}) void testInvalidCustomDeleteConfigs(boolean configureCustomDeleteKey, @@ -149,11 +128,6 @@ void testInvalidCustomDeleteConfigs(boolean configureCustomDeleteKey, exception.getMessage()); } - private void mockDeleteRecord(DeleteRecord deleteRecord, - Comparable orderingValue) { - when(deleteRecord.getOrderingValue()).thenReturn(orderingValue); - } - @Test void testIsCustomDeleteRecord() { String customDeleteKey = "op"; 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 3289614165afd..67b03db2c0b21 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,6 +70,7 @@ public void testDefaultFileGroupBufferRecordLoader(String fileGroupRecordBufferT readerContext.initRecordMerger(new TypedProperties()); FileGroupReaderSchemaHandler fileGroupReaderSchemaHandler = mock(FileGroupReaderSchemaHandler.class); when(fileGroupReaderSchemaHandler.getRequiredSchema()).thenReturn(SCHEMA); + when(fileGroupReaderSchemaHandler.getRequestedSchema()).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/TestReusableKeyBasedRecordBuffer.java b/hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestReusableKeyBasedRecordBuffer.java index 6865899166af3..9f3f6d8d44e16 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 @@ -65,7 +65,8 @@ void testRemainingLogEntryHandling() throws IOException { preMergedLogRecords.put("3", new BufferedRecord<>("3", 10, new TestRecord("3", 3), 0, null)); preMergedLogRecords.put("4", new BufferedRecord<>("4", 10, new TestRecord("4", 4), 0, null)); HoodieReadStats readStats = new HoodieReadStats(); - UpdateProcessor updateProcessor = UpdateProcessor.create(readStats, mockReaderContext, false, Option.empty()); + when(mockReaderContext.getPayloadClasses(any())).thenReturn(Option.empty()); + UpdateProcessor updateProcessor = UpdateProcessor.create(readStats, mockReaderContext, false, Option.empty(), new TypedProperties()); // 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"))); 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 13e8feba92c05..a1ec3f66caca8 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 @@ -201,7 +201,8 @@ private SortedKeyBasedFileGroupRecordBuffer buildSortedKeyBasedFileG RecordMergeMode recordMergeMode = RecordMergeMode.COMMIT_TIME_ORDERING; PartialUpdateMode partialUpdateMode = PartialUpdateMode.NONE; TypedProperties props = new TypedProperties(); - UpdateProcessor updateProcessor = UpdateProcessor.create(readStats, mockReaderContext, false, Option.empty()); + when(mockReaderContext.getPayloadClasses(any())).thenReturn(Option.empty()); + UpdateProcessor updateProcessor = UpdateProcessor.create(readStats, mockReaderContext, false, Option.empty(), props); return new SortedKeyBasedFileGroupRecordBuffer<>( mockReaderContext, mockMetaClient, recordMergeMode, partialUpdateMode, props, Collections.emptyList(), updateProcessor); } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java index 18f84084b0344..103e43b39d7e5 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java @@ -1108,7 +1108,7 @@ public Stream generateUniqueDeleteStream(Integer n) { * @param n Number of unique records * @return stream of hoodie records for delete */ - public Stream generateUniqueDeleteRecordStream(String instantTime, Integer n) { + public Stream generateUniqueDeleteRecordStream(String instantTime, Integer n, boolean updatePartition) { final Set used = new HashSet<>(); Map existingKeys = existingKeysBySchema.get(TRIP_EXAMPLE_SCHEMA); Integer numExistingKeys = numKeysBySchema.get(TRIP_EXAMPLE_SCHEMA); @@ -1128,8 +1128,14 @@ public Stream generateUniqueDeleteRecordStream(String instantTime, existingKeys.remove(numExistingKeys - 1); numExistingKeys--; used.add(kp); + HoodieKey key = kp.key; + if (updatePartition) { + String updatedPartitionPath = Arrays.stream(partitionPaths).filter(p -> !p.equals(kp.partitionPath)) + .findAny().orElseThrow(() -> new HoodieIOException("No other partition path found to update")); + key = new HoodieKey(key.getRecordKey(), updatedPartitionPath); + } try { - result.add(new HoodieAvroRecord(kp.key, generateRandomDeleteValue(kp.key, instantTime))); + result.add(new HoodieAvroRecord(key, generateRandomDeleteValue(key, instantTime))); } catch (IOException e) { throw new HoodieIOException(e.getMessage(), e); } @@ -1146,7 +1152,11 @@ public Stream generateUniqueDeleteRecordStream(String instantTime, * @return List of hoodie records for delete */ public List generateUniqueDeleteRecords(String instantTime, Integer n) { - return generateUniqueDeleteRecordStream(instantTime, n).collect(Collectors.toList()); + return generateUniqueDeleteRecordStream(instantTime, n, false).collect(Collectors.toList()); + } + + public List generateUniqueDeleteRecordsWithUpdatedPartition(String instantTime, Integer n) { + return generateUniqueDeleteRecordStream(instantTime, n, true).collect(Collectors.toList()); } public boolean deleteExistingKeyIfPresent(HoodieKey key) { @@ -1184,6 +1194,14 @@ public int getNumExistingKeys(String schemaStr) { return numKeysBySchema.getOrDefault(schemaStr, 0); } + public List getExistingKeys() { + return getExistingKeys(TRIP_EXAMPLE_SCHEMA); + } + + public List getExistingKeys(String schemaStr) { + return existingKeysBySchema.get(schemaStr).values().stream().map(kp -> kp.key.getRecordKey()).collect(Collectors.toList()); + } + /** * Object containing the key and partition path for testing. */ @@ -1200,10 +1218,10 @@ public void close() { private static long genRandomTimeMillis(Random r) { // Fri Feb 13 15:31:30 PST 2009 - long anchorTs = 1234567890L; + long anchorTs = 1234567890000L; // NOTE: To provide for certainty and not generate overly random dates, we will limit // dispersion to be w/in +/- 3 days from the anchor date - return anchorTs + r.nextLong() % 259200000L; + return anchorTs + r.nextInt(259200000); } public static UUID genPseudoRandomUUID(Random r) { diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/read/TestCustomRecordMerger.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/read/TestCustomRecordMerger.java index a1a73a3d20a90..02eee5387621f 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/read/TestCustomRecordMerger.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/read/TestCustomRecordMerger.java @@ -239,7 +239,6 @@ private static Stream testArgs() { public static class CustomAvroMerger implements HoodieRecordMerger { public static final String KEEP_CERTAIN_TIMESTAMP_VALUE_ONLY = "KEEP_CERTAIN_TIMESTAMP_VALUE_ONLY"; - public static final String TIMESTAMP = "timestamp"; private String[] orderingFields; @Override @@ -276,18 +275,6 @@ public Option> merge( return Option.empty(); } - @Override - public boolean shouldFlush( - HoodieRecord record, - Schema schema, - TypedProperties props - ) { - long timestamp = (long) ((HoodieAvroIndexedRecord) record) - .getData() - .get(schema.getField(TIMESTAMP).pos()); - return timestamp % 3 == 0L; - } - @Override public HoodieRecord.HoodieRecordType getRecordType() { return AVRO; 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 a9ce1a8163021..e232bf953e851 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 @@ -513,7 +513,7 @@ class CDCFileGroupIterator(split: HoodieCDCFileGroupSplit, keyBasedFileGroupRecordBuffer.ifPresent(k => k.close()) keyBasedFileGroupRecordBuffer = Option.of(new KeyBasedFileGroupRecordBuffer[InternalRow](readerContext, metaClient, readerContext.getMergeMode, metaClient.getTableConfig.getPartialUpdateMode, readerProperties, metaClient.getTableConfig.getPreCombineFields, - UpdateProcessor.create(stats, readerContext, true, Option.empty()))) + UpdateProcessor.create(stats, readerContext, true, Option.empty(), props))) HoodieMergedLogRecordReader.newBuilder[InternalRow] .withStorage(metaClient.getStorage) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3ParquetSchemaEvolutionUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3ParquetSchemaEvolutionUtils.scala index 912879a602847..63a168fc8a918 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3ParquetSchemaEvolutionUtils.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3ParquetSchemaEvolutionUtils.scala @@ -51,7 +51,7 @@ class Spark3ParquetSchemaEvolutionUtils(sharedConf: Configuration, // Fetch internal schema private lazy val querySchemaOption: util.Option[InternalSchema] = pruneInternalSchema(internalSchemaOpt, requiredSchema) - var shouldUseInternalSchema: Boolean = querySchemaOption.isPresent + var shouldUseInternalSchema: Boolean = querySchemaOption.isPresent && tablePath != null private lazy val schemaUtils: HoodieSchemaUtils = sparkAdapter.getSchemaUtils diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodieMergeHandleWithSparkMerger.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodieMergeHandleWithSparkMerger.java index 8cdd41803a85a..1b15f52654268 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodieMergeHandleWithSparkMerger.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodieMergeHandleWithSparkMerger.java @@ -22,6 +22,7 @@ import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteClientTestUtils; +import org.apache.hudi.common.config.RecordMergeMode; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; @@ -59,21 +60,23 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Properties; import java.util.stream.Collectors; import java.util.stream.Stream; import static org.apache.hudi.common.config.HoodieReaderConfig.FILE_GROUP_READER_ENABLED; import static org.apache.hudi.common.config.HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT; -import static org.apache.hudi.common.model.HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY; import static org.apache.hudi.config.HoodieWriteConfig.RECORD_MERGE_IMPL_CLASSES; +import static org.apache.hudi.config.HoodieWriteConfig.RECORD_MERGE_MODE; +import static org.apache.hudi.config.HoodieWriteConfig.RECORD_MERGE_STRATEGY_ID; import static org.apache.hudi.config.HoodieWriteConfig.WRITE_RECORD_POSITIONS; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertTrue; public class TestHoodieMergeHandleWithSparkMerger extends SparkClientFunctionalTestHarness { private static final Schema SCHEMA = getAvroSchema("AvroSchema", "AvroSchemaNS"); + private final Map properties = new HashMap<>(); private HoodieTableMetaClient metaClient; public static String getPartitionPath() { @@ -82,41 +85,30 @@ public static String getPartitionPath() { @BeforeEach public void setUp() throws IOException { - Properties properties = new Properties(); - properties.setProperty( + properties.put( HoodieTableConfig.BASE_FILE_FORMAT.key(), HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().toString()); - properties.setProperty(HoodieTableConfig.PRECOMBINE_FIELDS.key(), "record_key"); - properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(),"partition_path"); - properties.setProperty(HoodieTableConfig.PARTITION_FIELDS.key(), "partition_path"); - metaClient = getHoodieMetaClient(storageConf(), basePath(), HoodieTableType.MERGE_ON_READ, properties); + properties.put(HoodieTableConfig.PRECOMBINE_FIELDS.key(), "record_key"); + properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(),"partition_path"); + properties.put(HoodieTableConfig.PARTITION_FIELDS.key(), "partition_path"); } @Test public void testDefaultMerger() throws Exception { HoodieWriteConfig writeConfig = buildDefaultWriteConfig(SCHEMA); HoodieRecordMerger merger = writeConfig.getRecordMerger(); - assertTrue(merger instanceof DefaultMerger); + assertInstanceOf(DefaultMerger.class, merger); assertTrue(writeConfig.getBooleanOrDefault(FILE_GROUP_READER_ENABLED.key(), false)); insertAndUpdate(writeConfig, 114); } - @Test - public void testNoFlushMerger() throws Exception { - HoodieWriteConfig writeConfig = buildNoFlushWriteConfig(SCHEMA); - HoodieRecordMerger merger = writeConfig.getRecordMerger(); - assertTrue(merger instanceof NoFlushMerger); - assertTrue(writeConfig.getBooleanOrDefault(FILE_GROUP_READER_ENABLED.key(), false)); - insertAndUpdate(writeConfig, 64); - } - @Test public void testCustomMerger() throws Exception { HoodieWriteConfig writeConfig = buildCustomWriteConfig(SCHEMA); HoodieRecordMerger merger = writeConfig.getRecordMerger(); - assertTrue(merger instanceof CustomMerger); + assertInstanceOf(CustomMerger.class, merger); assertTrue(writeConfig.getBooleanOrDefault(FILE_GROUP_READER_ENABLED.key(), false)); - insertAndUpdate(writeConfig, 95); + insertAndUpdate(writeConfig, 114); } public List generateRecords(int numOfRecords, String commitTime) throws Exception { @@ -157,44 +149,44 @@ private static Schema getAvroSchema(String schemaName, String schemaNameSpace) { return AvroConversionUtils.convertStructTypeToAvroSchema(SparkDatasetTestUtils.STRUCT_TYPE, schemaName, schemaNameSpace); } - public HoodieWriteConfig getWriteConfig(Schema avroSchema) { - Properties extraProperties = new Properties(); - extraProperties.setProperty( + public HoodieWriteConfig getWriteConfig(Schema avroSchema, String recordMergerImplClass, String mergeStrategyId, RecordMergeMode recordMergeMode) { + properties.put(RECORD_MERGE_STRATEGY_ID.key(), mergeStrategyId); + properties.put(HoodieTableConfig.RECORD_MERGE_STRATEGY_ID.key(), mergeStrategyId); + properties.put(RECORD_MERGE_MODE.key(), recordMergeMode.name()); + properties.put(HoodieTableConfig.RECORD_MERGE_MODE.key(), recordMergeMode.name()); + properties.put( RECORD_MERGE_IMPL_CLASSES.key(), - "org.apache.hudi.DefaultSparkRecordMerger"); - extraProperties.setProperty( + recordMergerImplClass); + properties.put( LOGFILE_DATA_BLOCK_FORMAT.key(), "parquet"); - extraProperties.setProperty( + properties.put( HoodieWriteConfig.PRECOMBINE_FIELD_NAME.key(), "record_key"); - extraProperties.setProperty( + properties.put( FILE_GROUP_READER_ENABLED.key(), "true"); - extraProperties.setProperty( + properties.put( WRITE_RECORD_POSITIONS.key(), "true"); - extraProperties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(),"partition_path"); + properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "partition_path"); return getConfigBuilder(true) .withPath(basePath()) .withSchema(avroSchema.toString()) - .withProperties(extraProperties) + .withProps(properties) .build(); } - public DefaultWriteConfig buildDefaultWriteConfig(Schema avroSchema) { - HoodieWriteConfig config = getWriteConfig(avroSchema); - return new DefaultWriteConfig(config); - } - - public NoFlushWriteConfig buildNoFlushWriteConfig(Schema avroSchema) { - HoodieWriteConfig config = getWriteConfig(avroSchema); - return new NoFlushWriteConfig(config); + public HoodieWriteConfig buildDefaultWriteConfig(Schema avroSchema) { + HoodieWriteConfig writeConfig = getWriteConfig(avroSchema, DefaultMerger.class.getName(), HoodieRecordMerger.EVENT_TIME_BASED_MERGE_STRATEGY_UUID, RecordMergeMode.EVENT_TIME_ORDERING); + metaClient = getHoodieMetaClient(storageConf(), basePath(), HoodieTableType.MERGE_ON_READ, writeConfig.getProps()); + return writeConfig; } - public CustomWriteConfig buildCustomWriteConfig(Schema avroSchema) { - HoodieWriteConfig config = getWriteConfig(avroSchema); - return new CustomWriteConfig(config); + public HoodieWriteConfig buildCustomWriteConfig(Schema avroSchema) { + HoodieWriteConfig writeConfig = getWriteConfig(avroSchema, CustomMerger.class.getName(), HoodieRecordMerger.CUSTOM_MERGE_STRATEGY_UUID, RecordMergeMode.CUSTOM); + metaClient = getHoodieMetaClient(storageConf(), basePath(), HoodieTableType.MERGE_ON_READ, writeConfig.getProps()); + return writeConfig; } public HoodieTableFileSystemView getFileSystemView() { @@ -227,22 +219,6 @@ public List getFileIds(String partitionPath) { } public void checkDataEquality(int numRecords) { - Map properties = new HashMap<>(); - properties.put( - RECORD_MERGE_IMPL_CLASSES.key(), - "org.apache.hudi.DefaultSparkRecordMerger"); - properties.put( - LOGFILE_DATA_BLOCK_FORMAT.key(), - "parquet"); - properties.put( - PAYLOAD_ORDERING_FIELD_PROP_KEY, - HoodieRecord.HoodieMetadataField.RECORD_KEY_METADATA_FIELD.getFieldName()); - properties.put( - FILE_GROUP_READER_ENABLED.key(), - "true"); - properties.put( - WRITE_RECORD_POSITIONS.key(), - "true"); Dataset rows = spark() .read() .options(properties) @@ -269,7 +245,7 @@ public void insertAndUpdate(HoodieWriteConfig writeConfig, int expectedRecordNum assertTrue(baseFileStream.findAny().isPresent()); // Check metadata files. - Option deltaCommit = reloadedMetaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant(); + Option deltaCommit = reloadedMetaClient.reloadActiveTimeline().getDeltaCommitTimeline().lastInstant(); assertTrue(deltaCommit.isPresent()); assertEquals(instantTime, deltaCommit.get().requestedTime(), "Delta commit should be specified value"); @@ -297,7 +273,7 @@ public void insertAndUpdate(HoodieWriteConfig writeConfig, int expectedRecordNum updateRecordsInMORTable(reloadedMetaClient, records2, writeClient, writeConfig, instantTime, false); // Check metadata files. - deltaCommit = reloadedMetaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant(); + deltaCommit = reloadedMetaClient.reloadActiveTimeline().getDeltaCommitTimeline().lastInstant(); assertTrue(deltaCommit.isPresent()); // Check data files. @@ -323,45 +299,6 @@ public void insertAndUpdate(HoodieWriteConfig writeConfig, int expectedRecordNum } } - public static class TestHoodieWriteConfig extends HoodieWriteConfig { - TestHoodieWriteConfig(HoodieWriteConfig writeConfig) { - super(writeConfig.getEngineType(), writeConfig.getProps()); - } - } - - public static class DefaultWriteConfig extends TestHoodieWriteConfig { - DefaultWriteConfig(HoodieWriteConfig writeConfig) { - super(writeConfig); - } - - @Override - public HoodieRecordMerger getRecordMerger() { - return new DefaultMerger(); - } - } - - public static class NoFlushWriteConfig extends TestHoodieWriteConfig { - NoFlushWriteConfig(HoodieWriteConfig writeConfig) { - super(writeConfig); - } - - @Override - public HoodieRecordMerger getRecordMerger() { - return new NoFlushMerger(); - } - } - - public static class CustomWriteConfig extends TestHoodieWriteConfig { - CustomWriteConfig(HoodieWriteConfig writeConfig) { - super(writeConfig); - } - - @Override - public HoodieRecordMerger getRecordMerger() { - return new CustomMerger(); - } - } - public static class DefaultMerger extends DefaultSparkRecordMerger { @Override public boolean shouldFlush(HoodieRecord record, Schema schema, TypedProperties props) { @@ -374,12 +311,18 @@ public static class NoFlushMerger extends DefaultSparkRecordMerger { public boolean shouldFlush(HoodieRecord record, Schema schema, TypedProperties props) { return false; } + + @Override + public String getMergingStrategy() { + return HoodieRecordMerger.CUSTOM_MERGE_STRATEGY_UUID; + } } public static class CustomMerger extends DefaultSparkRecordMerger { + @Override - public boolean shouldFlush(HoodieRecord record, Schema schema, TypedProperties props) throws IOException { - return !((HoodieSparkRecord) record).getData().getString(0).equals("001"); + public String getMergingStrategy() { + return HoodieRecordMerger.CUSTOM_MERGE_STRATEGY_UUID; } } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/io/TestMergeHandle.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/io/TestMergeHandle.java index 4452268fa4455..7abec9389d8ab 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/io/TestMergeHandle.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/io/TestMergeHandle.java @@ -18,59 +18,93 @@ package org.apache.hudi.io; +import org.apache.hudi.avro.AvroSchemaUtils; +import org.apache.hudi.client.SecondaryIndexStats; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.config.RecordMergeMode; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.HoodieLocalEngineContext; import org.apache.hudi.common.engine.LocalTaskContextSupplier; +import org.apache.hudi.common.model.DefaultHoodieRecordPayload; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileGroup; +import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordDelegate; +import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.ConfigUtils; +import org.apache.hudi.common.util.DateTimeUtils; +import org.apache.hudi.common.util.HoodieRecordUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ParquetUtils; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.keygen.constant.KeyGeneratorOptions; +import org.apache.hudi.storage.StoragePath; import org.apache.hudi.table.HoodieSparkCopyOnWriteTable; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.action.commit.HoodieMergeHelper; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; import org.apache.spark.api.java.JavaRDD; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION; +import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.AVRO_SCHEMA; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.AssertionsKt.assertNull; /** * Unit tests {@link HoodieMergeHandle}. */ public class TestMergeHandle extends BaseTestHandle { + private static final String ORDERING_FIELD = "timestamp"; + @Test public void testMergeHandleRLIAndSIStatsWithUpdatesAndDeletes() throws Exception { + + // delete and recreate + metaClient.getStorage().deleteDirectory(metaClient.getBasePath()); + Properties properties = new Properties(); + properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key"); + properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "partition_path"); + properties.put(HoodieWriteConfig.PRECOMBINE_FIELD_NAME.key(), ORDERING_FIELD); + initMetaClient(getTableType(), properties); + // init config and table - HoodieWriteConfig config = getConfigBuilder(basePath) - .withPopulateMetaFields(true) - .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder().withRemoteServerPort(timelineServicePort).build()) - .withMetadataConfig(HoodieMetadataConfig.newBuilder() - .enable(true) - .withEnableRecordIndex(true) - .withStreamingWriteEnabled(true) - .withSecondaryIndexEnabled(true) - .withSecondaryIndexName("sec-rider") - .withSecondaryIndexForColumn("rider") - .build()) - .withKeyGenerator(KeyGeneratorForDataGeneratorRecords.class.getCanonicalName()) - .build(); - config.setSchema(TRIP_EXAMPLE_SCHEMA); - HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, new HoodieLocalEngineContext(storageConf), metaClient); + HoodieWriteConfig config = getHoodieWriteConfigBuilder().build(); + HoodieSparkTable.create(config, new HoodieLocalEngineContext(storageConf), metaClient); // one round per partition String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0]; @@ -84,7 +118,7 @@ public void testMergeHandleRLIAndSIStatsWithUpdatesAndDeletes() throws Exception client.commit(instantTime, statuses, Option.empty(), COMMIT_ACTION, Collections.emptyMap(), Option.empty()); metaClient = HoodieTableMetaClient.reload(metaClient); - table = (HoodieSparkCopyOnWriteTable) HoodieSparkCopyOnWriteTable.create(config, context, metaClient); + HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkCopyOnWriteTable.create(config, context, metaClient); HoodieFileGroup fileGroup = table.getFileSystemView().getAllFileGroups(partitionPath).collect(Collectors.toList()).get(0); String fileId = fileGroup.getFileGroupId().getFileId(); @@ -106,15 +140,21 @@ public void testMergeHandleRLIAndSIStatsWithUpdatesAndDeletes() throws Exception // numUpdates + numDeletes - new record index updates assertEquals(numUpdates + numDeletes, writeStatus.getIndexStats().getWrittenRecordDelegates().size()); int numDeletedRecordDelegates = 0; + int numDeletedRecordDelegatesWithIgnoreIndexUpdate = 0; for (HoodieRecordDelegate recordDelegate : writeStatus.getIndexStats().getWrittenRecordDelegates()) { if (!recordDelegate.getNewLocation().isPresent()) { numDeletedRecordDelegates++; + if (recordDelegate.getIgnoreIndexUpdate()) { + numDeletedRecordDelegatesWithIgnoreIndexUpdate++; + } } else { assertTrue(recordDelegate.getNewLocation().isPresent()); assertEquals(fileId, recordDelegate.getNewLocation().get().getFileId()); assertEquals(instantTime, recordDelegate.getNewLocation().get().getInstantTime()); } } + // 5 of the deletes are marked with ignoreIndexUpdate in generateDeleteRecords + assertEquals(5, numDeletedRecordDelegatesWithIgnoreIndexUpdate); assertEquals(numDeletes, numDeletedRecordDelegates); // verify secondary index stats @@ -124,4 +164,483 @@ public void testMergeHandleRLIAndSIStatsWithUpdatesAndDeletes() throws Exception assertEquals(2 * numUpdates + numDeletes, writeStatus.getIndexStats().getSecondaryIndexStats().values().stream().findFirst().get().size()); validateSecondaryIndexStatsContent(writeStatus, numUpdates, numDeletes); } + + @ParameterizedTest + @ValueSource(strings = {"EVENT_TIME_ORDERING", "COMMIT_TIME_ORDERING", "CUSTOM", "CUSTOM_MERGER"}) + public void testFGReaderBasedMergeHandleInsertUpsertDelete(String mergeMode) throws IOException { + testFGReaderBasedMergeHandleInsertUpsertDeleteInternal(mergeMode, new Properties(), false); + } + + @Test + public void testFGReaderBasedMergeHandleEventTimeMetadata() throws IOException { + Properties properties = new Properties(); + properties.put("hoodie.write.track.event.time.watermark", "true"); + properties.put("hoodie.payload.event.time.field", "current_ts"); + testFGReaderBasedMergeHandleInsertUpsertDeleteInternal("EVENT_TIME_ORDERING", properties, true); + } + + private void testFGReaderBasedMergeHandleInsertUpsertDeleteInternal(String mergeMode, Properties writerProps, boolean validateEventTimeMetadata) throws IOException { + metaClient.getStorage().deleteDirectory(metaClient.getBasePath()); + + HoodieWriteConfig config = getHoodieWriteConfigBuilder().withProperties(writerProps).build(); + TypedProperties properties = new TypedProperties(); + writerProps.keySet().forEach((key -> properties.put(key, writerProps.get(key)))); + + properties.put(HoodieTableConfig.RECORDKEY_FIELDS.key(), "_row_key"); + properties.put(HoodieTableConfig.PARTITION_FIELDS.key(), "partition_path"); + properties.put(HoodieTableConfig.PRECOMBINE_FIELDS.key(), ORDERING_FIELD); + properties.put(HoodieTableConfig.RECORD_MERGE_MODE.key(), mergeMode); + if (mergeMode.equals("CUSTOM_MERGER")) { + config.setValue(HoodieWriteConfig.RECORD_MERGE_IMPL_CLASSES, CustomMerger.class.getName()); + properties.put(HoodieTableConfig.RECORD_MERGE_STRATEGY_ID.key(), CustomMerger.getStrategyId()); + properties.put(HoodieTableConfig.RECORD_MERGE_MODE.key(), "CUSTOM"); + } + String payloadClass = null; + if (mergeMode.equals(RecordMergeMode.CUSTOM.name()) || mergeMode.equals("CUSTOM_MERGER")) { + // set payload class as part of table properties. + properties.put(HoodieTableConfig.PAYLOAD_CLASS_NAME.key(), CustomPayload.class.getName()); + payloadClass = CustomPayload.class.getName(); + } else if (mergeMode.equals(RecordMergeMode.EVENT_TIME_ORDERING.name())) { + payloadClass = DefaultHoodieRecordPayload.class.getName(); + } else if (mergeMode.equals(RecordMergeMode.COMMIT_TIME_ORDERING.name())) { + payloadClass = OverwriteWithLatestAvroPayload.class.getName(); + } + initMetaClient(getTableType(), properties); + + String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0]; + HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[] {partitionPath}); + // initial write + List recordsBatch1 = initialWrite(config, dataGenerator, payloadClass, partitionPath); + Map recordsBatch1Map = recordsBatch1.stream().map(record -> Pair.of(record.getRecordKey(), record)) + .collect(Collectors.toMap(pair -> pair.getKey(), pair -> pair.getValue())); + + metaClient = HoodieTableMetaClient.reload(metaClient); + String commit1 = metaClient.getActiveTimeline().getWriteTimeline().filterCompletedInstants().getInstants().get(0).requestedTime(); + HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkCopyOnWriteTable.create(config, context, metaClient); + HoodieFileGroup fileGroup = table.getFileSystemView().getAllFileGroups(partitionPath).collect(Collectors.toList()).get(0); + String fileId = fileGroup.getFileGroupId().getFileId(); + + String instantTime = "001"; + InputAndExpectedDataSet inputAndExpectedDataSet = prepareInputFor2ndBatch(config, dataGenerator, payloadClass, partitionPath, mergeMode, recordsBatch1, instantTime, + fileGroup); + + Map newInsertRecordsMap = inputAndExpectedDataSet.getNewInserts().stream().map(record -> Pair.of(record.getRecordKey(), record)) + .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + setCurLocation(inputAndExpectedDataSet.getRecordsToMerge().stream().filter(record -> !newInsertRecordsMap.containsKey(record.getRecordKey())).collect(Collectors.toList()), + fileId, commit1); + Map validUpdatesRecordsMap = inputAndExpectedDataSet.getValidUpdates().stream().map(record -> Pair.of(record.getRecordKey(), record)) + .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + Map validDeletesMap = inputAndExpectedDataSet.getValidDeletes(); + Map untouchedRecordsFromBatch1 = recordsBatch1Map.entrySet().stream().filter(kv -> { + return (!validUpdatesRecordsMap.containsKey(kv.getKey()) && !validDeletesMap.containsKey(kv.getKey())); + }).collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + FileGroupReaderBasedMergeHandle fileGroupReaderBasedMergeHandle = new FileGroupReaderBasedMergeHandle( + config, instantTime, table, inputAndExpectedDataSet.getRecordsToMerge().iterator(), partitionPath, fileId, new LocalTaskContextSupplier(), + Option.empty()); + + fileGroupReaderBasedMergeHandle.doMerge(); + List writeStatuses = fileGroupReaderBasedMergeHandle.close(); + WriteStatus writeStatus = writeStatuses.get(0); + + // read the file and validate values. + String filePath = writeStatus.getStat().getPath(); + String fullPath = metaClient.getBasePath() + "/" + filePath; + + List actualRecords = new ParquetUtils().readAvroRecords(metaClient.getStorage(), new StoragePath(fullPath)); + Map actualRecordsMap = actualRecords.stream() + .map(genRec -> Pair.of(genRec.get("_row_key"), genRec)) + .collect(Collectors.toMap(pair -> pair.getKey().toString(), pair -> pair.getValue())); + + for (Map.Entry entry : inputAndExpectedDataSet.getExpectedRecordsMap().entrySet()) { + assertTrue(actualRecordsMap.containsKey(entry.getKey())); + GenericRecord genericRecord = (GenericRecord) ((HoodieRecordPayload) entry.getValue().getData()).getInsertValue(AVRO_SCHEMA, properties).get(); + assertEquals(genericRecord.get(ORDERING_FIELD).toString(), actualRecordsMap.get(entry.getKey()).get(ORDERING_FIELD).toString()); + } + + // validate that deleted records are not part of actual list + inputAndExpectedDataSet.getValidDeletes().keySet().forEach(deletedKey -> { + assertTrue(!actualRecordsMap.containsKey(deletedKey)); + }); + + HoodieWriteStat stat = writeStatus.getStat(); + assertEquals(inputAndExpectedDataSet.getExpectedUpdates(), stat.getNumUpdateWrites()); + assertEquals(inputAndExpectedDataSet.getExpectedDeletes(), stat.getNumDeletes()); + assertEquals(2, stat.getNumInserts()); + + validateWriteStatus(writeStatus, commit1, 10 - inputAndExpectedDataSet.getExpectedDeletes() + 2, + inputAndExpectedDataSet.getExpectedUpdates(), 2, inputAndExpectedDataSet.getExpectedDeletes()); + + // validate event time metadata if enabled + if (validateEventTimeMetadata) { + List records = new ArrayList<>(inputAndExpectedDataSet.getExpectedRecordsMap().values()); + validateEventTimeMetadata(writeStatus, writerProps.get("hoodie.payload.event.time.field").toString(), AVRO_SCHEMA, config, properties, records); + } else { + validateEventTimeMetadataNotSet(writeStatus); + } + + // validate RLI stats + List recordDelegates = writeStatus.getIndexStats().getWrittenRecordDelegates(); + recordDelegates.forEach(recordDelegate -> { + if (recordDelegate.getNewLocation().isPresent() && recordDelegate.getCurrentLocation().isPresent()) { + // updates + // inserts are also tagged as updates. To be fixed. + assertTrue(validUpdatesRecordsMap.containsKey(recordDelegate.getRecordKey()) || untouchedRecordsFromBatch1.containsKey(recordDelegate.getRecordKey())); + } else if (recordDelegate.getNewLocation().isPresent() && recordDelegate.getCurrentLocation().isEmpty()) { + // inserts + assertTrue(newInsertRecordsMap.containsKey(recordDelegate.getRecordKey())); + } else if (recordDelegate.getCurrentLocation().isPresent() && recordDelegate.getNewLocation().isEmpty()) { + // deletes + assertTrue(validDeletesMap.containsKey(recordDelegate.getRecordKey())); + } + }); + + // validate SI stats. + assertEquals(1, writeStatus.getIndexStats().getSecondaryIndexStats().size()); + assertEquals(inputAndExpectedDataSet.expectedDeletes + 2 * inputAndExpectedDataSet.expectedUpdates + inputAndExpectedDataSet.newInserts.size(), + writeStatus.getIndexStats().getSecondaryIndexStats().get("secondary_index_sec-rider").size()); + for (SecondaryIndexStats secondaryIndexStat : writeStatus.getIndexStats().getSecondaryIndexStats().get("secondary_index_sec-rider")) { + if (secondaryIndexStat.isDeleted()) { + // Either the record is deleted or record is updated. For updated record there are two SI entries + // one for older SI record deletion and another for new SI record creation + assertTrue(inputAndExpectedDataSet.validDeletes.containsKey(secondaryIndexStat.getRecordKey()) + || inputAndExpectedDataSet.getValidUpdates().stream().anyMatch(rec -> rec.getRecordKey().equals(secondaryIndexStat.getRecordKey()))); + } else { + HoodieRecord record = inputAndExpectedDataSet.expectedRecordsMap.get(secondaryIndexStat.getRecordKey()); + assertEquals(record.getColumnValueAsJava(AVRO_SCHEMA, "rider", properties).toString(), + secondaryIndexStat.getSecondaryKeyValue().toString()); + } + } + } + + private void validateEventTimeMetadataNotSet(WriteStatus writeStatus) { + assertNull(writeStatus.getStat().getMinEventTime()); + assertNull(writeStatus.getStat().getMaxEventTime()); + } + + private void validateEventTimeMetadata(WriteStatus writeStatus, String eventTimeFieldName, Schema schema, HoodieWriteConfig config, + TypedProperties props, List records) { + long actualMinEventTime = writeStatus.getStat().getMinEventTime(); + long actualMaxEventTime = writeStatus.getStat().getMaxEventTime(); + boolean keepConsistentLogicalTimestamp = ConfigUtils.shouldKeepConsistentLogicalTimestamp(config.getProps()); + + AtomicLong expectedMinValue = new AtomicLong(Long.MAX_VALUE); + AtomicLong expectedMaxValue = new AtomicLong(Long.MIN_VALUE); + + // Append event_time. + records.forEach(record -> { + Object eventTimeValue = record.getColumnValueAsJava(schema, eventTimeFieldName, props); + if (eventTimeValue != null) { + // Append event_time. + Option field = AvroSchemaUtils.findNestedField(schema, eventTimeFieldName); + // Field should definitely exist. + eventTimeValue = record.convertColumnValueForLogicalType( + field.get().schema(), eventTimeValue, keepConsistentLogicalTimestamp); + int length = eventTimeValue.toString().length(); + Long millisEventTime = null; + if (length == 10) { + millisEventTime = Long.parseLong(eventTimeValue.toString()) * 1000; + } else if (length == 13) { + // eventTimeVal in millis unit + millisEventTime = Long.parseLong(eventTimeValue.toString()); + } + long eventTime = DateTimeUtils.parseDateTime(Long.toString(millisEventTime)).toEpochMilli(); + expectedMinValue.set(Math.min(expectedMinValue.get(), eventTime)); + expectedMaxValue.set(Math.max(expectedMaxValue.get(), eventTime)); + } + }); + assertEquals(expectedMinValue.get(), actualMinEventTime, "Min event time does not match"); + assertEquals(expectedMaxValue.get(), actualMaxEventTime, "Max event time does not match"); + } + + private List initialWrite(HoodieWriteConfig config, HoodieTestDataGenerator dataGenerator, String payloadClass, String partitionPath) { + List insertRecords = null; + try (SparkRDDWriteClient client = getHoodieWriteClient(config)) { + String instantTime = client.startCommit(); + insertRecords = dataGenerator.generateInserts(instantTime, 10); + insertRecords = overrideOrderingValue(insertRecords, config, payloadClass, partitionPath, 5L); + JavaRDD writeRecords = jsc.parallelize(insertRecords, 1); + JavaRDD statuses = client.upsert(writeRecords, instantTime); + client.commit(instantTime, statuses, Option.empty(), COMMIT_ACTION, Collections.emptyMap(), Option.empty()); + } + return insertRecords; + } + + private InputAndExpectedDataSet prepareInputFor2ndBatch(HoodieWriteConfig config, HoodieTestDataGenerator dataGenerator, String payloadClass, + String partitionPath, String mergeMode, List recordsBatch1, + String instantTime, HoodieFileGroup fileGroup) { + List recordsToDelete = new ArrayList<>(); + Map validDeletes = new HashMap<>(); + List recordsToUpdate = new ArrayList<>(); + List validUpdates = new ArrayList<>(); + List newInserts = new ArrayList<>(); + int expectedUpdates = 0; + int expectedDeletes = 0; + + // Generate records to delete + List newRecords = dataGenerator.generateUniqueUpdates(instantTime, 5); + HoodieRecord deleteRecordSameOrderingValue = generateDeletes(Collections.singletonList(newRecords.get(2)), config, payloadClass, partitionPath, 10L).get(0); + HoodieRecord deleteRecordHigherOrderingValue = generateDeletes(Collections.singletonList(newRecords.get(3)), config, payloadClass, partitionPath, 20L).get(0); + HoodieRecord deleteRecordLowerOrderingValue = generateDeletes(Collections.singletonList(newRecords.get(4)), config, payloadClass, partitionPath, 2L).get(0); + recordsToDelete.add(deleteRecordSameOrderingValue); + recordsToDelete.add(deleteRecordLowerOrderingValue); + recordsToDelete.add(deleteRecordHigherOrderingValue); + + // Known Gap HUDI-9715: Currently the ordering provided by the custom mergers does not apply deletes. + validDeletes.put(deleteRecordSameOrderingValue.getRecordKey(), deleteRecordSameOrderingValue); + validDeletes.put(deleteRecordHigherOrderingValue.getRecordKey(), deleteRecordHigherOrderingValue); + expectedDeletes += 2; + + if (mergeMode.equals(RecordMergeMode.COMMIT_TIME_ORDERING.name())) { + validDeletes.put(deleteRecordLowerOrderingValue.getRecordKey(), deleteRecordLowerOrderingValue); + expectedDeletes += 1; + } + + // Generate records to update + GenericRecord genericRecord1 = getGenRecord(newRecords.get(0), config); + GenericRecord genericRecord2 = getGenRecord(newRecords.get(1), config); + genericRecord1.put(ORDERING_FIELD, 20L); + genericRecord2.put(ORDERING_FIELD, 2L); + recordsToUpdate.add(genericRecord1); + recordsToUpdate.add(genericRecord2); + List hoodieRecordsToUpdate = getHoodieRecords(payloadClass, recordsToUpdate, partitionPath); + if (!mergeMode.equals("CUSTOM_MERGER") && !mergeMode.equals("CUSTOM")) { + // Custom merger chooses record with lower ordering value + validUpdates.add(hoodieRecordsToUpdate.get(0)); + expectedUpdates += 1; + } + if (!mergeMode.equals(RecordMergeMode.EVENT_TIME_ORDERING.name())) { + validUpdates.add(hoodieRecordsToUpdate.get(1)); + expectedUpdates += 1; + } + + List recordsToMerge = hoodieRecordsToUpdate; + recordsToMerge.addAll(recordsToDelete); + // Generate records to insert + List recordsToInsert2 = dataGenerator.generateInserts(instantTime, 2); + recordsToInsert2 = overrideOrderingValue(recordsToInsert2, config, payloadClass, partitionPath, 15L); + recordsToMerge.addAll(recordsToInsert2); + newInserts.addAll(recordsToInsert2); + + // let's compute the expected record list + Map expectedRecordsMap = new HashMap<>(); + validUpdates.forEach(rec -> { + expectedRecordsMap.put(rec.getRecordKey(), rec); + }); + recordsBatch1.forEach(record -> { + // if not part of new update, if not valid delete, add records from 1st batch. + String recKey = record.getRecordKey(); + if (!expectedRecordsMap.containsKey(recKey) && !validDeletes.containsKey(recKey)) { + expectedRecordsMap.put(recKey, record); + } + }); + // add new inserts. + newInserts.forEach(record -> { + expectedRecordsMap.put(record.getRecordKey(), record); + }); + + return new InputAndExpectedDataSet(expectedRecordsMap, expectedUpdates, expectedDeletes, recordsToMerge, newInserts, validUpdates, validDeletes); + } + + HoodieWriteConfig.Builder getHoodieWriteConfigBuilder() { + return getConfigBuilder(basePath) + .withPopulateMetaFields(true) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder().withRemoteServerPort(timelineServicePort).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder() + .enable(true) + .withEnableRecordIndex(true) + .withStreamingWriteEnabled(true) + .withSecondaryIndexEnabled(true) + .withSecondaryIndexName("sec-rider") + .withSecondaryIndexForColumn("rider") + .build()) + .withKeyGenerator(KeyGeneratorForDataGeneratorRecords.class.getCanonicalName()) + .withSchema(TRIP_EXAMPLE_SCHEMA); + } + + private List overrideOrderingValue(List hoodieRecords, HoodieWriteConfig config, String payloadClass, String partitionPath, long orderingValue) { + + List genericRecords = hoodieRecords.stream().map(insertRecord -> { + try { + GenericRecord genericRecord = (GenericRecord) ((HoodieRecordPayload) insertRecord.getData()).getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA, config.getProps()).get(); + genericRecord.put(ORDERING_FIELD, orderingValue); + return genericRecord; + } catch (IOException e) { + throw new HoodieIOException("Failed to deser ", e); + } + }).collect(Collectors.toList()); + + return getHoodieRecords(payloadClass, genericRecords, partitionPath); + } + + private List generateDeletes(List hoodieRecords, HoodieWriteConfig config, String payloadClass, String partitionPath, long orderingValue) { + List genericRecords = hoodieRecords.stream().map(deleteRecord -> { + try { + GenericRecord genericRecord = (GenericRecord) ((HoodieRecordPayload) deleteRecord.getData()).getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA, config.getProps()).get(); + genericRecord.put(ORDERING_FIELD, orderingValue); + genericRecord.put(HoodieRecord.HOODIE_IS_DELETED_FIELD, true); + return genericRecord; + } catch (IOException e) { + throw new HoodieIOException("Failed to deser ", e); + } + }).collect(Collectors.toList()); + return getHoodieRecords(payloadClass, genericRecords, partitionPath); + } + + private GenericRecord getGenRecord(HoodieRecord hoodieRecord, HoodieWriteConfig config) { + try { + return (GenericRecord) ((HoodieRecordPayload) hoodieRecord.getData()).getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA, config.getProps()).get(); + } catch (IOException e) { + throw new HoodieIOException("Failed to deser record ", e); + } + } + + private List getHoodieRecords(String payloadClass, List genericRecords, String partitionPath) { + return genericRecords.stream().map(genericRecord -> { + return (HoodieRecord) new HoodieAvroRecord<>(new HoodieKey(genericRecord.get("_row_key").toString(), partitionPath), + HoodieRecordUtils.loadPayload(payloadClass, genericRecord, (Comparable) genericRecord.get(ORDERING_FIELD))); + }).collect(Collectors.toList()); + } + + private void setCurLocation(List records, String fileId, String instantTime) { + records.forEach(record -> record.setCurrentLocation(new HoodieRecordLocation(instantTime, fileId))); + } + + private static void validateWriteStatus(WriteStatus writeStatus, String previousCommit, long expectedTotalRecordsWritten, long expectedTotalUpdatedRecords, + long expectedTotalInsertedRecords, long expectedTotalDeletedRecords) { + HoodieWriteStat writeStat = writeStatus.getStat(); + assertEquals(previousCommit, writeStat.getPrevCommit()); + assertNotNull(writeStat.getFileId()); + assertNotNull(writeStat.getPath()); + assertTrue(writeStat.getFileSizeInBytes() > 0); + assertTrue(writeStat.getTotalWriteBytes() > 0); + assertTrue(writeStat.getTotalLogBlocks() == 0); + assertTrue(writeStat.getTotalLogSizeCompacted() == 0); + assertTrue(writeStat.getTotalLogFilesCompacted() == 0); + assertTrue(writeStat.getTotalLogRecords() == 0); + assertEquals(expectedTotalRecordsWritten, writeStat.getNumWrites()); + assertEquals(expectedTotalUpdatedRecords, writeStat.getNumUpdateWrites()); + assertEquals(expectedTotalInsertedRecords, writeStat.getNumInserts()); + assertEquals(expectedTotalDeletedRecords, writeStat.getNumDeletes()); + } + + class InputAndExpectedDataSet { + private final Map expectedRecordsMap; + private final int expectedUpdates; + private final int expectedDeletes; + private final List recordsToMerge; + private final List newInserts; + private final List validUpdates; + private final Map validDeletes; + + public InputAndExpectedDataSet(Map expectedRecordsMap, int expectedUpdates, int expectedDeletes, + List recordsToMerge, List newInserts, List validUpdates, + Map validDeletes) { + this.expectedRecordsMap = expectedRecordsMap; + this.expectedUpdates = expectedUpdates; + this.expectedDeletes = expectedDeletes; + this.recordsToMerge = recordsToMerge; + this.validUpdates = validUpdates; + this.newInserts = newInserts; + this.validDeletes = validDeletes; + } + + public Map getExpectedRecordsMap() { + return expectedRecordsMap; + } + + public int getExpectedUpdates() { + return expectedUpdates; + } + + public int getExpectedDeletes() { + return expectedDeletes; + } + + public List getRecordsToMerge() { + return recordsToMerge; + } + + public List getNewInserts() { + return newInserts; + } + + public List getValidUpdates() { + return validUpdates; + } + + public Map getValidDeletes() { + return validDeletes; + } + } + + public static class CustomMerger implements HoodieRecordMerger { + private static final String STRATEGY_ID = UUID.randomUUID().toString(); + + public static String getStrategyId() { + return STRATEGY_ID; + } + + @Override + public Option> merge(HoodieRecord older, Schema oldSchema, HoodieRecord newer, Schema newSchema, TypedProperties props) throws IOException { + GenericRecord olderData = (GenericRecord) older.getData(); + GenericRecord newerData = (GenericRecord) newer.getData(); + Long olderTimestamp = (Long) olderData.get("timestamp"); + Long newerTimestamp = (Long) newerData.get("timestamp"); + if (olderTimestamp.equals(newerTimestamp)) { + // If the timestamps are the same, we do not update + return Option.of(Pair.of(older, oldSchema)); + } else if (olderTimestamp < newerTimestamp) { + // Custom merger chooses record with lower ordering value + return Option.of(Pair.of(older, oldSchema)); + } else { + // Custom merger chooses record with lower ordering value + return Option.of(Pair.of(newer, newSchema)); + } + } + + @Override + public HoodieRecord.HoodieRecordType getRecordType() { + return HoodieRecord.HoodieRecordType.AVRO; + } + + @Override + public String getMergingStrategy() { + return STRATEGY_ID; + } + } + + public static class CustomPayload implements HoodieRecordPayload { + private final GenericRecord record; + + public CustomPayload(GenericRecord record, Comparable orderingValue) { + this.record = record; + } + + @Override + public CustomPayload preCombine(CustomPayload other) { + return this; // No-op for this test + } + + @Override + public Option combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException { + Long olderTimestamp = (Long) ((GenericRecord) currentValue).get("timestamp"); + Long newerTimestamp = (Long) record.get("timestamp"); + if (olderTimestamp.equals(newerTimestamp)) { + // If the timestamps are the same, we do not update + return Option.of(currentValue); + } else if (olderTimestamp < newerTimestamp) { + // Custom merger chooses record with lower ordering value + return Option.of(currentValue); + } else { + // Custom merger chooses record with lower ordering value + return Option.of(record); + } + } + + @Override + public Option getInsertValue(Schema schema) throws IOException { + return Option.of(record); + } + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java index 4012293a2bca4..04b10818a3cfa 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java @@ -100,6 +100,7 @@ import static org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings; import static org.apache.hudi.config.HoodieWriteConfig.WRITE_TABLE_VERSION; import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; +import static org.apache.hudi.testutils.DataSourceTestUtils.validateCommitMetadata; import static org.apache.hudi.testutils.HoodieSparkClientTestHarness.buildProfile; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -418,7 +419,7 @@ public void testLogFileCountsAfterCompaction(IndexType indexType) throws Excepti expectedInserts = 0; expectedUpdates = 80; } - validateCompactionMetadata(compactionMetadata, previousCommit, 90, expectedUpdates, expectedInserts, 10); + validateCommitMetadata(compactionMetadata, previousCommit, 90, expectedUpdates, expectedInserts, 10); // Verify that recently written compacted data file has no log file metaClient = HoodieTableMetaClient.reload(metaClient); @@ -521,7 +522,7 @@ public void testLogBlocksCountsAfterLogCompaction(boolean populateMetaFields, St String logCompactionInstantTime = writeClient.scheduleLogCompaction(Option.empty()).get().toString(); HoodieWriteMetadata> result = writeClient.logCompact(logCompactionInstantTime, true); HoodieCommitMetadata compactionMetadata = metaClient.getActiveTimeline().readCommitMetadata(metaClient.getActiveTimeline().reload().getCommitsAndCompactionTimeline().lastInstant().get()); - validateCompactionMetadata(compactionMetadata, firstCommitTime, 80, 80, 0, 10); + validateCommitMetadata(compactionMetadata, firstCommitTime, 80, 80, 0, 10); validateLogCompactionMetadataHeaders(compactionMetadata, metaClient.getBasePath(), "102,101"); // Verify that recently written compacted data file has no log file @@ -600,33 +601,6 @@ private static void validateWriteStatuses(List statuses, long expec assertEquals(expectedDeletes, totalDeletes); } - private static void validateCompactionMetadata(HoodieCommitMetadata compactionMetadata, String previousCommit, long expectedTotalRecordsWritten, long expectedTotalUpdatedRecords, - long expectedTotalInsertedRecords, long expectedTotalDeletedRecords) { - long totalRecordsWritten = 0; - long totalDeletedRecords = 0; - long totalUpdatedRecords = 0; - long totalInsertedRecords = 0; - for (HoodieWriteStat writeStat : compactionMetadata.getWriteStats()) { - totalRecordsWritten += writeStat.getNumWrites(); - totalDeletedRecords += writeStat.getNumDeletes(); - totalUpdatedRecords += writeStat.getNumUpdateWrites(); - totalInsertedRecords += writeStat.getNumInserts(); - assertEquals(previousCommit, writeStat.getPrevCommit()); - assertNotNull(writeStat.getFileId()); - assertNotNull(writeStat.getPath()); - assertTrue(writeStat.getFileSizeInBytes() > 0); - assertTrue(writeStat.getTotalWriteBytes() > 0); - assertTrue(writeStat.getTotalLogBlocks() > 0); - assertTrue(writeStat.getTotalLogSizeCompacted() > 0); - assertTrue(writeStat.getTotalLogFilesCompacted() > 0); - assertTrue(writeStat.getTotalLogRecords() > 0); - } - assertEquals(expectedTotalRecordsWritten, totalRecordsWritten); - assertEquals(expectedTotalUpdatedRecords, totalUpdatedRecords); - assertEquals(expectedTotalInsertedRecords, totalInsertedRecords); - assertEquals(expectedTotalDeletedRecords, totalDeletedRecords); - } - private void validateLogCompactionMetadataHeaders(HoodieCommitMetadata compactionMetadata, StoragePath basePath, String expectedCompactedBlockTimes) { compactionMetadata.getFileIdAndFullPaths(basePath).values().stream() .map(StoragePath::new) diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/testutils/DataSourceTestUtils.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/testutils/DataSourceTestUtils.java index 1daa65b4ab86d..f8ba80496fb6e 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/testutils/DataSourceTestUtils.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/testutils/DataSourceTestUtils.java @@ -19,6 +19,9 @@ package org.apache.hudi.testutils; import org.apache.hudi.HoodieDataSourceHelpers; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; @@ -48,6 +51,9 @@ import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * Test utils for data source tests. @@ -204,4 +210,33 @@ public static String latestDeltaCommitRequest(HoodieStorage storage, String base .filter(instant -> HoodieTimeline.DELTA_COMMIT_ACTION.equals(instant.getAction())) .lastInstant().map(instant -> instant.requestedTime()).orElse(null); } + + public static void validateCommitMetadata(HoodieCommitMetadata commitMetadata, String previousCommit, long expectedTotalRecordsWritten, long expectedTotalUpdatedRecords, + long expectedTotalInsertedRecords, long expectedTotalDeletedRecords) { + long totalRecordsWritten = 0; + long totalDeletedRecords = 0; + long totalUpdatedRecords = 0; + long totalInsertedRecords = 0; + for (HoodieWriteStat writeStat : commitMetadata.getWriteStats()) { + totalRecordsWritten += writeStat.getNumWrites(); + totalDeletedRecords += writeStat.getNumDeletes(); + totalUpdatedRecords += writeStat.getNumUpdateWrites(); + totalInsertedRecords += writeStat.getNumInserts(); + assertEquals(previousCommit, writeStat.getPrevCommit()); + assertNotNull(writeStat.getFileId()); + assertNotNull(writeStat.getPath()); + assertTrue(writeStat.getFileSizeInBytes() > 0); + assertTrue(writeStat.getTotalWriteBytes() > 0); + if (commitMetadata.getOperationType() == WriteOperationType.COMPACT) { + assertTrue(writeStat.getTotalLogBlocks() > 0); + assertTrue(writeStat.getTotalLogSizeCompacted() > 0); + assertTrue(writeStat.getTotalLogFilesCompacted() > 0); + assertTrue(writeStat.getTotalLogRecords() > 0); + } + } + assertEquals(expectedTotalRecordsWritten, totalRecordsWritten); + assertEquals(expectedTotalUpdatedRecords, totalUpdatedRecords); + assertEquals(expectedTotalInsertedRecords, totalInsertedRecords); + assertEquals(expectedTotalDeletedRecords, totalDeletedRecords); + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala index 78ca3d149ed45..793c7ef657e1a 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala @@ -1000,7 +1000,8 @@ def testBulkInsertForDropPartitionColumn(): Unit = { HoodieWriteConfig.TBL_NAME.key -> "hoodie_test", HoodieWriteConfig.COMBINE_BEFORE_UPSERT.key -> "false", "hoodie.insert.shuffle.parallelism" -> "1", - "hoodie.upsert.shuffle.parallelism" -> "1" + "hoodie.upsert.shuffle.parallelism" -> "1", + HoodieWriteConfig.MERGE_HANDLE_CLASS_NAME.key -> "org.apache.hudi.io.HoodieWriteMergeHandle" ) val df = spark.range(0, 10).toDF("keyid") diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala index 9c5d34d0960e0..468f2f11e97cc 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala @@ -733,6 +733,73 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup assertEquals(snapshotDF2.count(), 80) } + @Test + def testCopyOnWriteUpserts(): Unit = { + val recordType = HoodieRecordType.AVRO + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + + // Insert Operation + val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).asScala.toList + val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) + inputDF1.write.format("org.apache.hudi") + .options(writeOpts) + .mode(SaveMode.Overwrite) + .save(basePath) + + assertTrue(HoodieDataSourceHelpers.hasNewCommits(storage, basePath, "000")) + + val snapshotDF1 = spark.read.format("org.apache.hudi").options(readOpts).load(basePath) + assertEquals(100, snapshotDF1.count()) + + val records2 = recordsToStrings(dataGen.generateUniqueUpdates("001", 20)).asScala.toList + val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2)) + + inputDF2.write.format("org.apache.hudi") + .options(writeOpts) + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) + .option("hoodie.write.merge.handle.class", "org.apache.hudi.io.FileGroupReaderBasedMergeHandle") + .mode(SaveMode.Append) + .save(basePath) + val metaClient = HoodieTableMetaClient.builder().setBasePath(basePath).setConf(storageConf).build() + val timeline = metaClient.reloadActiveTimeline() + val firstCommit = timeline.getCommitTimeline.getInstants.get(timeline.countInstants() - 2) + val secondCommit = timeline.getCommitTimeline.getInstants.get(timeline.countInstants() - 1) + val commitMetadata = timeline.readCommitMetadata(secondCommit) + DataSourceTestUtils.validateCommitMetadata(commitMetadata, firstCommit.requestedTime(), 100, 20, 0, 0) + + val snapshotDF2 = spark.read.format("org.apache.hudi").options(readOpts).load(basePath) + .drop(HoodieRecord.HOODIE_META_COLUMNS.asScala.toSeq: _*) + snapshotDF2.cache() + assertEquals(snapshotDF2.count(), 100) + + assertEquals(0, inputDF2.except(snapshotDF2).count()) + + val updates3 = recordsToStrings(dataGen.generateUniqueUpdates("002", 5)).asScala.toList + val inserts3 = recordsToStrings(dataGen.generateInserts("002", 5)).asScala.toList + val inputDF3 = spark.read.json(spark.sparkContext.parallelize(updates3, 1) + .union(spark.sparkContext.parallelize(inserts3, 1))) + inputDF3.cache() + + inputDF3.write.format("org.apache.hudi") + .options(writeOpts) + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.DELETE_OPERATION_OPT_VAL) + .mode(SaveMode.Append) + .save(basePath) + + val snapshotDF3 = spark.read.format("org.apache.hudi").options(readOpts).load(basePath) + .drop(HoodieRecord.HOODIE_META_COLUMNS.asScala.toSeq: _*) + snapshotDF3.cache() + assertEquals(snapshotDF3.count(), 95) + assertEquals(inputDF3.count(), inputDF3.except(snapshotDF3).count()) // none of the deleted records should be part of snapshot read + val counts = spark.read.format("org.apache.hudi").options(readOpts).load(basePath) + .groupBy("_hoodie_commit_time").count().orderBy("_hoodie_commit_time").collect() + // validate the commit time metadata is not updated for the update operation + assertEquals(2, counts.length) + assertEquals(firstCommit.requestedTime(), counts.apply(0).getAs[String](0)) + assertEquals(secondCommit.requestedTime(), counts.apply(1).getAs[String](0)) + assertTrue(counts.apply(0).getAs[Long](1) > counts.apply(1).getAs[Long](1)) + } + /** * Test retries on conflict failures. */ diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndex.scala index 5127924ac08f9..dc962e8a593a6 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndex.scala @@ -35,15 +35,19 @@ import org.apache.hudi.metadata.{HoodieBackedTableMetadata, MetadataPartitionTyp import org.apache.hudi.util.JavaConversions import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema import org.apache.spark.sql.functions.lit +import org.apache.spark.sql.types.{StringType, StructField, StructType} import org.junit.jupiter.api._ import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{Arguments, CsvSource, EnumSource, MethodSource} import org.junit.jupiter.params.provider.Arguments.arguments +import java.util import java.util.{Collections, Properties} import java.util.concurrent.Executors +import java.util.stream.Collectors import scala.collection.JavaConverters._ import scala.concurrent.{Await, ExecutionContext, Future} @@ -278,23 +282,56 @@ class TestRecordLevelIndex extends RecordLevelIndexTestBase { @ParameterizedTest @EnumSource(classOf[HoodieTableType]) def testRLIForDeletesWithHoodieIsDeletedColumn(tableType: HoodieTableType): Unit = { - val hudiOpts = commonOpts + (DataSourceWriteOptions.TABLE_TYPE.key -> tableType.name()) + val hudiOpts = commonOpts + (DataSourceWriteOptions.TABLE_TYPE.key -> tableType.name()) + + (HoodieIndexConfig.INDEX_TYPE.key -> "RECORD_INDEX") + + (HoodieIndexConfig.RECORD_INDEX_UPDATE_PARTITION_PATH_ENABLE.key -> "true") val insertDf = doWriteAndValidateDataAndRecordIndex(hudiOpts, operation = DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL, saveMode = SaveMode.Overwrite) insertDf.cache() - val deleteBatch = recordsToStrings(dataGen.generateUniqueDeleteRecords(getNewInstantTime, 1)).asScala + val instantTime = getNewInstantTime + // Issue two deletes, one with the original partition and one with an updated partition + val recordsToDelete = dataGen.generateUniqueDeleteRecords(instantTime, 1) + recordsToDelete.addAll(dataGen.generateUniqueDeleteRecordsWithUpdatedPartition(instantTime, 1)) + val deleteBatch = recordsToStrings(recordsToDelete).asScala val deleteDf = spark.read.json(spark.sparkContext.parallelize(deleteBatch.toSeq, 1)) deleteDf.cache() - val recordKeyToDelete = deleteDf.collectAsList().get(0).getAs("_row_key").asInstanceOf[String] + val recordKeyToDelete1 = deleteDf.collectAsList().get(0).getAs("_row_key").asInstanceOf[String] + val recordKeyToDelete2 = deleteDf.collectAsList().get(1).getAs("_row_key").asInstanceOf[String] deleteDf.write.format("org.apache.hudi") .options(hudiOpts) .mode(SaveMode.Append) .save(basePath) val prevDf = mergedDfList.last - mergedDfList = mergedDfList :+ prevDf.filter(row => row.getAs("_row_key").asInstanceOf[String] != recordKeyToDelete) + mergedDfList = mergedDfList :+ prevDf.filter(row => row.getAs("_row_key").asInstanceOf[String] != recordKeyToDelete1 && + row.getAs("_row_key").asInstanceOf[String] != recordKeyToDelete2) + validateDataAndRecordIndices(hudiOpts, deleteDf) + deleteDf.unpersist() + } + + @ParameterizedTest + @EnumSource(classOf[HoodieTableType]) + def testRLIForDeletesWithSQLDelete(tableType: HoodieTableType): Unit = { + val hudiOpts = commonOpts + (DataSourceWriteOptions.TABLE_TYPE.key -> tableType.name()) + val insertDf = doWriteAndValidateDataAndRecordIndex(hudiOpts, + operation = DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL, + saveMode = SaveMode.Overwrite) + insertDf.cache() + + spark.sql(s"CREATE TABLE IF NOT EXISTS hudi_indexed_table USING hudi OPTIONS (hoodie.metadata.enable = 'true', hoodie.metadata.record.index.enable = 'true', hoodie.write.merge.handle.class = 'org.apache.hudi.io.FileGroupReaderBasedMergeHandle') LOCATION '$basePath'") + val existingKeys = dataGen.getExistingKeys + spark.sql(s"DELETE FROM hudi_indexed_table WHERE _row_key IN ('${existingKeys.get(0)}', '${existingKeys.get(1)}')") + + val prevDf = mergedDfList.last + mergedDfList = mergedDfList :+ prevDf.filter(row => row.getAs("_row_key").asInstanceOf[String] != existingKeys.get(0) && + row.getAs("_row_key").asInstanceOf[String] != existingKeys.get(1)) + val structType = new StructType(Array(StructField("_row_key", StringType))) + val convertToRow: Function[String, Row] = key => new GenericRowWithSchema(Array(key), structType) + val rows: java.util.List[Row] = util.Arrays.asList(convertToRow.apply(existingKeys.get(0)), convertToRow.apply(existingKeys.get(1))) + val deleteDf = spark.createDataFrame(rows, structType) validateDataAndRecordIndices(hudiOpts, deleteDf) + deleteDf.unpersist() } @ParameterizedTest diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCDataFrameSuite.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCDataFrameSuite.scala index 1416470fec40d..e9107a90e6ca0 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCDataFrameSuite.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCDataFrameSuite.scala @@ -54,10 +54,16 @@ class TestCDCDataFrameSuite extends HoodieCDCTestBase { * Step8: Bulk_Insert 20 */ @ParameterizedTest - @EnumSource(classOf[HoodieCDCSupplementalLoggingMode]) - def testCOWDataSourceWrite(loggingMode: HoodieCDCSupplementalLoggingMode): Unit = { + @CsvSource(Array("OP_KEY_ONLY,org.apache.hudi.io.HoodieWriteMergeHandle", + "DATA_BEFORE,org.apache.hudi.io.HoodieWriteMergeHandle", + "DATA_BEFORE_AFTER,org.apache.hudi.io.HoodieWriteMergeHandle", + "OP_KEY_ONLY,org.apache.hudi.io.FileGroupReaderBasedMergeHandle", + "DATA_BEFORE,org.apache.hudi.io.FileGroupReaderBasedMergeHandle", + "DATA_BEFORE_AFTER,org.apache.hudi.io.FileGroupReaderBasedMergeHandle")) + def testCOWDataSourceWrite(loggingMode: HoodieCDCSupplementalLoggingMode, mergeHandleClassName: String): Unit = { val options = commonOpts ++ Map( - HoodieTableConfig.CDC_SUPPLEMENTAL_LOGGING_MODE.key -> loggingMode.name() + HoodieTableConfig.CDC_SUPPLEMENTAL_LOGGING_MODE.key -> loggingMode.name(), + HoodieWriteConfig.MERGE_HANDLE_CLASS_NAME.key() -> mergeHandleClassName ) var totalInsertedCnt = 0L diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestMergeIntoTable2.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestMergeIntoTable2.scala index 13b8def272b09..56a1fd916096b 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestMergeIntoTable2.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/others/TestMergeIntoTable2.scala @@ -26,6 +26,7 @@ import org.apache.hudi.testutils.HoodieClientTestUtils.createMetaClient import org.apache.spark.sql.Row import org.apache.spark.sql.hudi.common.HoodieSparkSqlTestBase +import org.junit.jupiter.api.Disabled import org.slf4j.LoggerFactory class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { @@ -838,7 +839,10 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { } } - test("Test only insert for source table in dup key without preCombineField") { + /** + * This test relies on duplicate entries with a record key and will be re-enabled as part of HUDI-9708 + */ + ignore("Test only insert for source table in dup key without preCombineField") { spark.sql(s"set ${MERGE_SMALL_FILE_GROUP_CANDIDATES_LIMIT.key} = ${MERGE_SMALL_FILE_GROUP_CANDIDATES_LIMIT.defaultValue()}") Seq("cow", "mor").foreach { tableType => { diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestGcsEventsHoodieIncrSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestGcsEventsHoodieIncrSource.java index 58be73acebe14..fd526f63f22a7 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestGcsEventsHoodieIncrSource.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestGcsEventsHoodieIncrSource.java @@ -99,7 +99,7 @@ public class TestGcsEventsHoodieIncrSource extends SparkClientFunctionalTestHarness { private static final Schema GCS_METADATA_SCHEMA = SchemaTestUtil.getSchemaFromResource( - TestGcsEventsHoodieIncrSource.class, "/streamer-config/gcs-metadata.avsc", true); + TestGcsEventsHoodieIncrSource.class, "/streamer-config/gcs-metadata.avsc", false); private static final String IGNORE_FILE_EXTENSION = ".ignore"; private ObjectMapper mapper = new ObjectMapper(); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java index 08e73d36bc044..b4fc6ad4c8f85 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java @@ -138,7 +138,7 @@ protected static Stream fetchNextBatch(TypedProperties props, int if (!reachedMax && numUpdates >= 50) { LOG.info("After adjustments => NumInserts={}, NumUpdates={}, NumDeletes=50, maxUniqueRecords={}", numInserts, (numUpdates - 50), maxUniqueKeys); // if we generate update followed by deletes -> some keys in update batch might be picked up for deletes. Hence generating delete batch followed by updates - deleteStream = dataGenerator.generateUniqueDeleteRecordStream(instantTime, 50).map(AbstractBaseTestSource::toGenericRecord); + deleteStream = dataGenerator.generateUniqueDeleteRecordStream(instantTime, 50, false).map(AbstractBaseTestSource::toGenericRecord); updateStream = dataGenerator.generateUniqueUpdatesStream(instantTime, numUpdates - 50, HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) .map(AbstractBaseTestSource::toGenericRecord); } else {