diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java index 56e00aa24cd7c..22f70480a3f4d 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java @@ -217,6 +217,7 @@ public String showLogFileRecords( .withSpillableMapBasePath(HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH.defaultValue()) .withDiskMapType(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue()) .withBitCaskDiskMapCompressionEnabled(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue()) + .withUseScanV2(Boolean.parseBoolean(HoodieCompactionConfig.USE_LOG_RECORD_READER_SCAN_V2.defaultValue())) .build(); for (HoodieRecord hoodieRecord : scanner) { Option record = hoodieRecord.getData().getInsertValue(readerSchema); diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java index e93ad0c8cad4e..25d54b648715f 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java @@ -228,6 +228,7 @@ public void testShowLogFileRecordsWithMerge() throws IOException, InterruptedExc .withSpillableMapBasePath(HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH.defaultValue()) .withDiskMapType(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue()) .withBitCaskDiskMapCompressionEnabled(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue()) + .withUseScanV2(Boolean.parseBoolean(HoodieCompactionConfig.USE_LOG_RECORD_READER_SCAN_V2.defaultValue())) .build(); Iterator> records = scanner.iterator(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java index d9f260e633cfb..146889c8dbfaf 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java @@ -141,6 +141,7 @@ public abstract class BaseHoodieWriteClient> extraMetadata) { protected void rollbackFailedBootstrap() { LOG.info("Rolling back pending bootstrap if present"); HoodieTable table = createTable(config, hadoopConf); - HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction(); + HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingMajorAndMinorCompaction(); Option instant = Option.fromJavaOptional( inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp).findFirst()); if (instant.isPresent() && HoodieTimeline.compareTimestamps(instant.get(), HoodieTimeline.LESSER_THAN_OR_EQUALS, @@ -562,6 +563,15 @@ protected void runTableServicesInline(HoodieTable table, HoodieCommitMetadata me inlineScheduleCompaction(extraMetadata); } + // Do an inline log compaction if enabled + if (config.inlineLogCompactionEnabled()) { + runAnyPendingLogCompactions(table); + metadata.addMetadata(HoodieCompactionConfig.INLINE_LOG_COMPACT.key(), "true"); + inlineLogCompact(extraMetadata); + } else { + metadata.addMetadata(HoodieCompactionConfig.INLINE_LOG_COMPACT.key(), "false"); + } + // Do an inline clustering if enabled if (config.inlineClusteringEnabled()) { runAnyPendingClustering(table); @@ -589,6 +599,14 @@ protected void runAnyPendingCompactions(HoodieTable table) { }); } + protected void runAnyPendingLogCompactions(HoodieTable table) { + table.getActiveTimeline().getWriteTimeline().filterPendingLogCompactionTimeline().getInstants() + .forEach(instant -> { + LOG.info("Running previously failed inflight log compaction at instant " + instant); + logCompact(instant.getTimestamp(), true); + }); + } + protected void runAnyPendingClustering(HoodieTable table) { table.getActiveTimeline().filterPendingReplaceTimeline().getInstants().forEach(instant -> { Option> instantPlan = ClusteringUtils.getClusteringPlan(table.getMetaClient(), instant); @@ -1077,13 +1095,60 @@ public abstract void commitCompaction(String compactionInstantTime, HoodieCommit */ protected abstract void completeCompaction(HoodieCommitMetadata metadata, HoodieTable table, String compactionCommitTime); + /** + * Schedules a new log compaction instant. + * @param extraMetadata Extra Metadata to be stored + */ + public Option scheduleLogCompaction(Option> extraMetadata) throws HoodieIOException { + String instantTime = HoodieActiveTimeline.createNewInstantTime(); + return scheduleLogCompactionAtInstant(instantTime, extraMetadata) ? Option.of(instantTime) : Option.empty(); + } + + /** + * Schedules a new log compaction instant with passed-in instant time. + * @param instantTime Log Compaction Instant Time + * @param extraMetadata Extra Metadata to be stored + */ + public boolean scheduleLogCompactionAtInstant(String instantTime, Option> extraMetadata) throws HoodieIOException { + return scheduleTableService(instantTime, extraMetadata, TableServiceType.LOG_COMPACT).isPresent(); + } + + /** + * Performs Log Compaction for the workload stored in instant-time. + * + * @param logCompactionInstantTime Log Compaction Instant Time + * @return Collection of WriteStatus to inspect errors and counts + */ + public HoodieWriteMetadata logCompact(String logCompactionInstantTime) { + return logCompact(logCompactionInstantTime, config.shouldAutoCommit()); + } + + /** + * Commit a log compaction operation. Allow passing additional meta-data to be stored in commit instant file. + * + * @param logCompactionInstantTime Log Compaction Instant Time + * @param metadata All the metadata that gets stored along with a commit + * @param extraMetadata Extra Metadata to be stored + */ + public void commitLogCompaction(String logCompactionInstantTime, HoodieCommitMetadata metadata, + Option> extraMetadata) { + throw new UnsupportedOperationException("Log compaction is not supported yet."); + } + + /** + * Commit Log Compaction and track metrics. + */ + protected void completeLogCompaction(HoodieCommitMetadata metadata, HoodieTable table, String logCompactionCommitTime) { + throw new UnsupportedOperationException("Log compaction is not supported yet."); + } + /** * Get inflight time line exclude compaction and clustering. * @param metaClient * @return */ private HoodieTimeline getInflightTimelineExcludeCompactionAndClustering(HoodieTableMetaClient metaClient) { - HoodieTimeline inflightTimelineWithReplaceCommit = metaClient.getCommitsTimeline().filterPendingExcludingCompaction(); + HoodieTimeline inflightTimelineWithReplaceCommit = metaClient.getCommitsTimeline().filterPendingExcludingMajorAndMinorCompaction(); HoodieTimeline inflightTimelineExcludeClusteringCommit = inflightTimelineWithReplaceCommit.filter(instant -> { if (instant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)) { Option> instantPlan = ClusteringUtils.getClusteringPlan(metaClient, instant); @@ -1139,7 +1204,7 @@ protected Map> getPendingRollbackInfos try { String action = rollbackPlan.getInstantToRollback().getAction(); if (ignoreCompactionAndClusteringInstants) { - if (!HoodieTimeline.COMPACTION_ACTION.equals(action)) { + if (!HoodieTimeline.COMPACTION_ACTION.equals(action) && !HoodieTimeline.LOG_COMPACTION_ACTION.equals(action)) { boolean isClustering = HoodieTimeline.REPLACE_COMMIT_ACTION.equals(action) && ClusteringUtils.getClusteringPlan(metaClient, new HoodieInstant(true, rollbackPlan.getInstantToRollback().getAction(), rollbackPlan.getInstantToRollback().getCommitTime())).isPresent(); @@ -1253,6 +1318,28 @@ protected Option inlineScheduleCompaction(Option> ex return scheduleCompaction(extraMetadata); } + /** + * Ensures compaction instant is in expected state and performs Log Compaction for the workload stored in instant-time.s + * + * @param compactionInstantTime Compaction Instant Time + * @return Collection of Write Status + */ + protected HoodieWriteMetadata logCompact(String compactionInstantTime, boolean shouldComplete) { + throw new UnsupportedOperationException("Log compaction is not supported yet."); + } + + /** + * Performs a log compaction operation on a table, serially before or after an insert/upsert action. + */ + protected Option inlineLogCompact(Option> extraMetadata) { + Option logCompactionInstantTimeOpt = scheduleLogCompaction(extraMetadata); + logCompactionInstantTimeOpt.ifPresent(logCompactInstantTime -> { + // inline log compaction should auto commit as the user is never given control + logCompact(logCompactInstantTime, true); + }); + return logCompactionInstantTimeOpt; + } + /** * Schedules a new clustering instant. * @param extraMetadata Extra Metadata to be stored @@ -1348,6 +1435,11 @@ private Option scheduleTableServiceInternal(String instantTime, Option compactionPlan = createTable(config, hadoopConf) .scheduleCompaction(context, instantTime, extraMetadata); return compactionPlan.isPresent() ? Option.of(instantTime) : Option.empty(); + case LOG_COMPACT: + LOG.info("Scheduling log compaction at instant time :" + instantTime); + Option logCompactionPlan = createTable(config, hadoopConf) + .scheduleLogCompaction(context, instantTime, extraMetadata); + return logCompactionPlan.isPresent() ? Option.of(instantTime) : Option.empty(); case CLEAN: LOG.info("Scheduling cleaning at instant time :" + instantTime); Option cleanerPlan = createTable(config, hadoopConf) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentOperation.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentOperation.java index 35580229e3867..4e69b69cf4712 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentOperation.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentOperation.java @@ -36,6 +36,7 @@ import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_ACTION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.LOG_COMPACTION_ACTION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION; import static org.apache.hudi.common.util.CommitUtils.getFileIdWithoutSuffixAndRelativePathsFromSpecificRecord; @@ -149,6 +150,7 @@ private void init(HoodieInstant instant) { switch (getInstantActionType()) { case COMMIT_ACTION: case DELTA_COMMIT_ACTION: + case LOG_COMPACTION_ACTION: this.mutatedFileIds = CommitUtils.getFileIdWithoutSuffixAndRelativePaths(this.metadataWrapper.getCommitMetadata().getPartitionToWriteStats()).keySet(); this.operationType = this.metadataWrapper.getCommitMetadata().getOperationType(); break; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleConcurrentFileWritesConflictResolutionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleConcurrentFileWritesConflictResolutionStrategy.java index 938a40684a092..d17db6302b5da 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleConcurrentFileWritesConflictResolutionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleConcurrentFileWritesConflictResolutionStrategy.java @@ -94,8 +94,14 @@ public Option resolveConflict(HoodieTable table, // to consider it as conflict if we see overlapping file ids. Once concurrent updates are // supported for CLUSTER (https://issues.apache.org/jira/browse/HUDI-1042), // add that to the below check so that concurrent updates do not conflict. - if (otherOperation.getOperationType() == WriteOperationType.COMPACT - && HoodieTimeline.compareTimestamps(otherOperation.getInstantTimestamp(), HoodieTimeline.LESSER_THAN, thisOperation.getInstantTimestamp())) { + if (otherOperation.getOperationType() == WriteOperationType.COMPACT) { + if (HoodieTimeline.compareTimestamps(otherOperation.getInstantTimestamp(), HoodieTimeline.LESSER_THAN, thisOperation.getInstantTimestamp())) { + return thisOperation.getCommitMetadataOption(); + } + } else if (HoodieTimeline.LOG_COMPACTION_ACTION.equals(thisOperation.getInstantActionType())) { + // Since log compaction is a rewrite operation, it can be committed along with other delta commits. + // The ordering of the commits is taken care by AbstractHoodieLogRecordReader scan method. + // Conflict arises only if the log compaction commit has a lesser timestamp compared to compaction commit. return thisOperation.getCommitMetadataOption(); } // just abort the current write if conflicts are found diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MetadataConversionUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MetadataConversionUtils.java index 342de74a11395..08d9d34ba2e98 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MetadataConversionUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MetadataConversionUtils.java @@ -118,6 +118,12 @@ public static HoodieArchivedMetaEntry createMetaWrapper(HoodieInstant hoodieInst archivedMetaWrapper.setActionType(ActionType.compaction.name()); break; } + case HoodieTimeline.LOG_COMPACTION_ACTION: { + HoodieCompactionPlan plan = CompactionUtils.getLogCompactionPlan(metaClient, hoodieInstant.getTimestamp()); + archivedMetaWrapper.setHoodieCompactionPlan(plan); + archivedMetaWrapper.setActionType(ActionType.logcompaction.name()); + break; + } default: { throw new UnsupportedOperationException("Action not fully supported yet"); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java index b6fe6d8aa026f..bdb346bd86568 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java @@ -61,6 +61,12 @@ public class HoodieCompactionConfig extends HoodieConfig { + "but users are expected to trigger async job for execution. If `hoodie.compact.inline` is set to true, regular writers will do both scheduling and " + "execution inline for compaction"); + public static final ConfigProperty INLINE_LOG_COMPACT = ConfigProperty + .key("hoodie.log.compaction.inline") + .defaultValue("false") + .withDocumentation("When set to true, logcompaction service is triggered after each write. While being " + + " simpler operationally, this adds extra latency on the write path."); + public static final ConfigProperty INLINE_COMPACT_NUM_DELTA_COMMITS = ConfigProperty .key("hoodie.compact.inline.max.delta.commits") .defaultValue("5") @@ -173,6 +179,18 @@ public class HoodieCompactionConfig extends HoodieConfig { + "record size estimate compute dynamically based on commit metadata. " + " This is critical in computing the insert parallelism and bin-packing inserts into small files."); + public static final ConfigProperty LOG_COMPACTION_BLOCKS_THRESHOLD = ConfigProperty + .key("hoodie.log.compaction.blocks.threshold") + .defaultValue("5") + .withDocumentation("Log compaction can be scheduled if the no. of log blocks crosses this threshold value. " + + "This is effective only when log compaction is enabled via " + INLINE_LOG_COMPACT.key()); + + public static final ConfigProperty USE_LOG_RECORD_READER_SCAN_V2 = ConfigProperty + .key("hoodie.log.record.reader.use.scanV2") + .defaultValue("false") + .sinceVersion("0.13.0") + .withDocumentation("ScanV2 logic address all the multiwriter challenges while appending to log files. " + + "It also differentiates original blocks written by ingestion writers and compacted blocks written log compaction."); /** @deprecated Use {@link #INLINE_COMPACT} and its methods instead */ @Deprecated @@ -321,6 +339,11 @@ public Builder withScheduleInlineCompaction(Boolean scheduleAsyncCompaction) { return this; } + public Builder withInlineLogCompaction(Boolean inlineLogCompaction) { + compactionConfig.setValue(INLINE_LOG_COMPACT, String.valueOf(inlineLogCompaction)); + return this; + } + public Builder withInlineCompactionTriggerStrategy(CompactionTriggerStrategy compactionTriggerStrategy) { compactionConfig.setValue(INLINE_COMPACT_TRIGGER_STRATEGY, compactionTriggerStrategy.name()); return this; @@ -401,6 +424,16 @@ public Builder withPreserveCommitMetadata(boolean preserveCommitMetadata) { return this; } + public Builder withLogCompactionBlocksThreshold(String logCompactionBlocksThreshold) { + compactionConfig.setValue(LOG_COMPACTION_BLOCKS_THRESHOLD, logCompactionBlocksThreshold); + return this; + } + + public Builder withLogRecordReaderScanV2(String useLogRecordReaderScanV2) { + compactionConfig.setValue(USE_LOG_RECORD_READER_SCAN_V2, useLogRecordReaderScanV2); + return this; + } + public HoodieCompactionConfig build() { compactionConfig.setDefaults(HoodieCompactionConfig.class.getName()); return compactionConfig; 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 fd7972a7199c8..438f35ade4884 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 @@ -1148,6 +1148,15 @@ public boolean populateMetaFields() { /** * compaction properties. */ + + public int getLogCompactionBlocksThreshold() { + return getInt(HoodieCompactionConfig.LOG_COMPACTION_BLOCKS_THRESHOLD); + } + + public boolean useScanV2ForLogRecordReader() { + return getBoolean(HoodieCompactionConfig.USE_LOG_RECORD_READER_SCAN_V2); + } + public HoodieCleaningPolicy getCleanerPolicy() { return HoodieCleaningPolicy.valueOf(getString(CLEANER_POLICY)); } @@ -1252,6 +1261,10 @@ public boolean scheduleInlineCompaction() { return getBoolean(HoodieCompactionConfig.SCHEDULE_INLINE_COMPACT); } + public boolean inlineLogCompactionEnabled() { + return getBoolean(HoodieCompactionConfig.INLINE_LOG_COMPACT); + } + public CompactionTriggerStrategy getInlineCompactTriggerStrategy() { return CompactionTriggerStrategy.valueOf(getString(HoodieCompactionConfig.INLINE_COMPACT_TRIGGER_STRATEGY)); } @@ -2095,7 +2108,7 @@ public Boolean doSkipDefaultPartitionValidation() { */ public Boolean areAnyTableServicesExecutedInline() { return areTableServicesEnabled() - && (inlineClusteringEnabled() || inlineCompactionEnabled() + && (inlineClusteringEnabled() || inlineCompactionEnabled() || inlineLogCompactionEnabled() || (isAutoClean() && !isAsyncClean()) || (isAutoArchive() && !isAsyncArchive())); } 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 3cb149427aa52..8db927d569b82 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 @@ -118,9 +118,27 @@ public class HoodieAppendHandle extends // Header metadata for a log block protected final Map header = new HashMap<>(); private SizeEstimator sizeEstimator; + // Instant time of the basefile on which append operation is performed. + private String baseInstantTime; + // This is used to distinguish between normal append and logcompaction's append operation. + private boolean isLogCompaction = false; + // use writer schema for log compaction. + private boolean useWriterSchema = false; private Properties recordProperties = new Properties(); + /** + * This is used by log compaction only. + */ + public HoodieAppendHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, + String partitionPath, String fileId, Iterator> recordItr, + TaskContextSupplier taskContextSupplier, Map header) { + this(config, instantTime, hoodieTable, partitionPath, fileId, recordItr, taskContextSupplier); + this.useWriterSchema = true; + this.isLogCompaction = true; + this.header.putAll(header); + } + public HoodieAppendHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, String partitionPath, String fileId, Iterator> recordItr, TaskContextSupplier taskContextSupplier) { super(config, instantTime, partitionPath, fileId, hoodieTable, taskContextSupplier); @@ -215,7 +233,7 @@ private Option getIndexedRecord(HoodieRecord hoodieRecord) { // If the format can not record the operation field, nullify the DELETE payload manually. boolean nullifyPayload = HoodieOperation.isDelete(hoodieRecord.getOperation()) && !config.allowOperationMetadataField(); recordProperties.put(HoodiePayloadProps.PAYLOAD_IS_UPDATE_RECORD_FOR_MOR, String.valueOf(isUpdateRecord)); - Option avroRecord = nullifyPayload ? Option.empty() : hoodieRecord.getData().getInsertValue(tableSchema, recordProperties); + Option avroRecord = nullifyPayload ? Option.empty() : getInsertValue(hoodieRecord); if (avroRecord.isPresent()) { if (avroRecord.get().equals(IGNORE_RECORD)) { return avroRecord; @@ -228,12 +246,14 @@ private Option getIndexedRecord(HoodieRecord hoodieRecord) { if (config.populateMetaFields()) { HoodieAvroUtils.addHoodieKeyToRecord(rewriteRecord, hoodieRecord.getRecordKey(), hoodieRecord.getPartitionPath(), fileId); - HoodieAvroUtils.addCommitMetadataToRecord(rewriteRecord, instantTime, seqId); + if (!this.isLogCompaction) { + HoodieAvroUtils.addCommitMetadataToRecord(rewriteRecord, instantTime, seqId); + } } if (config.allowOperationMetadataField()) { HoodieAvroUtils.addOperationToRecord(rewriteRecord, hoodieRecord.getOperation()); } - if (isUpdateRecord) { + if (isUpdateRecord || isLogCompaction) { updatedRecordsWritten++; } else { insertRecordsWritten++; @@ -256,6 +276,14 @@ private Option getIndexedRecord(HoodieRecord hoodieRecord) { return Option.empty(); } + private Option getInsertValue(HoodieRecord hoodieRecord) throws IOException { + if (useWriterSchema) { + return hoodieRecord.getData().getInsertValue(tableSchemaWithMetaFields, recordProperties); + } else { + return hoodieRecord.getData().getInsertValue(tableSchema, recordProperties); + } + } + private void initNewStatus() { HoodieDeltaWriteStat prevStat = (HoodieDeltaWriteStat) this.writeStatus.getStat(); // Make a new write status and copy basic fields over. @@ -381,14 +409,19 @@ public void doAppend() { while (recordItr.hasNext()) { HoodieRecord record = recordItr.next(); init(record); - flushToDiskIfRequired(record); + flushToDiskIfRequired(record, false); writeToBuffer(record); } - appendDataAndDeleteBlocks(header); + appendDataAndDeleteBlocks(header, true); estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords; } - protected void appendDataAndDeleteBlocks(Map header) { + /** + * Appends data and delete blocks. When appendDeleteBlocks value is false, only data blocks are appended. + * This is done so that all the data blocks are created first and then a single delete block is added. + * Otherwise what can end up happening is creation of multiple small delete blocks get added after each data block. + */ + protected void appendDataAndDeleteBlocks(Map header, boolean appendDeleteBlocks) { try { header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, instantTime); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writeSchemaWithMetaFields.toString()); @@ -401,7 +434,7 @@ protected void appendDataAndDeleteBlocks(Map header) blocks.add(getBlock(config, pickLogDataBlockFormat(), recordList, header, keyField)); } - if (recordsToDelete.size() > 0) { + if (appendDeleteBlocks && recordsToDelete.size() > 0) { blocks.add(new HoodieDeleteBlock(recordsToDelete.toArray(new DeleteRecord[0]), header)); } @@ -409,7 +442,9 @@ protected void appendDataAndDeleteBlocks(Map header) AppendResult appendResult = writer.appendBlocks(blocks); processAppendResult(appendResult, recordList); recordList.clear(); - recordsToDelete.clear(); + if (appendDeleteBlocks) { + recordsToDelete.clear(); + } } } catch (Exception e) { throw new HoodieAppendException("Failed while appending records to " + writer.getLogFile().getPath(), e); @@ -427,7 +462,7 @@ public void write(HoodieRecord record, Option insertValue) { Option> recordMetadata = ((HoodieRecordPayload) record.getData()).getMetadata(); try { init(record); - flushToDiskIfRequired(record); + flushToDiskIfRequired(record, false); writeToBuffer(record); } catch (Throwable t) { // Not throwing exception from here, since we don't want to fail the entire job @@ -441,7 +476,7 @@ public void write(HoodieRecord record, Option insertValue) { public List close() { try { // flush any remaining records to disk - appendDataAndDeleteBlocks(header); + appendDataAndDeleteBlocks(header, true); recordItr = null; if (writer != null) { writer.close(); @@ -461,6 +496,21 @@ public List close() { } } + public void write(Map> recordMap) { + try { + for (Map.Entry> entry: recordMap.entrySet()) { + HoodieRecord record = (HoodieRecord) entry.getValue(); + init(record); + flushToDiskIfRequired(record, false); + writeToBuffer(record); + } + appendDataAndDeleteBlocks(header, true); + estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords; + } catch (Exception e) { + throw new HoodieUpsertException("Failed to compact blocks for fileId " + fileId, e); + } + } + @Override public IOType getIOType() { return IOType.APPEND; @@ -508,14 +558,15 @@ private void writeToBuffer(HoodieRecord record) { /** * Checks if the number of records have reached the set threshold and then flushes the records to disk. */ - private void flushToDiskIfRequired(HoodieRecord record) { + private void flushToDiskIfRequired(HoodieRecord record, boolean appendDeleteBlocks) { // Append if max number of records reached to achieve block size if (numberOfRecords >= (int) (maxBlockSize / averageRecordSize)) { // Recompute averageRecordSize before writing a new block and update existing value with // avg of new and old LOG.info("AvgRecordSize => " + averageRecordSize); averageRecordSize = (averageRecordSize + sizeEstimator.sizeEstimate(record)) / 2; - appendDataAndDeleteBlocks(header); + // Delete blocks will be appended after appending all the data blocks. + appendDataAndDeleteBlocks(header, appendDeleteBlocks); estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords; numberOfRecords = 0; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index 962875fb924fe..7da0a4a57ade7 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -278,6 +278,7 @@ private HoodieWriteConfig createMetadataWriteConfig(HoodieWriteConfig writeConfi // by default, the HFile does not keep the metadata fields, set up as false // to always use the metadata of the new record. .withPreserveCommitMetadata(false) + .withLogRecordReaderScanV2(String.valueOf(writeConfig.useScanV2ForLogRecordReader())) .build()) .withParallelism(parallelism, parallelism) .withDeleteParallelism(parallelism) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java index 69ef7917b284f..6bc91d0459323 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java @@ -39,6 +39,7 @@ public class HoodieMetrics { public String rollbackTimerName = null; public String cleanTimerName = null; public String commitTimerName = null; + public String logCompactionTimerName = null; public String deltaCommitTimerName = null; public String replaceCommitTimerName = null; public String finalizeTimerName = null; @@ -55,6 +56,7 @@ public class HoodieMetrics { private Timer deltaCommitTimer = null; private Timer finalizeTimer = null; private Timer compactionTimer = null; + private Timer logCompactionTimer = null; private Timer clusteringTimer = null; private Timer indexTimer = null; private Timer conflictResolutionTimer = null; @@ -73,6 +75,7 @@ public HoodieMetrics(HoodieWriteConfig config) { this.replaceCommitTimerName = getMetricsName("timer", HoodieTimeline.REPLACE_COMMIT_ACTION); this.finalizeTimerName = getMetricsName("timer", "finalize"); this.compactionTimerName = getMetricsName("timer", HoodieTimeline.COMPACTION_ACTION); + this.logCompactionTimerName = getMetricsName("timer", HoodieTimeline.LOG_COMPACTION_ACTION); this.indexTimerName = getMetricsName("timer", "index"); this.conflictResolutionTimerName = getMetricsName("timer", "conflict_resolution"); this.conflictResolutionSuccessCounterName = getMetricsName("counter", "conflict_resolution.success"); @@ -98,6 +101,13 @@ public Timer.Context getCompactionCtx() { return compactionTimer == null ? null : compactionTimer.time(); } + public Timer.Context getLogCompactionCtx() { + if (config.isMetricsOn() && logCompactionTimer == null) { + logCompactionTimer = createTimer(commitTimerName); + } + return logCompactionTimer == null ? null : logCompactionTimer.time(); + } + public Timer.Context getClusteringCtx() { if (config.isMetricsOn() && clusteringTimer == null) { clusteringTimer = createTimer(replaceCommitTimerName); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieCompactionHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieCompactionHandler.java index eeb287abd543c..7c0ad4a4ad647 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieCompactionHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieCompactionHandler.java @@ -23,6 +23,8 @@ import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.log.block.HoodieLogBlock; +import org.apache.hudi.exception.HoodieNotSupportedException; import java.io.IOException; import java.util.Iterator; @@ -40,4 +42,10 @@ Iterator> handleUpdate(String instantTime, String partitionPat Iterator> handleInsert(String instantTime, String partitionPath, String fileId, Map> recordMap); + + default Iterator> handleInsertsForLogCompaction(String instantTime, String partitionPath, String fileId, + Map> recordMap, + Map header) { + throw new HoodieNotSupportedException("Operation is not yet supported"); + } } 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 5ca3aee764afe..4a6f4ae1f4cb7 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 @@ -329,7 +329,7 @@ public HoodieTimeline getCompletedCommitTimeline() { * Get only the inflights (no-completed) commit timeline. */ public HoodieTimeline getPendingCommitTimeline() { - return metaClient.getCommitsTimeline().filterPendingExcludingCompaction(); + return metaClient.getCommitsTimeline().filterPendingExcludingMajorAndMinorCompaction(); } /** @@ -410,6 +410,31 @@ public abstract Option scheduleCompaction(HoodieEngineCont public abstract HoodieWriteMetadata compact(HoodieEngineContext context, String compactionInstantTime); + /** + * Schedule log compaction for the instant time. + * + * @param context HoodieEngineContext + * @param instantTime Instant Time for scheduling log compaction + * @param extraMetadata additional metadata to write into plan + * @return + */ + public Option scheduleLogCompaction(HoodieEngineContext context, + String instantTime, + Option> extraMetadata) { + throw new UnsupportedOperationException("Log compaction is not supported for this table type"); + } + + /** + * Run Log Compaction on the table. Log Compaction arranges the data so that it is optimized for data access. + * + * @param context HoodieEngineContext + * @param logCompactionInstantTime Instant Time + */ + public HoodieWriteMetadata logCompact(HoodieEngineContext context, + String logCompactionInstantTime) { + throw new UnsupportedOperationException("Log compaction is not supported for this table type"); + } + /** * Schedule clustering for the instant time. * @@ -540,6 +565,10 @@ public void rollbackInflightCompaction(HoodieInstant inflightInstant) { rollbackInflightCompaction(inflightInstant, s -> Option.empty()); } + public void rollbackInflightLogCompaction(HoodieInstant inflightInstant) { + rollbackInflightLogCompaction(inflightInstant, s -> Option.empty()); + } + /** * Rollback failed compactions. Inflight rollbacks for compactions revert the .inflight file * to the .requested file. @@ -579,6 +608,19 @@ private void rollbackInflightInstant(HoodieInstant inflightInstant, getActiveTimeline().revertInstantFromInflightToRequested(inflightInstant); } + /** + * Rollback failed compactions. Inflight rollbacks for compactions revert the .inflight file + * to the .requested file. + * + * @param inflightInstant Inflight Compaction Instant + */ + public void rollbackInflightLogCompaction(HoodieInstant inflightInstant, Function> getPendingRollbackInstantFunc) { + final String commitTime = getPendingRollbackInstantFunc.apply(inflightInstant.getTimestamp()).map(entry + -> entry.getRollbackInstant().getTimestamp()).orElse(HoodieActiveTimeline.createNewInstantTime()); + scheduleRollback(context, commitTime, inflightInstant, false, config.shouldRollbackUsingMarkers()); + rollback(context, commitTime, inflightInstant, true, false); + } + /** * Finalize the written data onto storage. Perform any final cleanups. * diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java index 9027ab045a2fd..c08bec8a267e1 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java @@ -85,6 +85,7 @@ public class CleanPlanner implements Ser private final SyncableFileSystemView fileSystemView; private final HoodieTimeline commitTimeline; private final Map fgIdToPendingCompactionOperations; + private final Map fgIdToPendingLogCompactionOperations; private HoodieTable hoodieTable; private HoodieWriteConfig config; private transient HoodieEngineContext context; @@ -95,12 +96,16 @@ public CleanPlanner(HoodieEngineContext context, HoodieTable hoodieT this.fileSystemView = hoodieTable.getHoodieView(); this.commitTimeline = hoodieTable.getCompletedCommitsTimeline(); this.config = config; - this.fgIdToPendingCompactionOperations = - ((SyncableFileSystemView) hoodieTable.getSliceView()).getPendingCompactionOperations() + SyncableFileSystemView fileSystemView = (SyncableFileSystemView) hoodieTable.getSliceView(); + this.fgIdToPendingCompactionOperations = fileSystemView + .getPendingCompactionOperations() .map(entry -> Pair.of( new HoodieFileGroupId(entry.getValue().getPartitionPath(), entry.getValue().getFileId()), entry.getValue())) .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + this.fgIdToPendingLogCompactionOperations = fileSystemView.getPendingLogCompactionOperations() + .map(entry -> Pair.of(new HoodieFileGroupId(entry.getValue().getPartitionPath(), entry.getValue().getFileId()), entry.getValue())) + .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); } /** @@ -243,8 +248,10 @@ private Pair> getFilesToCleanKeepingLatestVersions( int keepVersions = config.getCleanerFileVersionsRetained(); // do not cleanup slice required for pending compaction Iterator fileSliceIterator = - fileGroup.getAllFileSlices().filter(fs -> !isFileSliceNeededForPendingCompaction(fs)).iterator(); - if (isFileGroupInPendingCompaction(fileGroup)) { + fileGroup.getAllFileSlices() + .filter(fs -> !isFileSliceNeededForPendingMajorOrMinorCompaction(fs)) + .iterator(); + if (isFileGroupInPendingMajorOrMinorCompaction(fileGroup)) { // We have already saved the last version of file-groups for pending compaction Id keepVersions--; } @@ -352,7 +359,7 @@ private Pair> getFilesToCleanKeepingLatestCommits(S } // Always keep the last commit - if (!isFileSliceNeededForPendingCompaction(aSlice) && HoodieTimeline + if (!isFileSliceNeededForPendingMajorOrMinorCompaction(aSlice) && HoodieTimeline .compareTimestamps(earliestCommitToRetain.getTimestamp(), HoodieTimeline.GREATER_THAN, fileCommitTime)) { // this is a commit, that should be cleaned. aFile.ifPresent(hoodieDataFile -> { @@ -503,6 +510,15 @@ public String getLastCompletedCommitTimestamp() { } } + /* + * Determine if file slice needed to be preserved for pending compaction or log compaction. + * @param fileSlice File slice + * @return true if file slice needs to be preserved, false otherwise. + */ + private boolean isFileSliceNeededForPendingMajorOrMinorCompaction(FileSlice fileSlice) { + return isFileSliceNeededForPendingCompaction(fileSlice) || isFileSliceNeededForPendingLogCompaction(fileSlice); + } + /** * Determine if file slice needed to be preserved for pending compaction. * @@ -519,7 +535,24 @@ private boolean isFileSliceNeededForPendingCompaction(FileSlice fileSlice) { return false; } - private boolean isFileGroupInPendingCompaction(HoodieFileGroup fg) { - return fgIdToPendingCompactionOperations.containsKey(fg.getFileGroupId()); + /** + * Determine if file slice needed to be preserved for pending logcompaction. + * + * @param fileSlice File Slice + * @return true if file slice needs to be preserved, false otherwise. + */ + private boolean isFileSliceNeededForPendingLogCompaction(FileSlice fileSlice) { + CompactionOperation op = fgIdToPendingLogCompactionOperations.get(fileSlice.getFileGroupId()); + if (null != op) { + // If file slice's instant time is newer or same as that of operation, do not clean + return HoodieTimeline.compareTimestamps(fileSlice.getBaseInstantTime(), HoodieTimeline.GREATER_THAN_OR_EQUALS, op.getBaseInstantTime() + ); + } + return false; + } + + private boolean isFileGroupInPendingMajorOrMinorCompaction(HoodieFileGroup fg) { + return fgIdToPendingCompactionOperations.containsKey(fg.getFileGroupId()) + || fgIdToPendingLogCompactionOperations.containsKey(fg.getFileGroupId()); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java index dd827ff5a94c0..2df1824c5f72f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java @@ -116,14 +116,15 @@ public boolean checkPrecondition() { */ protected Stream getFileSlicesEligibleForClustering(String partition) { SyncableFileSystemView fileSystemView = (SyncableFileSystemView) getHoodieTable().getSliceView(); - Set fgIdsInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations() - .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId()) - .collect(Collectors.toSet()); - fgIdsInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getKey).collect(Collectors.toSet())); + Set fgIdsInPendingCompactionLogCompactionAndClustering = + Stream.concat(fileSystemView.getPendingCompactionOperations(), fileSystemView.getPendingLogCompactionOperations()) + .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId()) + .collect(Collectors.toSet()); + fgIdsInPendingCompactionLogCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getKey).collect(Collectors.toSet())); return hoodieTable.getSliceView().getLatestFileSlices(partition) // file ids already in clustering are not eligible - .filter(slice -> !fgIdsInPendingCompactionAndClustering.contains(slice.getFileGroupId())); + .filter(slice -> !fgIdsInPendingCompactionLogCompactionAndClustering.contains(slice.getFileGroupId())); } /** diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java index 3379d16f4c035..a4daacfedbeec 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java @@ -83,4 +83,16 @@ public void completeInflightCompaction(HoodieTable table, String compactionCommi "Failed to commit " + table.getMetaClient().getBasePath() + " at time " + compactionCommitTime, e); } } + + public void completeInflightLogCompaction(HoodieTable table, String logCompactionCommitTime, HoodieCommitMetadata commitMetadata) { + HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); + try { + activeTimeline.transitionLogCompactionInflightToComplete( + HoodieTimeline.getLogCompactionInflightInstant(logCompactionCommitTime), + Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + } catch (IOException e) { + throw new HoodieCompactionException( + "Failed to commit " + table.getMetaClient().getBasePath() + " at time " + logCompactionCommitTime, e); + } + } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactionExecutionHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactionExecutionHelper.java new file mode 100644 index 0000000000000..f402a673598dc --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactionExecutionHelper.java @@ -0,0 +1,78 @@ +/* + * 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.table.action.compact; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.CompactionOperation; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieCompactionHandler; +import org.apache.hudi.table.HoodieTable; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Iterator; +import java.util.List; + +public class CompactionExecutionHelper implements Serializable { + + protected void transitionRequestedToInflight(HoodieTable table, String compactionInstantTime) { + HoodieActiveTimeline timeline = table.getActiveTimeline(); + HoodieInstant instant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime); + // Mark instant as compaction inflight + timeline.transitionCompactionRequestedToInflight(instant); + } + + protected String instantTimeToUseForScanning(String compactionInstantTime, String maxInstantTime) { + return maxInstantTime; + } + + protected boolean shouldPreserveCommitMetadata() { + return false; + } + + protected Iterator> writeFileAndGetWriteStats(HoodieCompactionHandler compactionHandler, + CompactionOperation operation, + String instantTime, + HoodieMergedLogRecordScanner scanner, + Option oldDataFileOpt) throws IOException { + Iterator> result; + // If the dataFile is present, perform updates else perform inserts into a new base file. + if (oldDataFileOpt.isPresent()) { + result = compactionHandler.handleUpdate(instantTime, operation.getPartitionPath(), + operation.getFileId(), scanner.getRecords(), + oldDataFileOpt.get()); + } else { + result = compactionHandler.handleInsert(instantTime, operation.getPartitionPath(), operation.getFileId(), + scanner.getRecords()); + } + return result; + } + + protected boolean useScanV2(HoodieWriteConfig writeConfig) { + return writeConfig.useScanV2ForLogRecordReader(); + } + +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java index 75954872aedd5..51ed7639a4923 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java @@ -19,34 +19,25 @@ package org.apache.hudi.table.action.compact; import org.apache.hudi.avro.HoodieAvroUtils; -import org.apache.hudi.avro.model.HoodieCompactionOperation; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.common.data.HoodieAccumulator; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; 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.HoodieBaseFile; -import org.apache.hudi.common.model.HoodieFileGroupId; -import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; -import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.table.view.TableFileSystemView.SliceView; import org.apache.hudi.common.util.CollectionUtils; -import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.StringUtils; -import org.apache.hudi.common.util.ValidationUtils; -import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.internal.schema.InternalSchema; import org.apache.hudi.internal.schema.utils.SerDeHelper; @@ -67,7 +58,6 @@ import java.util.Collection; import java.util.Iterator; import java.util.List; -import java.util.Set; import java.util.stream.StreamSupport; import static java.util.stream.Collectors.toList; @@ -84,10 +74,10 @@ public abstract class HoodieCompactor im * * @param table {@link HoodieTable} instance to use. * @param pendingCompactionTimeline pending compaction timeline. - * @param compactionInstantTime compaction instant + * @param instantTime compaction instant */ public abstract void preCompact( - HoodieTable table, HoodieTimeline pendingCompactionTimeline, String compactionInstantTime); + HoodieTable table, HoodieTimeline pendingCompactionTimeline, WriteOperationType operationType, String instantTime); /** * Maybe persist write status. @@ -107,10 +97,10 @@ public HoodieData compact( || (compactionPlan.getOperations().isEmpty())) { return context.emptyHoodieData(); } - HoodieActiveTimeline timeline = table.getActiveTimeline(); - HoodieInstant instant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime); - // Mark instant as compaction inflight - timeline.transitionCompactionRequestedToInflight(instant); + CompactionExecutionHelper executionHelper = getCompactionExecutionStrategy(compactionPlan); + + // Transition requested to inflight file. + executionHelper.transitionRequestedToInflight(table, compactionInstantTime); table.getMetaClient().reloadActiveTimeline(); HoodieTableMetaClient metaClient = table.getMetaClient(); @@ -133,10 +123,12 @@ public HoodieData compact( .map(CompactionOperation::convertFromAvroRecordInstance).collect(toList()); LOG.info("Compactor compacting " + operations + " files"); + String maxInstantTime = getMaxInstantTime(metaClient); + context.setJobStatus(this.getClass().getSimpleName(), "Compacting file slices: " + config.getTableName()); TaskContextSupplier taskContextSupplier = table.getTaskContextSupplier(); return context.parallelize(operations).map(operation -> compact( - compactionHandler, metaClient, config, operation, compactionInstantTime, taskContextSupplier)) + compactionHandler, metaClient, config, operation, compactionInstantTime, maxInstantTime, taskContextSupplier, executionHelper)) .flatMap(List::iterator); } @@ -148,7 +140,23 @@ public List compact(HoodieCompactionHandler compactionHandler, HoodieWriteConfig config, CompactionOperation operation, String instantTime, + String maxInstantTime, TaskContextSupplier taskContextSupplier) throws IOException { + return compact(compactionHandler, metaClient, config, operation, instantTime, maxInstantTime, + taskContextSupplier, new CompactionExecutionHelper()); + } + + /** + * Execute a single compaction operation and report back status. + */ + public List compact(HoodieCompactionHandler compactionHandler, + HoodieTableMetaClient metaClient, + HoodieWriteConfig config, + CompactionOperation operation, + String instantTime, + String maxInstantTime, + TaskContextSupplier taskContextSupplier, + CompactionExecutionHelper executionHelper) throws IOException { FileSystem fs = metaClient.getFs(); Schema readerSchema; Option internalSchemaOption = Option.empty(); @@ -161,7 +169,7 @@ public List compact(HoodieCompactionHandler compactionHandler, readerSchema = HoodieAvroUtils.addMetadataFields( new Schema.Parser().parse(config.getSchema()), config.allowOperationMetadataField()); } - LOG.info("Compacting base " + operation.getDataFileName() + " with delta files " + operation.getDeltaFileNames() + LOG.info("Compaction operation started for base file: " + operation.getDataFileName() + " and delta files: " + operation.getDeltaFileNames() + " for commit " + instantTime); // TODO - FIX THIS // Reads the entire avro file. Always only specific blocks should be read from the avro file @@ -169,10 +177,7 @@ public List compact(HoodieCompactionHandler compactionHandler, // Load all the delta commits since the last compaction commit and get all the blocks to be // loaded and load it using CompositeAvroLogReader // Since a DeltaCommit is not defined yet, reading all the records. revisit this soon. - String maxInstantTime = metaClient - .getActiveTimeline().getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, - HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION)) - .filterCompletedInstants().lastInstant().get().getTimestamp(); + long maxMemoryPerCompaction = IOUtils.getMaxMemoryPerCompaction(taskContextSupplier, config); LOG.info("MaxMemoryPerCompaction => " + maxMemoryPerCompaction); @@ -184,7 +189,7 @@ public List compact(HoodieCompactionHandler compactionHandler, .withBasePath(metaClient.getBasePath()) .withLogFilePaths(logFiles) .withReaderSchema(readerSchema) - .withLatestInstantTime(maxInstantTime) + .withLatestInstantTime(executionHelper.instantTimeToUseForScanning(instantTime, maxInstantTime)) .withInternalSchema(internalSchemaOption.orElse(InternalSchema.getEmptyInternalSchema())) .withMaxMemorySizeInBytes(maxMemoryPerCompaction) .withReadBlocksLazily(config.getCompactionLazyBlockReadEnabled()) @@ -195,6 +200,7 @@ public List compact(HoodieCompactionHandler compactionHandler, .withBitCaskDiskMapCompressionEnabled(config.getCommonConfig().isBitCaskDiskMapCompressionEnabled()) .withOperationField(config.allowOperationMetadataField()) .withPartition(operation.getPartitionPath()) + .withUseScanV2(executionHelper.useScanV2(config)) .build(); Option oldDataFileOpt = @@ -222,15 +228,7 @@ public List compact(HoodieCompactionHandler compactionHandler, // Compacting is very similar to applying updates to existing file Iterator> result; - // If the dataFile is present, perform updates else perform inserts into a new base file. - if (oldDataFileOpt.isPresent()) { - result = compactionHandler.handleUpdate(instantTime, operation.getPartitionPath(), - operation.getFileId(), scanner.getRecords(), - oldDataFileOpt.get()); - } else { - result = compactionHandler.handleInsert(instantTime, operation.getPartitionPath(), operation.getFileId(), - scanner.getRecords()); - } + result = executionHelper.writeFileAndGetWriteStats(compactionHandler, operation, instantTime, scanner, oldDataFileOpt); scanner.close(); Iterable> resultIterable = () -> result; return StreamSupport.stream(resultIterable.spliterator(), false).flatMap(Collection::stream).peek(s -> { @@ -249,82 +247,21 @@ public List compact(HoodieCompactionHandler compactionHandler, }).collect(toList()); } - /** - * Generate a new compaction plan for scheduling. - * - * @param context HoodieEngineContext - * @param hoodieTable Hoodie Table - * @param config Hoodie Write Configuration - * @param compactionCommitTime scheduled compaction commit time - * @param fgIdsInPendingCompactionAndClustering partition-fileId pairs for which compaction is pending - * @return Compaction Plan - * @throws IOException when encountering errors - */ - HoodieCompactionPlan generateCompactionPlan( - HoodieEngineContext context, HoodieTable hoodieTable, HoodieWriteConfig config, - String compactionCommitTime, Set fgIdsInPendingCompactionAndClustering) throws IOException { - // Accumulator to keep track of total log files for a table - HoodieAccumulator totalLogFiles = context.newAccumulator(); - // Accumulator to keep track of total log file slices for a table - HoodieAccumulator totalFileSlices = context.newAccumulator(); - - ValidationUtils.checkArgument(hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ, - "Can only compact table of type " + HoodieTableType.MERGE_ON_READ + " and not " - + hoodieTable.getMetaClient().getTableType().name()); - - // TODO : check if maxMemory is not greater than JVM or executor memory - // TODO - rollback any compactions in flight - HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); - LOG.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommitTime); - List partitionPaths = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), metaClient.getBasePath()); - - // filter the partition paths if needed to reduce list status - partitionPaths = config.getCompactionStrategy().filterPartitionPaths(config, partitionPaths); - - if (partitionPaths.isEmpty()) { - // In case no partitions could be picked, return no compaction plan - return null; - } - - SliceView fileSystemView = hoodieTable.getSliceView(); - LOG.info("Compaction looking for files to compact in " + partitionPaths + " partitions"); - context.setJobStatus(this.getClass().getSimpleName(), "Looking for files to compact: " + config.getTableName()); - - List operations = context.flatMap(partitionPaths, partitionPath -> fileSystemView - .getLatestFileSlices(partitionPath) - .filter(slice -> !fgIdsInPendingCompactionAndClustering.contains(slice.getFileGroupId())) - .map(s -> { - List logFiles = - s.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()).collect(toList()); - totalLogFiles.add(logFiles.size()); - totalFileSlices.add(1L); - // Avro generated classes are not inheriting Serializable. Using CompactionOperation POJO - // for Map operations and collecting them finally in Avro generated classes for storing - // into meta files. - Option dataFile = s.getBaseFile(); - return new CompactionOperation(dataFile, partitionPath, logFiles, - config.getCompactionStrategy().captureMetrics(config, s)); - }) - .filter(c -> !c.getDeltaFileNames().isEmpty()), partitionPaths.size()).stream() - .map(CompactionUtils::buildHoodieCompactionOperation).collect(toList()); + public String getMaxInstantTime(HoodieTableMetaClient metaClient) { + String maxInstantTime = metaClient + .getActiveTimeline().getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, + HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION)) + .filterCompletedInstants().lastInstant().get().getTimestamp(); + return maxInstantTime; + } - LOG.info("Total of " + operations.size() + " compactions are retrieved"); - LOG.info("Total number of latest files slices " + totalFileSlices.value()); - LOG.info("Total number of log files " + totalLogFiles.value()); - LOG.info("Total number of file slices " + totalFileSlices.value()); - // Filter the compactions with the passed in filter. This lets us choose most effective - // compactions only - HoodieCompactionPlan compactionPlan = config.getCompactionStrategy().generateCompactionPlan(config, operations, - CompactionUtils.getAllPendingCompactionPlans(metaClient).stream().map(Pair::getValue).collect(toList())); - ValidationUtils.checkArgument( - compactionPlan.getOperations().stream().noneMatch( - op -> fgIdsInPendingCompactionAndClustering.contains(new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()))), - "Bad Compaction Plan. FileId MUST NOT have multiple pending compactions. " - + "Please fix your strategy implementation. FileIdsWithPendingCompactions :" + fgIdsInPendingCompactionAndClustering - + ", Selected workload :" + compactionPlan); - if (compactionPlan.getOperations().isEmpty()) { - LOG.warn("After filtering, Nothing to compact for " + metaClient.getBasePath()); + public CompactionExecutionHelper getCompactionExecutionStrategy(HoodieCompactionPlan compactionPlan) { + if (compactionPlan.getStrategy() == null || StringUtils.isNullOrEmpty(compactionPlan.getStrategy().getCompactorClassName())) { + return new CompactionExecutionHelper(); + } else { + CompactionExecutionHelper executionStrategy = ReflectionUtils.loadClass(compactionPlan.getStrategy().getCompactorClassName()); + return executionStrategy; } - return compactionPlan; } + } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/LogCompactionExecutionHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/LogCompactionExecutionHelper.java new file mode 100644 index 0000000000000..0e49267507c2e --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/LogCompactionExecutionHelper.java @@ -0,0 +1,86 @@ +/* + * 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.table.action.compact; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.CompactionOperation; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; +import org.apache.hudi.common.table.log.block.HoodieLogBlock; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieCompactionHandler; +import org.apache.hudi.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + * Strategy class to execute log compaction operations. + */ +public class LogCompactionExecutionHelper + extends CompactionExecutionHelper { + + private static final Logger LOG = LogManager.getLogger(LogCompactionExecutionHelper.class); + + @Override + protected void transitionRequestedToInflight(HoodieTable table, String logCompactionInstantTime) { + HoodieActiveTimeline timeline = table.getActiveTimeline(); + HoodieInstant instant = HoodieTimeline.getLogCompactionRequestedInstant(logCompactionInstantTime); + // Mark instant as compaction inflight + timeline.transitionLogCompactionRequestedToInflight(instant); + } + + protected String instantTimeToUseForScanning(String logCompactionInstantTime, String maxInstantTime) { + return logCompactionInstantTime; + } + + protected boolean shouldPreserveCommitMetadata() { + return true; + } + + @Override + protected Iterator> writeFileAndGetWriteStats(HoodieCompactionHandler compactionHandler, + CompactionOperation operation, + String instantTime, + HoodieMergedLogRecordScanner scanner, + Option oldDataFileOpt) throws IOException { + Map header = new HashMap<>(); + header.put(HoodieLogBlock.HeaderMetadataType.COMPACTED_BLOCK_TIMES, + StringUtils.join(scanner.getValidBlockInstants(), ",")); + // Compacting is very similar to applying updates to existing file + return compactionHandler.handleInsertsForLogCompaction(instantTime, operation.getPartitionPath(), + operation.getFileId(), scanner.getRecords(), header); + } + + @Override + protected boolean useScanV2(HoodieWriteConfig writeConfig) { + return true; + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java index fc4ae986e6d55..a0e5ec22f9f5f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java @@ -41,38 +41,47 @@ import org.apache.hudi.table.action.BaseActionExecutor; import org.apache.hudi.table.action.HoodieWriteMetadata; -import java.io.IOException; import java.util.List; +import static org.apache.hudi.common.util.ValidationUtils.checkArgument; + @SuppressWarnings("checkstyle:LineLength") public class RunCompactionActionExecutor extends BaseActionExecutor>, HoodieData, HoodieData, HoodieWriteMetadata>> { private final HoodieCompactor compactor; private final HoodieCompactionHandler compactionHandler; + private WriteOperationType operationType; public RunCompactionActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, String instantTime, HoodieCompactor compactor, - HoodieCompactionHandler compactionHandler) { + HoodieCompactionHandler compactionHandler, + WriteOperationType operationType) { super(context, config, table, instantTime); this.compactor = compactor; this.compactionHandler = compactionHandler; + this.operationType = operationType; + checkArgument(operationType == WriteOperationType.COMPACT || operationType == WriteOperationType.LOG_COMPACT, + "Only COMPACT and LOG_COMPACT is supported"); } @Override public HoodieWriteMetadata> execute() { - HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline(); - compactor.preCompact(table, pendingCompactionTimeline, instantTime); + HoodieTimeline pendingMajorOrMinorCompactionTimeline = WriteOperationType.COMPACT.equals(operationType) + ? table.getActiveTimeline().filterPendingCompactionTimeline() + : table.getActiveTimeline().filterPendingLogCompactionTimeline(); + compactor.preCompact(table, pendingMajorOrMinorCompactionTimeline, this.operationType, instantTime); HoodieWriteMetadata> compactionMetadata = new HoodieWriteMetadata<>(); try { // generate compaction plan // should support configurable commit metadata - HoodieCompactionPlan compactionPlan = - CompactionUtils.getCompactionPlan(table.getMetaClient(), instantTime); + HoodieCompactionPlan compactionPlan = operationType.equals(WriteOperationType.COMPACT) + ? CompactionUtils.getCompactionPlan(table.getMetaClient(), instantTime) + : CompactionUtils.getLogCompactionPlan(table.getMetaClient(), instantTime); // try to load internalSchema to support schema Evolution HoodieWriteConfig configCopy = config; @@ -100,11 +109,11 @@ public HoodieWriteMetadata> execute() { metadata.addMetadata(SerDeHelper.LATEST_SCHEMA, schemaPair.getLeft().get()); metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, schemaPair.getRight().get()); } - metadata.setOperationType(WriteOperationType.COMPACT); + metadata.setOperationType(operationType); compactionMetadata.setWriteStatuses(statuses); compactionMetadata.setCommitted(false); compactionMetadata.setCommitMetadata(Option.of(metadata)); - } catch (IOException e) { + } catch (Exception e) { throw new HoodieCompactionException("Could not compact " + config.getBasePath(), e); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java index 4fb5f9f7ddba5..8010cfbe82761 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java @@ -21,13 +21,13 @@ import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; -import org.apache.hudi.common.table.view.SyncableFileSystemView; import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; @@ -38,6 +38,9 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.BaseActionExecutor; +import org.apache.hudi.table.action.compact.plan.generators.BaseHoodieCompactionPlanGenerator; +import org.apache.hudi.table.action.compact.plan.generators.HoodieCompactionPlanGenerator; +import org.apache.hudi.table.action.compact.plan.generators.HoodieLogCompactionPlanGenerator; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -45,35 +48,50 @@ import java.text.ParseException; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.stream.Collectors; +import static org.apache.hudi.common.util.ValidationUtils.checkArgument; + public class ScheduleCompactionActionExecutor extends BaseActionExecutor> { private static final Logger LOG = LogManager.getLogger(ScheduleCompactionActionExecutor.class); - + private WriteOperationType operationType; private final Option> extraMetadata; - private final HoodieCompactor compactor; + private BaseHoodieCompactionPlanGenerator planGenerator; public ScheduleCompactionActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, String instantTime, Option> extraMetadata, - HoodieCompactor compactor) { + WriteOperationType operationType) { super(context, config, table, instantTime); this.extraMetadata = extraMetadata; - this.compactor = compactor; + this.operationType = operationType; + checkArgument(operationType == WriteOperationType.COMPACT || operationType == WriteOperationType.LOG_COMPACT, + "Only COMPACT and LOG_COMPACT is supported"); + initPlanGenerator(context, config, table); + } + + private void initPlanGenerator(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table) { + if (WriteOperationType.COMPACT.equals(operationType)) { + planGenerator = new HoodieCompactionPlanGenerator(table, context, config); + } else { + planGenerator = new HoodieLogCompactionPlanGenerator(table, context, config); + } } @Override public Option execute() { + ValidationUtils.checkArgument(this.table.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ, + "Can only compact table of type " + HoodieTableType.MERGE_ON_READ + " and not " + + this.table.getMetaClient().getTableType().name()); if (!config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl() && !config.getFailedWritesCleanPolicy().isLazy()) { // TODO(yihua): this validation is removed for Java client used by kafka-connect. Need to revisit this. if (config.getEngineType() == EngineType.SPARK) { // if there are inflight writes, their instantTime must not be less than that of compaction instant time - table.getActiveTimeline().getCommitsTimeline().filterPendingExcludingCompaction().firstInstant() + table.getActiveTimeline().getCommitsTimeline().filterPendingExcludingMajorAndMinorCompaction().firstInstant() .ifPresent(earliestInflight -> ValidationUtils.checkArgument( HoodieTimeline.compareTimestamps(earliestInflight.getTimestamp(), HoodieTimeline.GREATER_THAN, instantTime), "Earliest write inflight instant time must be later than compaction time. Earliest :" + earliestInflight @@ -93,11 +111,18 @@ public Option execute() { HoodieCompactionPlan plan = scheduleCompaction(); if (plan != null && (plan.getOperations() != null) && (!plan.getOperations().isEmpty())) { extraMetadata.ifPresent(plan::setExtraMetadata); - HoodieInstant compactionInstant = - new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instantTime); try { - table.getActiveTimeline().saveToCompactionRequested(compactionInstant, - TimelineMetadataUtils.serializeCompactionPlan(plan)); + if (operationType.equals(WriteOperationType.COMPACT)) { + HoodieInstant compactionInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, + HoodieTimeline.COMPACTION_ACTION, instantTime); + table.getActiveTimeline().saveToCompactionRequested(compactionInstant, + TimelineMetadataUtils.serializeCompactionPlan(plan)); + } else { + HoodieInstant logCompactionInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, + HoodieTimeline.LOG_COMPACTION_ACTION, instantTime); + table.getActiveTimeline().saveToLogCompactionRequested(logCompactionInstant, + TimelineMetadataUtils.serializeCompactionPlan(plan)); + } } catch (IOException ioe) { throw new HoodieIOException("Exception scheduling compaction", ioe); } @@ -113,19 +138,12 @@ private HoodieCompactionPlan scheduleCompaction() { if (compactable) { LOG.info("Generating compaction plan for merge on read table " + config.getBasePath()); try { - SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView(); - Set fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations() - .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId()) - .collect(Collectors.toSet()); - // exclude files in pending clustering from compaction. - fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet())); - context.setJobStatus(this.getClass().getSimpleName(), "Compaction: generating compaction plan: " + config.getTableName()); - return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering); + context.setJobStatus(this.getClass().getSimpleName(), "Compaction: generating compaction plan"); + return planGenerator.generateCompactionPlan(); } catch (IOException e) { throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e); } } - return new HoodieCompactionPlan(); } @@ -159,6 +177,9 @@ private boolean needCompact(CompactionTriggerStrategy compactionTriggerStrategy) return false; } Pair latestDeltaCommitInfo = latestDeltaCommitInfoOption.get(); + if (WriteOperationType.LOG_COMPACT.equals(operationType)) { + return true; + } int inlineCompactDeltaCommitMax = config.getInlineCompactDeltaCommitMax(); int inlineCompactDeltaSecondsMax = config.getInlineCompactDeltaSecondsMax(); switch (compactionTriggerStrategy) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/BaseHoodieCompactionPlanGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/BaseHoodieCompactionPlanGenerator.java new file mode 100644 index 0000000000000..5e5d6de92d3e6 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/BaseHoodieCompactionPlanGenerator.java @@ -0,0 +1,165 @@ +/* + * 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.table.action.compact.plan.generators; + +import org.apache.hudi.avro.model.HoodieCompactionOperation; +import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.common.data.HoodieAccumulator; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.CompactionOperation; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.view.SyncableFileSystemView; +import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.util.CompactionUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import static java.util.stream.Collectors.toList; + +public abstract class BaseHoodieCompactionPlanGenerator implements Serializable { + private static final Logger LOG = LogManager.getLogger(BaseHoodieCompactionPlanGenerator.class); + + protected final HoodieTable hoodieTable; + protected final HoodieWriteConfig writeConfig; + protected final transient HoodieEngineContext engineContext; + + public BaseHoodieCompactionPlanGenerator(HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig writeConfig) { + this.hoodieTable = table; + this.writeConfig = writeConfig; + this.engineContext = engineContext; + } + + public HoodieCompactionPlan generateCompactionPlan() throws IOException { + // Accumulator to keep track of total log files for a table + HoodieAccumulator totalLogFiles = engineContext.newAccumulator(); + // Accumulator to keep track of total log file slices for a table + HoodieAccumulator totalFileSlices = engineContext.newAccumulator(); + + // TODO : check if maxMemory is not greater than JVM or executor memory + // TODO - rollback any compactions in flight + HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); + List partitionPaths = FSUtils.getAllPartitionPaths(engineContext, writeConfig.getMetadataConfig(), metaClient.getBasePath()); + + // filter the partition paths if needed to reduce list status + partitionPaths = filterPartitionPathsByStrategy(writeConfig, partitionPaths); + + if (partitionPaths.isEmpty()) { + // In case no partitions could be picked, return no compaction plan + return null; + } + LOG.info("Looking for files to compact in " + partitionPaths + " partitions"); + engineContext.setJobStatus(this.getClass().getSimpleName(), "Looking for files to compact: " + writeConfig.getTableName()); + + SyncableFileSystemView fileSystemView = (SyncableFileSystemView) this.hoodieTable.getSliceView(); + Set fgIdsInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations() + .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId()) + .collect(Collectors.toSet()); + + // Exclude files in pending clustering from compaction. + fgIdsInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet())); + + if (filterLogCompactionOperations()) { + fgIdsInPendingCompactionAndClustering.addAll(fileSystemView.getPendingLogCompactionOperations() + .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId()) + .collect(Collectors.toList())); + } + + String lastCompletedInstantTime = hoodieTable.getMetaClient() + .getActiveTimeline().getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, + HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION)) + .filterCompletedInstants().lastInstant().get().getTimestamp(); + + List operations = engineContext.flatMap(partitionPaths, partitionPath -> fileSystemView + .getLatestFileSlices(partitionPath) + .filter(slice -> filterFileSlice(slice, lastCompletedInstantTime, fgIdsInPendingCompactionAndClustering)) + .map(s -> { + List logFiles = + s.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()).collect(toList()); + totalLogFiles.add(logFiles.size()); + totalFileSlices.add(1L); + // Avro generated classes are not inheriting Serializable. Using CompactionOperation POJO + // for Map operations and collecting them finally in Avro generated classes for storing + // into meta files.6 + Option dataFile = s.getBaseFile(); + return new CompactionOperation(dataFile, partitionPath, logFiles, + writeConfig.getCompactionStrategy().captureMetrics(writeConfig, s)); + }), partitionPaths.size()).stream() + .map(CompactionUtils::buildHoodieCompactionOperation).collect(toList()); + + LOG.info("Total of " + operations.size() + " compaction operations are retrieved"); + LOG.info("Total number of latest files slices " + totalFileSlices.value()); + LOG.info("Total number of log files " + totalLogFiles.value()); + LOG.info("Total number of file slices " + totalFileSlices.value()); + + if (operations.isEmpty()) { + LOG.warn("No operations are retrieved for " + metaClient.getBasePath()); + return null; + } + + // Filter the compactions with the passed in filter. This lets us choose most effective compactions only + HoodieCompactionPlan compactionPlan = getCompactionPlan(metaClient, operations); + ValidationUtils.checkArgument( + compactionPlan.getOperations().stream().noneMatch( + op -> fgIdsInPendingCompactionAndClustering.contains(new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()))), + "Bad Compaction Plan. FileId MUST NOT have multiple pending compactions. " + + "Please fix your strategy implementation. FileIdsWithPendingCompactions :" + fgIdsInPendingCompactionAndClustering + + ", Selected workload :" + compactionPlan); + if (compactionPlan.getOperations().isEmpty()) { + LOG.warn("After filtering, Nothing to compact for " + metaClient.getBasePath()); + } + return compactionPlan; + } + + protected abstract HoodieCompactionPlan getCompactionPlan(HoodieTableMetaClient metaClient, List operations); + + protected abstract boolean filterLogCompactionOperations(); + + protected List filterPartitionPathsByStrategy(HoodieWriteConfig writeConfig, List partitionPaths) { + return partitionPaths; + } + + protected boolean filterFileSlice(FileSlice fileSlice, String lastCompletedInstantTime, Set pendingFileGroupIds) { + return fileSlice.getLogFiles().count() > 0 && !pendingFileGroupIds.contains(fileSlice.getFileGroupId()); + } + + protected Map getStrategyParams() { + return Collections.emptyMap(); + } + +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/HoodieCompactionPlanGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/HoodieCompactionPlanGenerator.java new file mode 100644 index 0000000000000..2adac577b7da3 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/HoodieCompactionPlanGenerator.java @@ -0,0 +1,63 @@ +/* + * 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.table.action.compact.plan.generators; + +import org.apache.hudi.avro.model.HoodieCompactionOperation; +import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.CompactionUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.List; + +import static java.util.stream.Collectors.toList; + +public class HoodieCompactionPlanGenerator + extends BaseHoodieCompactionPlanGenerator { + + private static final Logger LOG = LogManager.getLogger(HoodieCompactionPlanGenerator.class); + + public HoodieCompactionPlanGenerator(HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + @Override + protected HoodieCompactionPlan getCompactionPlan(HoodieTableMetaClient metaClient, List operations) { + // Filter the compactions with the passed in filter. This lets us choose most effective + // compactions only + return writeConfig.getCompactionStrategy().generateCompactionPlan(writeConfig, operations, + CompactionUtils.getAllPendingCompactionPlans(metaClient).stream().map(Pair::getValue).collect(toList())); + } + + @Override + protected List filterPartitionPathsByStrategy(HoodieWriteConfig writeConfig, List partitionPaths) { + return writeConfig.getCompactionStrategy().filterPartitionPaths(writeConfig, partitionPaths); + } + + @Override + protected boolean filterLogCompactionOperations() { + return false; + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/HoodieLogCompactionPlanGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/HoodieLogCompactionPlanGenerator.java new file mode 100644 index 0000000000000..7357054b4e233 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/HoodieLogCompactionPlanGenerator.java @@ -0,0 +1,104 @@ +/* + * 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.table.action.compact.plan.generators; + +import org.apache.hudi.avro.model.HoodieCompactionOperation; +import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.avro.model.HoodieCompactionStrategy; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.log.HoodieUnMergedLogRecordScanner; +import org.apache.hudi.common.util.CompactionUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.compact.LogCompactionExecutionHelper; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +public class HoodieLogCompactionPlanGenerator extends BaseHoodieCompactionPlanGenerator { + + private static final Logger LOG = LogManager.getLogger(HoodieLogCompactionPlanGenerator.class); + + public HoodieLogCompactionPlanGenerator(HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + @Override + protected HoodieCompactionPlan getCompactionPlan(HoodieTableMetaClient metaClient, List operations) { + HoodieCompactionStrategy compactionStrategy = HoodieCompactionStrategy.newBuilder() + .setStrategyParams(getStrategyParams()) + .setCompactorClassName(LogCompactionExecutionHelper.class.getName()) + .build(); + return HoodieCompactionPlan.newBuilder() + .setOperations(operations) + .setVersion(CompactionUtils.LATEST_COMPACTION_METADATA_VERSION) + .setStrategy(compactionStrategy) + .setPreserveHoodieMetadata(true) + .build(); + } + + @Override + protected boolean filterFileSlice(FileSlice fileSlice, String lastCompletedInstantTime, Set pendingFileGroupIds) { + return isFileSliceEligibleForLogCompaction(fileSlice, lastCompletedInstantTime) + && super.filterFileSlice(fileSlice, lastCompletedInstantTime, pendingFileGroupIds); + } + + @Override + protected boolean filterLogCompactionOperations() { + return true; + } + + /** + * Can schedule logcompaction if log files count is greater than 4 or total log blocks is greater than 4. + * @param fileSlice File Slice under consideration. + * @return Boolean value that determines whether log compaction will be scheduled or not. + */ + private boolean isFileSliceEligibleForLogCompaction(FileSlice fileSlice, String maxInstantTime) { + LOG.info("Checking if fileId " + fileSlice.getFileId() + " and partition " + + fileSlice.getPartitionPath() + " eligible for log compaction."); + HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); + HoodieUnMergedLogRecordScanner scanner = HoodieUnMergedLogRecordScanner.newBuilder() + .withFileSystem(metaClient.getFs()) + .withBasePath(hoodieTable.getMetaClient().getBasePath()) + .withLogFilePaths(fileSlice.getLogFiles() + .sorted(HoodieLogFile.getLogFileComparator()) + .map(file -> file.getPath().toString()) + .collect(Collectors.toList())) + .withLatestInstantTime(maxInstantTime) + .withBufferSize(writeConfig.getMaxDFSStreamBufferSize()) + .withUseScanV2(true) + .build(); + scanner.scanInternal(Option.empty(), true); + int totalBlocks = scanner.getCurrentInstantLogBlocks().size(); + LOG.info("Total blocks seen are " + totalBlocks); + + // If total blocks in the file slice is > blocks threshold value(default value is 5). + // Log compaction can be scheduled. + return totalBlocks >= writeConfig.getLogCompactionBlocksThreshold(); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackHelper.java index 8d5e767307d78..7cd09930c1f08 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackHelper.java @@ -210,7 +210,7 @@ protected Map generateHeader(String c header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, metaClient.getActiveTimeline().lastInstant().get().getTimestamp()); header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, commit); header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, - String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); + String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_BLOCK.ordinal())); return header; } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java index ce7a18515137b..c37244f7fc2e3 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java @@ -65,7 +65,7 @@ static Map generateHeader(String inst header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, rollbackInstantTime); header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, instantToRollback); header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, - String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); + String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_BLOCK.ordinal())); return header; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java index 14fe8e2b88713..f70fd4ade09c5 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java @@ -43,7 +43,7 @@ public Map downgrade( SupportsUpgradeDowngrade upgradeDowngradeHelper) { HoodieTable table = upgradeDowngradeHelper.getTable(config, context); // fetch pending commit info - HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction(); + HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingMajorAndMinorCompaction(); List commits = inflightTimeline.getReverseOrderedInstants().collect(Collectors.toList()); for (HoodieInstant inflightInstant : commits) { // delete existing markers diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/TwoToOneDowngradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/TwoToOneDowngradeHandler.java index de1a1067fe111..5cde65e257270 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/TwoToOneDowngradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/TwoToOneDowngradeHandler.java @@ -59,7 +59,7 @@ public Map downgrade( HoodieTableMetaClient metaClient = table.getMetaClient(); // re-create marker files if any partial timeline server based markers are found - HoodieTimeline inflightTimeline = metaClient.getCommitsTimeline().filterPendingExcludingCompaction(); + HoodieTimeline inflightTimeline = metaClient.getCommitsTimeline().filterPendingExcludingMajorAndMinorCompaction(); List commits = inflightTimeline.getReverseOrderedInstants().collect(Collectors.toList()); for (HoodieInstant inflightInstant : commits) { // Converts the markers in new format to old format of direct markers diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java index 95f22bba27d5f..935f827ab6db5 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java @@ -55,7 +55,7 @@ public Map upgrade( SupportsUpgradeDowngrade upgradeDowngradeHelper) { // fetch pending commit info HoodieTable table = upgradeDowngradeHelper.getTable(config, context); - HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction(); + HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingMajorAndMinorCompaction(); List commits = inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp) .collect(Collectors.toList()); if (commits.size() > 0 && instantTime != null) { diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java index aa8adde7353c9..d69534c0611ee 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java @@ -25,6 +25,7 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -107,8 +108,7 @@ public Option scheduleCompaction( String instantTime, Option> extraMetadata) { ScheduleCompactionActionExecutor scheduleCompactionExecutor = new ScheduleCompactionActionExecutor( - context, config, this, instantTime, extraMetadata, - new HoodieFlinkMergeOnReadTableCompactor()); + context, config, this, instantTime, extraMetadata, WriteOperationType.COMPACT); return scheduleCompactionExecutor.execute(); } @@ -117,7 +117,7 @@ public HoodieWriteMetadata> compact( HoodieEngineContext context, String compactionInstantTime) { RunCompactionActionExecutor compactionExecutor = new RunCompactionActionExecutor( context, config, this, compactionInstantTime, new HoodieFlinkMergeOnReadTableCompactor(), - new HoodieFlinkCopyOnWriteTable(config, context, getMetaClient())); + new HoodieFlinkCopyOnWriteTable(config, context, getMetaClient()), WriteOperationType.COMPACT); return convertMetadata(compactionExecutor.execute()); } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/HoodieFlinkMergeOnReadTableCompactor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/HoodieFlinkMergeOnReadTableCompactor.java index 03b9f8e7ee090..d47da217169ef 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/HoodieFlinkMergeOnReadTableCompactor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/HoodieFlinkMergeOnReadTableCompactor.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.config.HoodieWriteConfig; @@ -43,8 +44,11 @@ public class HoodieFlinkMergeOnReadTableCompactor @Override public void preCompact( - HoodieTable table, HoodieTimeline pendingCompactionTimeline, String compactionInstantTime) { - HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime); + HoodieTable table, HoodieTimeline pendingCompactionTimeline, WriteOperationType operationType, String instantTime) { + if (WriteOperationType.LOG_COMPACT.equals(operationType)) { + throw new UnsupportedOperationException("Log compaction is not supported for this execution engine."); + } + HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(instantTime); if (pendingCompactionTimeline.containsInstant(inflightInstant)) { table.rollbackInflightCompaction(inflightInstant); table.getMetaClient().reloadActiveTimeline(); diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaMergeOnReadTable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaMergeOnReadTable.java index 32d30f704ecbb..f5bc09b4d1cc7 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaMergeOnReadTable.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaMergeOnReadTable.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +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.util.Option; @@ -69,8 +70,7 @@ public HoodieWriteMetadata> bulkInsertPrepped(HoodieEngineCont @Override public Option scheduleCompaction(HoodieEngineContext context, String instantTime, Option> extraMetadata) { ScheduleCompactionActionExecutor scheduleCompactionExecutor = new ScheduleCompactionActionExecutor( - context, config, this, instantTime, extraMetadata, - new HoodieJavaMergeOnReadTableCompactor()); + context, config, this, instantTime, extraMetadata, WriteOperationType.COMPACT); return scheduleCompactionExecutor.execute(); } @@ -79,7 +79,7 @@ public HoodieWriteMetadata> compact( HoodieEngineContext context, String compactionInstantTime) { RunCompactionActionExecutor compactionExecutor = new RunCompactionActionExecutor( context, config, this, compactionInstantTime, new HoodieJavaMergeOnReadTableCompactor(), - new HoodieJavaCopyOnWriteTable(config, context, getMetaClient())); + new HoodieJavaCopyOnWriteTable(config, context, getMetaClient()), WriteOperationType.COMPACT); return convertMetadata(compactionExecutor.execute()); } } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/compact/HoodieJavaMergeOnReadTableCompactor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/compact/HoodieJavaMergeOnReadTableCompactor.java index 30bdcda759ce0..6dc65649a1d40 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/compact/HoodieJavaMergeOnReadTableCompactor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/compact/HoodieJavaMergeOnReadTableCompactor.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.config.HoodieWriteConfig; @@ -41,8 +42,11 @@ public class HoodieJavaMergeOnReadTableCompactor @Override public void preCompact( - HoodieTable table, HoodieTimeline pendingCompactionTimeline, String compactionInstantTime) { - HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime); + HoodieTable table, HoodieTimeline pendingCompactionTimeline, WriteOperationType operationType, String instantTime) { + if (WriteOperationType.LOG_COMPACT.equals(operationType)) { + throw new UnsupportedOperationException("Log compaction is not supported for this execution engine."); + } + HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(instantTime); if (pendingCompactionTimeline.containsInstant(inflightInstant)) { table.rollbackInflightCompaction(inflightInstant); table.getMetaClient().reloadActiveTimeline(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java index 7110e26bb068f..ef37dd18356b2 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java @@ -46,6 +46,7 @@ import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.exception.HoodieClusteringException; import org.apache.hudi.exception.HoodieWriteConflictException; +import org.apache.hudi.exception.HoodieException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.SparkHoodieIndexFactory; import org.apache.hudi.metadata.HoodieTableMetadataWriter; @@ -214,7 +215,6 @@ public HoodieWriteResult insertOverwrite(JavaRDD> records, final /** * Removes all existing records of the Hoodie table and inserts the given HoodieRecords, into the table. - * @param records HoodieRecords to insert * @param instantTime Instant time of the commit * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts @@ -347,6 +347,64 @@ protected HoodieWriteMetadata> compact(String compactionIns return compactionMetadata; } + @Override + public void commitLogCompaction(String logCompactionInstantTime, HoodieCommitMetadata metadata, Option> extraMetadata) { + HoodieSparkTable table = HoodieSparkTable.create(config, context); + extraMetadata.ifPresent(m -> m.forEach(metadata::addMetadata)); + completeLogCompaction(metadata, table, logCompactionInstantTime); + } + + @Override + protected void completeLogCompaction(HoodieCommitMetadata metadata, + HoodieTable table, + String logCompactionCommitTime) { + this.context.setJobStatus(this.getClass().getSimpleName(), "Collect log compaction write status and commit compaction"); + List writeStats = metadata.getWriteStats(); + final HoodieInstant logCompactionInstant = new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.LOG_COMPACTION_ACTION, logCompactionCommitTime); + try { + this.txnManager.beginTransaction(Option.of(logCompactionInstant), Option.empty()); + preCommit(logCompactionInstant, metadata); + finalizeWrite(table, logCompactionCommitTime, writeStats); + // commit to data table after committing to metadata table. + updateTableMetadata(table, metadata, logCompactionInstant); + LOG.info("Committing Log Compaction " + logCompactionCommitTime + ". Finished with result " + metadata); + CompactHelpers.getInstance().completeInflightLogCompaction(table, logCompactionCommitTime, metadata); + } finally { + this.txnManager.endTransaction(Option.of(logCompactionInstant)); + } + WriteMarkersFactory.get(config.getMarkersType(), table, logCompactionCommitTime) + .quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); + if (compactionTimer != null) { + long durationInMs = metrics.getDurationInMs(compactionTimer.stop()); + HoodieActiveTimeline.parseDateFromInstantTimeSafely(logCompactionCommitTime).ifPresent(parsedInstant -> + metrics.updateCommitMetrics(parsedInstant.getTime(), durationInMs, metadata, HoodieActiveTimeline.LOG_COMPACTION_ACTION) + ); + } + LOG.info("Log Compacted successfully on commit " + logCompactionCommitTime); + } + + @Override + protected HoodieWriteMetadata> logCompact(String logCompactionInstantTime, boolean shouldComplete) { + HoodieSparkTable table = HoodieSparkTable.create(config, context); + preWrite(logCompactionInstantTime, WriteOperationType.LOG_COMPACT, table.getMetaClient()); + HoodieTimeline pendingLogCompactionTimeline = table.getActiveTimeline().filterPendingLogCompactionTimeline(); + HoodieInstant inflightInstant = HoodieTimeline.getLogCompactionInflightInstant(logCompactionInstantTime); + if (pendingLogCompactionTimeline.containsInstant(inflightInstant)) { + LOG.info("Found Log compaction inflight file. Rolling back the commit and exiting."); + table.rollbackInflightLogCompaction(inflightInstant, commitToRollback -> getPendingRollbackInfo(table.getMetaClient(), commitToRollback, false)); + table.getMetaClient().reloadActiveTimeline(); + throw new HoodieException("Inflight logcompaction file exists"); + } + logCompactionTimer = metrics.getLogCompactionCtx(); + WriteMarkersFactory.get(config.getMarkersType(), table, logCompactionInstantTime); + HoodieWriteMetadata> writeMetadata = table.logCompact(context, logCompactionInstantTime); + HoodieWriteMetadata> logCompactionMetadata = writeMetadata.clone(HoodieJavaRDD.getJavaRDD(writeMetadata.getWriteStatuses())); + if (shouldComplete && logCompactionMetadata.getCommitMetadata().isPresent()) { + completeTableService(TableServiceType.LOG_COMPACT, logCompactionMetadata.getCommitMetadata().get(), table, logCompactionInstantTime); + } + return logCompactionMetadata; + } + @Override public HoodieWriteMetadata> cluster(String clusteringInstant, boolean shouldComplete) { HoodieSparkTable table = HoodieSparkTable.create(config, context); @@ -470,6 +528,9 @@ private void completeTableService(TableServiceType tableServiceType, HoodieCommi case COMPACT: completeCompaction(metadata, table, commitInstant); break; + case LOG_COMPACT: + completeLogCompaction(metadata, table, commitInstant); + break; default: throw new IllegalArgumentException("This table service is not valid " + tableServiceType); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java index eab98f2f1907c..171f54fc2dfa2 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java @@ -294,6 +294,7 @@ private HoodieData> readRecordsForGroupWithLogs(JavaSparkContext .withBufferSize(config.getMaxDFSStreamBufferSize()) .withSpillableMapBasePath(config.getSpillableMapBasePath()) .withPartition(clusteringOp.getPartitionPath()) + .withUseScanV2(config.useScanV2ForLogRecordReader()) .withDiskMapType(config.getCommonConfig().getSpillableDiskMapType()) .withBitCaskDiskMapCompressionEnabled(config.getCommonConfig().isBitCaskDiskMapCompressionEnabled()) .build(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java index efc667af297be..bf88aa2690f26 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java @@ -30,12 +30,15 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; 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.log.block.HoodieLogBlock; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.io.HoodieAppendHandle; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; import org.apache.hudi.table.action.bootstrap.SparkBootstrapDeltaCommitActionExecutor; @@ -54,6 +57,8 @@ import org.apache.hudi.table.action.rollback.MergeOnReadRollbackActionExecutor; import org.apache.hudi.table.action.rollback.RestorePlanActionExecutor; +import java.util.Collections; +import java.util.Iterator; import java.util.List; import java.util.Map; @@ -73,7 +78,7 @@ * action *

*/ -public class HoodieSparkMergeOnReadTable extends HoodieSparkCopyOnWriteTable { +public class HoodieSparkMergeOnReadTable extends HoodieSparkCopyOnWriteTable implements HoodieCompactionHandler { HoodieSparkMergeOnReadTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) { super(config, context, metaClient); @@ -127,9 +132,8 @@ public HoodieWriteMetadata> bulkInsertPrepped(HoodieEngi @Override public Option scheduleCompaction(HoodieEngineContext context, String instantTime, Option> extraMetadata) { - ScheduleCompactionActionExecutor scheduleCompactionExecutor = new ScheduleCompactionActionExecutor<>( - context, config, this, instantTime, extraMetadata, - new HoodieSparkMergeOnReadTableCompactor<>()); + ScheduleCompactionActionExecutor scheduleCompactionExecutor = new ScheduleCompactionActionExecutor( + context, config, this, instantTime, extraMetadata, WriteOperationType.COMPACT); return scheduleCompactionExecutor.execute(); } @@ -138,7 +142,7 @@ public HoodieWriteMetadata> compact( HoodieEngineContext context, String compactionInstantTime) { RunCompactionActionExecutor compactionExecutor = new RunCompactionActionExecutor<>( context, config, this, compactionInstantTime, new HoodieSparkMergeOnReadTableCompactor<>(), - new HoodieSparkCopyOnWriteTable<>(config, context, getMetaClient())); + new HoodieSparkCopyOnWriteTable<>(config, context, getMetaClient()), WriteOperationType.COMPACT); return compactionExecutor.execute(); } @@ -147,6 +151,21 @@ public HoodieBootstrapWriteMetadata> bootstrap(HoodieEng return new SparkBootstrapDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, extraMetadata).execute(); } + @Override + public Option scheduleLogCompaction(HoodieEngineContext context, String instantTime, Option> extraMetadata) { + ScheduleCompactionActionExecutor scheduleLogCompactionExecutor = new ScheduleCompactionActionExecutor( + context, config, this, instantTime, extraMetadata, WriteOperationType.LOG_COMPACT); + return scheduleLogCompactionExecutor.execute(); + } + + @Override + public HoodieWriteMetadata> logCompact( + HoodieEngineContext context, String logCompactionInstantTime) { + RunCompactionActionExecutor logCompactionExecutor = new RunCompactionActionExecutor(context, config, this, + logCompactionInstantTime, new HoodieSparkMergeOnReadTableCompactor<>(), this, WriteOperationType.LOG_COMPACT); + return logCompactionExecutor.execute(); + } + @Override public void rollbackBootstrap(HoodieEngineContext context, String instantTime) { new RestorePlanActionExecutor<>(context, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute(); @@ -161,6 +180,17 @@ public Option scheduleRollback(HoodieEngineContext context, shouldRollbackUsingMarkers).execute(); } + @Override + public Iterator> handleInsertsForLogCompaction(String instantTime, String partitionPath, String fileId, + Map> recordMap, + Map header) { + HoodieAppendHandle appendHandle = new HoodieAppendHandle(config, instantTime, this, + partitionPath, fileId, recordMap.values().iterator(), taskContextSupplier, header); + appendHandle.write(recordMap); + List writeStatuses = appendHandle.close(); + return Collections.singletonList(writeStatuses).iterator(); + } + @Override public HoodieRollbackMetadata rollback(HoodieEngineContext context, String rollbackInstantTime, diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java index 61cb1ffd27bd1..af3e4960ad071 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.config.HoodieWriteConfig; @@ -41,11 +42,13 @@ public class HoodieSparkMergeOnReadTableCompactor @Override public void preCompact( - HoodieTable table, HoodieTimeline pendingCompactionTimeline, String compactionInstantTime) { - HoodieInstant instant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime); - if (!pendingCompactionTimeline.containsInstant(instant)) { + HoodieTable table, HoodieTimeline pendingCompactionTimeline, WriteOperationType operationType, String instantTime) { + HoodieInstant requestedCompactionInstantTime = WriteOperationType.COMPACT.equals(operationType) + ? HoodieTimeline.getCompactionRequestedInstant(instantTime) + : HoodieTimeline.getLogCompactionRequestedInstant(instantTime); + if (!pendingCompactionTimeline.containsInstant(requestedCompactionInstantTime)) { throw new IllegalStateException( - "No Compaction request available at " + compactionInstantTime + " to run compaction"); + "No Compaction request available at " + requestedCompactionInstantTime.getTimestamp() + " to run compaction"); } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java index 67d82578fccbf..a5cc430b6d1c7 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java @@ -100,7 +100,7 @@ public void testUnscheduleCompactionFileId() throws Exception { Stream.of("001", "003", "005", "007").map(instant -> { try { return Pair.of(instant, CompactionUtils.getCompactionPlan(metaClient, instant)); - } catch (IOException ioe) { + } catch (Exception ioe) { throw new HoodieException(ioe); } }).map(instantWithPlan -> instantWithPlan.getRight().getOperations().stream() diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestDataValidationCheckForLogCompactionActions.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestDataValidationCheckForLogCompactionActions.java new file mode 100644 index 0000000000000..eb9d2b462865a --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestDataValidationCheckForLogCompactionActions.java @@ -0,0 +1,419 @@ +/* + * 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.client.functional; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.fs.ConsistencyGuardConfig; +import org.apache.hudi.common.model.HoodieAvroRecord; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; +import org.apache.hudi.common.table.view.FileSystemViewStorageType; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.testutils.HoodieTestTable; +import org.apache.hudi.common.testutils.HoodieTestUtils; +import org.apache.hudi.common.testutils.RawTripTestPayload; +import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieCleanConfig; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.testutils.GenericRecordValidationTestUtils; +import org.apache.hudi.testutils.HoodieClientTestBase; +import org.apache.hudi.testutils.HoodieSparkWriteableTestTable; +import org.apache.hudi.testutils.MetadataMergeWriteStatus; +import org.apache.spark.api.java.JavaRDD; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Random; +import java.util.stream.Collectors; + +import static org.apache.hudi.common.model.HoodieRecord.COMMIT_SEQNO_METADATA_FIELD; +import static org.apache.hudi.common.model.HoodieRecord.COMMIT_TIME_METADATA_FIELD; +import static org.apache.hudi.common.model.HoodieRecord.FILENAME_METADATA_FIELD; +import static org.apache.hudi.common.model.HoodieRecord.OPERATION_METADATA_FIELD; +import static org.apache.hudi.common.model.HoodieRecord.RECORD_KEY_METADATA_FIELD; +import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; +import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; +import static org.apache.hudi.testutils.GenericRecordValidationTestUtils.assertGenericRecords; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestDataValidationCheckForLogCompactionActions extends HoodieClientTestBase { + + private HoodieTestTable testTable; + Random random = new Random(); + public static final String RECORD_KEY_APPEND_VALUE = "-EXP"; + + @TempDir + java.nio.file.Path secondTableBasePath; + + Function3, HoodieTestDataGenerator, String, Integer> insertsGenFunction = + HoodieTestDataGenerator::generateInserts; + Function3, HoodieTestDataGenerator, String, Integer> updatesGenFunction = + HoodieTestDataGenerator::generateUniqueUpdates; + Function2, HoodieTestDataGenerator, Integer> deletesGenFunction = + HoodieTestDataGenerator::generateUniqueDeletes; + + Function3, SparkRDDWriteClient, JavaRDD, String> insertsFunction = SparkRDDWriteClient::insert; + Function3, SparkRDDWriteClient, JavaRDD, String> updatesFunction = SparkRDDWriteClient::upsert; + Function3, SparkRDDWriteClient, JavaRDD, String> deletesFunction = SparkRDDWriteClient::delete; + + @BeforeEach + public void setUpTestTable() { + testTable = HoodieSparkWriteableTestTable.of(metaClient); + } + + //TODO: include both the table's contents. + /** + * Cleanups resource group for the subclasses of {@link HoodieClientTestBase}. + */ + @AfterEach + public void cleanupResources() throws IOException { + cleanupTimelineService(); + cleanupClients(); + cleanupSparkContexts(); + cleanupTestDataGenerator(); + cleanupFileSystem(); + cleanupDFS(); + cleanupExecutorService(); + System.gc(); + } + + /** + * Stress test logcompaction along with compaction by following approach. + * a. Create a random seed to do insert/upsert/deleting operations on main table and replicate same action on experiment table. + * b. Schedule inline major compaction to run for every 5 deltacommits on both the tables. + * c. After writes on both the tables configure log compaction to run on second table and keep no. of blocks threshold to 2. + * d. After every commit operation refresh the timeline and run a validation query for all the records. + */ + @ParameterizedTest + @ValueSource(ints = {17}) + public void stressTestCompactionAndLogCompactionOperations(int seed) throws Exception { + + // Set seed. + random.setSeed(seed); + + // Setup First table. + TestTableContents mainTable = setupTestTable1(); + + // Setup second table. + TestTableContents experimentTable = setupTestTable2(); + + // Total ingestion writes. + int totalWrites = 15; + + LOG.warn("Starting trial with seed " + seed); + + // Current ingestion commit. + int curr = 1; + while (curr < totalWrites) { + LOG.warn("Starting write No. " + curr); + + // Pick an action. It can be insert/update/delete and write data to main table. + boolean status = writeOnMainTable(mainTable, curr); + if (status) { + // Write data into experiment table. + writeOnExperimentTable(mainTable, experimentTable); + + // schedule and run log compaction on second table. + scheduleLogCompactionOnExperimentTable(experimentTable); + + // Verify that no compaction plans are left on the timeline. + assertEquals(0, mainTable.metaClient.reloadActiveTimeline().filterPendingCompactionTimeline().getInstants().count()); + assertEquals(0, experimentTable.metaClient.reloadActiveTimeline().filterPendingCompactionTimeline().getInstants().count()); + assertEquals(0, experimentTable.metaClient.reloadActiveTimeline().filterPendingLogCompactionTimeline().getInstants().count()); + + // Verify the records in both the tables. + verifyRecords(mainTable, experimentTable); + LOG.warn("For write No." + curr + ", verification passed. Last ingestion commit timestamp is " + mainTable.commitTimeOnMainTable); + } + curr++; + } + + } + + private void verifyRecords(TestTableContents mainTable, TestTableContents experimentTable) { + Map mainRecordsMap = + GenericRecordValidationTestUtils.getRecordsMap(mainTable.config, hadoopConf, dataGen); + Map experimentRecordsMap = + GenericRecordValidationTestUtils.getRecordsMap(experimentTable.config, hadoopConf, dataGen); + + // Verify row count. + assertEquals(mainRecordsMap.size(), experimentRecordsMap.size()); + + Schema readerSchema = new Schema.Parser().parse(mainTable.config.getSchema()); + List excludeFields = CollectionUtils.createImmutableList(COMMIT_TIME_METADATA_FIELD, COMMIT_SEQNO_METADATA_FIELD, + FILENAME_METADATA_FIELD, OPERATION_METADATA_FIELD, RECORD_KEY_METADATA_FIELD); + + // Verify every field. + mainRecordsMap.forEach((key, value) -> { + assertTrue(experimentRecordsMap.containsKey(key + RECORD_KEY_APPEND_VALUE)); + assertGenericRecords(value, experimentRecordsMap.get(key + RECORD_KEY_APPEND_VALUE), readerSchema, excludeFields); + }); + } + + private void scheduleLogCompactionOnExperimentTable(TestTableContents experimentTable) { + Option logCompactionTimeStamp = experimentTable.logCompactionClient.scheduleLogCompaction(Option.empty()); + if (logCompactionTimeStamp.isPresent()) { + experimentTable.logCompactionClient.logCompact(logCompactionTimeStamp.get()); + } + } + + private boolean writeOnMainTable(TestTableContents mainTable, int curr) throws IOException { + String commitTime = HoodieActiveTimeline.createNewInstantTime(); + mainTable.client.startCommitWithTime(commitTime); + + int actionType = pickAWriteAction(); + JavaRDD result; + if (curr == 1 || actionType == 0) { + result = insertDataIntoMainTable(mainTable, commitTime); + } else { + try { + if (actionType == 1) { + result = updateDataIntoMainTable(mainTable, commitTime); + } else { + result = deleteDataIntoMainTable(mainTable, commitTime); + } + } catch (IllegalArgumentException e) { + LOG.warn(e.getMessage() + " ignoring current command."); + return false; + } + } + verifyWriteStatus(result); + return true; + } + + /** + * This method has 50% chance to pick an insert, 30% chance to pick an update and 20% chance to pick a delete operation + */ + private int pickAWriteAction() { + int val = random.nextInt(10); + if (val < 5) { + return 0; + } else if (val < 8) { + return 1; + } + return 2; + } + + private void writeOnExperimentTable(TestTableContents mainTable, TestTableContents experimentTable) throws IOException { + String commitTime = mainTable.commitTimeOnMainTable; + experimentTable.client.startCommitWithTime(commitTime); + int actionType = mainTable.previousActionType; + JavaRDD result; + if (actionType == 0) { + result = insertDataIntoExperimentTable(mainTable, experimentTable); + } else if (actionType == 1) { + result = updateDataIntoExperimentTable(mainTable, experimentTable); + } else { + result = deleteDataIntoExperimentTable(mainTable, experimentTable); + } + verifyWriteStatus(result); + } + + private JavaRDD insertDataIntoMainTable(TestTableContents mainTable, String commitTime) throws IOException { + int numRecords = 50 + random.nextInt(10); + List records = insertsGenFunction.apply(dataGen, commitTime, numRecords); + mainTable.updatePreviousGeneration(records, commitTime, 0); + JavaRDD writeRecords = jsc.parallelize(records, 1); + return insertsFunction.apply(mainTable.client, writeRecords, commitTime); + } + + private JavaRDD updateDataIntoMainTable(TestTableContents mainTable, String commitTime) throws IOException { + int numRecords = 10 + random.nextInt(10); + List records = updatesGenFunction.apply(dataGen, commitTime, numRecords); + mainTable.updatePreviousGeneration(records, commitTime, 1); + JavaRDD writeRecords = jsc.parallelize(records, 1); + return updatesFunction.apply(mainTable.client, writeRecords, commitTime); + } + + private JavaRDD deleteDataIntoMainTable(TestTableContents mainTable, String commitTime) throws IOException { + int numRecords = 5 + random.nextInt(10); + List keys = deletesGenFunction.apply(dataGen, numRecords); + mainTable.updatePreviousGenerationForDelete(keys, commitTime); + JavaRDD deleteKeys = jsc.parallelize(keys, 1); + return deletesFunction.apply(mainTable.client, deleteKeys, commitTime); + } + + private JavaRDD insertDataIntoExperimentTable(TestTableContents mainTable, TestTableContents experimentTable) throws IOException { + JavaRDD writeRecords = jsc.parallelize(mainTable.generatedRecords, 1); + return insertsFunction.apply(experimentTable.client, writeRecords, mainTable.commitTimeOnMainTable); + } + + private JavaRDD updateDataIntoExperimentTable(TestTableContents mainTable, TestTableContents experimentTable) throws IOException { + JavaRDD writeRecords = jsc.parallelize(mainTable.generatedRecords, 1); + return updatesFunction.apply(experimentTable.client, writeRecords, mainTable.commitTimeOnMainTable); + } + + private JavaRDD deleteDataIntoExperimentTable(TestTableContents mainTable, TestTableContents experimentTable) throws IOException { + JavaRDD writeKeys = jsc.parallelize(mainTable.generatedKeysForDelete, 1); + return deletesFunction.apply(experimentTable.client, writeKeys, mainTable.commitTimeOnMainTable); + } + + private void verifyWriteStatus(JavaRDD writeStatuses) { + List statuses = writeStatuses.collect(); + assertNoWriteErrors(statuses); + } + + private class TestTableContents { + final String basePath; + final String tableName; + final HoodieTableMetaClient metaClient; + final HoodieWriteConfig config; + final SparkRDDWriteClient client; + String commitTimeOnMainTable = ""; + List generatedRecords = new ArrayList<>(); + List generatedKeysForDelete = new ArrayList<>(); + // 0 means insert, 1 means update, 2 means delete. + int previousActionType = 0; + + final SparkRDDWriteClient logCompactionClient; + + public TestTableContents(String basePath, String tableName, HoodieTableMetaClient metaClient, + HoodieWriteConfig config, SparkRDDWriteClient client) { + this(basePath, tableName, metaClient, config, client, null); + } + + public TestTableContents(String basePath, String tableName, HoodieTableMetaClient metaClient, HoodieWriteConfig config, + SparkRDDWriteClient client, SparkRDDWriteClient logCompactionClient) { + this.basePath = basePath; + this.tableName = tableName; + this.metaClient = metaClient; + this.config = config; + this.client = client; + this.logCompactionClient = logCompactionClient; + } + + private void updatePreviousGeneration(List generatedRecords, String commitTimeOnMainTable, int previousActionType) { + Schema schema = new Schema.Parser().parse(this.config.getSchema()); + this.generatedRecords = generatedRecords.stream().map(rec -> deepCopyAndModifyRecordKey(rec)).collect(Collectors.toList()); + this.commitTimeOnMainTable = commitTimeOnMainTable; + this.previousActionType = previousActionType; + } + + private HoodieRecord deepCopyAndModifyRecordKey(HoodieRecord record) { + HoodieKey key = deepCopyAndModifyRecordKey(record.getKey()); + RawTripTestPayload payload = ((RawTripTestPayload)record.getData()).clone(); + return new HoodieAvroRecord(key, payload); + } + + private HoodieKey deepCopyAndModifyRecordKey(HoodieKey key) { + return new HoodieKey(key.getRecordKey() + RECORD_KEY_APPEND_VALUE, key.getPartitionPath()); + } + + private void updatePreviousGenerationForDelete(List generatedKeysForDelete, String commitTimeOnMainTable) { + this.generatedKeysForDelete = generatedKeysForDelete.stream().map(this::deepCopyAndModifyRecordKey).collect(Collectors.toList()); + this.commitTimeOnMainTable = commitTimeOnMainTable; + this.previousActionType = 2; + } + } + + private TestTableContents setupTestTable1() { + Properties properties = new Properties(); + properties.setProperty("hoodie.parquet.small.file.limit", "0"); + HoodieWriteConfig config = getConfigBuilder(TRIP_EXAMPLE_SCHEMA, HoodieIndex.IndexType.INMEMORY) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withInlineCompaction(true).build()) + .withAutoCommit(true) + .withProperties(properties) + .build(); + SparkRDDWriteClient client = new SparkRDDWriteClient(context, config); + return new TestTableContents(basePath, tableName, metaClient, config, client); + } + + private TestTableContents setupTestTable2() throws IOException { + String tableName2 = "test-trip-table2"; + String basePath2 = createBasePathForSecondTable(secondTableBasePath); + Properties properties = new Properties(); + properties.put(HoodieTableConfig.NAME.key(), tableName2); + + // Create metaclient + HoodieTableMetaClient metaClient2 = HoodieTestUtils.init(hadoopConf, basePath2, + HoodieTableType.MERGE_ON_READ, properties); + HoodieWriteConfig config2 = getConfigBuilderForSecondTable(tableName2, basePath2, + TRIP_EXAMPLE_SCHEMA, HoodieIndex.IndexType.INMEMORY) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withInlineCompaction(true).build()) + .withAutoCommit(true).build(); + + // Create writeClient + SparkRDDWriteClient client2 = new SparkRDDWriteClient(context, config2); + + // Create logcompaction client. + HoodieWriteConfig logCompactionConfig = HoodieWriteConfig.newBuilder().withProps(config2.getProps()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withLogCompactionBlocksThreshold("2").build()) + .build(); + SparkRDDWriteClient logCompactionClient = new SparkRDDWriteClient(context, logCompactionConfig); + + return new TestTableContents(basePath2, tableName2, metaClient2, config2, client2, logCompactionClient); + } + + private String createBasePathForSecondTable(java.nio.file.Path secondTableBasePath) throws IOException { + java.nio.file.Path basePath = secondTableBasePath.resolve("dataset2"); + java.nio.file.Files.createDirectories(basePath); + return basePath.toString(); + } + + private HoodieWriteConfig.Builder getConfigBuilderForSecondTable(String tableName, String basePath, String schemaStr, HoodieIndex.IndexType indexType) { + Properties properties = new Properties(); + properties.setProperty("hoodie.parquet.small.file.limit", "0"); + return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr) + .withParallelism(2, 2).withBulkInsertParallelism(2).withFinalizeWriteParallelism(2).withDeleteParallelism(2) + .withTimelineLayoutVersion(TimelineLayoutVersion.CURR_VERSION) + .withWriteStatusClass(MetadataMergeWriteStatus.class) + .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .compactionSmallFileSize(1024 * 1024).build()) + .withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).orcMaxFileSize(1024 * 1024).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER).build()) + .forTable(tableName) + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build()) + .withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() + .withEnableBackupForRemoteFileSystemView(false) // Fail test if problem connecting to timeline-server + .withRemoteServerPort(timelineServicePort) + .withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build()) + .withProperties(properties); + } + + @Override + protected HoodieTableType getTableType() { + return HoodieTableType.MERGE_ON_READ; + } +} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnMergeOnReadStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnMergeOnReadStorage.java new file mode 100644 index 0000000000000..e54ca074da742 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnMergeOnReadStorage.java @@ -0,0 +1,544 @@ +/* + * 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.client.functional; + +import org.apache.avro.generic.GenericRecord; +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.client.transaction.lock.InProcessLockProvider; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.model.WriteConcurrencyMode; +import org.apache.hudi.common.table.log.HoodieUnMergedLogRecordScanner; +import org.apache.hudi.common.table.marker.MarkerType; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.view.SyncableFileSystemView; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.testutils.HoodieTestTable; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieArchivalConfig; +import org.apache.hudi.config.HoodieCleanConfig; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieLockConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.testutils.GenericRecordValidationTestUtils; +import org.apache.hudi.testutils.HoodieClientTestBase; +import org.apache.hudi.testutils.HoodieSparkWriteableTestTable; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.hudi.testutils.GenericRecordValidationTestUtils.assertDataInMORTable; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestHoodieClientOnMergeOnReadStorage extends HoodieClientTestBase { + + private HoodieTestTable testTable; + + @BeforeEach + public void setUpTestTable() { + testTable = HoodieSparkWriteableTestTable.of(metaClient); + } + + @Test + public void testReadingMORTableWithoutBaseFile() throws Exception { + HoodieWriteConfig config = getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA, + HoodieIndex.IndexType.INMEMORY).withAutoCommit(true) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withMaxNumDeltaCommitsBeforeCompaction(2).build()) + .build(); + SparkRDDWriteClient client = getHoodieWriteClient(config); + + // Do insert and updates thrice one after the other. + // Insert + String commitTime = HoodieActiveTimeline.createNewInstantTime(); + insertBatch(config, client, commitTime, "000", 10, SparkRDDWriteClient::insert, + false, false, 10, 10, 1, Option.empty()); + + // Update + String commitTimeBetweenPrevAndNew = commitTime; + commitTime = HoodieActiveTimeline.createNewInstantTime(); + updateBatch(config, client, commitTime, commitTimeBetweenPrevAndNew, + Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), "000", 5, SparkRDDWriteClient::upsert, + false, false, 5, 10, 2, config.populateMetaFields()); + + // Delete 5 records + String prevCommitTime = commitTime; + commitTime = HoodieActiveTimeline.createNewInstantTime(); + deleteBatch(config, client, commitTime, prevCommitTime, + "000", 2, SparkRDDWriteClient::delete, false, false, + 0, 150); + + // Verify all the records. + metaClient.reloadActiveTimeline(); + Map recordMap = GenericRecordValidationTestUtils.getRecordsMap(config, hadoopConf, dataGen); + assertEquals(8, recordMap.size()); + } + + @Test + public void testCompactionOnMORTable() throws Exception { + HoodieWriteConfig config = getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA, + HoodieIndex.IndexType.INMEMORY).withAutoCommit(true) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withMaxNumDeltaCommitsBeforeCompaction(2).build()) + .build(); + SparkRDDWriteClient client = getHoodieWriteClient(config); + + // Do insert and updates thrice one after the other. + // Insert + String commitTime = HoodieActiveTimeline.createNewInstantTime(); + insertBatch(config, client, commitTime, "000", 10, SparkRDDWriteClient::insert, + false, false, 10, 10, 1, Option.empty()); + + // Update + String commitTimeBetweenPrevAndNew = commitTime; + commitTime = HoodieActiveTimeline.createNewInstantTime(); + updateBatch(config, client, commitTime, commitTimeBetweenPrevAndNew, + Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), "000", 5, SparkRDDWriteClient::upsert, + false, false, 5, 10, 2, config.populateMetaFields()); + + // Schedule and execute compaction. + Option timeStamp = client.scheduleCompaction(Option.empty()); + assertTrue(timeStamp.isPresent()); + client.compact(timeStamp.get()); + + // Verify all the records. + metaClient.reloadActiveTimeline(); + assertDataInMORTable(config, commitTime, timeStamp.get(), hadoopConf, Arrays.asList(dataGen.getPartitionPaths())); + } + + @Test + public void testLogCompactionOnMORTable() throws Exception { + HoodieCompactionConfig compactionConfig = HoodieCompactionConfig.newBuilder() + .withLogCompactionBlocksThreshold("1") + .build(); + HoodieWriteConfig config = getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA, + HoodieIndex.IndexType.INMEMORY).withAutoCommit(true).withCompactionConfig(compactionConfig).build(); + SparkRDDWriteClient client = getHoodieWriteClient(config); + + // First insert + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + insertBatch(config, client, newCommitTime, "000", 10, + SparkRDDWriteClient::insert, false, false, 10, 100, + 1, Option.empty()); + + String prevCommitTime = newCommitTime; + for (int i = 0; i < 5; i++) { + // Upsert + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + updateBatch(config, client, newCommitTime, prevCommitTime, + Option.of(Arrays.asList(prevCommitTime)), "000", 2, SparkRDDWriteClient::upsert, + false, false, 50, 10, i + 2, config.populateMetaFields()); + prevCommitTime = newCommitTime; + } + + // Schedule and execute compaction. + Option compactionTimeStamp = client.scheduleCompaction(Option.empty()); + assertTrue(compactionTimeStamp.isPresent()); + client.compact(compactionTimeStamp.get()); + + prevCommitTime = compactionTimeStamp.get(); + //TODO: Below commits are creating duplicates when all the tests are run together. but individually they are passing. + for (int i = 0; i < 2; i++) { + // Upsert + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + updateBatch(config, client, newCommitTime, prevCommitTime, + Option.of(Arrays.asList(prevCommitTime)), "000", 2, SparkRDDWriteClient::upsert, + false, false, 50, 10, i + 8, config.populateMetaFields()); + prevCommitTime = newCommitTime; + } + String lastCommitBeforeLogCompaction = prevCommitTime; + + // Schedule and execute compaction. + Option logCompactionTimeStamp = client.scheduleLogCompaction(Option.empty()); + assertTrue(logCompactionTimeStamp.isPresent()); + client.logCompact(logCompactionTimeStamp.get()); + + // Verify all the records. + assertDataInMORTable(config, lastCommitBeforeLogCompaction, logCompactionTimeStamp.get(), + hadoopConf, Arrays.asList(dataGen.getPartitionPaths())); + } + + /** + * Test logcompaction before any compaction is scheduled. Here base file is not yet created. + */ + @Test + public void testLogCompactionOnMORTableWithoutBaseFile() throws Exception { + HoodieCompactionConfig compactionConfig = HoodieCompactionConfig.newBuilder() + .withLogCompactionBlocksThreshold("1") + .withLogRecordReaderScanV2("true") + .build(); + HoodieWriteConfig config = getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA, + HoodieIndex.IndexType.INMEMORY).withAutoCommit(true).withCompactionConfig(compactionConfig).build(); + SparkRDDWriteClient client = getHoodieWriteClient(config); + + // First insert 10 records + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + insertBatch(config, client, newCommitTime, "000", 10, + SparkRDDWriteClient::insert, false, false, 10, 10, + 1, Option.of(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)); + + // Upsert 5 records + String prevCommitTime = newCommitTime; + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + updateBatch(config, client, newCommitTime, prevCommitTime, + Option.of(Arrays.asList(prevCommitTime)), "000", 5, SparkRDDWriteClient::upsert, + false, false, 5, 10, 2, config.populateMetaFields()); + prevCommitTime = newCommitTime; + + // Delete 3 records + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + deleteBatch(config, client, newCommitTime, prevCommitTime, + "000", 3, SparkRDDWriteClient::delete, false, false, + 0, 10); + + String lastCommitBeforeLogCompaction = newCommitTime; + // Schedule and execute compaction. + Option timeStamp = client.scheduleLogCompaction(Option.empty()); + assertTrue(timeStamp.isPresent()); + client.logCompact(timeStamp.get()); + // Verify all the records. + assertDataInMORTable(config, lastCommitBeforeLogCompaction, timeStamp.get(), + hadoopConf, Arrays.asList(dataGen.getPartitionPaths())); + } + + /** + * Test scheduling logcompaction right after scheduling compaction. This should fail. + */ + @Test + public void testSchedulingLogCompactionAfterSchedulingCompaction() throws Exception { + HoodieCompactionConfig compactionConfig = HoodieCompactionConfig.newBuilder() + .withMaxNumDeltaCommitsBeforeCompaction(1) + .withLogCompactionBlocksThreshold("1") + .build(); + HoodieWriteConfig config = getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA, + HoodieIndex.IndexType.INMEMORY).withAutoCommit(true).withCompactionConfig(compactionConfig).build(); + SparkRDDWriteClient client = getHoodieWriteClient(config); + + // First insert + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + insertBatch(config, client, newCommitTime, "000", 100, + SparkRDDWriteClient::insert, false, false, 10, 100, + 1, Option.empty()); + + String prevCommitTime = newCommitTime; + // Upsert + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + updateBatch(config, client, newCommitTime, prevCommitTime, + Option.of(Arrays.asList(prevCommitTime)), "000", 50, SparkRDDWriteClient::upsert, + false, false, 50, 10, 2, config.populateMetaFields()); + + // Schedule compaction + Option compactionTimeStamp = client.scheduleCompaction(Option.empty()); + assertTrue(compactionTimeStamp.isPresent()); + + // Try scheduing log compaction, it wont succeed + Option logCompactionTimeStamp = client.scheduleLogCompaction(Option.empty()); + assertFalse(logCompactionTimeStamp.isPresent()); + } + + /** + * Test scheduling compaction right after scheduling logcompaction. This should fail. + */ + @Test + public void testSchedulingCompactionAfterSchedulingLogCompaction() throws Exception { + HoodieCompactionConfig compactionConfig = HoodieCompactionConfig.newBuilder() + .withMaxNumDeltaCommitsBeforeCompaction(1) + .withLogCompactionBlocksThreshold("1") + .build(); + HoodieWriteConfig config = getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA, + HoodieIndex.IndexType.INMEMORY).withAutoCommit(true) + .withCompactionConfig(compactionConfig) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .withAutoClean(false).build()) + .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) + // Timeline-server-based markers are not used for multi-writer tests + .withMarkersType(MarkerType.DIRECT.name()) + .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(InProcessLockProvider.class) + .build()) + .build(); + SparkRDDWriteClient client = getHoodieWriteClient(config); + + // First insert + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + insertBatch(config, client, newCommitTime, "000", 100, + SparkRDDWriteClient::insert, false, false, 10, 100, + 1, Option.empty()); + + String prevCommitTime = newCommitTime; + // Upsert + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + updateBatch(config, client, newCommitTime, prevCommitTime, + Option.of(Arrays.asList(prevCommitTime)), "000", 50, SparkRDDWriteClient::upsert, + false, false, 50, 10, 2, config.populateMetaFields()); + + // Schedule log compaction + Option logCompactionTimeStamp = client.scheduleLogCompaction(Option.empty()); + assertTrue(logCompactionTimeStamp.isPresent()); + + // Try scheduling compaction, it wont succeed + Option compactionTimeStamp = client.scheduleCompaction(Option.empty()); + assertTrue(compactionTimeStamp.isPresent()); + client.compact(compactionTimeStamp.get()); + assertThrows(Exception.class, () -> client.logCompact(logCompactionTimeStamp.get())); + } + + @Test + public void testCleanFunctionalityWhenCompactionRequestedInstantIsPresent() throws Exception { + HoodieWriteConfig config = getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA, + HoodieIndex.IndexType.INMEMORY).withAutoCommit(true) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withMaxNumDeltaCommitsBeforeCompaction(1) + .build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(4).build()) + .build(); + SparkRDDWriteClient client = getHoodieWriteClient(config); + + // First insert. Here First file slice gets added to file group. + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + insertBatch(config, client, newCommitTime, "000", 100, + SparkRDDWriteClient::insert, false, false, 10, 100, + 1, Option.empty()); + + // Schedule and execute compaction. Here, second file slice gets added. + Option compactionTimeStamp = client.scheduleCompaction(Option.empty()); + assertTrue(compactionTimeStamp.isPresent()); + client.compact(compactionTimeStamp.get()); + String prevCommitTime = compactionTimeStamp.get(); + + // First upsert on second file slice. + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + updateBatch(config, client, newCommitTime, prevCommitTime, + Option.of(Arrays.asList(prevCommitTime)), "000", 50, SparkRDDWriteClient::upsert, + false, false, 50, 10, 2, config.populateMetaFields()); + prevCommitTime = newCommitTime; + + // Schedule compaction. Third file slice gets added, compaction is not complete so base file is not created yet. + compactionTimeStamp = client.scheduleCompaction(Option.empty()); + assertTrue(compactionTimeStamp.isPresent()); + prevCommitTime = compactionTimeStamp.get(); + + for (int i = 0; i < 6; i++) { + // First upsert on third file slice. + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + updateBatch(config, client, newCommitTime, prevCommitTime, + Option.of(Arrays.asList(prevCommitTime)), "000", 50, SparkRDDWriteClient::upsert, + false, false, 50, 10, 2, config.populateMetaFields()); + prevCommitTime = newCommitTime; + if (i == 2) { + // Since retain commits is 4 exactly after 6th completed commit there will be some files to be cleaned, + // since a version older than the earliest commit is also retained. + HoodieInstant cleanInstant = metaClient.reloadActiveTimeline().lastInstant().get(); + assertEquals(HoodieTimeline.CLEAN_ACTION, cleanInstant.getAction()); + } else { + // Make sure clean is never triggered for other commits. The cleaner is blocked due to pending compaction instant. + assertEquals(HoodieTimeline.DELTA_COMMIT_ACTION, metaClient.reloadActiveTimeline().lastInstant().get().getAction()); + } + } + } + + @Test + public void testRollbackOnLogCompaction() throws Exception { + HoodieCompactionConfig compactionConfig = HoodieCompactionConfig.newBuilder() + .withLogCompactionBlocksThreshold("1") + .build(); + HoodieWriteConfig lcConfig = getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA, HoodieIndex.IndexType.INMEMORY) + .withAutoCommit(false).withCompactionConfig(compactionConfig).build(); + SparkRDDWriteClient lcClient = new SparkRDDWriteClient(context, lcConfig); + HoodieWriteConfig config = getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA, HoodieIndex.IndexType.INMEMORY) + .withAutoCommit(true).build(); + SparkRDDWriteClient client = new SparkRDDWriteClient(context, config); + + // First insert + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + insertBatch(config, client, newCommitTime, "000", 100, + SparkRDDWriteClient::insert, false, false, 10, 100, + 1, Option.empty()); + String prevCommitTime = newCommitTime; + + // Upsert + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + updateBatch(config, client, newCommitTime, prevCommitTime, + Option.of(Arrays.asList(prevCommitTime)), "000", 10, SparkRDDWriteClient::upsert, + false, false, 10, 10, 4, config.populateMetaFields()); + prevCommitTime = newCommitTime; + + // Schedule and execute logcompaction but do not commit. + Option logCompactionTimeStamp = lcClient.scheduleLogCompaction(Option.empty()); + assertTrue(logCompactionTimeStamp.isPresent()); + lcClient.logCompact(logCompactionTimeStamp.get()); + + // Rollback the log compaction commit. + HoodieInstant instant = new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.LOG_COMPACTION_ACTION, logCompactionTimeStamp.get()); + getHoodieTable(metaClient, config).rollbackInflightLogCompaction(instant); + + // Validate timeline. + HoodieTimeline activeTimeline = metaClient.reloadActiveTimeline(); + HoodieInstant rollbackInstant = activeTimeline.lastInstant().get(); + assertEquals(3, activeTimeline.countInstants()); + assertEquals(HoodieTimeline.ROLLBACK_ACTION, rollbackInstant.getAction()); + + // Validate block instant times. + validateBlockInstantsBeforeAndAfterRollback(config, prevCommitTime, rollbackInstant.getTimestamp()); + prevCommitTime = rollbackInstant.getTimestamp(); + + // Do one more upsert + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + updateBatch(config, client, newCommitTime, prevCommitTime, + Option.of(Arrays.asList(prevCommitTime)), "000", 10, SparkRDDWriteClient::upsert, + false, false, 10, 10, 4, config.populateMetaFields()); + prevCommitTime = newCommitTime; + + // Complete logcompaction now. + logCompactionTimeStamp = lcClient.scheduleLogCompaction(Option.empty()); + assertTrue(logCompactionTimeStamp.isPresent()); + HoodieWriteMetadata metadata = lcClient.logCompact(logCompactionTimeStamp.get()); + lcClient.commitLogCompaction(logCompactionTimeStamp.get(), (HoodieCommitMetadata) metadata.getCommitMetadata().get(), Option.empty()); + assertDataInMORTable(config, prevCommitTime, logCompactionTimeStamp.get(), hadoopConf, Arrays.asList(dataGen.getPartitionPaths())); + } + + private void validateBlockInstantsBeforeAndAfterRollback(HoodieWriteConfig config, String instant, String currentInstant) { + HoodieTable table = getHoodieTable(metaClient, config); + SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView(); + List partitionPaths = Stream.of(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS).collect(Collectors.toList()); + for (String partitionPath: partitionPaths) { + fileSystemView.getLatestFileSlices(partitionPath).forEach(slice -> { + HoodieUnMergedLogRecordScanner scanner = HoodieUnMergedLogRecordScanner.newBuilder() + .withFileSystem(metaClient.getFs()) + .withBasePath(table.getMetaClient().getBasePath()) + .withLogFilePaths(slice.getLogFiles() + .sorted(HoodieLogFile.getLogFileComparator()) + .map(file -> file.getPath().toString()) + .collect(Collectors.toList())) + .withLatestInstantTime(instant) + .withBufferSize(config.getMaxDFSStreamBufferSize()) + .withUseScanV2(true) + .build(); + scanner.scanInternal(Option.empty(), true); + List prevInstants = scanner.getValidBlockInstants(); + HoodieUnMergedLogRecordScanner scanner2 = HoodieUnMergedLogRecordScanner.newBuilder() + .withFileSystem(metaClient.getFs()) + .withBasePath(table.getMetaClient().getBasePath()) + .withLogFilePaths(slice.getLogFiles() + .sorted(HoodieLogFile.getLogFileComparator()) + .map(file -> file.getPath().toString()) + .collect(Collectors.toList())) + .withLatestInstantTime(currentInstant) + .withBufferSize(config.getMaxDFSStreamBufferSize()) + .withUseScanV2(true) + .build(); + scanner2.scanInternal(Option.empty(), true); + List currentInstants = scanner2.getValidBlockInstants(); + assertEquals(prevInstants, currentInstants); + }); + } + } + + @Test + public void testArchivalOnLogCompaction() throws Exception { + HoodieCompactionConfig logCompactionConfig = HoodieCompactionConfig.newBuilder() + .withLogCompactionBlocksThreshold("2") + .build(); + HoodieWriteConfig lcWriteConfig = getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA, + HoodieIndex.IndexType.INMEMORY).withAutoCommit(true).withCompactionConfig(logCompactionConfig).build(); + SparkRDDWriteClient lcWriteClient = new SparkRDDWriteClient(context, lcWriteConfig); + + HoodieCompactionConfig compactionConfig = HoodieCompactionConfig.newBuilder() + .withMaxNumDeltaCommitsBeforeCompaction(1) + .build(); + HoodieWriteConfig config = getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA, HoodieIndex.IndexType.INMEMORY) + .withAutoCommit(true).withCompactionConfig(compactionConfig) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(2).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(3, 4).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withMaxNumDeltaCommitsBeforeCompaction(2).build()) + .build(); + SparkRDDWriteClient client = new SparkRDDWriteClient(context, config); + + // First insert + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + insertBatch(config, client, newCommitTime, "000", 100, + SparkRDDWriteClient::insert, false, false, 10, 100, + 1, Option.empty()); + String prevCommitTime = newCommitTime; + List logCompactionInstantTimes = new ArrayList<>(); + + for (int i = 0; i < 6; i++) { + if (i % 4 == 0) { + // Schedule compaction. + Option compactionTimeStamp = client.scheduleCompaction(Option.empty()); + assertTrue(compactionTimeStamp.isPresent()); + client.compact(compactionTimeStamp.get()); + prevCommitTime = compactionTimeStamp.get(); + } + + // Upsert + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + updateBatch(config, client, newCommitTime, prevCommitTime, + Option.of(Arrays.asList(prevCommitTime)), "000", 50, SparkRDDWriteClient::upsert, + false, false, 50, 10, 0, config.populateMetaFields()); + // Schedule log compaction. + Option logCompactionTimeStamp = lcWriteClient.scheduleLogCompaction(Option.empty()); + if (logCompactionTimeStamp.isPresent()) { + logCompactionInstantTimes.add(logCompactionTimeStamp.get()); + lcWriteClient.logCompact(logCompactionTimeStamp.get()); + prevCommitTime = logCompactionTimeStamp.get(); + } + } + boolean logCompactionInstantArchived = false; + Map> instantsMap = metaClient.getArchivedTimeline().getInstants() + .collect(Collectors.groupingBy(HoodieInstant::getTimestamp)); + for (String logCompactionTimeStamp: logCompactionInstantTimes) { + List instants = instantsMap.get(logCompactionTimeStamp); + if (instants == null) { + continue; + } + assertEquals(3, instants.size()); + for (HoodieInstant instant: instants) { + if (instant.isCompleted()) { + assertEquals(HoodieTimeline.DELTA_COMMIT_ACTION, instant.getAction()); + } else { + assertEquals(HoodieTimeline.LOG_COMPACTION_ACTION, instant.getAction()); + } + } + logCompactionInstantArchived = true; + } + assertTrue(logCompactionInstantArchived); + } + + @Override + protected HoodieTableType getTableType() { + return HoodieTableType.MERGE_ON_READ; + } + +} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java index 18f764c1fa25f..5ba0f33f9bc53 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java @@ -18,9 +18,11 @@ package org.apache.hudi.table; +import org.apache.hudi.client.HoodieReadClient; import org.apache.hudi.client.SparkRDDReadClient; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -39,6 +41,7 @@ import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieClusteringConfig; +import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.index.HoodieIndex; @@ -80,6 +83,7 @@ import static org.apache.hudi.testutils.HoodieClientTestHarness.buildProfile; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertTrue; public class TestHoodieMergeOnReadTable extends SparkClientFunctionalTestHarness { @@ -300,6 +304,98 @@ public void testLogFileCountsAfterCompaction(boolean preserveCommitMeta) throws } } + @ParameterizedTest + @ValueSource(booleans = {true}) + public void testLogBlocksCountsAfterLogCompaction(boolean populateMetaFields) throws Exception { + + HoodieCompactionConfig compactionConfig = HoodieCompactionConfig.newBuilder() + .withInlineCompaction(false) + .withLogCompactionBlocksThreshold("1") + .build(); + // insert 100 recordsx + HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(true) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()) + .withCompactionConfig(compactionConfig); + addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields); + HoodieWriteConfig config = cfgBuilder.build(); + setUp(config.getProps()); + + try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config)) { + String newCommitTime = "100"; + writeClient.startCommitWithTime(newCommitTime); + + List records = dataGen.generateInserts(newCommitTime, 100); + JavaRDD recordsRDD = jsc().parallelize(records, 1); + writeClient.insert(recordsRDD, newCommitTime).collect(); + + // Update all the 100 records + newCommitTime = "101"; + List updatedRecords = dataGen.generateUpdates(newCommitTime, records); + JavaRDD updatedRecordsRDD = jsc().parallelize(updatedRecords, 1); + + HoodieReadClient readClient = new HoodieReadClient(context(), config); + JavaRDD updatedTaggedRecordsRDD = readClient.tagLocation(updatedRecordsRDD); + + writeClient.startCommitWithTime(newCommitTime); + writeClient.upsertPreppedRecords(updatedTaggedRecordsRDD, newCommitTime).collect(); + + + newCommitTime = "102"; + writeClient.startCommitWithTime(newCommitTime); + writeClient.upsertPreppedRecords(updatedTaggedRecordsRDD, newCommitTime).collect(); + + + // Write them to corresponding avro logfiles + metaClient = HoodieTableMetaClient.reload(metaClient); + + HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create( + writeClient.getEngineContext().getHadoopConf().get(), config, writeClient.getEngineContext()); + HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable + .of(metaClient, HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS, metadataWriter); + + Set allPartitions = updatedRecords.stream() + .map(record -> record.getPartitionPath()) + .collect(Collectors.groupingBy(partitionPath -> partitionPath)) + .keySet(); + assertEquals(allPartitions.size(), testTable.listAllBaseFiles().length); + + // Verify that all data file has one log file + HoodieTable table = HoodieSparkTable.create(config, context(), metaClient); + for (String partitionPath : dataGen.getPartitionPaths()) { + List groupedLogFiles = + table.getSliceView().getLatestFileSlices(partitionPath).collect(Collectors.toList()); + for (FileSlice fileSlice : groupedLogFiles) { + assertEquals(2, fileSlice.getLogFiles().count(), + "There should be 1 log file written for the latest data file - " + fileSlice); + } + } + + // Do a log compaction + String logCompactionInstantTime = writeClient.scheduleLogCompaction(Option.empty()).get().toString(); + HoodieWriteMetadata> result = writeClient.logCompact(logCompactionInstantTime); + + // Verify that recently written compacted data file has no log file + metaClient = HoodieTableMetaClient.reload(metaClient); + table = HoodieSparkTable.create(config, context(), metaClient); + HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); + + assertTrue(HoodieTimeline + .compareTimestamps(timeline.lastInstant().get().getTimestamp(), HoodieTimeline.GREATER_THAN, newCommitTime), + "Compaction commit should be > than last insert"); + + for (String partitionPath : dataGen.getPartitionPaths()) { + List fileSlices = + table.getSliceView().getLatestFileSlices(partitionPath).collect(Collectors.toList()); + assertEquals(1, fileSlices.size()); + for (FileSlice slice : fileSlices) { + assertEquals(3, slice.getLogFiles().count(), "After compaction there will still be one log file."); + assertNotNull(slice.getBaseFile(), "Base file is not created by log compaction operation."); + } + assertTrue(result.getCommitMetadata().get().getWritePartitionPaths().stream().anyMatch(part -> part.contentEquals(partitionPath))); + } + } + } + /** * Test to ensure metadata stats are correctly written to metadata file. */ diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/GenericRecordValidationTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/GenericRecordValidationTestUtils.java new file mode 100644 index 0000000000000..36ef74b60558a --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/GenericRecordValidationTestUtils.java @@ -0,0 +1,127 @@ +/* + * 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.testutils; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.ArrayWritable; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieValidationException; +import org.apache.hudi.hadoop.config.HoodieRealtimeConfig; +import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils; + +import java.nio.file.Paths; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.hudi.common.model.HoodieRecord.COMMIT_SEQNO_METADATA_FIELD; +import static org.apache.hudi.common.model.HoodieRecord.COMMIT_TIME_METADATA_FIELD; +import static org.apache.hudi.common.model.HoodieRecord.FILENAME_METADATA_FIELD; +import static org.apache.hudi.common.model.HoodieRecord.OPERATION_METADATA_FIELD; +import static org.apache.hudi.common.model.HoodieRecord.RECORD_KEY_METADATA_FIELD; +import static org.apache.hudi.hadoop.utils.HoodieHiveUtils.HOODIE_CONSUME_COMMIT; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class GenericRecordValidationTestUtils { + + public static void assertGenericRecords(GenericRecord record1, GenericRecord record2, + Schema schema, List excludeFields) { + for (Schema.Field f: schema.getFields()) { + String fieldName = f.name(); + if (excludeFields.contains(fieldName)) { + continue; + } + Object value1 = record1.get(fieldName); + Object value2 = record2.get(fieldName); + if (value1 != null && value2 != null) { + if (value1 instanceof ArrayWritable) { + assertEquals(HoodieRealtimeRecordReaderUtils.arrayWritableToString((ArrayWritable) value1), + HoodieRealtimeRecordReaderUtils.arrayWritableToString((ArrayWritable) value2)); + } else { + assertEquals(value1, value2, "Field name " + fieldName + " is not same." + + " Val1: " + value1 + ", Val2:" + value2); + } + } else if (value1 != null || value2 != null) { + throw new HoodieValidationException("Field name " + fieldName + " is not same." + + " Val1: " + value1 + ", Val2:" + value2); + } + } + } + + public static void assertDataInMORTable(HoodieWriteConfig config, String instant1, String instant2, + Configuration hadoopConf, List partitionPaths) { + List excludeFields = CollectionUtils.createImmutableList(COMMIT_TIME_METADATA_FIELD, COMMIT_SEQNO_METADATA_FIELD, + FILENAME_METADATA_FIELD, OPERATION_METADATA_FIELD); + assertDataInMORTable(config, instant1, instant2, hadoopConf, partitionPaths, excludeFields); + } + + public static void assertDataInMORTable(HoodieWriteConfig config, String instant1, String instant2, + Configuration hadoopConf, List partitionPaths, List excludeFields) { + JobConf jobConf = new JobConf(hadoopConf); + List fullPartitionPaths = partitionPaths.stream() + .map(partitionPath -> Paths.get(config.getBasePath(), partitionPath).toString()) + .collect(Collectors.toList()); + + jobConf.set(String.format(HOODIE_CONSUME_COMMIT, config.getTableName()), instant1); + jobConf.set(HoodieRealtimeConfig.USE_LOG_RECORD_READER_SCAN_V2, "true"); + List records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( + hadoopConf, fullPartitionPaths, config.getBasePath(), jobConf, true); + Map prevRecordsMap = records.stream() + .collect(Collectors.toMap(rec -> rec.get(RECORD_KEY_METADATA_FIELD).toString(), Function.identity())); + + jobConf.set(String.format(HOODIE_CONSUME_COMMIT, config.getTableName()), instant2); + List records1 = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( + hadoopConf, fullPartitionPaths, config.getBasePath(), jobConf, true); + Map newRecordsMap = records1.stream() + .collect(Collectors.toMap(rec -> rec.get(RECORD_KEY_METADATA_FIELD).toString(), Function.identity())); + + // Verify row count. + assertEquals(prevRecordsMap.size(), newRecordsMap.size()); + + Schema readerSchema = HoodieAvroUtils.addMetadataFields( + new Schema.Parser().parse(config.getSchema()), config.allowOperationMetadataField()); + + // Verify every field. + prevRecordsMap.forEach((key, value) -> { + assertTrue(newRecordsMap.containsKey(key)); + assertGenericRecords(value, newRecordsMap.get(key), readerSchema, excludeFields); + }); + } + + public static Map getRecordsMap(HoodieWriteConfig config, Configuration hadoopConf, + HoodieTestDataGenerator dataGen) { + JobConf jobConf = new JobConf(hadoopConf); + List fullPartitionPaths = Arrays.stream(dataGen.getPartitionPaths()) + .map(partitionPath -> Paths.get(config.getBasePath(), partitionPath).toString()) + .collect(Collectors.toList()); + return HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( + hadoopConf, fullPartitionPaths, config.getBasePath(), jobConf, true).stream() + .collect(Collectors.toMap(rec -> rec.get(RECORD_KEY_METADATA_FIELD).toString(), Function.identity())); + } + +} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java index 900674a677588..424bb6c53e37e 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java @@ -65,6 +65,7 @@ import java.util.function.Function; import java.util.stream.Collectors; +import static org.apache.hudi.common.testutils.HoodieTestUtils.RAW_TRIPS_TEST_NAME; import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -150,7 +151,7 @@ public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr, IndexType in .withCleanConfig(HoodieCleanConfig.newBuilder().withFailedWritesCleaningPolicy(cleaningPolicy).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build()) .withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).orcMaxFileSize(1024 * 1024).build()) - .forTable("test-trip-table") + .forTable(RAW_TRIPS_TEST_NAME) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build()) .withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withEnableBackupForRemoteFileSystemView(false) // Fail test if problem connecting to timeline-server diff --git a/hudi-common/pom.xml b/hudi-common/pom.xml index ae5303cc70726..87b8e5e0beea3 100644 --- a/hudi-common/pom.xml +++ b/hudi-common/pom.xml @@ -68,6 +68,7 @@ ${basedir}/src/main/avro/HoodieCommitMetadata.avsc + ${basedir}/src/main/avro/HoodieCompactionStrategy.avsc ${basedir}/src/main/avro/HoodieCompactionOperation.avsc ${basedir}/src/main/avro/HoodieSavePointMetadata.avsc ${basedir}/src/main/avro/HoodieCompactionMetadata.avsc diff --git a/hudi-common/src/main/avro/HoodieCompactionOperation.avsc b/hudi-common/src/main/avro/HoodieCompactionOperation.avsc index 2095a9518c53e..bab7321f29cae 100644 --- a/hudi-common/src/main/avro/HoodieCompactionOperation.avsc +++ b/hudi-common/src/main/avro/HoodieCompactionOperation.avsc @@ -84,6 +84,18 @@ "name":"version", "type":["int", "null"], "default": 1 + }, + { + "name":"strategy", + "type":[ + "null", "HoodieCompactionStrategy" + ], + "default": null + }, + { + "name":"preserveHoodieMetadata", + "type":["boolean", "null"], + "default": false } ] } diff --git a/hudi-common/src/main/avro/HoodieCompactionStrategy.avsc b/hudi-common/src/main/avro/HoodieCompactionStrategy.avsc new file mode 100644 index 0000000000000..eff500d1f5f18 --- /dev/null +++ b/hudi-common/src/main/avro/HoodieCompactionStrategy.avsc @@ -0,0 +1,44 @@ +/* + * 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. + */ +{ + "namespace": "org.apache.hudi.avro.model", + "name": "HoodieCompactionStrategy", + "type": "record", + "fields": [ + { + "name":"compactorClassName", + "doc": "The class name that is provided here should extend CompactionExecutionStrategy abstract class, idea is that the plan generator will also provide the execution strategy to use.", + "type":["null","string"], + "default": null + }, + { + "name":"strategyParams", + "doc": "These parameters play a key role in determining which action type it is i.e. compaction or logcompaction and it will also include configs that determine how compaction needs to be executed.", + "type":["null", { + "type":"map", + "values":"string" + }], + "default": null + }, + { + "name":"version", + "type":["int", "null"], + "default": 1 + } + ] +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java index b16373ef83436..793d79e256aff 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java @@ -234,6 +234,13 @@ public final class HoodieMetadataConfig extends HoodieConfig { + "metadata table which are never added before. This config determines how to handle " + "such spurious deletes"); + public static final ConfigProperty USE_LOG_RECORD_READER_SCAN_V2 = ConfigProperty + .key(METADATA_PREFIX + ".log.record.reader.use.scanV2") + .defaultValue(false) + .sinceVersion("0.13.0") + .withDocumentation("ScanV2 logic address all the multiwriter challenges while appending to log files. " + + "It also differentiates original blocks written by ingestion writers and compacted blocks written log compaction."); + private HoodieMetadataConfig() { super(); } @@ -318,6 +325,10 @@ public boolean ignoreSpuriousDeletes() { return getBoolean(IGNORE_SPURIOUS_DELETES); } + public boolean getUseLogRecordReaderScanV2() { + return getBoolean(USE_LOG_RECORD_READER_SCAN_V2); + } + public static class Builder { private EngineType engineType = EngineType.SPARK; @@ -461,6 +472,11 @@ public Builder withProperties(Properties properties) { return this; } + public Builder withLogRecordReaderScanV2(boolean useLogRecordReaderScanV2) { + metadataConfig.setValue(USE_LOG_RECORD_READER_SCAN_V2, String.valueOf(useLogRecordReaderScanV2)); + return this; + } + public HoodieMetadataConfig build() { metadataConfig.setDefaultValue(ENABLE, getDefaultMetadataEnable(engineType)); metadataConfig.setDefaults(HoodieMetadataConfig.class.getName()); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/ActionType.java b/hudi-common/src/main/java/org/apache/hudi/common/model/ActionType.java index c10c99d8dc8ee..eb7e578522ee8 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/ActionType.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/ActionType.java @@ -22,5 +22,5 @@ * The supported action types. */ public enum ActionType { - commit, savepoint, compaction, clean, rollback, replacecommit, deltacommit + commit, savepoint, compaction, clean, rollback, replacecommit, deltacommit, logcompaction } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/TableServiceType.java b/hudi-common/src/main/java/org/apache/hudi/common/model/TableServiceType.java index 69dd30782ff77..2aa5d08f06a8e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/TableServiceType.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/TableServiceType.java @@ -24,7 +24,7 @@ * Supported runtime table services. */ public enum TableServiceType { - ARCHIVE, COMPACT, CLUSTER, CLEAN; + ARCHIVE, COMPACT, CLUSTER, CLEAN, LOG_COMPACT; public String getAction() { switch (this) { @@ -37,6 +37,8 @@ public String getAction() { return HoodieTimeline.CLEAN_ACTION; case CLUSTER: return HoodieTimeline.REPLACE_COMMIT_ACTION; + case LOG_COMPACT: + return HoodieTimeline.LOG_COMPACTION_ACTION; default: throw new IllegalArgumentException("Unknown table service " + this); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java index f2f3809cf5c3a..69867f71ebd8f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java @@ -53,7 +53,8 @@ public enum WriteOperationType { // alter schema ALTER_SCHEMA("alter_schema"), - + // log compact + LOG_COMPACT("logcompact"), // used for old version UNKNOWN("unknown"); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java index 5bfb395dbc54c..88da6aa1f0669 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java @@ -57,17 +57,23 @@ import java.io.IOException; import java.util.ArrayDeque; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.Deque; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Properties; +import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; +import static org.apache.hudi.common.table.log.block.HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_BLOCK; +import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType.COMPACTED_BLOCK_TIMES; import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType.INSTANT_TIME; +import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME; import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.COMMAND_BLOCK; import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.CORRUPT_BLOCK; @@ -140,6 +146,10 @@ public abstract class AbstractHoodieLogRecordReader { private Option partitionName; // Populate meta fields for the records private boolean populateMetaFields = true; + // Collect all the block instants after scanning all the log files. + private List validBlockInstants = new ArrayList<>(); + // Use scanV2 method. + private boolean useScanV2; protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List logFilePaths, Schema readerSchema, @@ -147,14 +157,14 @@ protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List instantRange, boolean withOperationField) { this(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize, - instantRange, withOperationField, true, Option.empty(), InternalSchema.getEmptyInternalSchema()); + instantRange, withOperationField, true, Option.empty(), InternalSchema.getEmptyInternalSchema(), false); } protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List logFilePaths, Schema readerSchema, String latestInstantTime, boolean readBlocksLazily, boolean reverseReader, int bufferSize, Option instantRange, boolean withOperationField, boolean forceFullScan, - Option partitionName, InternalSchema internalSchema) { + Option partitionName, InternalSchema internalSchema, boolean useScanV2) { this.readerSchema = readerSchema; this.latestInstantTime = latestInstantTime; this.hoodieTableMetaClient = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(basePath).build(); @@ -176,6 +186,7 @@ protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List keys) { - scanInternal(Option.of(new KeySpec(keys, true))); + scanInternal(Option.of(new KeySpec(keys, true)), false); } - protected synchronized void scanInternal(Option keySpecOpt) { + public synchronized void scanInternal(Option keySpecOpt, boolean skipProcessingBlocks) { + if (useScanV2) { + scanInternalV2(keySpecOpt, skipProcessingBlocks); + } else { + scanInternal(keySpecOpt); + } + } + + private synchronized void scanInternal(Option keySpecOpt) { currentInstantLogBlocks = new ArrayDeque<>(); progress = 0.0f; totalLogFiles = new AtomicLong(0); @@ -295,7 +314,7 @@ protected synchronized void scanInternal(Option keySpecOpt) { String targetInstantForCommandBlock = logBlock.getLogBlockHeader().get(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME); switch (commandBlock.getType()) { // there can be different types of command blocks - case ROLLBACK_PREVIOUS_BLOCK: + case ROLLBACK_BLOCK: // Rollback the last read log block // Get commit time from last record block, compare with targetCommitTime, // rollback only if equal, this is required in scenarios of invalid/extra @@ -368,6 +387,232 @@ protected synchronized void scanInternal(Option keySpecOpt) { } } + private synchronized void scanInternalV2(Option keySpecOption, boolean skipProcessingBlocks) { + currentInstantLogBlocks = new ArrayDeque<>(); + progress = 0.0f; + totalLogFiles = new AtomicLong(0); + totalRollbacks = new AtomicLong(0); + totalCorruptBlocks = new AtomicLong(0); + totalLogBlocks = new AtomicLong(0); + totalLogRecords = new AtomicLong(0); + HoodieLogFormatReader logFormatReaderWrapper = null; + HoodieTimeline commitsTimeline = this.hoodieTableMetaClient.getCommitsTimeline(); + HoodieTimeline completedInstantsTimeline = commitsTimeline.filterCompletedInstants(); + HoodieTimeline inflightInstantsTimeline = commitsTimeline.filterInflights(); + try { + + // Get the key field based on populate meta fields config + // and the table type + final String keyField = getKeyField(); + + boolean enableRecordLookups = !forceFullScan; + // Iterate over the paths + logFormatReaderWrapper = new HoodieLogFormatReader(fs, + logFilePaths.stream().map(logFile -> new HoodieLogFile(new Path(logFile))).collect(Collectors.toList()), + readerSchema, readBlocksLazily, reverseReader, bufferSize, enableRecordLookups, keyField, internalSchema); + + /** + * Scanning log blocks and placing the compacted blocks at the right place require two traversals. + * First traversal to identify the rollback blocks and valid data and compacted blocks. + * + * Scanning blocks is easy to do in single writer mode, where the rollback block is right after the effected data blocks. + * With multiwriter mode the blocks can be out of sync. An example scenario. + * B1, B2, B3, B4, R1(B3), B5 + * In this case, rollback block R1 is invalidating the B3 which is not the previous block. + * This becomes more complicated if we have compacted blocks, which are data blocks created using log compaction. + * + * To solve this, run a single traversal, collect all the valid blocks that are not corrupted + * along with the block instant times and rollback block's target instant times. + * + * As part of second traversal iterate block instant times in reverse order. + * While iterating in reverse order keep a track of final compacted instant times for each block. + * In doing so, when a data block is seen include the final compacted block if it is not already added. + * + * find the final compacted block which contains the merged contents. + * For example B1 and B2 are merged and created a compacted block called M1 and now M1, B3 and B4 are merged and + * created another compacted block called M2. So, now M2 is the final block which contains all the changes of B1,B2,B3,B4. + * So, blockTimeToCompactionBlockTimeMap will look like + * (B1 -> M2), (B2 -> M2), (B3 -> M2), (B4 -> M2), (M1 -> M2) + * This map is updated while iterating and is used to place the compacted blocks in the correct position. + * This way we can have multiple layers of merge blocks and still be able to find the correct positions of merged blocks. + */ + + // Collect targetRollbackInstants, using which we can determine which blocks are invalid. + Set targetRollbackInstants = new HashSet<>(); + + // This holds block instant time to list of blocks. Note here the log blocks can be normal data blocks or compacted log blocks. + Map> instantToBlocksMap = new HashMap<>(); + + // Order of Instants. + List orderedInstantsList = new ArrayList<>(); + + Set scannedLogFiles = new HashSet<>(); + + /* + * 1. First step to traverse in forward direction. While traversing the log blocks collect following, + * a. instant times + * b. instant to logblocks map. + * c. targetRollbackInstants. + */ + while (logFormatReaderWrapper.hasNext()) { + HoodieLogFile logFile = logFormatReaderWrapper.getLogFile(); + LOG.info("Scanning log file " + logFile); + scannedLogFiles.add(logFile); + totalLogFiles.set(scannedLogFiles.size()); + // Use the HoodieLogFileReader to iterate through the blocks in the log file + HoodieLogBlock logBlock = logFormatReaderWrapper.next(); + final String instantTime = logBlock.getLogBlockHeader().get(INSTANT_TIME); + totalLogBlocks.incrementAndGet(); + // Ignore the corrupt blocks. No further handling is required for them. + if (logBlock.getBlockType().equals(CORRUPT_BLOCK)) { + LOG.info("Found a corrupt block in " + logFile.getPath()); + totalCorruptBlocks.incrementAndGet(); + continue; + } + if (!HoodieTimeline.compareTimestamps(logBlock.getLogBlockHeader().get(INSTANT_TIME), + HoodieTimeline.LESSER_THAN_OR_EQUALS, this.latestInstantTime)) { + // hit a block with instant time greater than should be processed, stop processing further + break; + } + if (logBlock.getBlockType() != COMMAND_BLOCK) { + if (!completedInstantsTimeline.containsOrBeforeTimelineStarts(instantTime) + || inflightInstantsTimeline.containsInstant(instantTime)) { + // hit an uncommitted block possibly from a failed write, move to the next one and skip processing this one + continue; + } + if (instantRange.isPresent() && !instantRange.get().isInRange(instantTime)) { + // filter the log block by instant range + continue; + } + } + + switch (logBlock.getBlockType()) { + case HFILE_DATA_BLOCK: + case AVRO_DATA_BLOCK: + case DELETE_BLOCK: + List logBlocksList = instantToBlocksMap.getOrDefault(instantTime, new ArrayList<>()); + if (logBlocksList.size() == 0) { + // Keep a track of instant Times in the order of arrival. + orderedInstantsList.add(instantTime); + } + logBlocksList.add(logBlock); + instantToBlocksMap.put(instantTime, logBlocksList); + break; + case COMMAND_BLOCK: + LOG.info("Reading a command block from file " + logFile.getPath()); + // This is a command block - take appropriate action based on the command + HoodieCommandBlock commandBlock = (HoodieCommandBlock) logBlock; + + // Rollback blocks contain information of instants that are failed, collect them in a set.. + if (commandBlock.getType().equals(ROLLBACK_BLOCK)) { + totalRollbacks.incrementAndGet(); + String targetInstantForCommandBlock = + logBlock.getLogBlockHeader().get(TARGET_INSTANT_TIME); + targetRollbackInstants.add(targetInstantForCommandBlock); + } else { + throw new UnsupportedOperationException("Command type not yet supported."); + } + break; + default: + throw new UnsupportedOperationException("Block type not yet supported."); + } + } + + if (LOG.isDebugEnabled()) { + LOG.debug("Ordered instant times seen " + orderedInstantsList); + } + + int numBlocksRolledBack = 0; + + // All the block's instants time that are added to the queue are collected in this set. + Set instantTimesIncluded = new HashSet<>(); + + // Key will have details related to instant time and value will be empty if that instant is not compacted. + // Ex: B1(i1), B2(i2), CB(i3,[i1,i2]) entries will be like i1 -> i3, i2 -> i3. + Map blockTimeToCompactionBlockTimeMap = new HashMap<>(); + + /* + * 2. Iterate the instants list in reverse order to get the latest instants first. + * While iterating update the blockTimeToCompactionBlockTimesMap and include the compacted blocks in right position. + */ + for (int i = orderedInstantsList.size() - 1; i >= 0; i--) { + String instantTime = orderedInstantsList.get(i); + + // Exclude the blocks which are included in targetRollbackInstants set. + // Here, rollback can include instants affiliated to deltacommits or log compaction commits. + if (targetRollbackInstants.contains(instantTime)) { + numBlocksRolledBack += instantToBlocksMap.get(instantTime).size(); + continue; + } + List instantsBlocks = instantToBlocksMap.get(instantTime); + if (instantsBlocks.size() == 0) { + throw new HoodieException("Data corrupted while writing. Found zero blocks for an instant " + instantTime); + } + HoodieLogBlock firstBlock = instantsBlocks.get(0); + + // For compacted blocks COMPACTED_BLOCK_TIMES entry is present under its headers. + if (firstBlock.getLogBlockHeader().containsKey(COMPACTED_BLOCK_TIMES)) { + // When compacted blocks are seen update the blockTimeToCompactionBlockTimeMap. + Arrays.stream(firstBlock.getLogBlockHeader().get(COMPACTED_BLOCK_TIMES).split(",")) + .forEach(originalInstant -> { + String finalInstant = blockTimeToCompactionBlockTimeMap.getOrDefault(instantTime, instantTime); + blockTimeToCompactionBlockTimeMap.put(originalInstant, finalInstant); + }); + } else { + // When a data block is found check if it is already compacted. + String compactedFinalInstantTime = blockTimeToCompactionBlockTimeMap.get(instantTime); + if (compactedFinalInstantTime == null) { + // If it is not compacted then add the blocks related to the instant time at the end of the queue and continue. + List logBlocks = instantToBlocksMap.get(instantTime); + Collections.reverse(logBlocks); + logBlocks.forEach(block -> currentInstantLogBlocks.addLast(block)); + instantTimesIncluded.add(instantTime); + validBlockInstants.add(instantTime); + continue; + } + // If the compacted block exists and it is already included in the dequeue then ignore and continue. + if (instantTimesIncluded.contains(compactedFinalInstantTime)) { + continue; + } + // If the compacted block exists and it is not already added then add all the blocks related to that instant time. + List logBlocks = instantToBlocksMap.get(compactedFinalInstantTime); + Collections.reverse(logBlocks); + logBlocks.forEach(block -> currentInstantLogBlocks.addLast(block)); + instantTimesIncluded.add(compactedFinalInstantTime); + validBlockInstants.add(compactedFinalInstantTime); + } + } + LOG.info("Number of applied rollback blocks " + numBlocksRolledBack); + + if (LOG.isDebugEnabled()) { + LOG.info("Final view of the Block time to compactionBlockMap " + blockTimeToCompactionBlockTimeMap); + } + + // merge the last read block when all the blocks are done reading + if (!currentInstantLogBlocks.isEmpty() && !skipProcessingBlocks) { + LOG.info("Merging the final data blocks"); + processQueuedBlocksForInstant(currentInstantLogBlocks, scannedLogFiles.size(), keySpecOption); + } + // Done + progress = 1.0f; + } catch (IOException e) { + LOG.error("Got IOException when reading log file", e); + throw new HoodieIOException("IOException when reading log file ", e); + } catch (Exception e) { + LOG.error("Got exception when reading log file", e); + throw new HoodieException("Exception when reading log file ", e); + } finally { + try { + if (null != logFormatReaderWrapper) { + logFormatReaderWrapper.close(); + } + } catch (IOException ioe) { + // Eat exception as we do not want to mask the original exception that can happen + LOG.error("Unable to close log format reader", ioe); + } + } + } + /** * Checks if the current logblock belongs to a later instant. */ @@ -551,6 +796,14 @@ public KeySpec(List keys, boolean fullKey) { } } + public Deque getCurrentInstantLogBlocks() { + return currentInstantLogBlocks; + } + + public List getValidBlockInstants() { + return validBlockInstants; + } + /** * Builder used to build {@code AbstractHoodieLogRecordScanner}. */ @@ -584,6 +837,10 @@ public Builder withOperationField(boolean withOperationField) { throw new UnsupportedOperationException(); } + public Builder withUseScanV2(boolean useScanV2) { + throw new UnsupportedOperationException(); + } + public abstract AbstractHoodieLogRecordReader build(); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java index d7e725544aa65..756e031cea6c7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java @@ -88,10 +88,11 @@ protected HoodieMergedLogRecordScanner(FileSystem fs, String basePath, List partitionName, InternalSchema internalSchema) { + Option partitionName, InternalSchema internalSchema, + boolean useScanV2) { super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize, instantRange, withOperationField, - forceFullScan, partitionName, internalSchema); + forceFullScan, partitionName, internalSchema, useScanV2); try { // Store merged records for all versions for this log file, set the in-memory footprint to maxInMemoryMapSize this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, spillableMapBasePath, new DefaultSizeEstimator(), @@ -156,7 +157,11 @@ protected void processNextRecord(HoodieRecord hoo // If combinedValue is oldValue, no need rePut oldRecord if (combinedValue != oldValue) { HoodieOperation operation = hoodieRecord.getOperation(); - records.put(key, new HoodieAvroRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue, operation)); + HoodieRecord latestHoodieRecord = new HoodieAvroRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue, operation); + latestHoodieRecord.unseal(); + latestHoodieRecord.setCurrentLocation(hoodieRecord.getCurrentLocation()); + latestHoodieRecord.seal(); + records.put(key, latestHoodieRecord); } } else { // Put the record as is @@ -221,8 +226,12 @@ public static class Builder extends AbstractHoodieLogRecordReader.Builder { // incremental filtering protected Option instantRange = Option.empty(); protected String partitionName; + // auto scan default true + private boolean autoScan = true; // operation field default false private boolean withOperationField = false; + // Use scanV2 method. + private boolean useScanV2 = false; @Override public Builder withFileSystem(FileSystem fs) { @@ -316,6 +325,12 @@ public Builder withPartition(String partitionName) { return this; } + @Override + public Builder withUseScanV2(boolean useScanV2) { + this.useScanV2 = useScanV2; + return this; + } + @Override public HoodieMergedLogRecordScanner build() { if (this.partitionName == null && CollectionUtils.nonEmpty(this.logFilePaths)) { @@ -325,7 +340,7 @@ public HoodieMergedLogRecordScanner build() { latestInstantTime, maxMemorySizeInBytes, readBlocksLazily, reverseReader, bufferSize, spillableMapBasePath, instantRange, diskMapType, isBitCaskDiskMapCompressionEnabled, withOperationField, true, - Option.ofNullable(partitionName), internalSchema); + Option.ofNullable(partitionName), internalSchema, useScanV2); } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java index 8ea34d6f2fa0d..7eb87e8e7fdb4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java @@ -25,6 +25,7 @@ import org.apache.avro.Schema; import org.apache.hadoop.fs.FileSystem; +import org.apache.hudi.internal.schema.InternalSchema; import java.util.List; @@ -37,8 +38,9 @@ public class HoodieUnMergedLogRecordScanner extends AbstractHoodieLogRecordReade private HoodieUnMergedLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, Schema readerSchema, String latestInstantTime, boolean readBlocksLazily, boolean reverseReader, int bufferSize, - LogRecordScannerCallback callback, Option instantRange) { - super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize, instantRange, false); + LogRecordScannerCallback callback, Option instantRange, boolean useScanV2) { + super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize, instantRange, + false, true, Option.empty(), InternalSchema.getEmptyInternalSchema(), useScanV2); this.callback = callback; } @@ -84,6 +86,7 @@ public static class Builder extends AbstractHoodieLogRecordReader.Builder { private Option instantRange = Option.empty(); // specific configurations private LogRecordScannerCallback callback; + private boolean useScanV2; public Builder withFileSystem(FileSystem fs) { this.fs = fs; @@ -135,10 +138,16 @@ public Builder withLogRecordScannerCallback(LogRecordScannerCallback callback) { return this; } + @Override + public Builder withUseScanV2(boolean useScanV2) { + this.useScanV2 = useScanV2; + return this; + } + @Override public HoodieUnMergedLogRecordScanner build() { return new HoodieUnMergedLogRecordScanner(fs, basePath, logFilePaths, readerSchema, - latestInstantTime, readBlocksLazily, reverseReader, bufferSize, callback, instantRange); + latestInstantTime, readBlocksLazily, reverseReader, bufferSize, callback, instantRange, useScanV2); } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieCommandBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieCommandBlock.java index 0ff3a77b5007b..c44f1950144b5 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieCommandBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieCommandBlock.java @@ -36,7 +36,7 @@ public class HoodieCommandBlock extends HoodieLogBlock { * Hoodie command block type enum. */ public enum HoodieCommandBlockTypeEnum { - ROLLBACK_PREVIOUS_BLOCK + ROLLBACK_BLOCK } public HoodieCommandBlock(Map header) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieLogBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieLogBlock.java index 1718e7dd02457..f8307333d257c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieLogBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieLogBlock.java @@ -113,6 +113,14 @@ public Option getContent() { return content; } + /** + * Compacted blocks are created using log compaction which basically merges the consecutive blocks together and create + * huge block with all the changes. + */ + public boolean isCompactedLogBlock() { + return logBlockHeader.containsKey(HeaderMetadataType.COMPACTED_BLOCK_TIMES); + } + /** * Type of the log block WARNING: This enum is serialized as the ordinal. Only add new enums at the end. */ @@ -144,7 +152,7 @@ public static HoodieLogBlockType fromId(String id) { * new enums at the end. */ public enum HeaderMetadataType { - INSTANT_TIME, TARGET_INSTANT_TIME, SCHEMA, COMMAND_BLOCK_TYPE + INSTANT_TIME, TARGET_INSTANT_TIME, SCHEMA, COMMAND_BLOCK_TYPE, COMPACTED_BLOCK_TIMES } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java index 2b27d3ab5e568..be2febdff3f57 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java @@ -70,6 +70,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline { CLEAN_EXTENSION, REQUESTED_CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION, INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION, REQUESTED_RESTORE_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION, + INFLIGHT_LOG_COMPACTION_EXTENSION, REQUESTED_LOG_COMPACTION_EXTENSION, ROLLBACK_EXTENSION, REQUESTED_ROLLBACK_EXTENSION, INFLIGHT_ROLLBACK_EXTENSION, REQUESTED_REPLACE_COMMIT_EXTENSION, INFLIGHT_REPLACE_COMMIT_EXTENSION, REPLACE_COMMIT_EXTENSION, REQUESTED_INDEX_COMMIT_EXTENSION, INFLIGHT_INDEX_COMMIT_EXTENSION, INDEX_COMMIT_EXTENSION, @@ -225,7 +226,7 @@ public void saveAsComplete(HoodieInstant instant, Option data) { public HoodieInstant revertToInflight(HoodieInstant instant) { LOG.info("Reverting instant to inflight " + instant); - HoodieInstant inflight = HoodieTimeline.getInflightInstant(instant, metaClient.getTableType()); + HoodieInstant inflight = HoodieTimeline.getInflightInstant(instant, metaClient); revertCompleteToInflight(instant, inflight); LOG.info("Reverted " + instant + " to inflight " + inflight); return inflight; @@ -406,6 +407,27 @@ public HoodieInstant revertInstantFromInflightToRequested(HoodieInstant inflight return requestedInstant; } + /** + * TODO: This method is not needed, since log compaction plan is not a immutable plan. + * Revert logcompaction State from inflight to requested. + * + * @param inflightInstant Inflight Instant + * @return requested instant + */ + public HoodieInstant revertLogCompactionInflightToRequested(HoodieInstant inflightInstant) { + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION)); + ValidationUtils.checkArgument(inflightInstant.isInflight()); + HoodieInstant requestedInstant = + new HoodieInstant(State.REQUESTED, LOG_COMPACTION_ACTION, inflightInstant.getTimestamp()); + if (metaClient.getTimelineLayoutVersion().isNullVersion()) { + // Pass empty data since it is read from the corresponding .aux/.compaction instant file + transitionState(inflightInstant, requestedInstant, Option.empty()); + } else { + deleteInflight(inflightInstant); + } + return requestedInstant; + } + /** * Transition Compaction State from requested to inflight. * @@ -421,6 +443,21 @@ public HoodieInstant transitionCompactionRequestedToInflight(HoodieInstant reque return inflightInstant; } + /** + * Transition LogCompaction State from requested to inflight. + * + * @param requestedInstant Requested instant + * @return inflight instant + */ + public HoodieInstant transitionLogCompactionRequestedToInflight(HoodieInstant requestedInstant) { + ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION)); + ValidationUtils.checkArgument(requestedInstant.isRequested()); + HoodieInstant inflightInstant = + new HoodieInstant(State.INFLIGHT, LOG_COMPACTION_ACTION, requestedInstant.getTimestamp()); + transitionState(requestedInstant, inflightInstant, Option.empty()); + return inflightInstant; + } + /** * Transition Compaction State from inflight to Committed. * @@ -436,6 +473,21 @@ public HoodieInstant transitionCompactionInflightToComplete(HoodieInstant inflig return commitInstant; } + /** + * Transition Log Compaction State from inflight to Committed. + * + * @param inflightInstant Inflight instant + * @param data Extra Metadata + * @return commit instant + */ + public HoodieInstant transitionLogCompactionInflightToComplete(HoodieInstant inflightInstant, Option data) { + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION)); + ValidationUtils.checkArgument(inflightInstant.isInflight()); + HoodieInstant commitInstant = new HoodieInstant(State.COMPLETED, DELTA_COMMIT_ACTION, inflightInstant.getTimestamp()); + transitionState(inflightInstant, commitInstant, data); + return commitInstant; + } + private void createFileInAuxiliaryFolder(HoodieInstant instant, Option data) { // This will be removed in future release. See HUDI-546 Path fullPath = new Path(metaClient.getMetaAuxiliaryPath(), instant.getFileName()); @@ -653,6 +705,17 @@ public void saveToCompactionRequested(HoodieInstant instant, Option cont createFileInMetaPath(instant.getFileName(), content, overwrite); } + public void saveToLogCompactionRequested(HoodieInstant instant, Option content) { + saveToLogCompactionRequested(instant, content, false); + } + + public void saveToLogCompactionRequested(HoodieInstant instant, Option content, boolean overwrite) { + ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION)); + // Write workload to auxiliary folder + createFileInAuxiliaryFolder(instant, content); + createFileInMetaPath(instant.getFileName(), content, overwrite); + } + /** * Saves content for requested REPLACE instant. */ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java index 4df30b115e0ea..2f0ebffbefba3 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java @@ -205,6 +205,7 @@ private Option getMetadataKey(String action) { case HoodieTimeline.SAVEPOINT_ACTION: return Option.of("hoodieSavePointMetadata"); case HoodieTimeline.COMPACTION_ACTION: + case HoodieTimeline.LOG_COMPACTION_ACTION: return Option.of("hoodieCompactionPlan"); case HoodieTimeline.REPLACE_COMMIT_ACTION: return Option.of("hoodieReplaceCommitMetadata"); @@ -363,7 +364,7 @@ private int getArchivedFileSuffix(FileStatus f) { @Override public HoodieDefaultTimeline getWriteTimeline() { // filter in-memory instants - Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, REPLACE_COMMIT_ACTION); + Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, LOG_COMPACTION_ACTION, REPLACE_COMMIT_ACTION); return new HoodieDefaultTimeline(getInstants().filter(i -> readCommits.containsKey(i.getTimestamp())) .filter(s -> validActions.contains(s.getAction())), details); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java index 7324421894c0d..0803faeab27cd 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java @@ -97,6 +97,20 @@ public HoodieTimeline filterPendingExcludingCompaction() { && (!instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION))), details); } + @Override + public HoodieTimeline filterPendingExcludingLogCompaction() { + return new HoodieDefaultTimeline(instants.stream().filter(instant -> (!instant.isCompleted()) + && (!instant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION))), details); + } + + //TODO: Use a better naming convention for this. + @Override + public HoodieTimeline filterPendingExcludingMajorAndMinorCompaction() { + return new HoodieDefaultTimeline(instants.stream().filter(instant -> (!instant.isCompleted()) + && (!instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION) + || !instant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION))), details); + } + @Override public HoodieTimeline filterCompletedInstants() { return new HoodieDefaultTimeline(instants.stream().filter(HoodieInstant::isCompleted), details); @@ -108,9 +122,21 @@ public HoodieTimeline filterCompletedAndCompactionInstants() { || s.getAction().equals(HoodieTimeline.COMPACTION_ACTION)), details); } + @Override + public HoodieTimeline filterCompletedOrMajorOrMinorCompactionInstants() { + return new HoodieDefaultTimeline(instants.stream().filter(s -> s.isCompleted() + || s.getAction().equals(HoodieTimeline.COMPACTION_ACTION) || s.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION)), details); + } + + @Override + public HoodieDefaultTimeline filterCompletedInstantsOrRewriteTimeline() { + Set validActions = CollectionUtils.createSet(COMPACTION_ACTION, LOG_COMPACTION_ACTION, REPLACE_COMMIT_ACTION); + return new HoodieDefaultTimeline(instants.stream().filter(s -> s.isCompleted() || validActions.contains(s.getAction())), details); + } + @Override public HoodieDefaultTimeline getWriteTimeline() { - Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, REPLACE_COMMIT_ACTION); + Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, LOG_COMPACTION_ACTION, REPLACE_COMMIT_ACTION); return new HoodieDefaultTimeline(instants.stream().filter(s -> validActions.contains(s.getAction())), details); } @@ -148,6 +174,23 @@ public HoodieTimeline filterPendingCompactionTimeline() { instants.stream().filter(s -> s.getAction().equals(HoodieTimeline.COMPACTION_ACTION) && !s.isCompleted()), details); } + @Override + public HoodieTimeline filterPendingLogCompactionTimeline() { + return new HoodieDefaultTimeline( + instants.stream().filter(s -> s.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION) && !s.isCompleted()), details); + } + + /** + * Compaction and logcompaction operation on MOR table is called major and minor compaction respectively. + */ + @Override + public HoodieTimeline filterPendingMajorOrMinorCompactionTimeline() { + return new HoodieDefaultTimeline( + instants.stream().filter(s -> s.getAction().equals(HoodieTimeline.COMPACTION_ACTION) + || s.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION) + && !s.isCompleted()), details); + } + @Override public HoodieDefaultTimeline findInstantsInRange(String startTs, String endTs) { return new HoodieDefaultTimeline( @@ -216,13 +259,15 @@ public HoodieTimeline getCommitsTimeline() { */ public HoodieTimeline getAllCommitsTimeline() { return getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, - CLEAN_ACTION, COMPACTION_ACTION, SAVEPOINT_ACTION, ROLLBACK_ACTION, REPLACE_COMMIT_ACTION, INDEXING_ACTION)); + CLEAN_ACTION, COMPACTION_ACTION, SAVEPOINT_ACTION, ROLLBACK_ACTION, REPLACE_COMMIT_ACTION, INDEXING_ACTION, + LOG_COMPACTION_ACTION)); } /** * Get only pure commits (inflight and completed) in the active timeline. */ public HoodieTimeline getCommitTimeline() { + //TODO: Make sure this change does not break existing functionality. return getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, REPLACE_COMMIT_ACTION)); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java index 8b1cb875c09f6..0115742e07a08 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java @@ -18,11 +18,11 @@ package org.apache.hudi.common.table.timeline; -import org.apache.hudi.common.util.CollectionUtils; import org.apache.hadoop.fs.FileStatus; import java.io.Serializable; import java.util.Comparator; +import java.util.HashMap; import java.util.Map; import java.util.Objects; @@ -38,8 +38,7 @@ public class HoodieInstant implements Serializable, Comparable { * A COMPACTION action eventually becomes COMMIT when completed. So, when grouping instants * for state transitions, this needs to be taken into account */ - private static final Map COMPARABLE_ACTIONS = - CollectionUtils.createImmutableMap(HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.COMMIT_ACTION); + private static final Map COMPARABLE_ACTIONS = createComparableActionsMap(); public static final Comparator ACTION_COMPARATOR = Comparator.comparing(instant -> getComparableAction(instant.getAction())); @@ -164,6 +163,14 @@ public String getFileName() { } else { return HoodieTimeline.makeCommitFileName(timestamp); } + } else if (HoodieTimeline.LOG_COMPACTION_ACTION.equals(action)) { + if (isInflight()) { + return HoodieTimeline.makeInflightLogCompactionFileName(timestamp); + } else if (isRequested()) { + return HoodieTimeline.makeRequestedLogCompactionFileName(timestamp); + } else { + return HoodieTimeline.makeDeltaFileName(timestamp); + } } else if (HoodieTimeline.RESTORE_ACTION.equals(action)) { return isInflight() ? HoodieTimeline.makeInflightRestoreFileName(timestamp) : isRequested() ? HoodieTimeline.makeRequestedRestoreFileName(timestamp) @@ -184,6 +191,13 @@ public String getFileName() { throw new IllegalArgumentException("Cannot get file name for unknown action " + action); } + private static final Map createComparableActionsMap() { + Map comparableMap = new HashMap<>(); + comparableMap.put(HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.COMMIT_ACTION); + comparableMap.put(HoodieTimeline.LOG_COMPACTION_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION); + return comparableMap; + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java index e52a2795969ab..a210783095019 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java @@ -53,6 +53,7 @@ public interface HoodieTimeline extends Serializable { // With Async Compaction, compaction instant can be in 3 states : // (compaction-requested), (compaction-inflight), (completed) String COMPACTION_ACTION = "compaction"; + String LOG_COMPACTION_ACTION = "logcompaction"; String REQUESTED_EXTENSION = ".requested"; String RESTORE_ACTION = "restore"; String INDEXING_ACTION = "indexing"; @@ -93,6 +94,10 @@ public interface HoodieTimeline extends Serializable { String SAVE_SCHEMA_ACTION_EXTENSION = "." + SCHEMA_COMMIT_ACTION; String INFLIGHT_SAVE_SCHEMA_ACTION_EXTENSION = "." + SCHEMA_COMMIT_ACTION + INFLIGHT_EXTENSION; String REQUESTED_SAVE_SCHEMA_ACTION_EXTENSION = "." + SCHEMA_COMMIT_ACTION + REQUESTED_EXTENSION; + // Log compaction action + String REQUESTED_LOG_COMPACTION_SUFFIX = StringUtils.join(LOG_COMPACTION_ACTION, REQUESTED_EXTENSION); + String REQUESTED_LOG_COMPACTION_EXTENSION = StringUtils.join(".", REQUESTED_LOG_COMPACTION_SUFFIX); + String INFLIGHT_LOG_COMPACTION_EXTENSION = StringUtils.join(".", LOG_COMPACTION_ACTION, INFLIGHT_EXTENSION); String INVALID_INSTANT_TS = "0"; @@ -124,6 +129,20 @@ public interface HoodieTimeline extends Serializable { */ HoodieTimeline filterPendingExcludingCompaction(); + /** + * Filter this timeline to just include the in-flights excluding logcompaction instants. + * + * @return New instance of HoodieTimeline with just in-flights excluding compaction instants + */ + HoodieTimeline filterPendingExcludingLogCompaction(); + + /** + * Filter this timeline to just include the in-flights excluding major and minor compaction instants. + * + * @return New instance of HoodieTimeline with just in-flights excluding majoe and minor compaction instants + */ + HoodieTimeline filterPendingExcludingMajorAndMinorCompaction(); + /** * Filter this timeline to just include the completed instants. * @@ -131,6 +150,7 @@ public interface HoodieTimeline extends Serializable { */ HoodieTimeline filterCompletedInstants(); + // TODO: Check if logcompaction also needs to be included in this API. /** * Filter this timeline to just include the completed + compaction (inflight + requested) instants A RT filesystem * view is constructed with this timeline so that file-slice after pending compaction-requested instant-time is also @@ -141,6 +161,15 @@ public interface HoodieTimeline extends Serializable { */ HoodieTimeline filterCompletedAndCompactionInstants(); + HoodieTimeline filterCompletedOrMajorOrMinorCompactionInstants(); + + /** + * Timeline to just include completed commits or all rewrites like compaction, logcompaction and replace actions. + * + * @return + */ + HoodieTimeline filterCompletedInstantsOrRewriteTimeline(); + /** * Timeline to just include commits (commit/deltacommit), compaction and replace actions. * @@ -171,6 +200,20 @@ public interface HoodieTimeline extends Serializable { */ HoodieTimeline filterPendingCompactionTimeline(); + /** + * Filter this timeline to just include requested and inflight log compaction instants. + * + * @return + */ + HoodieTimeline filterPendingLogCompactionTimeline(); + + /** + * Filter this timeline to just include requested and inflight from both major and minor compaction instants. + * + * @return + */ + HoodieTimeline filterPendingMajorOrMinorCompactionTimeline(); + /** * Filter this timeline to just include requested and inflight replacecommit instants. */ @@ -366,6 +409,16 @@ static HoodieInstant getCompactionInflightInstant(final String timestamp) { return new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, timestamp); } + // Returns Log compaction requested instant + static HoodieInstant getLogCompactionRequestedInstant(final String timestamp) { + return new HoodieInstant(State.REQUESTED, LOG_COMPACTION_ACTION, timestamp); + } + + // Returns Log compaction inflight instant + static HoodieInstant getLogCompactionInflightInstant(final String timestamp) { + return new HoodieInstant(State.INFLIGHT, LOG_COMPACTION_ACTION, timestamp); + } + static HoodieInstant getReplaceCommitRequestedInstant(final String timestamp) { return new HoodieInstant(State.REQUESTED, REPLACE_COMMIT_ACTION, timestamp); } @@ -388,14 +441,26 @@ static HoodieInstant getIndexInflightInstant(final String timestamp) { /** * Returns the inflight instant corresponding to the instant being passed. Takes care of changes in action names - * between inflight and completed instants (compaction <=> commit). + * between inflight and completed instants (compaction <=> commit) and (logcompaction <==> deltacommit). * @param instant Hoodie Instant - * @param tableType Hoodie Table Type + * @param metaClient Hoodie metaClient to fetch tableType and fileSystem. * @return Inflight Hoodie Instant */ - static HoodieInstant getInflightInstant(final HoodieInstant instant, final HoodieTableType tableType) { - if ((tableType == HoodieTableType.MERGE_ON_READ) && instant.getAction().equals(COMMIT_ACTION)) { - return new HoodieInstant(true, COMPACTION_ACTION, instant.getTimestamp()); + static HoodieInstant getInflightInstant(final HoodieInstant instant, final HoodieTableMetaClient metaClient) { + if (metaClient.getTableType() == HoodieTableType.MERGE_ON_READ) { + if (instant.getAction().equals(COMMIT_ACTION)) { + return new HoodieInstant(true, COMPACTION_ACTION, instant.getTimestamp()); + } else if (instant.getAction().equals(DELTA_COMMIT_ACTION)) { + // Deltacommit is used by both ingestion and logcompaction. + // So, distinguish both of them check for the inflight file being present. + HoodieActiveTimeline rawActiveTimeline = new HoodieActiveTimeline(metaClient, false); + Option logCompactionInstant = Option.fromJavaOptional(rawActiveTimeline.getInstants() + .filter(hoodieInstant -> hoodieInstant.getTimestamp().equals(instant.getTimestamp()) + && LOG_COMPACTION_ACTION.equals(hoodieInstant.getAction())).findFirst()); + if (logCompactionInstant.isPresent()) { + return new HoodieInstant(true, LOG_COMPACTION_ACTION, instant.getTimestamp()); + } + } } return new HoodieInstant(true, instant.getAction(), instant.getTimestamp()); } @@ -464,6 +529,15 @@ static String makeRequestedCompactionFileName(String instantTime) { return StringUtils.join(instantTime, HoodieTimeline.REQUESTED_COMPACTION_EXTENSION); } + // Log comaction action + static String makeInflightLogCompactionFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.INFLIGHT_LOG_COMPACTION_EXTENSION); + } + + static String makeRequestedLogCompactionFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.REQUESTED_LOG_COMPACTION_EXTENSION); + } + static String makeRestoreFileName(String instant) { return StringUtils.join(instant, HoodieTimeline.RESTORE_EXTENSION); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineDiffHelper.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineDiffHelper.java index 5659dcbdf7fae..29399705f00db 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineDiffHelper.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineDiffHelper.java @@ -72,8 +72,14 @@ public static TimelineDiffResult getNewInstantsForIncrementalSync(HoodieTimeline && instantPair.getValue().isCompleted()) .map(Pair::getKey).collect(Collectors.toList()); - newT.getInstants().filter(instant -> !oldTimelineInstants.contains(instant)).forEach(newInstants::add); - return new TimelineDiffResult(newInstants, finishedCompactionInstants, true); + newTimeline.getInstants().filter(instant -> !oldTimelineInstants.contains(instant)).forEach(newInstants::add); + + List> logCompactionInstants = getPendingLogCompactionTransitions(oldTimeline, newTimeline); + List finishedOrRemovedLogCompactionInstants = logCompactionInstants.stream() + .filter(instantPair -> !instantPair.getKey().isCompleted() + && (instantPair.getValue() == null || instantPair.getValue().isCompleted())) + .map(Pair::getKey).collect(Collectors.toList()); + return new TimelineDiffResult(newInstants, finishedCompactionInstants, finishedOrRemovedLogCompactionInstants, true); } else { // One or more timelines is empty LOG.warn("One or more timelines is empty"); @@ -81,6 +87,35 @@ public static TimelineDiffResult getNewInstantsForIncrementalSync(HoodieTimeline } } + /** + * Getting pending log compaction transitions. + */ + private static List> getPendingLogCompactionTransitions(HoodieTimeline oldTimeline, + HoodieTimeline newTimeline) { + Set newTimelineInstants = newTimeline.getInstants().collect(Collectors.toSet()); + + return oldTimeline.filterPendingLogCompactionTimeline().getInstants().map(instant -> { + if (newTimelineInstants.contains(instant)) { + return Pair.of(instant, instant); + } else { + HoodieInstant logCompacted = + new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instant.getTimestamp()); + if (newTimelineInstants.contains(logCompacted)) { + return Pair.of(instant, logCompacted); + } + HoodieInstant inflightLogCompacted = + new HoodieInstant(State.INFLIGHT, HoodieTimeline.LOG_COMPACTION_ACTION, instant.getTimestamp()); + if (newTimelineInstants.contains(inflightLogCompacted)) { + return Pair.of(instant, inflightLogCompacted); + } + return Pair.of(instant, null); + } + }).collect(Collectors.toList()); + } + + /** + * Getting pending compaction transitions. + */ private static List> getPendingCompactionTransitions(HoodieTimeline oldTimeline, HoodieTimeline newTimeline) { Set newTimelineInstants = newTimeline.getInstants().collect(Collectors.toSet()); @@ -94,6 +129,11 @@ private static List> getPendingCompactionTran if (newTimelineInstants.contains(compacted)) { return Pair.of(instant, compacted); } + HoodieInstant inflightCompacted = + new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, instant.getTimestamp()); + if (newTimelineInstants.contains(inflightCompacted)) { + return Pair.of(instant, inflightCompacted); + } return Pair.of(instant, null); } }).collect(Collectors.toList()); @@ -106,14 +146,17 @@ public static class TimelineDiffResult { private final List newlySeenInstants; private final List finishedCompactionInstants; + private final List finishedOrRemovedLogCompactionInstants; private final boolean canSyncIncrementally; - public static final TimelineDiffResult UNSAFE_SYNC_RESULT = new TimelineDiffResult(null, null, false); + public static final TimelineDiffResult UNSAFE_SYNC_RESULT = + new TimelineDiffResult(null, null, null, false); public TimelineDiffResult(List newlySeenInstants, List finishedCompactionInstants, - boolean canSyncIncrementally) { + List finishedOrRemovedLogCompactionInstants, boolean canSyncIncrementally) { this.newlySeenInstants = newlySeenInstants; this.finishedCompactionInstants = finishedCompactionInstants; + this.finishedOrRemovedLogCompactionInstants = finishedOrRemovedLogCompactionInstants; this.canSyncIncrementally = canSyncIncrementally; } @@ -125,14 +168,22 @@ public List getFinishedCompactionInstants() { return finishedCompactionInstants; } + public List getFinishedOrRemovedLogCompactionInstants() { + return finishedOrRemovedLogCompactionInstants; + } + public boolean canSyncIncrementally() { return canSyncIncrementally; } @Override public String toString() { - return "TimelineDiffResult{newlySeenInstants=" + newlySeenInstants + ", finishedCompactionInstants=" - + finishedCompactionInstants + ", canSyncIncrementally=" + canSyncIncrementally + '}'; + return "TimelineDiffResult{" + + "newlySeenInstants=" + newlySeenInstants + + ", finishedCompactionInstants=" + finishedCompactionInstants + + ", finishedOrRemovedLogCompactionInstants=" + finishedOrRemovedLogCompactionInstants + + ", canSyncIncrementally=" + canSyncIncrementally + + '}'; } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java index ed4bfd760161f..6dbbf85fa578b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java @@ -82,6 +82,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV protected HoodieTableMetaClient metaClient; // This is the commits timeline that will be visible for all views extending this view + // This is nothing but the write timeline, which contains both ingestion and compaction(major and minor) writers. private HoodieTimeline visibleCommitsAndCompactionTimeline; // Used to concurrently load and populate partition views @@ -110,6 +111,10 @@ protected void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActi // Load Pending Compaction Operations resetPendingCompactionOperations(CompactionUtils.getAllPendingCompactionOperations(metaClient).values().stream() .map(e -> Pair.of(e.getKey(), CompactionOperation.convertFromAvroRecordInstance(e.getValue())))); + // Load Pending LogCompaction Operations. + resetPendingLogCompactionOperations(CompactionUtils.getAllPendingLogCompactionOperations(metaClient).values().stream() + .map(e -> Pair.of(e.getKey(), CompactionOperation.convertFromAvroRecordInstance(e.getValue())))); + resetBootstrapBaseFileMapping(Stream.empty()); resetFileGroupsInPendingClustering(ClusteringUtils.getAllFileGroupsInPendingClusteringPlans(metaClient)); } @@ -489,6 +494,16 @@ public final List getPartitionPaths() { } } + @Override + public final Stream> getPendingLogCompactionOperations() { + try { + readLock.lock(); + return fetchPendingLogCompactionOperations(); + } finally { + readLock.unlock(); + } + } + @Override public final Stream getLatestBaseFiles(String partitionStr) { try { @@ -819,6 +834,35 @@ public final Stream> getFileGroupsInPendi */ abstract void removePendingCompactionOperations(Stream> operations); + /** + * Check if there is an outstanding log compaction scheduled for this file. + * + * @param fgId File-Group Id + * @return true if there is a pending log compaction, false otherwise + */ + protected abstract boolean isPendingLogCompactionScheduledForFileId(HoodieFileGroupId fgId); + + /** + * resets the pending Log compaction operation and overwrite with the new list. + * + * @param operations Pending Log Compaction Operations + */ + abstract void resetPendingLogCompactionOperations(Stream> operations); + + /** + * Add pending Log compaction operations to store. + * + * @param operations Pending Log compaction operations to be added + */ + abstract void addPendingLogCompactionOperations(Stream> operations); + + /** + * Remove pending Log compaction operations from store. + * + * @param operations Pending Log compaction operations to be removed + */ + abstract void removePendingLogCompactionOperations(Stream> operations); + /** * Check if there is an outstanding clustering operation (requested/inflight) scheduled for this file. * @@ -861,11 +905,24 @@ public final Stream> getFileGroupsInPendi protected abstract Option> getPendingCompactionOperationWithInstant( HoodieFileGroupId fileGroupId); + /** + * Return pending Log compaction operation for a file-group. + * + * @param fileGroupId File-Group Id + */ + protected abstract Option> getPendingLogCompactionOperationWithInstant( + HoodieFileGroupId fileGroupId); + /** * Fetch all pending compaction operations. */ abstract Stream> fetchPendingCompactionOperations(); + /** + * Fetch all pending log compaction operations. + */ + abstract Stream> fetchPendingLogCompactionOperations(); + /** * Check if there is an bootstrap base file present for this file. * @@ -1147,7 +1204,7 @@ public HoodieTimeline getTimeline() { @Override public void sync() { HoodieTimeline oldTimeline = getTimeline(); - HoodieTimeline newTimeline = metaClient.reloadActiveTimeline().filterCompletedAndCompactionInstants(); + HoodieTimeline newTimeline = metaClient.reloadActiveTimeline().filterCompletedOrMajorOrMinorCompactionInstants(); try { writeLock.lock(); runSync(oldTimeline, newTimeline); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java index 92937f61e2c2c..2f43fb9b4f3b6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java @@ -85,6 +85,11 @@ public class FileSystemViewStorageConfig extends HoodieConfig { .defaultValue(0.8) .withDocumentation("Fraction of the file system view memory, to be used for holding compaction related metadata."); + public static final ConfigProperty SPILLABLE_LOG_COMPACTION_MEM_FRACTION = ConfigProperty + .key("hoodie.filesystem.view.spillable.log.compaction.mem.fraction") + .defaultValue(0.8) + .withDocumentation("Fraction of the file system view memory, to be used for holding log compaction related metadata."); + public static final ConfigProperty BOOTSTRAP_BASE_FILE_MEM_FRACTION = ConfigProperty .key("hoodie.filesystem.view.spillable.bootstrap.base.file.mem.fraction") .defaultValue(0.05) @@ -206,6 +211,12 @@ public long getMaxMemoryForPendingCompaction() { .longValue(); } + public long getMaxMemoryForPendingLogCompaction() { + long totalMemory = getLong(SPILLABLE_MEMORY); + return new Double(totalMemory * getDouble(SPILLABLE_LOG_COMPACTION_MEM_FRACTION)) + .longValue(); + } + public long getMaxMemoryForBootstrapBaseFile() { long totalMemory = getLong(SPILLABLE_MEMORY); long reservedForExternalDataFile = diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java index 9dac36081384b..ea72f305b0738 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java @@ -62,6 +62,11 @@ public class HoodieTableFileSystemView extends IncrementalTimelineSyncFileSystem */ protected Map> fgIdToPendingCompaction; + /** + * PartitionPath + File-Id to pending compaction instant time. + */ + protected Map> fgIdToPendingLogCompaction; + /** * PartitionPath + File-Id to bootstrap base File (Index Only bootstrapped). */ @@ -143,6 +148,11 @@ protected Map> createFileId return fileIdToPendingCompaction; } + protected Map> createFileIdToPendingLogCompactionMap( + Map> fileIdToPendingLogCompaction) { + return fileIdToPendingLogCompaction; + } + protected Map createFileIdToBootstrapBaseFileMap( Map fileGroupIdBootstrapBaseFileMap) { return fileGroupIdBootstrapBaseFileMap; @@ -213,6 +223,39 @@ protected void removePendingCompactionOperations(Stream> operations) { + // Build fileId to Pending Log Compaction Instants + this.fgIdToPendingLogCompaction = createFileIdToPendingLogCompactionMap(operations.map(entry -> + Pair.of(entry.getValue().getFileGroupId(), Pair.of(entry.getKey(), entry.getValue()))).collect(Collectors.toMap(Pair::getKey, Pair::getValue))); + } + + @Override + protected void addPendingLogCompactionOperations(Stream> operations) { + operations.forEach(opInstantPair -> { + ValidationUtils.checkArgument(!fgIdToPendingLogCompaction.containsKey(opInstantPair.getValue().getFileGroupId()), + "Duplicate FileGroupId found in pending log compaction operations. FgId :" + + opInstantPair.getValue().getFileGroupId()); + fgIdToPendingLogCompaction.put(opInstantPair.getValue().getFileGroupId(), + Pair.of(opInstantPair.getKey(), opInstantPair.getValue())); + }); + } + + @Override + protected void removePendingLogCompactionOperations(Stream> operations) { + operations.forEach(opInstantPair -> { + ValidationUtils.checkArgument(fgIdToPendingLogCompaction.containsKey(opInstantPair.getValue().getFileGroupId()), + "Trying to remove a FileGroupId which is not found in pending log compaction operations. FgId :" + + opInstantPair.getValue().getFileGroupId()); + fgIdToPendingLogCompaction.remove(opInstantPair.getValue().getFileGroupId()); + }); + } + @Override protected boolean isPendingClusteringScheduledForFileId(HoodieFileGroupId fgId) { return fgIdToPendingClustering.containsKey(fgId); @@ -273,6 +316,11 @@ public Stream getAllFileGroups() { @Override Stream> fetchPendingCompactionOperations() { return fgIdToPendingCompaction.values().stream(); + } + + @Override + Stream> fetchPendingLogCompactionOperations() { + return fgIdToPendingLogCompaction.values().stream(); } @@ -323,6 +371,11 @@ protected Option> getPendingCompactionOperatio return Option.ofNullable(fgIdToPendingCompaction.get(fgId)); } + @Override + protected Option> getPendingLogCompactionOperationWithInstant(HoodieFileGroupId fgId) { + return Option.ofNullable(fgIdToPendingLogCompaction.get(fgId)); + } + @Override protected boolean isPartitionAvailableInStore(String partitionPath) { return partitionToFileGroupsMap.containsKey(partitionPath); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java index 9f9fd1f7ce601..4c2aa79f27c6d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java @@ -116,9 +116,20 @@ private void runIncrementalSync(HoodieTimeline timeline, TimelineDiffResult diff } }); - // Add new completed instants found in the latest timeline + // Now remove pending log compaction instants which were completed or removed + diffResult.getFinishedOrRemovedLogCompactionInstants().stream().forEach(instant -> { + try { + removePendingLogCompactionInstant(timeline, instant); + } catch (IOException e) { + throw new HoodieException(e); + } + }); + + // Add new completed instants found in the latest timeline, this also contains inflight instants. diffResult.getNewlySeenInstants().stream() - .filter(instant -> instant.isCompleted() || instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)) + .filter(instant -> instant.isCompleted() + || instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION) + || instant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION)) .forEach(instant -> { try { if (instant.getAction().equals(HoodieTimeline.COMMIT_ACTION) @@ -130,6 +141,8 @@ private void runIncrementalSync(HoodieTimeline timeline, TimelineDiffResult diff addCleanInstant(timeline, instant); } else if (instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)) { addPendingCompactionInstant(timeline, instant); + } else if (instant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION)) { + addPendingLogCompactionInstant(instant); } else if (instant.getAction().equals(HoodieTimeline.ROLLBACK_ACTION)) { addRollbackInstant(timeline, instant); } else if (instant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)) { @@ -155,6 +168,21 @@ private void removePendingCompactionInstant(HoodieTimeline timeline, HoodieInsta CompactionOperation.convertFromAvroRecordInstance(instantPair.getValue().getValue())))); } + /** + * Remove Pending compaction instant. This is called when logcompaction is converted to delta commit, + * so you no longer need to track them as pending. + * + * @param timeline New Hoodie Timeline + * @param instant Log Compaction Instant to be removed + */ + private void removePendingLogCompactionInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException { + LOG.info("Removing completed log compaction instant (" + instant + ")"); + HoodieCompactionPlan plan = CompactionUtils.getLogCompactionPlan(metaClient, instant.getTimestamp()); + removePendingLogCompactionOperations(CompactionUtils.getPendingCompactionOperations(instant, plan) + .map(instantPair -> Pair.of(instantPair.getValue().getKey(), + CompactionOperation.convertFromAvroRecordInstance(instantPair.getValue().getValue())))); + } + /** * Add newly found compaction instant. * @@ -186,6 +214,24 @@ private void addPendingCompactionInstant(HoodieTimeline timeline, HoodieInstant }); } + /** + * Add newly found compaction instant. + * + * @param instant Compaction Instant + */ + private void addPendingLogCompactionInstant(HoodieInstant instant) throws IOException { + LOG.info("Syncing pending log compaction instant (" + instant + ")"); + HoodieCompactionPlan compactionPlan = CompactionUtils.getLogCompactionPlan(metaClient, instant.getTimestamp()); + List> pendingOps = + CompactionUtils.getPendingCompactionOperations(instant, compactionPlan) + .map(p -> Pair.of(p.getValue().getKey(), + CompactionOperation.convertFromAvroRecordInstance(p.getValue().getValue()))) + .collect(Collectors.toList()); + // Update Pending log compaction instants. + // Since logcompaction works similar to a deltacommit. Updating the partition view is not required. + addPendingLogCompactionOperations(pendingOps.stream()); + } + /** * Add newly found commit/delta-commit instant. * diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java index ff44c7cef017b..62edc4daa33e0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java @@ -224,6 +224,11 @@ public Stream> getPendingCompactionOperations( return execute(preferredView::getPendingCompactionOperations, secondaryView::getPendingCompactionOperations); } + @Override + public Stream> getPendingLogCompactionOperations() { + return execute(preferredView::getPendingLogCompactionOperations, secondaryView::getPendingLogCompactionOperations); + } + @Override public Stream> getFileGroupsInPendingClustering() { return execute(preferredView::getFileGroupsInPendingClustering, secondaryView::getFileGroupsInPendingClustering); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java index bd18ba22a25d6..759ef70c6d80f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java @@ -74,6 +74,7 @@ public class RemoteHoodieTableFileSystemView implements SyncableFileSystemView, String.format("%s/%s", BASE_URL, "slices/beforeoron/latest/"); public static final String PENDING_COMPACTION_OPS = String.format("%s/%s", BASE_URL, "compactions/pending/"); + public static final String PENDING_LOG_COMPACTION_OPS = String.format("%s/%s", BASE_URL, "logcompactions/pending/"); public static final String LATEST_PARTITION_DATA_FILES_URL = String.format("%s/%s", BASE_URL, "datafiles/latest/partition"); @@ -436,6 +437,18 @@ public Stream> getPendingCompactionOperations( } } + @Override + public Stream> getPendingLogCompactionOperations() { + Map paramsMap = getParams(); + try { + List dtos = executeRequest(PENDING_LOG_COMPACTION_OPS, paramsMap, + new TypeReference>() {}, RequestMethod.GET); + return dtos.stream().map(CompactionOpDTO::toCompactionOperation); + } catch (IOException e) { + throw new HoodieRemoteException(e); + } + } + @Override public Stream> getFileGroupsInPendingClustering() { Map paramsMap = getParams(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java index ebd759f88060b..2389f267818ec 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java @@ -135,6 +135,49 @@ void removePendingCompactionOperations(Stream> ); } + @Override + protected boolean isPendingLogCompactionScheduledForFileId(HoodieFileGroupId fgId) { + return getPendingLogCompactionOperationWithInstant(fgId).isPresent(); + } + + @Override + protected void resetPendingLogCompactionOperations(Stream> operations) { + rocksDB.writeBatch(batch -> { + operations.forEach(opPair -> + rocksDB.putInBatch(batch, schemaHelper.getColFamilyForPendingLogCompaction(), + schemaHelper.getKeyForPendingLogCompactionLookup(opPair.getValue().getFileGroupId()), opPair) + ); + LOG.info("Initializing pending Log compaction operations. Count=" + batch.count()); + }); + } + + @Override + protected void addPendingLogCompactionOperations(Stream> operations) { + rocksDB.writeBatch(batch -> + operations.forEach(opInstantPair -> { + ValidationUtils.checkArgument(!isPendingLogCompactionScheduledForFileId(opInstantPair.getValue().getFileGroupId()), + "Duplicate FileGroupId found in pending log compaction operations. FgId :" + + opInstantPair.getValue().getFileGroupId()); + rocksDB.putInBatch(batch, schemaHelper.getColFamilyForPendingLogCompaction(), + schemaHelper.getKeyForPendingLogCompactionLookup(opInstantPair.getValue().getFileGroupId()), opInstantPair); + }) + ); + } + + @Override + void removePendingLogCompactionOperations(Stream> operations) { + rocksDB.writeBatch(batch -> + operations.forEach(opInstantPair -> { + ValidationUtils.checkArgument( + getPendingLogCompactionOperationWithInstant(opInstantPair.getValue().getFileGroupId()) != null, + "Trying to remove a FileGroupId which is not found in pending Log compaction operations. FgId :" + + opInstantPair.getValue().getFileGroupId()); + rocksDB.deleteInBatch(batch, schemaHelper.getColFamilyForPendingLogCompaction(), + schemaHelper.getKeyForPendingLogCompactionLookup(opInstantPair.getValue().getFileGroupId())); + }) + ); + } + @Override protected boolean isPendingClusteringScheduledForFileId(HoodieFileGroupId fgId) { return getPendingClusteringInstant(fgId).isPresent(); @@ -171,7 +214,7 @@ void addFileGroupsInPendingClustering(Stream fileGroups.forEach(fgIdToClusterInstant -> { ValidationUtils.checkArgument(!isPendingClusteringScheduledForFileId(fgIdToClusterInstant.getLeft()), - "Duplicate FileGroupId found in pending compaction operations. FgId :" + "Duplicate FileGroupId found in pending clustering operations. FgId :" + fgIdToClusterInstant.getLeft()); rocksDB.putInBatch(batch, schemaHelper.getColFamilyForFileGroupsInPendingClustering(), @@ -186,7 +229,7 @@ void removeFileGroupsInPendingClustering(Stream { ValidationUtils.checkArgument( !isPendingClusteringScheduledForFileId(fgToPendingClusteringInstant.getLeft()), - "Trying to remove a FileGroupId which is not found in pending compaction operations. FgId :" + "Trying to remove a FileGroupId which is not found in pending clustering operations. FgId :" + fgToPendingClusteringInstant.getLeft()); rocksDB.deleteInBatch(batch, schemaHelper.getColFamilyForFileGroupsInPendingClustering(), schemaHelper.getKeyForFileGroupsInPendingClustering(fgToPendingClusteringInstant.getLeft())); @@ -210,6 +253,14 @@ protected Option> getPendingCompactionOperatio return Option.ofNullable(instantOperationPair); } + @Override + protected Option> getPendingLogCompactionOperationWithInstant(HoodieFileGroupId fgId) { + String lookupKey = schemaHelper.getKeyForPendingLogCompactionLookup(fgId); + Pair instantOperationPair = + rocksDB.get(schemaHelper.getColFamilyForPendingLogCompaction(), lookupKey); + return Option.ofNullable(instantOperationPair); + } + @Override protected boolean isPartitionAvailableInStore(String partitionPath) { String lookupKey = schemaHelper.getKeyForPartitionLookup(partitionPath); @@ -323,6 +374,12 @@ Stream> fetchPendingCompactionOperations() { .map(Pair::getValue); } + @Override + Stream> fetchPendingLogCompactionOperations() { + return rocksDB.>prefixSearch(schemaHelper.getColFamilyForPendingLogCompaction(), "") + .map(Pair::getValue); + } + @Override Stream fetchAllBaseFiles(String partitionPath) { return rocksDB.prefixSearch(schemaHelper.getColFamilyForView(), diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java index e4144420831c7..26d36d9c97bc0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java @@ -51,6 +51,7 @@ public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView { private final long maxMemoryForFileGroupMap; private final long maxMemoryForPendingCompaction; + private final long maxMemoryForPendingLogCompaction; private final long maxMemoryForBootstrapBaseFile; private final long maxMemoryForReplaceFileGroups; private final long maxMemoryForClusteringFileGroups; @@ -63,6 +64,7 @@ public SpillableMapBasedFileSystemView(HoodieTableMetaClient metaClient, HoodieT super(config.isIncrementalTimelineSyncEnabled()); this.maxMemoryForFileGroupMap = config.getMaxMemoryForFileGroupMap(); this.maxMemoryForPendingCompaction = config.getMaxMemoryForPendingCompaction(); + this.maxMemoryForPendingLogCompaction = config.getMaxMemoryForPendingLogCompaction(); this.maxMemoryForBootstrapBaseFile = config.getMaxMemoryForBootstrapBaseFile(); this.maxMemoryForReplaceFileGroups = config.getMaxMemoryForReplacedFileGroups(); this.maxMemoryForClusteringFileGroups = config.getMaxMemoryForPendingClusteringFileGroups(); @@ -109,6 +111,23 @@ maxMemoryForPendingCompaction, baseStoreDir, new DefaultSizeEstimator(), new Def } } + @Override + protected Map> createFileIdToPendingLogCompactionMap( + Map> fgIdToPendingLogCompaction) { + try { + LOG.info("Creating Pending Log Compaction map using external spillable Map. Max Mem=" + maxMemoryForPendingLogCompaction + + ", BaseDir=" + baseStoreDir); + new File(baseStoreDir).mkdirs(); + Map> pendingMap = new ExternalSpillableMap<>( + maxMemoryForPendingLogCompaction, baseStoreDir, new DefaultSizeEstimator(), new DefaultSizeEstimator<>(), + diskMapType, isBitCaskDiskMapCompressionEnabled); + pendingMap.putAll(fgIdToPendingLogCompaction); + return pendingMap; + } catch (IOException e) { + throw new RuntimeException(e); + } + } + @Override protected Map createFileIdToBootstrapBaseFileMap( Map fileGroupIdBootstrapBaseFileMap) { @@ -169,6 +188,11 @@ Stream> fetchPendingCompactionOperations() { return ((ExternalSpillableMap) fgIdToPendingCompaction).valueStream(); } + @Override + Stream> fetchPendingLogCompactionOperations() { + return ((ExternalSpillableMap) fgIdToPendingLogCompaction).valueStream(); + } + @Override Stream fetchBootstrapBaseFiles() { return ((ExternalSpillableMap) fgIdToBootstrapBaseFile).valueStream(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java index c32e2cabb1012..18c9a9af99817 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java @@ -156,6 +156,13 @@ interface SliceView extends SliceViewWithLatestSlice { */ Stream> getPendingCompactionOperations(); + /** + * Return Pending Compaction Operations. + * + * @return Pair> + */ + Stream> getPendingLogCompactionOperations(); + /** * Last Known Instant on which the view is built. */ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java index 90d6e6ae90fb0..3b94948707f5c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java @@ -261,4 +261,5 @@ private static Object[] checkElementsNotNull(Object[] array, int length) { private static Object checkElementNotNull(Object element, int index) { return Objects.requireNonNull(element, "Element is null at index " + index); } + } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/CompactionUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/CompactionUtils.java index cf9b5fb3ced8c..d909d209de256 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/CompactionUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/CompactionUtils.java @@ -35,9 +35,6 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; - import java.io.IOException; import java.util.HashMap; import java.util.List; @@ -51,8 +48,6 @@ */ public class CompactionUtils { - private static final Logger LOG = LogManager.getLogger(CompactionUtils.class); - public static final Integer COMPACTION_METADATA_VERSION_1 = CompactionV1MigrationHandler.VERSION; public static final Integer COMPACTION_METADATA_VERSION_2 = CompactionV2MigrationHandler.VERSION; public static final Integer LATEST_COMPACTION_METADATA_VERSION = COMPACTION_METADATA_VERSION_2; @@ -126,29 +121,79 @@ public static CompactionOperation buildCompactionOperation(HoodieCompactionOpera /** * Get all pending compaction plans along with their instants. - * * @param metaClient Hoodie Meta Client */ public static List> getAllPendingCompactionPlans( HoodieTableMetaClient metaClient) { - List pendingCompactionInstants = - metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants().collect(Collectors.toList()); - return pendingCompactionInstants.stream().map(instant -> { - try { - return Pair.of(instant, getCompactionPlan(metaClient, instant.getTimestamp())); - } catch (IOException e) { - throw new HoodieException(e); - } - }).collect(Collectors.toList()); + // This function returns pending compaction timeline. + Function getFilteredTimelineByActionType = + (hoodieTableMetaClient) -> hoodieTableMetaClient.getActiveTimeline().filterPendingCompactionTimeline(); + // Hoodie requested instant supplier + Function requestedInstantSupplier = HoodieTimeline::getCompactionRequestedInstant; + return getCompactionPlansByTimeline(metaClient, getFilteredTimelineByActionType, requestedInstantSupplier); + } + + /** + * Get all pending logcompaction plans along with their instants. + * @param metaClient Hoodie Meta Client + */ + public static List> getAllPendingLogCompactionPlans( + HoodieTableMetaClient metaClient) { + // This function returns pending logcompaction timeline. + Function filteredTimelineSupplier = + (hoodieTableMetaClient) -> hoodieTableMetaClient.getActiveTimeline().filterPendingLogCompactionTimeline(); + // Hoodie requested instant supplier + Function requestedInstantSupplier = HoodieTimeline::getLogCompactionRequestedInstant; + return getCompactionPlansByTimeline(metaClient, filteredTimelineSupplier, requestedInstantSupplier); + } + + /** + * Util method to get compaction plans by action_type(COMPACT or LOG_COMPACT) + * @param metaClient HoodieTable's metaclient + * @param filteredTimelineSupplier gives a timeline object, this can be either filtered to return pending compactions or log compaction instants. + * @param requestedInstantWrapper function that gives a requested Hoodie instant. + * @return List of pair of HoodieInstant and it's corresponding compaction plan. + * Note here the compaction plan can be related to a compaction instant or log compaction instant. + */ + private static List> getCompactionPlansByTimeline( + HoodieTableMetaClient metaClient, Function filteredTimelineSupplier, + Function requestedInstantWrapper) { + List filteredInstants = filteredTimelineSupplier.apply(metaClient).getInstants().collect(Collectors.toList()); + return filteredInstants.stream() + .map(instant -> Pair.of(instant, getCompactionPlan(metaClient, requestedInstantWrapper.apply(instant.getTimestamp())))) + .collect(Collectors.toList()); + } + + /** + * This method will serve only Compaction instants + * because we use same HoodieCompactionPlan for both the operations. + */ + public static HoodieCompactionPlan getCompactionPlan(HoodieTableMetaClient metaClient, String compactionInstant) { + HoodieInstant compactionRequestedInstant = HoodieTimeline.getCompactionRequestedInstant(compactionInstant); + return getCompactionPlan(metaClient, compactionRequestedInstant); } - public static HoodieCompactionPlan getCompactionPlan(HoodieTableMetaClient metaClient, String compactionInstant) - throws IOException { + /** + * This method will serve only log compaction instants, + * because we use same HoodieCompactionPlan for both the operations. + */ + public static HoodieCompactionPlan getLogCompactionPlan(HoodieTableMetaClient metaClient, String logCompactionInstant) { + HoodieInstant logCompactionRequestedInstant = HoodieTimeline.getLogCompactionRequestedInstant(logCompactionInstant); + return getCompactionPlan(metaClient, logCompactionRequestedInstant); + } + + /** + * Util method to fetch both compaction and log compaction plan from requestedInstant. + */ + private static HoodieCompactionPlan getCompactionPlan(HoodieTableMetaClient metaClient, HoodieInstant requestedInstant) { CompactionPlanMigrator migrator = new CompactionPlanMigrator(metaClient); - HoodieCompactionPlan compactionPlan = TimelineMetadataUtils.deserializeCompactionPlan( - metaClient.getActiveTimeline().readCompactionPlanAsBytes( - HoodieTimeline.getCompactionRequestedInstant(compactionInstant)).get()); - return migrator.upgradeToLatest(compactionPlan, compactionPlan.getVersion()); + try { + HoodieCompactionPlan compactionPlan = TimelineMetadataUtils.deserializeCompactionPlan( + metaClient.getActiveTimeline().readCompactionPlanAsBytes(requestedInstant).get()); + return migrator.upgradeToLatest(compactionPlan, compactionPlan.getVersion()); + } catch (IOException e) { + throw new HoodieException(e); + } } /** @@ -160,29 +205,49 @@ public static Map> ge HoodieTableMetaClient metaClient) { List> pendingCompactionPlanWithInstants = getAllPendingCompactionPlans(metaClient); + return getAllPendingCompactionOperationsInPendingCompactionPlans(pendingCompactionPlanWithInstants); + } + + /** + * Get all partition + file Ids with pending Log Compaction operations and their target log compaction instant time. + */ + public static Map> getAllPendingLogCompactionOperations( + HoodieTableMetaClient metaClient) { + List> pendingLogCompactionPlanWithInstants = + getAllPendingLogCompactionPlans(metaClient); + return getAllPendingCompactionOperationsInPendingCompactionPlans(pendingLogCompactionPlanWithInstants); + } - Map> fgIdToPendingCompactionWithInstantMap = - new HashMap<>(); - pendingCompactionPlanWithInstants.stream().flatMap(instantPlanPair -> + /** + * Get all partition + file Ids with pending Log Compaction operations and their target log compaction instant time. + */ + public static Map> getAllPendingCompactionOperationsInPendingCompactionPlans( + List> pendingLogCompactionPlanWithInstants) { + + Map> fgIdToPendingCompactionsWithInstantMap = new HashMap<>(); + pendingLogCompactionPlanWithInstants.stream().flatMap(instantPlanPair -> getPendingCompactionOperations(instantPlanPair.getKey(), instantPlanPair.getValue())).forEach(pair -> { - // Defensive check to ensure a single-fileId does not have more than one pending compaction with different + // Defensive check to ensure a single-fileId does not have more than one pending log compaction with different // file slices. If we find a full duplicate we assume it is caused by eventual nature of the move operation // on some DFSs. - if (fgIdToPendingCompactionWithInstantMap.containsKey(pair.getKey())) { + if (fgIdToPendingCompactionsWithInstantMap.containsKey(pair.getKey())) { HoodieCompactionOperation operation = pair.getValue().getValue(); - HoodieCompactionOperation anotherOperation = fgIdToPendingCompactionWithInstantMap.get(pair.getKey()).getValue(); + HoodieCompactionOperation anotherOperation = fgIdToPendingCompactionsWithInstantMap.get(pair.getKey()).getValue(); if (!operation.equals(anotherOperation)) { - String msg = "Hudi File Id (" + pair.getKey() + ") has more than 1 pending compactions. Instants: " - + pair.getValue() + ", " + fgIdToPendingCompactionWithInstantMap.get(pair.getKey()); + String msg = "Hudi File Id (" + pair.getKey() + ") has more than 1 pending operation. Instants: " + + pair.getValue() + ", " + fgIdToPendingCompactionsWithInstantMap.get(pair.getKey()); throw new IllegalStateException(msg); } } - fgIdToPendingCompactionWithInstantMap.put(pair.getKey(), pair.getValue()); + fgIdToPendingCompactionsWithInstantMap.put(pair.getKey(), pair.getValue()); }); - return fgIdToPendingCompactionWithInstantMap; + return fgIdToPendingCompactionsWithInstantMap; } + /** + * Get pending compaction operations for both major and minor compaction. + */ public static Stream>> getPendingCompactionOperations( HoodieInstant instant, HoodieCompactionPlan compactionPlan) { List ops = compactionPlan.getOperations(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBSchemaHelper.java b/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBSchemaHelper.java index e9377a6674383..45b2a13eb72ae 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBSchemaHelper.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBSchemaHelper.java @@ -45,6 +45,7 @@ public class RocksDBSchemaHelper { private final String colFamilyForView; private final String colFamilyForPendingCompaction; + private final String colFamilyForPendingLogCompaction; private final String colFamilyForBootstrapBaseFile; private final String colFamilyForStoredPartitions; private final String colFamilyForReplacedFileGroups; @@ -53,6 +54,7 @@ public class RocksDBSchemaHelper { public RocksDBSchemaHelper(HoodieTableMetaClient metaClient) { this.colFamilyForBootstrapBaseFile = "hudi_bootstrap_basefile_" + metaClient.getBasePath().replace("/", "_"); this.colFamilyForPendingCompaction = "hudi_pending_compaction_" + metaClient.getBasePath().replace("/", "_"); + this.colFamilyForPendingLogCompaction = "hudi_pending_log_compaction_" + metaClient.getBasePath().replace("/", "_"); this.colFamilyForStoredPartitions = "hudi_partitions_" + metaClient.getBasePath().replace("/", "_"); this.colFamilyForView = "hudi_view_" + metaClient.getBasePath().replace("/", "_"); this.colFamilyForReplacedFileGroups = "hudi_replaced_fg" + metaClient.getBasePath().replace("/", "_"); @@ -60,8 +62,9 @@ public RocksDBSchemaHelper(HoodieTableMetaClient metaClient) { } public List getAllColumnFamilies() { - return Arrays.asList(getColFamilyForView(), getColFamilyForPendingCompaction(), getColFamilyForBootstrapBaseFile(), - getColFamilyForStoredPartitions(), getColFamilyForReplacedFileGroups(), getColFamilyForFileGroupsInPendingClustering()); + return Arrays.asList(getColFamilyForView(), getColFamilyForPendingCompaction(), getColFamilyForPendingLogCompaction(), + getColFamilyForBootstrapBaseFile(), getColFamilyForStoredPartitions(), getColFamilyForReplacedFileGroups(), + getColFamilyForFileGroupsInPendingClustering()); } public String getKeyForPartitionLookup(String partition) { @@ -72,6 +75,10 @@ public String getKeyForPendingCompactionLookup(HoodieFileGroupId fgId) { return getPartitionFileIdBasedLookup(fgId); } + public String getKeyForPendingLogCompactionLookup(HoodieFileGroupId fgId) { + return getPartitionFileIdBasedLookup(fgId); + } + public String getKeyForBootstrapBaseFile(HoodieFileGroupId fgId) { return getPartitionFileIdBasedLookup(fgId); } @@ -130,6 +137,10 @@ public String getColFamilyForPendingCompaction() { return colFamilyForPendingCompaction; } + public String getColFamilyForPendingLogCompaction() { + return colFamilyForPendingLogCompaction; + } + public String getColFamilyForBootstrapBaseFile() { return colFamilyForBootstrapBaseFile; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java index a4f2c62437caa..af80af3db7329 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java @@ -78,6 +78,13 @@ public static String join(CharSequence delimiter, Iterable list, final String separator) { + if (list == null || list.size() == 0) { + return null; + } + return org.apache.hadoop.util.StringUtils.join(separator, list.toArray(new String[0])); + } + public static String toHexString(byte[] bytes) { StringBuilder sb = new StringBuilder(bytes.length * 2); for (byte b : bytes) { diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReaderFactory.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReaderFactory.java index f913df7e152a9..1c7663b371e6e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReaderFactory.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReaderFactory.java @@ -44,7 +44,6 @@ public static HoodieFileReader getFileReader(Config if (ORC.getFileExtension().equals(extension)) { return newOrcFileReader(conf, path); } - throw new UnsupportedOperationException(extension + " format not supported yet."); } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java index cb9fb8da14b5d..bcaf2bcab349f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java @@ -504,6 +504,7 @@ public Pair getLogRecordScanner(List< .withLogBlockTimestamps(validInstantTimestamps) .allowFullScan(allowFullScan) .withPartition(partitionName) + .withUseScanV2(metadataConfig.getUseLogRecordReaderScanV2()) .build(); Long logScannerOpenMs = timer.endTimer(); diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java index cbd7e6c17511c..b844f219cd2de 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java @@ -59,9 +59,10 @@ private HoodieMetadataMergedLogRecordReader(FileSystem fs, String basePath, Stri String spillableMapBasePath, ExternalSpillableMap.DiskMapType diskMapType, boolean isBitCaskDiskMapCompressionEnabled, - Option instantRange, boolean allowFullScan) { + Option instantRange, boolean allowFullScan, boolean useScanV2) { super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, maxMemorySizeInBytes, true, false, bufferSize, - spillableMapBasePath, instantRange, diskMapType, isBitCaskDiskMapCompressionEnabled, false, allowFullScan, Option.of(partitionName), InternalSchema.getEmptyInternalSchema()); + spillableMapBasePath, instantRange, diskMapType, isBitCaskDiskMapCompressionEnabled, false, allowFullScan, + Option.of(partitionName), InternalSchema.getEmptyInternalSchema(), useScanV2); } @Override @@ -106,7 +107,7 @@ public List> getRecordsByKeyPrefixes(List (HoodieRecord) record) @@ -137,8 +138,12 @@ protected String getKeyField() { * Builder used to build {@code HoodieMetadataMergedLogRecordScanner}. */ public static class Builder extends HoodieMergedLogRecordScanner.Builder { + private boolean allowFullScan = HoodieMetadataConfig.ENABLE_FULL_SCAN_LOG_FILES.defaultValue(); + // Use scanV2 method. + private boolean useScanV2 = false; + @Override public Builder withFileSystem(FileSystem fs) { this.fs = fs; @@ -225,11 +230,17 @@ public Builder allowFullScan(boolean enableFullScan) { return this; } + @Override + public Builder withUseScanV2(boolean useScanV2) { + this.useScanV2 = useScanV2; + return this; + } + @Override public HoodieMetadataMergedLogRecordReader build() { return new HoodieMetadataMergedLogRecordReader(fs, basePath, partitionName, logFilePaths, readerSchema, latestInstantTime, maxMemorySizeInBytes, bufferSize, spillableMapBasePath, - diskMapType, isBitCaskDiskMapCompressionEnabled, instantRange, allowFullScan); + diskMapType, isBitCaskDiskMapCompressionEnabled, instantRange, allowFullScan, useScanV2); } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java index f87e5a41b8439..a5df6252e45f7 100755 --- a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java @@ -644,7 +644,8 @@ public void testCDCBlock() throws IOException, InterruptedException { @MethodSource("testArguments") public void testBasicAppendAndScanMultipleFiles(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, - boolean readBlocksLazily) + boolean readBlocksLazily, + boolean useScanv2) throws IOException, URISyntaxException, InterruptedException { Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) @@ -685,6 +686,7 @@ public void testBasicAppendAndScanMultipleFiles(ExternalSpillableMap.DiskMapType .withSpillableMapBasePath(BASE_OUTPUT_PATH) .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) + .withUseScanV2(useScanv2) .build(); List scannedRecords = new ArrayList<>(); @@ -872,7 +874,8 @@ public void testValidateCorruptBlockEndPosition() throws IOException, URISyntaxE @MethodSource("testArguments") public void testAvroLogRecordReaderBasic(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, - boolean readBlocksLazily) + boolean readBlocksLazily, + boolean useScanv2) throws IOException, URISyntaxException, InterruptedException { Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); // Set a small threshold so that every block is a new version @@ -918,6 +921,7 @@ public void testAvroLogRecordReaderBasic(ExternalSpillableMap.DiskMapType diskMa .withSpillableMapBasePath(BASE_OUTPUT_PATH) .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) + .withUseScanV2(useScanv2) .build(); assertEquals(200, scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); @@ -934,7 +938,8 @@ public void testAvroLogRecordReaderBasic(ExternalSpillableMap.DiskMapType diskMa @MethodSource("testArguments") public void testAvroLogRecordReaderWithRollbackTombstone(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, - boolean readBlocksLazily) + boolean readBlocksLazily, + boolean useScanv2) throws IOException, URISyntaxException, InterruptedException { Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); // Set a small threshold so that every block is a new version @@ -954,6 +959,7 @@ public void testAvroLogRecordReaderWithRollbackTombstone(ExternalSpillableMap.Di writer.appendBlock(dataBlock); // Write 2 + header = new HashMap<>(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101"); List records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); @@ -961,13 +967,16 @@ public void testAvroLogRecordReaderWithRollbackTombstone(ExternalSpillableMap.Di writer.appendBlock(dataBlock); // Rollback the last write + header = new HashMap<>(); header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "101"); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "102"); header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, - String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); + String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_BLOCK.ordinal())); HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer.appendBlock(commandBlock); // Write 3 + header = new HashMap<>(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "102"); List records3 = SchemaTestUtil.generateHoodieTestRecords(0, 100); List copyOfRecords3 = records3.stream() @@ -996,6 +1005,7 @@ public void testAvroLogRecordReaderWithRollbackTombstone(ExternalSpillableMap.Di .withSpillableMapBasePath(BASE_OUTPUT_PATH) .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) + .withUseScanV2(useScanv2) .build(); assertEquals(200, scanner.getTotalLogRecords(), "We read 200 records from 2 write batches"); Set readKeys = new HashSet<>(200); @@ -1011,7 +1021,8 @@ public void testAvroLogRecordReaderWithRollbackTombstone(ExternalSpillableMap.Di @ParameterizedTest @MethodSource("testArguments") public void testAvroLogRecordReaderWithFailedPartialBlock(ExternalSpillableMap.DiskMapType diskMapType, - boolean isCompressionEnabled) + boolean isCompressionEnabled, + boolean useScanv2) throws IOException, URISyntaxException, InterruptedException { Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); // Set a small threshold so that every block is a new version @@ -1083,6 +1094,7 @@ public void testAvroLogRecordReaderWithFailedPartialBlock(ExternalSpillableMap.D .withSpillableMapBasePath(BASE_OUTPUT_PATH) .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) + .withUseScanV2(useScanv2) .build(); assertEquals(200, scanner.getTotalLogRecords(), "We would read 200 records"); Set readKeys = new HashSet<>(200); @@ -1099,7 +1111,8 @@ public void testAvroLogRecordReaderWithFailedPartialBlock(ExternalSpillableMap.D @MethodSource("testArguments") public void testAvroLogRecordReaderWithDeleteAndRollback(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, - boolean readBlocksLazily) + boolean readBlocksLazily, + boolean useScanv2) throws IOException, URISyntaxException, InterruptedException { Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); // Set a small threshold so that every block is a new version @@ -1125,17 +1138,17 @@ public void testAvroLogRecordReaderWithDeleteAndRollback(ExternalSpillableMap.Di dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records2, header); writer.appendBlock(dataBlock); - copyOfRecords1.addAll(copyOfRecords2); - List originalKeys = - copyOfRecords1.stream().map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) - .collect(Collectors.toList()); - // Delete 50 keys List deletedRecords = copyOfRecords1.stream() .map(s -> (DeleteRecord.create(((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(), ((GenericRecord) s).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString()))) .collect(Collectors.toList()).subList(0, 50); + copyOfRecords2.addAll(copyOfRecords1); + List originalKeys = + copyOfRecords2.stream().map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) + .collect(Collectors.toList()); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "102"); HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedRecords.toArray(new DeleteRecord[50]), header); writer.appendBlock(deleteBlock); @@ -1161,6 +1174,7 @@ public void testAvroLogRecordReaderWithDeleteAndRollback(ExternalSpillableMap.Di .withSpillableMapBasePath(BASE_OUTPUT_PATH) .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) + .withUseScanV2(useScanv2) .build(); assertEquals(200, scanner.getTotalLogRecords(), "We still would read 200 records"); @@ -1183,15 +1197,15 @@ public void testAvroLogRecordReaderWithDeleteAndRollback(ExternalSpillableMap.Di Collections.sort(readKeys); assertEquals(originalKeys, readKeys, "CompositeAvroLogReader should return 150 records from 2 versions"); - // Rollback the last block + // Rollback the 1st block i.e. a data block. header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "103"); - header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "102"); + header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "101"); header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, - String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); + String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_BLOCK.ordinal())); HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer.appendBlock(commandBlock); - FileCreateUtils.deleteDeltaCommit(basePath, "102", fs); + FileCreateUtils.deleteDeltaCommit(basePath, "101", fs); readKeys.clear(); scanner = HoodieMergedLogRecordScanner.newBuilder() @@ -1199,7 +1213,7 @@ public void testAvroLogRecordReaderWithDeleteAndRollback(ExternalSpillableMap.Di .withBasePath(basePath) .withLogFilePaths(allLogFiles) .withReaderSchema(schema) - .withLatestInstantTime("101") + .withLatestInstantTime("103") .withMaxMemorySizeInBytes(10240L) .withReadBlocksLazily(readBlocksLazily) .withReverseReader(false) @@ -1209,7 +1223,25 @@ public void testAvroLogRecordReaderWithDeleteAndRollback(ExternalSpillableMap.Di .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) .build(); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); - assertEquals(200, readKeys.size(), "Stream collect should return all 200 records after rollback of delete"); + assertEquals(100, readKeys.size(), "Stream collect should return all 200 records after rollback of delete"); + final List newEmptyPayloads = new ArrayList<>(); + scanner.forEach(s -> { + try { + if (!s.getData().getInsertValue(schema).isPresent()) { + newEmptyPayloads.add(true); + } + } catch (IOException io) { + throw new UncheckedIOException(io); + } + }); + assertEquals(100, readKeys.size(), "Stream collect should return 100 records, since 2nd block is rolled back"); + assertEquals(50, newEmptyPayloads.size(), "Stream collect should return all 50 records with empty payloads"); + List firstBlockRecords = + copyOfRecords1.stream().map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) + .collect(Collectors.toList()); + Collections.sort(firstBlockRecords); + Collections.sort(readKeys); + assertEquals(firstBlockRecords, readKeys, "CompositeAvroLogReader should return 150 records from 2 versions"); } @ParameterizedTest @@ -1333,7 +1365,8 @@ public void testAvroLogRecordReaderWithDisorderDelete(ExternalSpillableMap.DiskM @MethodSource("testArguments") public void testAvroLogRecordReaderWithFailedRollbacks(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, - boolean readBlocksLazily) + boolean readBlocksLazily, + boolean useScanv2) throws IOException, URISyntaxException, InterruptedException { // Write a Data block and Delete block with same InstantTime (written in same batch) @@ -1375,7 +1408,7 @@ public void testAvroLogRecordReaderWithFailedRollbacks(ExternalSpillableMap.Disk // Attempt 1 : Write rollback block for a failed write header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, - String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); + String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_BLOCK.ordinal())); HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); try { writer.appendBlock(commandBlock); @@ -1405,6 +1438,7 @@ public void testAvroLogRecordReaderWithFailedRollbacks(ExternalSpillableMap.Disk .withSpillableMapBasePath(BASE_OUTPUT_PATH) .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) + .withUseScanV2(useScanv2) .build(); assertEquals(0, scanner.getTotalLogRecords(), "We would have scanned 0 records because of rollback"); @@ -1418,7 +1452,8 @@ public void testAvroLogRecordReaderWithFailedRollbacks(ExternalSpillableMap.Disk @MethodSource("testArguments") public void testAvroLogRecordReaderWithInsertDeleteAndRollback(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, - boolean readBlocksLazily) + boolean readBlocksLazily, + boolean useScanv2) throws IOException, URISyntaxException, InterruptedException { // Write a Data block and Delete block with same InstantTime (written in same batch) @@ -1451,7 +1486,7 @@ public void testAvroLogRecordReaderWithInsertDeleteAndRollback(ExternalSpillable // Write 2 rollback blocks (1 data block + 1 delete bloc) for a failed write header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, - String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); + String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_BLOCK.ordinal())); HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer.appendBlock(commandBlock); writer.appendBlock(commandBlock); @@ -1473,6 +1508,7 @@ public void testAvroLogRecordReaderWithInsertDeleteAndRollback(ExternalSpillable .withSpillableMapBasePath(BASE_OUTPUT_PATH) .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) + .withUseScanV2(useScanv2) .build(); assertEquals(0, scanner.getTotalLogRecords(), "We would read 0 records"); FileCreateUtils.deleteDeltaCommit(basePath, "100", fs); @@ -1482,7 +1518,8 @@ public void testAvroLogRecordReaderWithInsertDeleteAndRollback(ExternalSpillable @MethodSource("testArguments") public void testAvroLogRecordReaderWithInvalidRollback(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, - boolean readBlocksLazily) + boolean readBlocksLazily, + boolean useScanv2) throws IOException, URISyntaxException, InterruptedException { Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); // Set a small threshold so that every block is a new version @@ -1503,7 +1540,7 @@ public void testAvroLogRecordReaderWithInvalidRollback(ExternalSpillableMap.Disk // Write invalid rollback for a failed write (possible for in-flight commits) header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "101"); header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, - String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); + String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_BLOCK.ordinal())); HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer.appendBlock(commandBlock); @@ -1524,6 +1561,7 @@ public void testAvroLogRecordReaderWithInvalidRollback(ExternalSpillableMap.Disk .withSpillableMapBasePath(BASE_OUTPUT_PATH) .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) + .withUseScanV2(useScanv2) .build(); assertEquals(100, scanner.getTotalLogRecords(), "We still would read 100 records"); final List readKeys = new ArrayList<>(100); @@ -1535,7 +1573,8 @@ public void testAvroLogRecordReaderWithInvalidRollback(ExternalSpillableMap.Disk @MethodSource("testArguments") public void testAvroLogRecordReaderWithInsertsDeleteAndRollback(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, - boolean readBlocksLazily) + boolean readBlocksLazily, + boolean useScanv2) throws IOException, URISyntaxException, InterruptedException { // Write a 3 Data blocs with same InstantTime (written in same batch) @@ -1573,7 +1612,7 @@ public void testAvroLogRecordReaderWithInsertsDeleteAndRollback(ExternalSpillabl header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101"); header.put(HeaderMetadataType.TARGET_INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, - String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); + String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_BLOCK.ordinal())); HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer.appendBlock(commandBlock); @@ -1594,6 +1633,7 @@ public void testAvroLogRecordReaderWithInsertsDeleteAndRollback(ExternalSpillabl .withSpillableMapBasePath(BASE_OUTPUT_PATH) .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) + .withUseScanV2(useScanv2) .build(); assertEquals(0, scanner.getTotalLogRecords(), "We would read 0 records"); } @@ -1602,7 +1642,8 @@ public void testAvroLogRecordReaderWithInsertsDeleteAndRollback(ExternalSpillabl @MethodSource("testArguments") public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, - boolean readBlocksLazily) + boolean readBlocksLazily, + boolean useScanv2) throws IOException, URISyntaxException, InterruptedException { // Write a 3 Data blocs with same InstantTime (written in same batch) @@ -1678,7 +1719,7 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback(ExternalS header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101"); header.put(HeaderMetadataType.TARGET_INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, - String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); + String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_BLOCK.ordinal())); HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer.appendBlock(commandBlock); writer.close(); @@ -1700,11 +1741,202 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback(ExternalS .withSpillableMapBasePath(BASE_OUTPUT_PATH) .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) + .withUseScanV2(useScanv2) .build(); assertEquals(0, scanner.getTotalLogRecords(), "We would read 0 records"); FileCreateUtils.deleteDeltaCommit(basePath, "100", fs); } + @ParameterizedTest + @MethodSource("testArgumentsWithoutScanV2Arg") + public void testAvroLogRecordReaderWithMixedInsertsCorruptsRollbackAndMergedLogBlock(ExternalSpillableMap.DiskMapType diskMapType, + boolean isCompressionEnabled, + boolean readBlocksLazily) + throws IOException, URISyntaxException, InterruptedException { + + // Write blocks in this manner. + // Ex: B1(i1), B2(i2), B3(i3), CRPB, CRPB, CB4(i4, [i1,i2]), CB5(i5, [CB4, B3]), B6(i6), B7(i7), B8(i8), CB9(i9, [B7, B8]) + // CRPB implies a corrupt block and CB implies a compacted block. + + // Write a 3 Data blocks with same InstantTime (written in same batch) + Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); + // Set a small threshold so that every block is a new version + Writer writer = + HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) + .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); + + // Write 1st data blocks multiple times. + List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); + Set recordKeysOfFirstTwoBatches = records1.stream() + .map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema) + .get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()).collect(Collectors.toSet()); + Map header = new HashMap<>(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, new ArrayList<>(records1), header); + writer.appendBlock(dataBlock); + + FileCreateUtils.createDeltaCommit(basePath, "100", fs); + + // Write 2nd data block + List records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100); + recordKeysOfFirstTwoBatches.addAll(records2.stream() + .map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema) + .get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()).collect(Collectors.toList())); + header = new HashMap<>(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, new ArrayList<>(records2), header); + writer.appendBlock(dataBlock); + + FileCreateUtils.createDeltaCommit(basePath, "101", fs); + + // Write 3rd data block + List records3 = SchemaTestUtil.generateHoodieTestRecords(0, 100); + Set recordKeysOfFirstThreeBatches = new HashSet<>(recordKeysOfFirstTwoBatches); + recordKeysOfFirstThreeBatches.addAll(records3.stream() + .map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema) + .get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()).collect(Collectors.toList())); + header = new HashMap<>(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "102"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, new ArrayList<>(records3), header); + writer.appendBlock(dataBlock); + + writer.close(); + FileCreateUtils.createDeltaCommit(basePath, "102", fs); + + // Append some arbit byte[] to the end of the log (mimics a partially written commit) + fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf()); + FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath()); + // create a block with + outputStream.write(HoodieLogFormat.MAGIC); + outputStream.writeLong(1000); + outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal()); + outputStream.writeInt(HoodieLogFormat.CURRENT_VERSION); + // Write out a length that does not confirm with the content + outputStream.writeLong(100); + outputStream.flush(); + outputStream.close(); + + // Append some arbit byte[] to the end of the log (mimics a partially written commit) + fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf()); + outputStream = fs.append(writer.getLogFile().getPath()); + // create a block with + outputStream.write(HoodieLogFormat.MAGIC); + outputStream.writeLong(1000); + outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal()); + outputStream.writeInt(HoodieLogFormat.CURRENT_VERSION); + // Write out a length that does not confirm with the content + outputStream.writeLong(100); + outputStream.flush(); + outputStream.close(); + + writer = + HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) + .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); + + // Create compacted block CB4 + List compactedRecords = Stream.of(records1, records2).flatMap(Collection::stream) + .collect(Collectors.toList()); + header = new HashMap<>(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "103"); + header.put(HeaderMetadataType.COMPACTED_BLOCK_TIMES, "100,101"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, new ArrayList<>(compactedRecords), header); + writer.appendBlock(dataBlock); + + FileCreateUtils.createDeltaCommit(basePath, "103", fs); + + // Create compacted block CB5 + List secondCompactedRecords = Stream.of(compactedRecords, records3).flatMap(Collection::stream) + .collect(Collectors.toList()); + header = new HashMap<>(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "104"); + header.put(HeaderMetadataType.COMPACTED_BLOCK_TIMES, "103,102"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, new ArrayList<>(secondCompactedRecords), header); + writer.appendBlock(dataBlock); + + FileCreateUtils.createDeltaCommit(basePath, "104", fs); + + // Write 6th data block + List records6 = SchemaTestUtil.generateHoodieTestRecords(0, 100); + header = new HashMap<>(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "105"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, new ArrayList<>(records6), header); + writer.appendBlock(dataBlock); + + FileCreateUtils.createDeltaCommit(basePath, "105", fs); + + // Write 7th data block + List records7 = SchemaTestUtil.generateHoodieTestRecords(0, 100); + header = new HashMap<>(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "106"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, new ArrayList<>(records7), header); + writer.appendBlock(dataBlock); + + FileCreateUtils.createDeltaCommit(basePath, "106", fs); + + // Write 8th data block + List records8 = SchemaTestUtil.generateHoodieTestRecords(0, 100); + header = new HashMap<>(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "107"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, new ArrayList<>(records8), header); + writer.appendBlock(dataBlock); + + FileCreateUtils.createDeltaCommit(basePath, "107", fs); + + // Create compacted block CB9 + List thirdCompactedBlockRecords = Stream.of(records7, records8).flatMap(Collection::stream) + .collect(Collectors.toList()); + header = new HashMap<>(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "108"); + header.put(HeaderMetadataType.COMPACTED_BLOCK_TIMES, "106,107"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, new ArrayList<>(thirdCompactedBlockRecords), header); + writer.appendBlock(dataBlock); + writer.close(); + + FileCreateUtils.createDeltaCommit(basePath, "108", fs); + + List allLogFiles = + FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") + .map(s -> s.getPath().toString()).collect(Collectors.toList()); + + HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(fs) + .withBasePath(basePath) + .withLogFilePaths(allLogFiles) + .withReaderSchema(schema) + .withLatestInstantTime("108") + .withMaxMemorySizeInBytes(10240L) + .withReadBlocksLazily(readBlocksLazily) + .withReverseReader(false) + .withBufferSize(bufferSize) + .withSpillableMapBasePath(BASE_OUTPUT_PATH) + .withDiskMapType(diskMapType) + .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) + .withUseScanV2(true) + .build(); + assertEquals(600, scanner.getTotalLogRecords(), "We would read 600 records from scanner"); + final List readKeys = new ArrayList<>(); + scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); + List expectedRecords = Stream.of(secondCompactedRecords, records6, thirdCompactedBlockRecords) + .flatMap(Collection::stream) + .map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) + .sorted() + .collect(Collectors.toList()); + List validBlockInstants = scanner.getValidBlockInstants(); + List expectedBlockInstants = Arrays.asList("108","105", "104"); + assertEquals(expectedBlockInstants, validBlockInstants); + Collections.sort(readKeys); + assertEquals(expectedRecords, readKeys, "Record keys read should be exactly same."); + } + /* * During a spark stage failure, when the stage is retried, tasks that are part of the previous attempt * of the stage would continue to run. As a result two different tasks could be performing the same operation. @@ -1721,7 +1953,8 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback(ExternalS private void testAvroLogRecordReaderMergingMultipleLogFiles(int numRecordsInLog1, int numRecordsInLog2, ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, - boolean readBlocksLazily) { + boolean readBlocksLazily, + boolean useScanv2) { try { // Write one Data block with same InstantTime (written in same batch) Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); @@ -1774,6 +2007,7 @@ private void testAvroLogRecordReaderMergingMultipleLogFiles(int numRecordsInLog1 .withSpillableMapBasePath(BASE_OUTPUT_PATH) .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) + .withUseScanV2(useScanv2) .build(); assertEquals(Math.max(numRecordsInLog1, numRecordsInLog2), scanner.getNumMergedRecordsInLog(), @@ -1788,39 +2022,42 @@ private void testAvroLogRecordReaderMergingMultipleLogFiles(int numRecordsInLog1 @MethodSource("testArguments") public void testAvroLogRecordReaderWithFailedTaskInFirstStageAttempt(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, - boolean readBlocksLazily) { + boolean readBlocksLazily, + boolean useScanv2) { /* * FIRST_ATTEMPT_FAILED: * Original task from the stage attempt failed, but subsequent stage retry succeeded. */ testAvroLogRecordReaderMergingMultipleLogFiles(77, 100, - diskMapType, isCompressionEnabled, readBlocksLazily); + diskMapType, isCompressionEnabled, readBlocksLazily, useScanv2); } @ParameterizedTest @MethodSource("testArguments") public void testAvroLogRecordReaderWithFailedTaskInSecondStageAttempt(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, - boolean readBlocksLazily) { + boolean readBlocksLazily, + boolean useScanv2) { /* * SECOND_ATTEMPT_FAILED: * Original task from stage attempt succeeded, but subsequent retry attempt failed. */ testAvroLogRecordReaderMergingMultipleLogFiles(100, 66, - diskMapType, isCompressionEnabled, readBlocksLazily); + diskMapType, isCompressionEnabled, readBlocksLazily, useScanv2); } @ParameterizedTest @MethodSource("testArguments") public void testAvroLogRecordReaderTasksSucceededInBothStageAttempts(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, - boolean readBlocksLazily) { + boolean readBlocksLazily, + boolean useScanv2) { /* * BOTH_ATTEMPTS_SUCCEEDED: * Original task from the stage attempt and duplicate task from the stage retry succeeded. */ testAvroLogRecordReaderMergingMultipleLogFiles(100, 100, - diskMapType, isCompressionEnabled, readBlocksLazily); + diskMapType, isCompressionEnabled, readBlocksLazily, useScanv2); } @ParameterizedTest @@ -2138,7 +2375,29 @@ private HoodieDataBlock getDataBlock(HoodieLogBlockType dataBlockType, List testArguments() { - // Arg1: ExternalSpillableMap Type, Arg2: isDiskMapCompressionEnabled, Arg3: readBlocksLazily + // Arg1: ExternalSpillableMap Type, Arg2: isDiskMapCompressionEnabled, Arg3: readBlocksLazily, Arg4: useScanv2 + return Stream.of( + arguments(ExternalSpillableMap.DiskMapType.BITCASK, false, false, true), + arguments(ExternalSpillableMap.DiskMapType.ROCKS_DB, false, false, true), + arguments(ExternalSpillableMap.DiskMapType.BITCASK, true, false, true), + arguments(ExternalSpillableMap.DiskMapType.ROCKS_DB, true, false, true), + arguments(ExternalSpillableMap.DiskMapType.BITCASK, false, true, true), + arguments(ExternalSpillableMap.DiskMapType.ROCKS_DB, false, true, true), + arguments(ExternalSpillableMap.DiskMapType.BITCASK, true, true, true), + arguments(ExternalSpillableMap.DiskMapType.ROCKS_DB, true, true, true), + arguments(ExternalSpillableMap.DiskMapType.BITCASK, false, false, false), + arguments(ExternalSpillableMap.DiskMapType.ROCKS_DB, false, false, false), + arguments(ExternalSpillableMap.DiskMapType.BITCASK, true, false, false), + arguments(ExternalSpillableMap.DiskMapType.ROCKS_DB, true, false, false), + arguments(ExternalSpillableMap.DiskMapType.BITCASK, false, true, false), + arguments(ExternalSpillableMap.DiskMapType.ROCKS_DB, false, true, false), + arguments(ExternalSpillableMap.DiskMapType.BITCASK, true, true, false), + arguments(ExternalSpillableMap.DiskMapType.ROCKS_DB, true, true, false) + ); + } + + private static Stream testArgumentsWithoutScanV2Arg() { + // Arg1: ExternalSpillableMap Type, Arg2: isDiskMapCompressionEnabled, Arg3: readBlocksLazily, Arg4: useScanv2 return Stream.of( arguments(ExternalSpillableMap.DiskMapType.BITCASK, false, false), arguments(ExternalSpillableMap.DiskMapType.ROCKS_DB, false, false), diff --git a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormatAppendFailure.java b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormatAppendFailure.java index 6c4d69a05b296..d825c294cc6b2 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormatAppendFailure.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormatAppendFailure.java @@ -143,7 +143,7 @@ public void testFailedToGetAppendStreamFromHDFSNameNode() .overBaseCommit("").withFs(fs).build(); header = new HashMap<>(); header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, - String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); + String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_BLOCK.ordinal())); writer.appendBlock(new HoodieCommandBlock(header)); // The log version should be different for this new writer diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java index 1c8d5ece242da..beca794dd636c 100755 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java @@ -116,7 +116,7 @@ public void testLoadingInstantsFromFiles() throws IOException { timeline.getCommitTimeline().filterCompletedInstants().getInstants(), "Check the instants stream"); assertStreamEquals(Stream.of(instant5), - timeline.getCommitTimeline().filterPendingExcludingCompaction().getInstants(), + timeline.getCommitTimeline().filterPendingExcludingMajorAndMinorCompaction().getInstants(), "Check the instants stream"); // Backwards compatibility testing for reading compaction plans @@ -180,7 +180,7 @@ public void testTimelineOperations() { .getInstants().map(HoodieInstant::getTimestamp), "findInstantsBefore 07 should return 3 instants"); assertFalse(timeline.empty()); - assertFalse(timeline.getCommitTimeline().filterPendingExcludingCompaction().empty()); + assertFalse(timeline.getCommitTimeline().filterPendingExcludingMajorAndMinorCompaction().empty()); assertEquals(12, timeline.countInstants()); assertEquals("01", timeline.firstInstant( HoodieTimeline.COMMIT_ACTION, State.COMPLETED).get().getTimestamp()); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java index ea32a36d96be5..c8b6372630e53 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java @@ -35,6 +35,7 @@ import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieLogFile; @@ -1770,6 +1771,145 @@ private List buildWriteStats(HashMap> part return HoodieTestTable.generateHoodieWriteStatForPartition(maps, commitTime, false); } + @Test + public void testPendingMajorAndMinorCompactionOperations() throws Exception { + String partitionPath = "2020/06/27"; + new File(basePath + "/" + partitionPath).mkdirs(); + + // Generate 2 fileIds + String fileId1 = UUID.randomUUID().toString(); + String fileId2 = UUID.randomUUID().toString(); + + // This is used for verifying file system view after every commit. + FileSystemViewExpectedState expectedState = new FileSystemViewExpectedState(); + + // First delta commit on partitionPath which creates 2 log files. + String commitTime1 = "001"; + String logFileName1 = FSUtils.makeLogFileName(fileId1, HoodieFileFormat.HOODIE_LOG.getFileExtension(), commitTime1, 1, TEST_WRITE_TOKEN); + String logFileName2 = FSUtils.makeLogFileName(fileId2, HoodieFileFormat.HOODIE_LOG.getFileExtension(), commitTime1, 1, TEST_WRITE_TOKEN); + new File(basePath + "/" + partitionPath + "/" + logFileName1).createNewFile(); + new File(basePath + "/" + partitionPath + "/" + logFileName2).createNewFile(); + expectedState.logFilesCurrentlyPresent.add(logFileName1); + expectedState.logFilesCurrentlyPresent.add(logFileName2); + + HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); + HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); + commitMetadata.addWriteStat(partitionPath, getHoodieWriteStat(partitionPath, fileId1, logFileName1)); + commitMetadata.addWriteStat(partitionPath, getHoodieWriteStat(partitionPath, fileId2, logFileName2)); + HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, commitTime1); + saveAsComplete(commitTimeline, instant1, Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + + SyncableFileSystemView fileSystemView = getFileSystemView(metaClient.reloadActiveTimeline(), true); + + // Verify file system view after 1st commit. + verifyFileSystemView(partitionPath, expectedState, fileSystemView); + + // Second ingestion commit on partitionPath1 + // First delta commit on partitionPath1 which creates 2 log files. + String commitTime2 = "002"; + String logFileName3 = FSUtils.makeLogFileName(fileId1, HoodieFileFormat.HOODIE_LOG.getFileExtension(), commitTime1, 2, TEST_WRITE_TOKEN); + new File(basePath + "/" + partitionPath + "/" + logFileName3).createNewFile(); + expectedState.logFilesCurrentlyPresent.add(logFileName3); + + commitTimeline = metaClient.getActiveTimeline(); + commitMetadata = new HoodieCommitMetadata(); + commitMetadata.addWriteStat(partitionPath, getHoodieWriteStat(partitionPath, fileId1, logFileName3)); + HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, commitTime2); + + saveAsComplete(commitTimeline, instant2, Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + + // Verify file system view after 2nd commit. + verifyFileSystemView(partitionPath, expectedState, fileSystemView); + + // Create compaction commit + List logFiles = Stream.of( + basePath + "/" + partitionPath + "/" + logFileName1, basePath + "/" + partitionPath + "/" + logFileName3) + .map(HoodieLogFile::new) + .collect(Collectors.toList()); + CompactionOperation compactionOperation = new CompactionOperation(Option.empty(), partitionPath, logFiles, Collections.emptyMap()); + HoodieCompactionPlan compactionPlan = getHoodieCompactionPlan(Collections.singletonList(compactionOperation)); + expectedState.pendingCompactionFgIdsCurrentlyPresent.add(fileId1); + + String commitTime3 = "003"; + HoodieInstant compactionInstant = + new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, commitTime3); + HoodieInstant compactionRequested = HoodieTimeline.getCompactionRequestedInstant(compactionInstant.getTimestamp()); + metaClient.getActiveTimeline().saveToCompactionRequested(compactionRequested, + TimelineMetadataUtils.serializeCompactionPlan(compactionPlan)); + metaClient.getActiveTimeline().transitionCompactionRequestedToInflight(compactionRequested); + + // Verify file system view after 3rd commit which is compaction.requested. + verifyFileSystemView(partitionPath, expectedState, fileSystemView); + + + // Create log compaction commit + logFiles = Collections.singletonList(new HoodieLogFile(basePath + "/" + partitionPath + "/" + logFileName2)); + CompactionOperation logCompactionOperation = new CompactionOperation(Option.empty(), partitionPath, logFiles, Collections.emptyMap()); + HoodieCompactionPlan logCompactionPlan = getHoodieCompactionPlan(Collections.singletonList(logCompactionOperation)); + expectedState.pendingLogCompactionFgIdsCurrentlyPresent.add(fileId2); + + String commitTime4 = "004"; + HoodieInstant logCompactionInstant = + new HoodieInstant(State.INFLIGHT, HoodieTimeline.LOG_COMPACTION_ACTION, commitTime4); + HoodieInstant logCompactionRequested = HoodieTimeline.getLogCompactionRequestedInstant(logCompactionInstant.getTimestamp()); + metaClient.getActiveTimeline().saveToLogCompactionRequested(logCompactionRequested, + TimelineMetadataUtils.serializeCompactionPlan(logCompactionPlan)); + metaClient.getActiveTimeline().transitionLogCompactionRequestedToInflight(logCompactionRequested); + + // Verify file system view after 4th commit which is logcompaction.requested. + verifyFileSystemView(partitionPath, expectedState, fileSystemView); + } + + private HoodieCompactionPlan getHoodieCompactionPlan(List operations) { + return HoodieCompactionPlan.newBuilder() + .setOperations(operations.stream() + .map(CompactionUtils::buildHoodieCompactionOperation) + .collect(Collectors.toList())) + .setVersion(CompactionUtils.LATEST_COMPACTION_METADATA_VERSION).build(); + } + + private HoodieWriteStat getHoodieWriteStat(String partitionPath, String fileId, String relativeFilePath) { + HoodieWriteStat writeStat = new HoodieWriteStat(); + writeStat.setFileId(fileId); + writeStat.setPath(partitionPath + "/" + relativeFilePath); + writeStat.setPartitionPath(partitionPath); + return writeStat; + } + + static class FileSystemViewExpectedState { + Set logFilesCurrentlyPresent = new HashSet<>(); + Set baseFilesCurrentlyPresent = new HashSet<>(); + Set pendingCompactionFgIdsCurrentlyPresent = new HashSet<>(); + Set pendingLogCompactionFgIdsCurrentlyPresent = new HashSet<>(); + } + + /** + * Used to verify fils system view on various file systems. + */ + protected void verifyFileSystemView(String partitionPath, FileSystemViewExpectedState expectedState, + SyncableFileSystemView tableFileSystemView) { + tableFileSystemView.sync(); + // Verify base files + assertEquals(expectedState.baseFilesCurrentlyPresent,tableFileSystemView.getLatestBaseFiles(partitionPath) + .map(HoodieBaseFile::getFileName) + .collect(Collectors.toSet())); + + // Verify log files + assertEquals(expectedState.logFilesCurrentlyPresent, tableFileSystemView.getAllFileSlices(partitionPath) + .flatMap(FileSlice::getLogFiles) + .map(logFile -> logFile.getPath().getName()) + .collect(Collectors.toSet())); + // Verify file groups part of pending compaction operations + assertEquals(expectedState.pendingCompactionFgIdsCurrentlyPresent, tableFileSystemView.getPendingCompactionOperations() + .map(pair -> pair.getValue().getFileGroupId().getFileId()) + .collect(Collectors.toSet())); + + // Verify file groups part of pending log compaction operations + assertEquals(expectedState.pendingLogCompactionFgIdsCurrentlyPresent, tableFileSystemView.getPendingLogCompactionOperations() + .map(pair -> pair.getValue().getFileGroupId().getFileId()) + .collect(Collectors.toSet())); + } + @Override protected HoodieTableType getTableType() { return HoodieTableType.MERGE_ON_READ; diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java index 2f284c5befd1e..2367ba3035a31 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java @@ -76,6 +76,7 @@ import java.util.stream.IntStream; import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.LOG_COMPACTION_ACTION; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -165,6 +166,69 @@ public void testAsyncCompaction() throws IOException { Collections.singletonList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "24")))); } + @Test + public void testAsyncMajorAndMinorCompaction() throws IOException { + SyncableFileSystemView view = getFileSystemView(metaClient); + view.sync(); + + // Run 3 ingestion on MOR table (3 delta commits) + Map> instantsToFiles = + testMultipleWriteSteps(view, Arrays.asList("11", "12", "13"), true, "11"); + + view.sync(); + // Schedule log Compaction + scheduleLogCompaction(view, "14", "11"); + + // Schedule Compaction + scheduleCompaction(view, "15"); + + view.sync(); + // Restore pending compaction + unscheduleLogCompaction(view, "14", "15", "15"); + + // Add one more delta instant - This will be added to second file-slice + instantsToFiles.putAll(testMultipleWriteSteps(view, Collections.singletonList("16"), true, "15", 2)); + + view.sync(); + // Run Compaction + testMultipleWriteSteps(view, Collections.singletonList("15"), false, "15", 2, + Collections.singletonList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "16"))); + + // Run 2 more ingest + instantsToFiles.putAll(testMultipleWriteSteps(view, Arrays.asList("17", "18"), true, "15", 2)); + + // Schedule Compaction again + scheduleLogCompaction(view, "19", "15"); + + // Clean first slice + testCleans(view, Collections.singletonList("20"), + new HashMap>() { + { + put("11", Arrays.asList("12", "13")); + } + }, instantsToFiles, Collections.singletonList("11"), 0, 0); + + // Add one more ingestion instant. This will be added to 1st slice now since cleaner removed the older file slice. + instantsToFiles.putAll(testMultipleWriteSteps(view, Collections.singletonList("21"), true, "15", 1)); + + // Restore last ingestion + testRestore(view, Collections.singletonList("22"), new HashMap<>(), Collections.singletonList(getHoodieCommitInstant("21", true)), "24", false); + + // Run one more ingestion. This is still on the 1st slice + instantsToFiles.putAll(testMultipleWriteSteps(view, Collections.singletonList("23"), true, "15", 1)); + + // Finish Log Compaction + instantsToFiles.putAll(testMultipleWriteSteps(view, Collections.singletonList("24"), true, "15", 1, + Collections.singletonList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "24")))); + + // Schedule Compaction again + scheduleCompaction(view, "25"); + + // Finish Compaction + instantsToFiles.putAll(testMultipleWriteSteps(view, Collections.singletonList("25"), false, "25", 2, + Collections.singletonList(new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "25")))); + } + @Test public void testIngestion() throws IOException { SyncableFileSystemView view = getFileSystemView(metaClient); @@ -611,17 +675,16 @@ private Map> deleteFiles(List files) { * Schedule a pending compaction and validate. * * @param view Hoodie View - * @param instantTime COmpaction Instant Time + * @param instantTime Compaction Instant Time */ - private void scheduleCompaction(SyncableFileSystemView view, String instantTime) throws IOException { + private void scheduleCompaction(SyncableFileSystemView view, String instantTime) throws IOException { List> slices = partitions.stream() .flatMap(p -> view.getLatestFileSlices(p).map(s -> Pair.of(p, s))).collect(Collectors.toList()); long initialExpTotalFileSlices = partitions.stream().mapToLong(p -> view.getAllFileSlices(p).count()).sum(); - + HoodieInstant compactionRequestedInstant = new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, instantTime); HoodieCompactionPlan plan = CompactionUtils.buildFromFileSlices(slices, Option.empty(), Option.empty()); - HoodieInstant compactionInstant = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instantTime); - metaClient.getActiveTimeline().saveToCompactionRequested(compactionInstant, + metaClient.getActiveTimeline().saveToCompactionRequested(compactionRequestedInstant, TimelineMetadataUtils.serializeCompactionPlan(plan)); view.sync(); @@ -642,6 +705,35 @@ private void scheduleCompaction(SyncableFileSystemView view, String instantTime) areViewsConsistent(view, newView, initialExpTotalFileSlices + partitions.size() * fileIdsPerPartition.size()); } + /** + * Schedule a pending Log compaction and validate. + * + * @param view Hoodie View + * @param instantTime Log Compaction Instant Time + */ + private void scheduleLogCompaction(SyncableFileSystemView view, String instantTime, String baseInstantTime) throws IOException { + List> slices = partitions.stream() + .flatMap(p -> view.getLatestFileSlices(p).map(s -> Pair.of(p, s))).collect(Collectors.toList()); + + long initialExpTotalFileSlices = partitions.stream().mapToLong(p -> view.getAllFileSlices(p).count()).sum(); + HoodieInstant logCompactionRequestedInstant = new HoodieInstant(State.REQUESTED, LOG_COMPACTION_ACTION, instantTime); + HoodieCompactionPlan plan = CompactionUtils.buildFromFileSlices(slices, Option.empty(), Option.empty()); + metaClient.getActiveTimeline().saveToLogCompactionRequested(logCompactionRequestedInstant, + TimelineMetadataUtils.serializeCompactionPlan(plan)); + + view.sync(); + partitions.forEach(p -> { + view.getLatestFileSlices(p).forEach(fs -> { + assertEquals(baseInstantTime, fs.getBaseInstantTime()); + assertEquals(p, fs.getPartitionPath()); + }); + }); + + metaClient.reloadActiveTimeline(); + SyncableFileSystemView newView = getFileSystemView(metaClient); + areViewsConsistent(view, newView, initialExpTotalFileSlices); + } + /** * Unschedule a compaction instant and validate incremental fs view. * @@ -661,6 +753,25 @@ private void unscheduleCompaction(SyncableFileSystemView view, String compaction partitions.forEach(p -> view.getLatestFileSlices(p).forEach(fs -> assertEquals(newBaseInstant, fs.getBaseInstantTime()))); } + /** + * Unschedule a log compaction instant and validate incremental fs view. + * + * @param view Hoodie View + * @param logCompactionInstantTime Log Compaction Instant to be removed + * @param newLastInstant New Last instant + * @param newBaseInstant New Base instant of last file-slice + */ + private void unscheduleLogCompaction(SyncableFileSystemView view, String logCompactionInstantTime, String newLastInstant, + String newBaseInstant) throws IOException { + HoodieInstant instant = new HoodieInstant(State.REQUESTED, LOG_COMPACTION_ACTION, logCompactionInstantTime); + boolean deleted = metaClient.getFs().delete(new Path(metaClient.getMetaPath(), instant.getFileName()), false); + ValidationUtils.checkArgument(deleted, "Unable to delete log compaction instant."); + + view.sync(); + assertEquals(newLastInstant, view.getLastInstant().get().getTimestamp()); + partitions.forEach(p -> view.getLatestFileSlices(p).forEach(fs -> assertEquals(newBaseInstant, fs.getBaseInstantTime()))); + } + /** * Perform one or more rounds of ingestion/compaction and validate incremental timeline syncing. * @@ -679,8 +790,8 @@ private Map> testMultipleWriteSteps(SyncableFileSystemView * Perform one or more rounds of ingestion/compaction and validate incremental timeline syncing. * * @param view Hoodie View - * @param instants Ingestion/Commit INstants - * @param deltaCommit Delta COmmit ? + * @param instants Ingestion/Commit Instants + * @param deltaCommit Delta Commit ? * @param baseInstantForDeltaCommit Base Instant to be used in case of delta-commit * @param begin initial file-slice offset * @return List of new file created @@ -814,6 +925,11 @@ private void areViewsConsistent(SyncableFileSystemView view1, SyncableFileSystem Set> ops1 = view1.getPendingCompactionOperations().collect(Collectors.toSet()); Set> ops2 = view2.getPendingCompactionOperations().collect(Collectors.toSet()); assertEquals(ops1, ops2); + + // Pending Log Compaction Operations Check + ops1 = view1.getPendingLogCompactionOperations().collect(Collectors.toSet()); + ops2 = view2.getPendingLogCompactionOperations().collect(Collectors.toSet()); + assertEquals(ops1, ops2); } private List> generateDataForInstant(String baseInstant, String instant, boolean deltaCommit) { diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestPriorityBasedFileSystemView.java b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestPriorityBasedFileSystemView.java index 9fa96216f4d53..29fdbefced0bd 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestPriorityBasedFileSystemView.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestPriorityBasedFileSystemView.java @@ -554,6 +554,35 @@ public void testGetPendingCompactionOperations() { }); } + @Test + public void testGetPendingLogCompactionOperations() { + Stream> actual; + Stream> expected = Collections.singleton( + (Pair) new ImmutablePair<>("test", new CompactionOperation())) + .stream(); + + when(primary.getPendingLogCompactionOperations()).thenReturn(expected); + actual = fsView.getPendingLogCompactionOperations(); + assertEquals(expected, actual); + + resetMocks(); + when(primary.getPendingLogCompactionOperations()).thenThrow(new RuntimeException()); + when(secondary.getPendingLogCompactionOperations()).thenReturn(expected); + actual = fsView.getPendingLogCompactionOperations(); + assertEquals(expected, actual); + + resetMocks(); + when(secondary.getPendingLogCompactionOperations()).thenReturn(expected); + actual = fsView.getPendingLogCompactionOperations(); + assertEquals(expected, actual); + + resetMocks(); + when(secondary.getPendingLogCompactionOperations()).thenThrow(new RuntimeException()); + assertThrows(RuntimeException.class, () -> { + fsView.getPendingLogCompactionOperations(); + }); + } + @Test public void testClose() { fsView.close(); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/CompactionTestUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/CompactionTestUtils.java index fb5f123e80234..292cdc76b5951 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/CompactionTestUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/CompactionTestUtils.java @@ -189,7 +189,7 @@ public static HoodieCompactionPlan createCompactionPlan(HoodieTableMetaClient me } }).collect(Collectors.toList()); return new HoodieCompactionPlan(ops.isEmpty() ? null : ops, new HashMap<>(), - CompactionUtils.LATEST_COMPACTION_METADATA_VERSION); + CompactionUtils.LATEST_COMPACTION_METADATA_VERSION, null, null); } /** diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieCommonTestHarness.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieCommonTestHarness.java index dc64856d3c76c..d9b3d4ecada1a 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieCommonTestHarness.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieCommonTestHarness.java @@ -104,7 +104,7 @@ protected SyncableFileSystemView getFileSystemView(HoodieTimeline timeline, bool } protected SyncableFileSystemView getFileSystemView(HoodieTableMetaClient metaClient) throws IOException { - return getFileSystemView(metaClient, metaClient.getActiveTimeline().filterCompletedAndCompactionInstants()); + return getFileSystemView(metaClient, metaClient.getActiveTimeline().filterCompletedOrMajorOrMinorCompactionInstants()); } protected SyncableFileSystemView getFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline timeline) diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/RawTripTestPayload.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/RawTripTestPayload.java index c052b63ab544b..cd437db8b3aa9 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/RawTripTestPayload.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/RawTripTestPayload.java @@ -188,4 +188,12 @@ private String unCompressData(byte[] data) throws IOException { } } + public RawTripTestPayload clone() { + try { + return new RawTripTestPayload(unCompressData(jsonDataCompressed), rowKey, partitionPath, null); + } catch (IOException e) { + return null; + } + } + } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactFunction.java index 3622ed491b9dd..14c5c1694142f 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactFunction.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactFunction.java @@ -21,6 +21,7 @@ import org.apache.hudi.client.HoodieFlinkWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.model.CompactionOperation; +import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.configuration.OptionsResolver; import org.apache.hudi.sink.utils.NonThrownExecutor; @@ -107,15 +108,17 @@ private void doCompaction(String instantTime, Collector collector, HoodieWriteConfig writeConfig) throws IOException { HoodieFlinkMergeOnReadTableCompactor compactor = new HoodieFlinkMergeOnReadTableCompactor<>(); + HoodieTableMetaClient metaClient = writeClient.getHoodieTable().getMetaClient(); + String maxInstantTime = compactor.getMaxInstantTime(metaClient); List writeStatuses = compactor.compact( new HoodieFlinkCopyOnWriteTable<>( writeConfig, writeClient.getEngineContext(), - writeClient.getHoodieTable().getMetaClient()), - writeClient.getHoodieTable().getMetaClient(), + metaClient), + metaClient, writeClient.getConfig(), compactionOperation, - instantTime, + instantTime, maxInstantTime, writeClient.getHoodieTable().getTaskContextSupplier()); collector.collect(new CompactionCommitEvent(instantTime, compactionOperation.getFileId(), writeStatuses, taskID)); } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionCommitSink.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionCommitSink.java index 8dadd2e2dcf62..5f47219a52fdb 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionCommitSink.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionCommitSink.java @@ -118,7 +118,7 @@ private void commitIfNecessary(String instant, Collection try { return CompactionUtils.getCompactionPlan( this.writeClient.getHoodieTable().getMetaClient(), instant); - } catch (IOException e) { + } catch (Exception e) { throw new HoodieException(e); } }); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java index cfe53c6039b72..b4b7906960cdc 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java @@ -44,7 +44,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; @@ -264,7 +263,7 @@ private void compact() throws Exception { .map(timestamp -> { try { return Pair.of(timestamp, CompactionUtils.getCompactionPlan(table.getMetaClient(), timestamp)); - } catch (IOException e) { + } catch (Exception e) { throw new HoodieException("Get compaction plan at instant " + timestamp + " error", e); } }) diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestCompactionUtil.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestCompactionUtil.java index 87c8379d6ab80..b4d0490aa1e51 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestCompactionUtil.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestCompactionUtil.java @@ -160,7 +160,7 @@ void testScheduleCompaction() throws Exception { */ private String generateCompactionPlan() { HoodieCompactionOperation operation = new HoodieCompactionOperation(); - HoodieCompactionPlan plan = new HoodieCompactionPlan(Collections.singletonList(operation), Collections.emptyMap(), 1); + HoodieCompactionPlan plan = new HoodieCompactionPlan(Collections.singletonList(operation), Collections.emptyMap(), 1, null, null); String instantTime = HoodieActiveTimeline.createNewInstantTime(); HoodieInstant compactionInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instantTime); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/config/HoodieRealtimeConfig.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/config/HoodieRealtimeConfig.java index 95945f38b27cd..de062bc251af2 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/config/HoodieRealtimeConfig.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/config/HoodieRealtimeConfig.java @@ -39,4 +39,5 @@ public final class HoodieRealtimeConfig { public static final String SPILLABLE_MAP_BASE_PATH_PROP = "hoodie.memory.spillable.map.path"; // Default file path prefix for spillable file public static final String DEFAULT_SPILLABLE_MAP_BASE_PATH = "/tmp/"; + public static final String USE_LOG_RECORD_READER_SCAN_V2 = "hoodie.log.record.reader.use.scanV2"; } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java index 1f1dd1b927470..5ba62113a8e48 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java @@ -89,7 +89,7 @@ public boolean getBelongsToIncrementalQuery() { } public boolean isSplitable() { - return !toString().isEmpty() && !includeBootstrapFilePath(); + return !toString().contains(".log") && !includeBootstrapFilePath(); } public PathWithBootstrapFileStatus getPathWithBootstrapFileStatus() { diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java index b917f004bcd06..b1bd3df50f3ba 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java @@ -91,6 +91,7 @@ private HoodieMergedLogRecordScanner getMergedLogRecordScanner() throws IOExcept .withDiskMapType(jobConf.getEnum(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.key(), HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue())) .withBitCaskDiskMapCompressionEnabled(jobConf.getBoolean(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.key(), HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue())) + .withUseScanV2(jobConf.getBoolean(HoodieRealtimeConfig.USE_LOG_RECORD_READER_SCAN_V2, false)) .build(); } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeRecordReaderUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeRecordReaderUtils.java index bf4cbff6665cf..782fb2d7a3616 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeRecordReaderUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeRecordReaderUtils.java @@ -57,6 +57,7 @@ import java.util.LinkedHashSet; import java.util.List; import java.util.Map; +import java.util.Random; import java.util.Set; import java.util.TreeMap; import java.util.stream.Collectors; @@ -97,9 +98,10 @@ public static String arrayWritableToString(ArrayWritable writable) { if (writable == null) { return "null"; } + Random random = new Random(2); StringBuilder builder = new StringBuilder(); Writable[] values = writable.get(); - builder.append("\"values_" + Math.random() + "_" + values.length + "\": {"); + builder.append("\"values_" + random.nextDouble() + "_" + values.length + "\": {"); int i = 0; for (Writable w : values) { if (w instanceof ArrayWritable) { @@ -195,7 +197,7 @@ public static Writable avroToArrayWritable(Object value, Schema schema) { try { fieldValue = record.get(field.name()); } catch (AvroRuntimeException e) { - LOG.debug("Field:" + field.name() + "not found in Schema:" + schema.toString()); + LOG.debug("Field:" + field.name() + "not found in Schema:" + schema); } recordValues[recordValueIndex++] = avroToArrayWritable(fieldValue, field.schema()); } diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java index a4471845c3e42..134a32663576e 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java @@ -366,7 +366,7 @@ public static HoodieLogFormat.Writer writeRollback(File partitionDir, FileSystem header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, newCommit); header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, rolledBackInstant); header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, - String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); + String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_BLOCK.ordinal())); // if update belongs to an existing log file writer.appendBlock(new HoodieCommandBlock(header)); return writer; @@ -420,7 +420,7 @@ public static HoodieLogFormat.Writer writeRollbackBlockToLogFile(File partitionD header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, oldCommit); header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, - String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); + String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_BLOCK.ordinal())); HoodieCommandBlock rollbackBlock = new HoodieCommandBlock(header); writer.appendBlock(rollbackBlock); return writer; diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java index 2648740f54e0f..6c951868bb9b3 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java @@ -33,6 +33,7 @@ import org.apache.hudi.common.table.view.TableFileSystemView; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieMemoryConfig; import org.apache.hudi.io.storage.HoodieFileReader; import org.apache.hudi.io.storage.HoodieFileReaderFactory; @@ -285,6 +286,7 @@ private Iterator readColumnarOrLogFiles(FileSlice fileSlice) thro .withSpillableMapBasePath(HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH.defaultValue()) .withDiskMapType(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue()) .withBitCaskDiskMapCompressionEnabled(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue()) + .withUseScanV2(Boolean.parseBoolean(HoodieCompactionConfig.USE_LOG_RECORD_READER_SCAN_V2.defaultValue())) .build(); // readAvro log files Iterable> iterable = () -> scanner.iterator(); diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java index f59a2ceba891c..14539c45b050f 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java @@ -124,7 +124,7 @@ private boolean isLocalViewBehind(Context ctx) { ctx.queryParam(RemoteHoodieTableFileSystemView.LAST_INSTANT_TS, HoodieTimeline.INVALID_INSTANT_TS); String timelineHashFromClient = ctx.queryParam(RemoteHoodieTableFileSystemView.TIMELINE_HASH, ""); HoodieTimeline localTimeline = - viewManager.getFileSystemView(basePath).getTimeline().filterCompletedAndCompactionInstants(); + viewManager.getFileSystemView(basePath).getTimeline().filterCompletedOrMajorOrMinorCompactionInstants(); if (LOG.isDebugEnabled()) { LOG.debug("Client [ LastTs=" + lastKnownInstantFromClient + ", TimelineHash=" + timelineHashFromClient + "], localTimeline=" + localTimeline.getInstants().collect(Collectors.toList())); @@ -375,6 +375,13 @@ private void registerFileSlicesAPI() { writeValueAsString(ctx, dtos); }, true)); + app.get(RemoteHoodieTableFileSystemView.PENDING_LOG_COMPACTION_OPS, new ViewHandler(ctx -> { + metricsRegistry.add("PEDING_LOG_COMPACTION_OPS", 1); + List dtos = sliceHandler.getPendingLogCompactionOperations( + ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow()); + writeValueAsString(ctx, dtos); + }, true)); + app.get(RemoteHoodieTableFileSystemView.ALL_FILEGROUPS_FOR_PARTITION_URL, new ViewHandler(ctx -> { metricsRegistry.add("ALL_FILEGROUPS_FOR_PARTITION", 1); List dtos = sliceHandler.getAllFileGroups( diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/FileSliceHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/FileSliceHandler.java index 8b906351e7c41..caf1e3c93997c 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/FileSliceHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/FileSliceHandler.java @@ -88,6 +88,12 @@ public List getPendingCompactionOperations(String basePath) { .collect(Collectors.toList()); } + public List getPendingLogCompactionOperations(String basePath) { + return viewManager.getFileSystemView(basePath).getPendingLogCompactionOperations() + .map(instantOp -> CompactionOpDTO.fromCompactionOperation(instantOp.getKey(), instantOp.getValue())) + .collect(Collectors.toList()); + } + public List getAllFileGroups(String basePath, String partitionPath) { return viewManager.getFileSystemView(basePath).getAllFileGroups(partitionPath).map(FileGroupDTO::fromFileGroup) .collect(Collectors.toList());