You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by si...@apache.org on 2022/10/09 23:41:47 UTC
[hudi] branch master updated: [HUDI-3900] [UBER] Support log compaction action for MOR tables (#5958)
This is an automated email from the ASF dual-hosted git repository.
sivabalan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hudi.git
The following commit(s) were added to refs/heads/master by this push:
new 86a1efbff1 [HUDI-3900] [UBER] Support log compaction action for MOR tables (#5958)
86a1efbff1 is described below
commit 86a1efbff1300603a8180111eae117c7f9dbd8a5
Author: Surya Prasanna <sy...@uber.com>
AuthorDate: Sun Oct 9 16:41:35 2022 -0700
[HUDI-3900] [UBER] Support log compaction action for MOR tables (#5958)
- Adding log compaction support to MOR table. subsequent log blocks can now be compacted into larger log blocks without needing to go for full compaction (by merging w/ base file).
- New timeline action is introduced for the purpose.
Co-authored-by: sivabalan <n....@gmail.com>
---
.../hudi/cli/commands/HoodieLogFileCommand.java | 1 +
.../cli/commands/TestHoodieLogFileCommand.java | 1 +
.../apache/hudi/client/BaseHoodieWriteClient.java | 98 +++-
.../client/transaction/ConcurrentOperation.java | 2 +
...urrentFileWritesConflictResolutionStrategy.java | 10 +-
.../hudi/client/utils/MetadataConversionUtils.java | 6 +
.../apache/hudi/config/HoodieCompactionConfig.java | 33 ++
.../org/apache/hudi/config/HoodieWriteConfig.java | 15 +-
.../org/apache/hudi/io/HoodieAppendHandle.java | 75 ++-
.../metadata/HoodieBackedTableMetadataWriter.java | 1 +
.../org/apache/hudi/metrics/HoodieMetrics.java | 10 +
.../apache/hudi/table/HoodieCompactionHandler.java | 8 +
.../java/org/apache/hudi/table/HoodieTable.java | 44 +-
.../hudi/table/action/clean/CleanPlanner.java | 47 +-
.../cluster/strategy/ClusteringPlanStrategy.java | 11 +-
.../hudi/table/action/compact/CompactHelpers.java | 12 +
.../action/compact/CompactionExecutionHelper.java | 78 +++
.../hudi/table/action/compact/HoodieCompactor.java | 155 ++----
.../compact/LogCompactionExecutionHelper.java | 86 ++++
.../compact/RunCompactionActionExecutor.java | 25 +-
.../compact/ScheduleCompactionActionExecutor.java | 63 ++-
.../BaseHoodieCompactionPlanGenerator.java | 165 +++++++
.../generators/HoodieCompactionPlanGenerator.java | 63 +++
.../HoodieLogCompactionPlanGenerator.java | 104 ++++
.../table/action/rollback/BaseRollbackHelper.java | 2 +-
.../hudi/table/action/rollback/RollbackUtils.java | 2 +-
.../table/upgrade/OneToZeroDowngradeHandler.java | 2 +-
.../table/upgrade/TwoToOneDowngradeHandler.java | 2 +-
.../table/upgrade/ZeroToOneUpgradeHandler.java | 2 +-
.../hudi/table/HoodieFlinkMergeOnReadTable.java | 6 +-
.../HoodieFlinkMergeOnReadTableCompactor.java | 8 +-
.../hudi/table/HoodieJavaMergeOnReadTable.java | 6 +-
.../HoodieJavaMergeOnReadTableCompactor.java | 8 +-
.../apache/hudi/client/SparkRDDWriteClient.java | 63 ++-
.../MultipleSparkJobExecutionStrategy.java | 1 +
.../hudi/table/HoodieSparkMergeOnReadTable.java | 40 +-
.../HoodieSparkMergeOnReadTableCompactor.java | 11 +-
.../hudi/client/TestCompactionAdminClient.java | 2 +-
...DataValidationCheckForLogCompactionActions.java | 419 ++++++++++++++++
.../TestHoodieClientOnMergeOnReadStorage.java | 544 +++++++++++++++++++++
.../hudi/table/TestHoodieMergeOnReadTable.java | 96 ++++
.../GenericRecordValidationTestUtils.java | 127 +++++
.../hudi/testutils/HoodieClientTestBase.java | 3 +-
hudi-common/pom.xml | 1 +
.../src/main/avro/HoodieCompactionOperation.avsc | 12 +
.../src/main/avro/HoodieCompactionStrategy.avsc | 44 ++
.../hudi/common/config/HoodieMetadataConfig.java | 16 +
.../org/apache/hudi/common/model/ActionType.java | 2 +-
.../apache/hudi/common/model/TableServiceType.java | 4 +-
.../hudi/common/model/WriteOperationType.java | 3 +-
.../table/log/AbstractHoodieLogRecordReader.java | 269 +++++++++-
.../table/log/HoodieMergedLogRecordScanner.java | 23 +-
.../table/log/HoodieUnMergedLogRecordScanner.java | 15 +-
.../common/table/log/block/HoodieCommandBlock.java | 2 +-
.../common/table/log/block/HoodieLogBlock.java | 10 +-
.../table/timeline/HoodieActiveTimeline.java | 65 ++-
.../table/timeline/HoodieArchivedTimeline.java | 3 +-
.../table/timeline/HoodieDefaultTimeline.java | 49 +-
.../hudi/common/table/timeline/HoodieInstant.java | 20 +-
.../hudi/common/table/timeline/HoodieTimeline.java | 84 +++-
.../common/table/timeline/TimelineDiffHelper.java | 63 ++-
.../table/view/AbstractTableFileSystemView.java | 59 ++-
.../table/view/FileSystemViewStorageConfig.java | 11 +
.../table/view/HoodieTableFileSystemView.java | 53 ++
.../IncrementalTimelineSyncFileSystemView.java | 50 +-
.../table/view/PriorityBasedFileSystemView.java | 5 +
.../view/RemoteHoodieTableFileSystemView.java | 13 +
.../table/view/RocksDbBasedFileSystemView.java | 61 ++-
.../view/SpillableMapBasedFileSystemView.java | 24 +
.../common/table/view/TableFileSystemView.java | 7 +
.../apache/hudi/common/util/CollectionUtils.java | 1 +
.../apache/hudi/common/util/CompactionUtils.java | 127 +++--
.../hudi/common/util/RocksDBSchemaHelper.java | 15 +-
.../org/apache/hudi/common/util/StringUtils.java | 7 +
.../hudi/io/storage/HoodieFileReaderFactory.java | 1 -
.../hudi/metadata/HoodieBackedTableMetadata.java | 1 +
.../HoodieMetadataMergedLogRecordReader.java | 19 +-
.../common/functional/TestHoodieLogFormat.java | 329 +++++++++++--
.../TestHoodieLogFormatAppendFailure.java | 2 +-
.../table/timeline/TestHoodieActiveTimeline.java | 4 +-
.../table/view/TestHoodieTableFileSystemView.java | 140 ++++++
.../table/view/TestIncrementalFSViewSync.java | 130 ++++-
.../view/TestPriorityBasedFileSystemView.java | 29 ++
.../hudi/common/testutils/CompactionTestUtils.java | 2 +-
.../common/testutils/HoodieCommonTestHarness.java | 2 +-
.../hudi/common/testutils/RawTripTestPayload.java | 8 +
.../apache/hudi/sink/compact/CompactFunction.java | 9 +-
.../hudi/sink/compact/CompactionCommitSink.java | 2 +-
.../hudi/sink/compact/HoodieFlinkCompactor.java | 3 +-
.../org/apache/hudi/utils/TestCompactionUtil.java | 2 +-
.../hudi/hadoop/config/HoodieRealtimeConfig.java | 1 +
.../hudi/hadoop/realtime/HoodieRealtimePath.java | 2 +-
.../realtime/RealtimeCompactedRecordReader.java | 1 +
.../utils/HoodieRealtimeRecordReaderUtils.java | 6 +-
.../hudi/hadoop/testutils/InputFormatTestUtil.java | 4 +-
.../reader/DFSHoodieDatasetInputReader.java | 2 +
.../hudi/timeline/service/RequestHandler.java | 9 +-
.../service/handlers/FileSliceHandler.java | 6 +
98 files changed, 3954 insertions(+), 335 deletions(-)
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 56e00aa24c..22f70480a3 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 class HoodieLogFileCommand {
.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<? extends HoodieRecordPayload> hoodieRecord : scanner) {
Option<IndexedRecord> 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 e93ad0c8ca..25d54b6487 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 class TestHoodieLogFileCommand extends CLIFunctionalTestHarness {
.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<HoodieRecord<? extends HoodieRecordPayload>> 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 d9f260e633..146889c8db 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<T extends HoodieRecordPayload, I, K,
protected transient Timer.Context writeTimer = null;
protected transient Timer.Context compactionTimer;
protected transient Timer.Context clusteringTimer;
+ protected transient Timer.Context logCompactionTimer;
protected transient AsyncCleanerService asyncCleanerService;
protected transient AsyncArchiveService asyncArchiveService;
@@ -366,7 +367,7 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
protected void rollbackFailedBootstrap() {
LOG.info("Rolling back pending bootstrap if present");
HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
- HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
+ HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingMajorAndMinorCompaction();
Option<String> 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 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
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 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
});
}
+ 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<Pair<HoodieInstant, HoodieClusteringPlan>> instantPlan = ClusteringUtils.getClusteringPlan(table.getMetaClient(), instant);
@@ -1077,13 +1095,60 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
*/
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<String> scheduleLogCompaction(Option<Map<String, String>> 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<Map<String, String>> 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<O> 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<Map<String, String>> extraMetadata) {
+ throw new UnsupportedOperationException("Log compaction is not supported yet.");
+ }
+
+ /**
+ * Commit Log Compaction and track metrics.
+ */
+ protected void completeLogCompaction(HoodieCommitMetadata metadata, HoodieTable<T, I, K, O> 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<Pair<HoodieInstant, HoodieClusteringPlan>> instantPlan = ClusteringUtils.getClusteringPlan(metaClient, instant);
@@ -1139,7 +1204,7 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
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 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
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<O> 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<String> inlineLogCompact(Option<Map<String, String>> extraMetadata) {
+ Option<String> 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 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
Option<HoodieCompactionPlan> 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<HoodieCompactionPlan> 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<HoodieCleanerPlan> 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 35580229e3..4e69b69cf4 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 java.util.stream.Collectors;
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 @@ public class ConcurrentOperation {
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 938a40684a..d17db6302b 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 class SimpleConcurrentFileWritesConflictResolutionStrategy
// 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 342de74a11..08d9d34ba2 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 class MetadataConversionUtils {
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 b6fe6d8aa0..bdb346bd86 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<String> 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<String> 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<String> 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<String> 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 class HoodieCompactionConfig extends HoodieConfig {
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 class HoodieCompactionConfig extends HoodieConfig {
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 3b3995c8f8..0c0966bc5f 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
@@ -1153,6 +1153,15 @@ public class HoodieWriteConfig extends HoodieConfig {
/**
* 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));
}
@@ -1257,6 +1266,10 @@ public class HoodieWriteConfig extends HoodieConfig {
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));
}
@@ -2100,7 +2113,7 @@ public class HoodieWriteConfig extends HoodieConfig {
*/
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 3cb149427a..8db927d569 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<T extends HoodieRecordPayload, I, K, O> extends
// Header metadata for a log block
protected final Map<HeaderMetadataType, String> header = new HashMap<>();
private SizeEstimator<HoodieRecord> 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<T, I, K, O> hoodieTable,
+ String partitionPath, String fileId, Iterator<HoodieRecord<T>> recordItr,
+ TaskContextSupplier taskContextSupplier, Map<HeaderMetadataType, String> 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<T, I, K, O> hoodieTable,
String partitionPath, String fileId, Iterator<HoodieRecord<T>> recordItr, TaskContextSupplier taskContextSupplier) {
super(config, instantTime, partitionPath, fileId, hoodieTable, taskContextSupplier);
@@ -215,7 +233,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
// 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<IndexedRecord> avroRecord = nullifyPayload ? Option.empty() : hoodieRecord.getData().getInsertValue(tableSchema, recordProperties);
+ Option<IndexedRecord> avroRecord = nullifyPayload ? Option.empty() : getInsertValue(hoodieRecord);
if (avroRecord.isPresent()) {
if (avroRecord.get().equals(IGNORE_RECORD)) {
return avroRecord;
@@ -228,12 +246,14 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
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 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
return Option.empty();
}
+ private Option<IndexedRecord> getInsertValue(HoodieRecord<T> 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 class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
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<HeaderMetadataType, String> 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<HeaderMetadataType, String> header, boolean appendDeleteBlocks) {
try {
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, instantTime);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writeSchemaWithMetaFields.toString());
@@ -401,7 +434,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
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 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
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 class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
Option<Map<String, String>> 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 class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
public List<WriteStatus> 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 class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
}
}
+ public void write(Map<String, HoodieRecord<? extends HoodieRecordPayload>> recordMap) {
+ try {
+ for (Map.Entry<String, HoodieRecord<? extends HoodieRecordPayload>> entry: recordMap.entrySet()) {
+ HoodieRecord<T> record = (HoodieRecord<T>) 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 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
/**
* 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 962875fb92..7da0a4a57a 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 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
// 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 69ef7917b2..6bc91d0459 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 class HoodieMetrics {
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 class HoodieMetrics {
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 eeb287abd5..7c0ad4a4ad 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.client.WriteStatus;
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 @@ public interface HoodieCompactionHandler<T extends HoodieRecordPayload> {
Iterator<List<WriteStatus>> handleInsert(String instantTime, String partitionPath, String fileId,
Map<String, HoodieRecord<? extends HoodieRecordPayload>> recordMap);
+
+ default Iterator<List<WriteStatus>> handleInsertsForLogCompaction(String instantTime, String partitionPath, String fileId,
+ Map<String, HoodieRecord<? extends HoodieRecordPayload>> recordMap,
+ Map<HoodieLogBlock.HeaderMetadataType, String> 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 5ca3aee764..4a6f4ae1f4 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 abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
* 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 class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
public abstract HoodieWriteMetadata<O> 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<HoodieCompactionPlan> scheduleLogCompaction(HoodieEngineContext context,
+ String instantTime,
+ Option<Map<String, String>> 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<O> 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 abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
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 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
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<String, Option<HoodiePendingRollbackInfo>> 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 9027ab045a..c08bec8a26 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<T extends HoodieRecordPayload, I, K, O> implements Ser
private final SyncableFileSystemView fileSystemView;
private final HoodieTimeline commitTimeline;
private final Map<HoodieFileGroupId, CompactionOperation> fgIdToPendingCompactionOperations;
+ private final Map<HoodieFileGroupId, CompactionOperation> fgIdToPendingLogCompactionOperations;
private HoodieTable<T, I, K, O> hoodieTable;
private HoodieWriteConfig config;
private transient HoodieEngineContext context;
@@ -95,12 +96,16 @@ public class CleanPlanner<T extends HoodieRecordPayload, I, K, O> implements Ser
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 @@ public class CleanPlanner<T extends HoodieRecordPayload, I, K, O> implements Ser
int keepVersions = config.getCleanerFileVersionsRetained();
// do not cleanup slice required for pending compaction
Iterator<FileSlice> 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 @@ public class CleanPlanner<T extends HoodieRecordPayload, I, K, O> implements Ser
}
// 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 class CleanPlanner<T extends HoodieRecordPayload, I, K, O> implements Ser
}
}
+ /*
+ * 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 @@ public class CleanPlanner<T extends HoodieRecordPayload, I, K, O> implements Ser
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 dd827ff5a9..2df1824c5f 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 abstract class ClusteringPlanStrategy<T extends HoodieRecordPayload,I,K,O
*/
protected Stream<FileSlice> getFileSlicesEligibleForClustering(String partition) {
SyncableFileSystemView fileSystemView = (SyncableFileSystemView) getHoodieTable().getSliceView();
- Set<HoodieFileGroupId> fgIdsInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
- .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
- .collect(Collectors.toSet());
- fgIdsInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getKey).collect(Collectors.toSet()));
+ Set<HoodieFileGroupId> 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 3379d16f4c..a4daacfedb 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 class CompactHelpers<T extends HoodieRecordPayload, I, K, O> {
"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 0000000000..f402a67359
--- /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<T extends HoodieRecordPayload, I, K, O> 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<List<WriteStatus>> writeFileAndGetWriteStats(HoodieCompactionHandler compactionHandler,
+ CompactionOperation operation,
+ String instantTime,
+ HoodieMergedLogRecordScanner scanner,
+ Option<HoodieBaseFile> oldDataFileOpt) throws IOException {
+ Iterator<List<WriteStatus>> 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 75954872ae..51ed7639a4 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.ArrayList;
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<T extends HoodieRecordPayload, I, K, O> 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 abstract class HoodieCompactor<T extends HoodieRecordPayload, I, K, O> im
|| (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 abstract class HoodieCompactor<T extends HoodieRecordPayload, I, K, O> im
.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 abstract class HoodieCompactor<T extends HoodieRecordPayload, I, K, O> im
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<WriteStatus> 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<InternalSchema> internalSchemaOption = Option.empty();
@@ -161,7 +169,7 @@ public abstract class HoodieCompactor<T extends HoodieRecordPayload, I, K, O> im
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 abstract class HoodieCompactor<T extends HoodieRecordPayload, I, K, O> im
// 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 abstract class HoodieCompactor<T extends HoodieRecordPayload, I, K, O> im
.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 abstract class HoodieCompactor<T extends HoodieRecordPayload, I, K, O> im
.withBitCaskDiskMapCompressionEnabled(config.getCommonConfig().isBitCaskDiskMapCompressionEnabled())
.withOperationField(config.allowOperationMetadataField())
.withPartition(operation.getPartitionPath())
+ .withUseScanV2(executionHelper.useScanV2(config))
.build();
Option<HoodieBaseFile> oldDataFileOpt =
@@ -222,15 +228,7 @@ public abstract class HoodieCompactor<T extends HoodieRecordPayload, I, K, O> im
// Compacting is very similar to applying updates to existing file
Iterator<List<WriteStatus>> 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<List<WriteStatus>> resultIterable = () -> result;
return StreamSupport.stream(resultIterable.spliterator(), false).flatMap(Collection::stream).peek(s -> {
@@ -249,82 +247,21 @@ public abstract class HoodieCompactor<T extends HoodieRecordPayload, I, K, O> im
}).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<T, I, K, O> hoodieTable, HoodieWriteConfig config,
- String compactionCommitTime, Set<HoodieFileGroupId> 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<String> 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<HoodieCompactionOperation> operations = context.flatMap(partitionPaths, partitionPath -> fileSystemView
- .getLatestFileSlices(partitionPath)
- .filter(slice -> !fgIdsInPendingCompactionAndClustering.contains(slice.getFileGroupId()))
- .map(s -> {
- List<HoodieLogFile> 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<HoodieBaseFile> 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 0000000000..0e49267507
--- /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<T extends HoodieRecordPayload, I, K, O>
+ extends CompactionExecutionHelper<T, I, K, O> {
+
+ 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<List<WriteStatus>> writeFileAndGetWriteStats(HoodieCompactionHandler compactionHandler,
+ CompactionOperation operation,
+ String instantTime,
+ HoodieMergedLogRecordScanner scanner,
+ Option<HoodieBaseFile> oldDataFileOpt) throws IOException {
+ Map<HoodieLogBlock.HeaderMetadataType, String> 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 fc4ae986e6..a0e5ec22f9 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.HoodieTable;
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<T extends HoodieRecordPayload> extends
BaseActionExecutor<T, HoodieData<HoodieRecord<T>>, HoodieData<HoodieKey>, HoodieData<WriteStatus>, HoodieWriteMetadata<HoodieData<WriteStatus>>> {
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<HoodieData<WriteStatus>> 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<HoodieData<WriteStatus>> 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 class RunCompactionActionExecutor<T extends HoodieRecordPayload> extends
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 4fb5f9f7dd..8010cfbe82 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 @@ package org.apache.hudi.table.action.compact;
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.exception.HoodieIOException;
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.io.IOException;
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<T extends HoodieRecordPayload, I, K, O> extends BaseActionExecutor<T, I, K, O, Option<HoodieCompactionPlan>> {
private static final Logger LOG = LogManager.getLogger(ScheduleCompactionActionExecutor.class);
-
+ private WriteOperationType operationType;
private final Option<Map<String, String>> extraMetadata;
- private final HoodieCompactor compactor;
+ private BaseHoodieCompactionPlanGenerator planGenerator;
public ScheduleCompactionActionExecutor(HoodieEngineContext context,
HoodieWriteConfig config,
HoodieTable<T, I, K, O> table,
String instantTime,
Option<Map<String, String>> 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<T, I, K, O> table) {
+ if (WriteOperationType.COMPACT.equals(operationType)) {
+ planGenerator = new HoodieCompactionPlanGenerator(table, context, config);
+ } else {
+ planGenerator = new HoodieLogCompactionPlanGenerator(table, context, config);
+ }
}
@Override
public Option<HoodieCompactionPlan> 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 class ScheduleCompactionActionExecutor<T extends HoodieRecordPayload, I,
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 @@ public class ScheduleCompactionActionExecutor<T extends HoodieRecordPayload, I,
if (compactable) {
LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
try {
- SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
- Set<HoodieFileGroupId> 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 @@ public class ScheduleCompactionActionExecutor<T extends HoodieRecordPayload, I,
return false;
}
Pair<Integer, String> 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 0000000000..5e5d6de92d
--- /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<T extends HoodieRecordPayload, I, K, O> implements Serializable {
+ private static final Logger LOG = LogManager.getLogger(BaseHoodieCompactionPlanGenerator.class);
+
+ protected final HoodieTable<T, I, K, O> 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<String> 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<HoodieFileGroupId> 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<HoodieCompactionOperation> operations = engineContext.flatMap(partitionPaths, partitionPath -> fileSystemView
+ .getLatestFileSlices(partitionPath)
+ .filter(slice -> filterFileSlice(slice, lastCompletedInstantTime, fgIdsInPendingCompactionAndClustering))
+ .map(s -> {
+ List<HoodieLogFile> 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<HoodieBaseFile> 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<HoodieCompactionOperation> operations);
+
+ protected abstract boolean filterLogCompactionOperations();
+
+ protected List<String> filterPartitionPathsByStrategy(HoodieWriteConfig writeConfig, List<String> partitionPaths) {
+ return partitionPaths;
+ }
+
+ protected boolean filterFileSlice(FileSlice fileSlice, String lastCompletedInstantTime, Set<HoodieFileGroupId> pendingFileGroupIds) {
+ return fileSlice.getLogFiles().count() > 0 && !pendingFileGroupIds.contains(fileSlice.getFileGroupId());
+ }
+
+ protected Map<String, String> 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 0000000000..2adac577b7
--- /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<T extends HoodieRecordPayload, I, K, O>
+ extends BaseHoodieCompactionPlanGenerator<T, I, K, O> {
+
+ 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<HoodieCompactionOperation> 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<String> filterPartitionPathsByStrategy(HoodieWriteConfig writeConfig, List<String> 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 0000000000..7357054b4e
--- /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<T extends HoodieRecordPayload, I, K, O> extends BaseHoodieCompactionPlanGenerator<T, I, K, O> {
+
+ 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<HoodieCompactionOperation> 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<HoodieFileGroupId> 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 8d5e767307..7cd09930c1 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 @@ public class BaseRollbackHelper implements Serializable {
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 ce7a185151..c37244f7fc 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 @@ public class RollbackUtils {
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 14fe8e2b88..f70fd4ade0 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 class OneToZeroDowngradeHandler implements DowngradeHandler {
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<HoodieInstant> 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 de1a1067fe..5cde65e257 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 class TwoToOneDowngradeHandler implements DowngradeHandler {
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<HoodieInstant> 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 95f22bba27..935f827ab6 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 class ZeroToOneUpgradeHandler implements UpgradeHandler {
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<String> 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 aa8adde735..d69534c061 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.client.WriteStatus;
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 class HoodieFlinkMergeOnReadTable<T extends HoodieRecordPayload>
String instantTime,
Option<Map<String, String>> 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 class HoodieFlinkMergeOnReadTable<T extends HoodieRecordPayload>
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 03b9f8e7ee..d47da21716 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.data.HoodieData;
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<T extends HoodieRecordPayload>
@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 32d30f704e..f5bc09b4d1 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.client.common.HoodieJavaEngineContext;
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 class HoodieJavaMergeOnReadTable<T extends HoodieRecordPayload> extends H
@Override
public Option<HoodieCompactionPlan> scheduleCompaction(HoodieEngineContext context, String instantTime, Option<Map<String, String>> 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 class HoodieJavaMergeOnReadTable<T extends HoodieRecordPayload> extends H
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 30bdcda759..6dc65649a1 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.data.HoodieData;
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<T extends HoodieRecordPayload>
@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 7110e26bb0..ef37dd1835 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.config.HoodieWriteConfig;
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 class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
/**
* 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 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
return compactionMetadata;
}
+ @Override
+ public void commitLogCompaction(String logCompactionInstantTime, HoodieCommitMetadata metadata, Option<Map<String, String>> extraMetadata) {
+ HoodieSparkTable<T> 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<HoodieWriteStat> 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<JavaRDD<WriteStatus>> logCompact(String logCompactionInstantTime, boolean shouldComplete) {
+ HoodieSparkTable<T> 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<HoodieData<WriteStatus>> writeMetadata = table.logCompact(context, logCompactionInstantTime);
+ HoodieWriteMetadata<JavaRDD<WriteStatus>> 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<JavaRDD<WriteStatus>> cluster(String clusteringInstant, boolean shouldComplete) {
HoodieSparkTable<T> table = HoodieSparkTable.create(config, context);
@@ -470,6 +528,9 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
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 eab98f2f19..171f54fc2d 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 @@ public abstract class MultipleSparkJobExecutionStrategy<T extends HoodieRecordPa
.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 efc667af29..bf88aa2690 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.HoodieKey;
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.BaseRollbackPlanActionExecutor;
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 @@ import java.util.Map;
* action
* </p>
*/
-public class HoodieSparkMergeOnReadTable<T extends HoodieRecordPayload> extends HoodieSparkCopyOnWriteTable<T> {
+public class HoodieSparkMergeOnReadTable<T extends HoodieRecordPayload> extends HoodieSparkCopyOnWriteTable<T> implements HoodieCompactionHandler<T> {
HoodieSparkMergeOnReadTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) {
super(config, context, metaClient);
@@ -127,9 +132,8 @@ public class HoodieSparkMergeOnReadTable<T extends HoodieRecordPayload> extends
@Override
public Option<HoodieCompactionPlan> scheduleCompaction(HoodieEngineContext context, String instantTime, Option<Map<String, String>> 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 class HoodieSparkMergeOnReadTable<T extends HoodieRecordPayload> extends
HoodieEngineContext context, String compactionInstantTime) {
RunCompactionActionExecutor<T> 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 class HoodieSparkMergeOnReadTable<T extends HoodieRecordPayload> extends
return new SparkBootstrapDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, extraMetadata).execute();
}
+ @Override
+ public Option<HoodieCompactionPlan> scheduleLogCompaction(HoodieEngineContext context, String instantTime, Option<Map<String, String>> extraMetadata) {
+ ScheduleCompactionActionExecutor scheduleLogCompactionExecutor = new ScheduleCompactionActionExecutor(
+ context, config, this, instantTime, extraMetadata, WriteOperationType.LOG_COMPACT);
+ return scheduleLogCompactionExecutor.execute();
+ }
+
+ @Override
+ public HoodieWriteMetadata<HoodieData<WriteStatus>> 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 class HoodieSparkMergeOnReadTable<T extends HoodieRecordPayload> extends
shouldRollbackUsingMarkers).execute();
}
+ @Override
+ public Iterator<List<WriteStatus>> handleInsertsForLogCompaction(String instantTime, String partitionPath, String fileId,
+ Map<String, HoodieRecord<? extends HoodieRecordPayload>> recordMap,
+ Map<HoodieLogBlock.HeaderMetadataType, String> header) {
+ HoodieAppendHandle appendHandle = new HoodieAppendHandle(config, instantTime, this,
+ partitionPath, fileId, recordMap.values().iterator(), taskContextSupplier, header);
+ appendHandle.write(recordMap);
+ List<WriteStatus> 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 61cb1ffd27..af3e4960ad 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.data.HoodieData;
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<T extends HoodieRecordPayload>
@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 67d82578fc..a5cc430b6d 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 class TestCompactionAdminClient extends HoodieClientTestBase {
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 0000000000..eb9d2b4628
--- /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<List<HoodieRecord>, HoodieTestDataGenerator, String, Integer> insertsGenFunction =
+ HoodieTestDataGenerator::generateInserts;
+ Function3<List<HoodieRecord>, HoodieTestDataGenerator, String, Integer> updatesGenFunction =
+ HoodieTestDataGenerator::generateUniqueUpdates;
+ Function2<List<HoodieKey>, HoodieTestDataGenerator, Integer> deletesGenFunction =
+ HoodieTestDataGenerator::generateUniqueDeletes;
+
+ Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> insertsFunction = SparkRDDWriteClient::insert;
+ Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> updatesFunction = SparkRDDWriteClient::upsert;
+ Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieKey>, 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<String, GenericRecord> mainRecordsMap =
+ GenericRecordValidationTestUtils.getRecordsMap(mainTable.config, hadoopConf, dataGen);
+ Map<String, GenericRecord> 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<String> 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<String> 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<WriteStatus> 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<WriteStatus> 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<WriteStatus> insertDataIntoMainTable(TestTableContents mainTable, String commitTime) throws IOException {
+ int numRecords = 50 + random.nextInt(10);
+ List<HoodieRecord> records = insertsGenFunction.apply(dataGen, commitTime, numRecords);
+ mainTable.updatePreviousGeneration(records, commitTime, 0);
+ JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
+ return insertsFunction.apply(mainTable.client, writeRecords, commitTime);
+ }
+
+ private JavaRDD<WriteStatus> updateDataIntoMainTable(TestTableContents mainTable, String commitTime) throws IOException {
+ int numRecords = 10 + random.nextInt(10);
+ List<HoodieRecord> records = updatesGenFunction.apply(dataGen, commitTime, numRecords);
+ mainTable.updatePreviousGeneration(records, commitTime, 1);
+ JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
+ return updatesFunction.apply(mainTable.client, writeRecords, commitTime);
+ }
+
+ private JavaRDD<WriteStatus> deleteDataIntoMainTable(TestTableContents mainTable, String commitTime) throws IOException {
+ int numRecords = 5 + random.nextInt(10);
+ List<HoodieKey> keys = deletesGenFunction.apply(dataGen, numRecords);
+ mainTable.updatePreviousGenerationForDelete(keys, commitTime);
+ JavaRDD<HoodieKey> deleteKeys = jsc.parallelize(keys, 1);
+ return deletesFunction.apply(mainTable.client, deleteKeys, commitTime);
+ }
+
+ private JavaRDD<WriteStatus> insertDataIntoExperimentTable(TestTableContents mainTable, TestTableContents experimentTable) throws IOException {
+ JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(mainTable.generatedRecords, 1);
+ return insertsFunction.apply(experimentTable.client, writeRecords, mainTable.commitTimeOnMainTable);
+ }
+
+ private JavaRDD<WriteStatus> updateDataIntoExperimentTable(TestTableContents mainTable, TestTableContents experimentTable) throws IOException {
+ JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(mainTable.generatedRecords, 1);
+ return updatesFunction.apply(experimentTable.client, writeRecords, mainTable.commitTimeOnMainTable);
+ }
+
+ private JavaRDD<WriteStatus> deleteDataIntoExperimentTable(TestTableContents mainTable, TestTableContents experimentTable) throws IOException {
+ JavaRDD<HoodieKey> writeKeys = jsc.parallelize(mainTable.generatedKeysForDelete, 1);
+ return deletesFunction.apply(experimentTable.client, writeKeys, mainTable.commitTimeOnMainTable);
+ }
+
+ private void verifyWriteStatus(JavaRDD<WriteStatus> writeStatuses) {
+ List<WriteStatus> 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<HoodieRecord> generatedRecords = new ArrayList<>();
+ List<HoodieKey> 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<HoodieRecord> 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<HoodieKey> 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 0000000000..e54ca074da
--- /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<String, GenericRecord> 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<String> 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<String> 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<String> 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<String> 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<String> compactionTimeStamp = client.scheduleCompaction(Option.empty());
+ assertTrue(compactionTimeStamp.isPresent());
+
+ // Try scheduing log compaction, it wont succeed
+ Option<String> 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<String> logCompactionTimeStamp = client.scheduleLogCompaction(Option.empty());
+ assertTrue(logCompactionTimeStamp.isPresent());
+
+ // Try scheduling compaction, it wont succeed
+ Option<String> 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<String> 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<String> 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<String> 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<String> 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<String> 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<String> logCompactionInstantTimes = new ArrayList<>();
+
+ for (int i = 0; i < 6; i++) {
+ if (i % 4 == 0) {
+ // Schedule compaction.
+ Option<String> 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<String> logCompactionTimeStamp = lcWriteClient.scheduleLogCompaction(Option.empty());
+ if (logCompactionTimeStamp.isPresent()) {
+ logCompactionInstantTimes.add(logCompactionTimeStamp.get());
+ lcWriteClient.logCompact(logCompactionTimeStamp.get());
+ prevCommitTime = logCompactionTimeStamp.get();
+ }
+ }
+ boolean logCompactionInstantArchived = false;
+ Map<String, List<HoodieInstant>> instantsMap = metaClient.getArchivedTimeline().getInstants()
+ .collect(Collectors.groupingBy(HoodieInstant::getTimestamp));
+ for (String logCompactionTimeStamp: logCompactionInstantTimes) {
+ List<HoodieInstant> 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 18f764c1fa..5ba0f33f9b 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.testutils.Transformations;
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.Assertions.assertNoWriteErrors;
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 class TestHoodieMergeOnReadTable extends SparkClientFunctionalTestHarness
}
}
+ @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<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
+ JavaRDD<HoodieRecord> recordsRDD = jsc().parallelize(records, 1);
+ writeClient.insert(recordsRDD, newCommitTime).collect();
+
+ // Update all the 100 records
+ newCommitTime = "101";
+ List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
+ JavaRDD<HoodieRecord> updatedRecordsRDD = jsc().parallelize(updatedRecords, 1);
+
+ HoodieReadClient readClient = new HoodieReadClient(context(), config);
+ JavaRDD<HoodieRecord> 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<String> 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<FileSlice> 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<JavaRDD<WriteStatus>> 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<FileSlice> 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 0000000000..36ef74b605
--- /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<String> 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<String> partitionPaths) {
+ List<String> 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<String> partitionPaths, List<String> excludeFields) {
+ JobConf jobConf = new JobConf(hadoopConf);
+ List<String> 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<GenericRecord> records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
+ hadoopConf, fullPartitionPaths, config.getBasePath(), jobConf, true);
+ Map<String, GenericRecord> 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<GenericRecord> records1 = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
+ hadoopConf, fullPartitionPaths, config.getBasePath(), jobConf, true);
+ Map<String, GenericRecord> 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<String, GenericRecord> getRecordsMap(HoodieWriteConfig config, Configuration hadoopConf,
+ HoodieTestDataGenerator dataGen) {
+ JobConf jobConf = new JobConf(hadoopConf);
+ List<String> 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 900674a677..424bb6c53e 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.Set;
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 class HoodieClientTestBase extends HoodieClientTestHarness {
.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 ae5303cc70..87b8e5e0be 100644
--- a/hudi-common/pom.xml
+++ b/hudi-common/pom.xml
@@ -68,6 +68,7 @@
<imports>
<!-- import avro files -->
<import>${basedir}/src/main/avro/HoodieCommitMetadata.avsc</import>
+ <import>${basedir}/src/main/avro/HoodieCompactionStrategy.avsc</import>
<import>${basedir}/src/main/avro/HoodieCompactionOperation.avsc</import>
<import>${basedir}/src/main/avro/HoodieSavePointMetadata.avsc</import>
<import>${basedir}/src/main/avro/HoodieCompactionMetadata.avsc</import>
diff --git a/hudi-common/src/main/avro/HoodieCompactionOperation.avsc b/hudi-common/src/main/avro/HoodieCompactionOperation.avsc
index 2095a9518c..bab7321f29 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 0000000000..eff500d1f5
--- /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 b16373ef83..793d79e256 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<Boolean> 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 final class HoodieMetadataConfig extends HoodieConfig {
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 final class HoodieMetadataConfig extends HoodieConfig {
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 c10c99d8dc..eb7e578522 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 @@ package org.apache.hudi.common.model;
* 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 69dd30782f..2aa5d08f06 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 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline;
* 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 enum TableServiceType {
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 e0574d5119..3be3004e53 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 5bfb395dbc..88da6aa1f0 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 org.apache.log4j.Logger;
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<String> partitionName;
// Populate meta fields for the records
private boolean populateMetaFields = true;
+ // Collect all the block instants after scanning all the log files.
+ private List<String> validBlockInstants = new ArrayList<>();
+ // Use scanV2 method.
+ private boolean useScanV2;
protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List<String> logFilePaths,
Schema readerSchema,
@@ -147,14 +157,14 @@ public abstract class AbstractHoodieLogRecordReader {
int bufferSize, Option<InstantRange> 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<String> logFilePaths,
Schema readerSchema, String latestInstantTime, boolean readBlocksLazily,
boolean reverseReader, int bufferSize, Option<InstantRange> instantRange,
boolean withOperationField, boolean forceFullScan,
- Option<String> partitionName, InternalSchema internalSchema) {
+ Option<String> 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 @@ public abstract class AbstractHoodieLogRecordReader {
this.forceFullScan = forceFullScan;
this.internalSchema = internalSchema == null ? InternalSchema.getEmptyInternalSchema() : internalSchema;
this.path = basePath;
+ this.useScanV2 = useScanV2;
// Key fields when populate meta fields is disabled (that is, virtual keys enabled)
if (!tableConfig.populateMetaFields()) {
@@ -195,14 +206,22 @@ public abstract class AbstractHoodieLogRecordReader {
}
public synchronized void scan() {
- scanInternal(Option.empty());
+ scanInternal(Option.empty(), false);
}
public synchronized void scan(List<String> keys) {
- scanInternal(Option.of(new KeySpec(keys, true)));
+ scanInternal(Option.of(new KeySpec(keys, true)), false);
}
- protected synchronized void scanInternal(Option<KeySpec> keySpecOpt) {
+ public synchronized void scanInternal(Option<KeySpec> keySpecOpt, boolean skipProcessingBlocks) {
+ if (useScanV2) {
+ scanInternalV2(keySpecOpt, skipProcessingBlocks);
+ } else {
+ scanInternal(keySpecOpt);
+ }
+ }
+
+ private synchronized void scanInternal(Option<KeySpec> keySpecOpt) {
currentInstantLogBlocks = new ArrayDeque<>();
progress = 0.0f;
totalLogFiles = new AtomicLong(0);
@@ -295,7 +314,7 @@ public abstract class AbstractHoodieLogRecordReader {
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 @@ public abstract class AbstractHoodieLogRecordReader {
}
}
+ private synchronized void scanInternalV2(Option<KeySpec> 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<String> 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<String, List<HoodieLogBlock>> instantToBlocksMap = new HashMap<>();
+
+ // Order of Instants.
+ List<String> orderedInstantsList = new ArrayList<>();
+
+ Set<HoodieLogFile> 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<HoodieLogBlock> 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<String> 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<String, String> 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<HoodieLogBlock> 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<HoodieLogBlock> 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<HoodieLogBlock> 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 abstract class AbstractHoodieLogRecordReader {
}
}
+ public Deque<HoodieLogBlock> getCurrentInstantLogBlocks() {
+ return currentInstantLogBlocks;
+ }
+
+ public List<String> getValidBlockInstants() {
+ return validBlockInstants;
+ }
+
/**
* Builder used to build {@code AbstractHoodieLogRecordScanner}.
*/
@@ -584,6 +837,10 @@ public abstract class AbstractHoodieLogRecordReader {
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 d7e725544a..756e031cea 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 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader
ExternalSpillableMap.DiskMapType diskMapType,
boolean isBitCaskDiskMapCompressionEnabled,
boolean withOperationField, boolean forceFullScan,
- Option<String> partitionName, InternalSchema internalSchema) {
+ Option<String> 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 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader
// 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 class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader
// incremental filtering
protected Option<InstantRange> 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 class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader
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 class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader
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 16d3a7a3e4..7ddb9f1236 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
@@ -26,6 +26,7 @@ import org.apache.hudi.common.util.Option;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem;
+import org.apache.hudi.internal.schema.InternalSchema;
import java.util.List;
import java.util.stream.Collectors;
@@ -39,8 +40,9 @@ public class HoodieUnMergedLogRecordScanner extends AbstractHoodieLogRecordReade
private HoodieUnMergedLogRecordScanner(FileSystem fs, String basePath, List<String> logFilePaths, Schema readerSchema,
String latestInstantTime, boolean readBlocksLazily, boolean reverseReader, int bufferSize,
- LogRecordScannerCallback callback, Option<InstantRange> instantRange) {
- super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize, instantRange, false);
+ LogRecordScannerCallback callback, Option<InstantRange> instantRange, boolean useScanV2) {
+ super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize, instantRange,
+ false, true, Option.empty(), InternalSchema.getEmptyInternalSchema(), useScanV2);
this.callback = callback;
}
@@ -86,6 +88,7 @@ public class HoodieUnMergedLogRecordScanner extends AbstractHoodieLogRecordReade
private Option<InstantRange> instantRange = Option.empty();
// specific configurations
private LogRecordScannerCallback callback;
+ private boolean useScanV2;
public Builder withFileSystem(FileSystem fs) {
this.fs = fs;
@@ -139,10 +142,16 @@ public class HoodieUnMergedLogRecordScanner extends AbstractHoodieLogRecordReade
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 0ff3a77b50..c44f195014 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<HeaderMetadataType, String> 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 1718e7dd02..f8307333d2 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 abstract class HoodieLogBlock {
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 abstract class HoodieLogBlock {
* 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 2b27d3ab5e..be2febdff3 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 class HoodieActiveTimeline extends HoodieDefaultTimeline {
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 class HoodieActiveTimeline extends HoodieDefaultTimeline {
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 class HoodieActiveTimeline extends HoodieDefaultTimeline {
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 class HoodieActiveTimeline extends HoodieDefaultTimeline {
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<byte[]> 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<byte[]> data) {
// This will be removed in future release. See HUDI-546
Path fullPath = new Path(metaClient.getMetaAuxiliaryPath(), instant.getFileName());
@@ -653,6 +705,17 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
createFileInMetaPath(instant.getFileName(), content, overwrite);
}
+ public void saveToLogCompactionRequested(HoodieInstant instant, Option<byte[]> content) {
+ saveToLogCompactionRequested(instant, content, false);
+ }
+
+ public void saveToLogCompactionRequested(HoodieInstant instant, Option<byte[]> 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 4df30b115e..2f0ebffbef 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 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
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 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
@Override
public HoodieDefaultTimeline getWriteTimeline() {
// filter in-memory instants
- Set<String> validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, REPLACE_COMMIT_ACTION);
+ Set<String> 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 7324421894..0803faeab2 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 class HoodieDefaultTimeline implements HoodieTimeline {
&& (!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 class HoodieDefaultTimeline implements HoodieTimeline {
|| 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<String> 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<String> validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, REPLACE_COMMIT_ACTION);
+ Set<String> 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 class HoodieDefaultTimeline implements HoodieTimeline {
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 class HoodieDefaultTimeline implements HoodieTimeline {
*/
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 8b1cb875c0..0115742e07 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<HoodieInstant> {
* 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<String, String> COMPARABLE_ACTIONS =
- CollectionUtils.createImmutableMap(HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.COMMIT_ACTION);
+ private static final Map<String, String> COMPARABLE_ACTIONS = createComparableActionsMap();
public static final Comparator<HoodieInstant> ACTION_COMPARATOR =
Comparator.comparing(instant -> getComparableAction(instant.getAction()));
@@ -164,6 +163,14 @@ public class HoodieInstant implements Serializable, Comparable<HoodieInstant> {
} 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 class HoodieInstant implements Serializable, Comparable<HoodieInstant> {
throw new IllegalArgumentException("Cannot get file name for unknown action " + action);
}
+ private static final Map<String, String> createComparableActionsMap() {
+ Map<String, String> 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 e52a279596..a210783095 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 @@ public interface HoodieTimeline extends Serializable {
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 @@ public interface HoodieTimeline extends Serializable {
/**
* 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<HoodieInstant> 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 @@ public interface HoodieTimeline extends Serializable {
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 5659dcbdf7..29399705f0 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 class TimelineDiffHelper {
&& 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<Pair<HoodieInstant, HoodieInstant>> logCompactionInstants = getPendingLogCompactionTransitions(oldTimeline, newTimeline);
+ List<HoodieInstant> 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 class TimelineDiffHelper {
}
}
+ /**
+ * Getting pending log compaction transitions.
+ */
+ private static List<Pair<HoodieInstant, HoodieInstant>> getPendingLogCompactionTransitions(HoodieTimeline oldTimeline,
+ HoodieTimeline newTimeline) {
+ Set<HoodieInstant> 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.<HoodieInstant, HoodieInstant>of(instant, null);
+ }
+ }).collect(Collectors.toList());
+ }
+
+ /**
+ * Getting pending compaction transitions.
+ */
private static List<Pair<HoodieInstant, HoodieInstant>> getPendingCompactionTransitions(HoodieTimeline oldTimeline,
HoodieTimeline newTimeline) {
Set<HoodieInstant> newTimelineInstants = newTimeline.getInstants().collect(Collectors.toSet());
@@ -94,6 +129,11 @@ public class TimelineDiffHelper {
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.<HoodieInstant, HoodieInstant>of(instant, null);
}
}).collect(Collectors.toList());
@@ -106,14 +146,17 @@ public class TimelineDiffHelper {
private final List<HoodieInstant> newlySeenInstants;
private final List<HoodieInstant> finishedCompactionInstants;
+ private final List<HoodieInstant> 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<HoodieInstant> newlySeenInstants, List<HoodieInstant> finishedCompactionInstants,
- boolean canSyncIncrementally) {
+ List<HoodieInstant> finishedOrRemovedLogCompactionInstants, boolean canSyncIncrementally) {
this.newlySeenInstants = newlySeenInstants;
this.finishedCompactionInstants = finishedCompactionInstants;
+ this.finishedOrRemovedLogCompactionInstants = finishedOrRemovedLogCompactionInstants;
this.canSyncIncrementally = canSyncIncrementally;
}
@@ -125,14 +168,22 @@ public class TimelineDiffHelper {
return finishedCompactionInstants;
}
+ public List<HoodieInstant> 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 976217ae07..a614523ba0 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 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
// 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 abstract class AbstractTableFileSystemView implements SyncableFileSystemV
}
}
+ @Override
+ public final Stream<Pair<String, CompactionOperation>> getPendingLogCompactionOperations() {
+ try {
+ readLock.lock();
+ return fetchPendingLogCompactionOperations();
+ } finally {
+ readLock.unlock();
+ }
+ }
+
@Override
public final Stream<HoodieBaseFile> getLatestBaseFiles(String partitionStr) {
try {
@@ -846,6 +861,35 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
*/
abstract void removePendingCompactionOperations(Stream<Pair<String, CompactionOperation>> 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<Pair<String, CompactionOperation>> operations);
+
+ /**
+ * Add pending Log compaction operations to store.
+ *
+ * @param operations Pending Log compaction operations to be added
+ */
+ abstract void addPendingLogCompactionOperations(Stream<Pair<String, CompactionOperation>> operations);
+
+ /**
+ * Remove pending Log compaction operations from store.
+ *
+ * @param operations Pending Log compaction operations to be removed
+ */
+ abstract void removePendingLogCompactionOperations(Stream<Pair<String, CompactionOperation>> operations);
+
/**
* Check if there is an outstanding clustering operation (requested/inflight) scheduled for this file.
*
@@ -888,11 +932,24 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
protected abstract Option<Pair<String, CompactionOperation>> getPendingCompactionOperationWithInstant(
HoodieFileGroupId fileGroupId);
+ /**
+ * Return pending Log compaction operation for a file-group.
+ *
+ * @param fileGroupId File-Group Id
+ */
+ protected abstract Option<Pair<String, CompactionOperation>> getPendingLogCompactionOperationWithInstant(
+ HoodieFileGroupId fileGroupId);
+
/**
* Fetch all pending compaction operations.
*/
abstract Stream<Pair<String, CompactionOperation>> fetchPendingCompactionOperations();
+ /**
+ * Fetch all pending log compaction operations.
+ */
+ abstract Stream<Pair<String, CompactionOperation>> fetchPendingLogCompactionOperations();
+
/**
* Check if there is an bootstrap base file present for this file.
*
@@ -1197,7 +1254,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
@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 92937f61e2..2f43fb9b4f 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<Double> 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<Double> BOOTSTRAP_BASE_FILE_MEM_FRACTION = ConfigProperty
.key("hoodie.filesystem.view.spillable.bootstrap.base.file.mem.fraction")
.defaultValue(0.05)
@@ -206,6 +211,12 @@ public class FileSystemViewStorageConfig extends HoodieConfig {
.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 9dac360813..ea72f305b0 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<HoodieFileGroupId, Pair<String, CompactionOperation>> fgIdToPendingCompaction;
+ /**
+ * PartitionPath + File-Id to pending compaction instant time.
+ */
+ protected Map<HoodieFileGroupId, Pair<String, CompactionOperation>> fgIdToPendingLogCompaction;
+
/**
* PartitionPath + File-Id to bootstrap base File (Index Only bootstrapped).
*/
@@ -143,6 +148,11 @@ public class HoodieTableFileSystemView extends IncrementalTimelineSyncFileSystem
return fileIdToPendingCompaction;
}
+ protected Map<HoodieFileGroupId, Pair<String, CompactionOperation>> createFileIdToPendingLogCompactionMap(
+ Map<HoodieFileGroupId, Pair<String, CompactionOperation>> fileIdToPendingLogCompaction) {
+ return fileIdToPendingLogCompaction;
+ }
+
protected Map<HoodieFileGroupId, BootstrapBaseFileMapping> createFileIdToBootstrapBaseFileMap(
Map<HoodieFileGroupId, BootstrapBaseFileMapping> fileGroupIdBootstrapBaseFileMap) {
return fileGroupIdBootstrapBaseFileMap;
@@ -213,6 +223,39 @@ public class HoodieTableFileSystemView extends IncrementalTimelineSyncFileSystem
});
}
+ @Override
+ protected boolean isPendingLogCompactionScheduledForFileId(HoodieFileGroupId fgId) {
+ return fgIdToPendingLogCompaction.containsKey(fgId);
+ }
+
+ @Override
+ protected void resetPendingLogCompactionOperations(Stream<Pair<String, CompactionOperation>> 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<Pair<String, CompactionOperation>> 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<Pair<String, CompactionOperation>> 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 class HoodieTableFileSystemView extends IncrementalTimelineSyncFileSystem
@Override
Stream<Pair<String, CompactionOperation>> fetchPendingCompactionOperations() {
return fgIdToPendingCompaction.values().stream();
+ }
+
+ @Override
+ Stream<Pair<String, CompactionOperation>> fetchPendingLogCompactionOperations() {
+ return fgIdToPendingLogCompaction.values().stream();
}
@@ -323,6 +371,11 @@ public class HoodieTableFileSystemView extends IncrementalTimelineSyncFileSystem
return Option.ofNullable(fgIdToPendingCompaction.get(fgId));
}
+ @Override
+ protected Option<Pair<String, CompactionOperation>> 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 9f9fd1f7ce..4c2aa79f27 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 @@ public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTabl
}
});
- // 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 @@ public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTabl
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 @@ public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTabl
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 @@ public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTabl
});
}
+ /**
+ * 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<Pair<String, CompactionOperation>> 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 ff44c7cef0..62edc4daa3 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 class PriorityBasedFileSystemView implements SyncableFileSystemView, Seri
return execute(preferredView::getPendingCompactionOperations, secondaryView::getPendingCompactionOperations);
}
+ @Override
+ public Stream<Pair<String, CompactionOperation>> getPendingLogCompactionOperations() {
+ return execute(preferredView::getPendingLogCompactionOperations, secondaryView::getPendingLogCompactionOperations);
+ }
+
@Override
public Stream<Pair<HoodieFileGroupId, HoodieInstant>> 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 bd18ba22a2..759ef70c6d 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 class RemoteHoodieTableFileSystemView implements SyncableFileSystemView,
}
}
+ @Override
+ public Stream<Pair<String, CompactionOperation>> getPendingLogCompactionOperations() {
+ Map<String, String> paramsMap = getParams();
+ try {
+ List<CompactionOpDTO> dtos = executeRequest(PENDING_LOG_COMPACTION_OPS, paramsMap,
+ new TypeReference<List<CompactionOpDTO>>() {}, RequestMethod.GET);
+ return dtos.stream().map(CompactionOpDTO::toCompactionOperation);
+ } catch (IOException e) {
+ throw new HoodieRemoteException(e);
+ }
+ }
+
@Override
public Stream<Pair<HoodieFileGroupId, HoodieInstant>> getFileGroupsInPendingClustering() {
Map<String, String> 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 ebd759f880..2389f26781 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 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
);
}
+ @Override
+ protected boolean isPendingLogCompactionScheduledForFileId(HoodieFileGroupId fgId) {
+ return getPendingLogCompactionOperationWithInstant(fgId).isPresent();
+ }
+
+ @Override
+ protected void resetPendingLogCompactionOperations(Stream<Pair<String, CompactionOperation>> 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<Pair<String, CompactionOperation>> 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<Pair<String, CompactionOperation>> 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 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
rocksDB.writeBatch(batch ->
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 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
fileGroups.forEach(fgToPendingClusteringInstant -> {
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 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
return Option.ofNullable(instantOperationPair);
}
+ @Override
+ protected Option<Pair<String, CompactionOperation>> getPendingLogCompactionOperationWithInstant(HoodieFileGroupId fgId) {
+ String lookupKey = schemaHelper.getKeyForPendingLogCompactionLookup(fgId);
+ Pair<String, CompactionOperation> 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 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
.map(Pair::getValue);
}
+ @Override
+ Stream<Pair<String, CompactionOperation>> fetchPendingLogCompactionOperations() {
+ return rocksDB.<Pair<String, CompactionOperation>>prefixSearch(schemaHelper.getColFamilyForPendingLogCompaction(), "")
+ .map(Pair::getValue);
+ }
+
@Override
Stream<HoodieBaseFile> fetchAllBaseFiles(String partitionPath) {
return rocksDB.<HoodieBaseFile>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 e414442083..26d36d9c97 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 class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView {
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 @@ public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView {
}
}
+ @Override
+ protected Map<HoodieFileGroupId, Pair<String, CompactionOperation>> createFileIdToPendingLogCompactionMap(
+ Map<HoodieFileGroupId, Pair<String, CompactionOperation>> fgIdToPendingLogCompaction) {
+ try {
+ LOG.info("Creating Pending Log Compaction map using external spillable Map. Max Mem=" + maxMemoryForPendingLogCompaction
+ + ", BaseDir=" + baseStoreDir);
+ new File(baseStoreDir).mkdirs();
+ Map<HoodieFileGroupId, Pair<String, CompactionOperation>> 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<HoodieFileGroupId, BootstrapBaseFileMapping> createFileIdToBootstrapBaseFileMap(
Map<HoodieFileGroupId, BootstrapBaseFileMapping> fileGroupIdBootstrapBaseFileMap) {
@@ -169,6 +188,11 @@ public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView {
return ((ExternalSpillableMap) fgIdToPendingCompaction).valueStream();
}
+ @Override
+ Stream<Pair<String, CompactionOperation>> fetchPendingLogCompactionOperations() {
+ return ((ExternalSpillableMap) fgIdToPendingLogCompaction).valueStream();
+ }
+
@Override
Stream<BootstrapBaseFileMapping> 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 c32e2cabb1..18c9a9af99 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 @@ public interface TableFileSystemView {
*/
Stream<Pair<String, CompactionOperation>> getPendingCompactionOperations();
+ /**
+ * Return Pending Compaction Operations.
+ *
+ * @return Pair<Pair<InstantTime,CompactionOperation>>
+ */
+ Stream<Pair<String, CompactionOperation>> 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 90d6e6ae90..3b94948707 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 @@ public class CollectionUtils {
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 cf9b5fb3ce..d909d209de 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.table.timeline.versioning.compaction.CompactionV2M
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 @@ import java.util.stream.Stream;
*/
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 class CompactionUtils {
/**
* Get all pending compaction plans along with their instants.
- *
* @param metaClient Hoodie Meta Client
*/
public static List<Pair<HoodieInstant, HoodieCompactionPlan>> getAllPendingCompactionPlans(
HoodieTableMetaClient metaClient) {
- List<HoodieInstant> 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<HoodieTableMetaClient, HoodieTimeline> getFilteredTimelineByActionType =
+ (hoodieTableMetaClient) -> hoodieTableMetaClient.getActiveTimeline().filterPendingCompactionTimeline();
+ // Hoodie requested instant supplier
+ Function<String, HoodieInstant> 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<Pair<HoodieInstant, HoodieCompactionPlan>> getAllPendingLogCompactionPlans(
+ HoodieTableMetaClient metaClient) {
+ // This function returns pending logcompaction timeline.
+ Function<HoodieTableMetaClient, HoodieTimeline> filteredTimelineSupplier =
+ (hoodieTableMetaClient) -> hoodieTableMetaClient.getActiveTimeline().filterPendingLogCompactionTimeline();
+ // Hoodie requested instant supplier
+ Function<String, HoodieInstant> 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<Pair<HoodieInstant, HoodieCompactionPlan>> getCompactionPlansByTimeline(
+ HoodieTableMetaClient metaClient, Function<HoodieTableMetaClient, HoodieTimeline> filteredTimelineSupplier,
+ Function<String, HoodieInstant> requestedInstantWrapper) {
+ List<HoodieInstant> 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 class CompactionUtils {
HoodieTableMetaClient metaClient) {
List<Pair<HoodieInstant, HoodieCompactionPlan>> 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<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> getAllPendingLogCompactionOperations(
+ HoodieTableMetaClient metaClient) {
+ List<Pair<HoodieInstant, HoodieCompactionPlan>> pendingLogCompactionPlanWithInstants =
+ getAllPendingLogCompactionPlans(metaClient);
+ return getAllPendingCompactionOperationsInPendingCompactionPlans(pendingLogCompactionPlanWithInstants);
+ }
- Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> 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<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> getAllPendingCompactionOperationsInPendingCompactionPlans(
+ List<Pair<HoodieInstant, HoodieCompactionPlan>> pendingLogCompactionPlanWithInstants) {
+
+ Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> 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<Pair<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>>> getPendingCompactionOperations(
HoodieInstant instant, HoodieCompactionPlan compactionPlan) {
List<HoodieCompactionOperation> 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 e9377a6674..45b2a13eb7 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 class RocksDBSchemaHelper {
}
public List<String> 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 class RocksDBSchemaHelper {
return getPartitionFileIdBasedLookup(fgId);
}
+ public String getKeyForPendingLogCompactionLookup(HoodieFileGroupId fgId) {
+ return getPartitionFileIdBasedLookup(fgId);
+ }
+
public String getKeyForBootstrapBaseFile(HoodieFileGroupId fgId) {
return getPartitionFileIdBasedLookup(fgId);
}
@@ -130,6 +137,10 @@ public class RocksDBSchemaHelper {
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 a4f2c62437..af80af3db7 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 class StringUtils {
return String.join(delimiter, elements);
}
+ public static String join(final List<String> 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 f913df7e15..1c7663b371 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 class HoodieFileReaderFactory {
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 cb9fb8da14..bcaf2bcab3 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 class HoodieBackedTableMetadata extends BaseTableMetadata {
.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 cbd7e6c175..b844f219cd 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 @@ public class HoodieMetadataMergedLogRecordReader extends HoodieMergedLogRecordSc
String spillableMapBasePath,
ExternalSpillableMap.DiskMapType diskMapType,
boolean isBitCaskDiskMapCompressionEnabled,
- Option<InstantRange> instantRange, boolean allowFullScan) {
+ Option<InstantRange> 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 class HoodieMetadataMergedLogRecordReader extends HoodieMergedLogRecordSc
// processing log block records as part of scan.
synchronized (this) {
records.clear();
- scanInternal(Option.of(new KeySpec(keyPrefixes, false)));
+ scanInternal(Option.of(new KeySpec(keyPrefixes, false)), false);
return records.values().stream()
.filter(Objects::nonNull)
.map(record -> (HoodieRecord<HoodieMetadataPayload>) record)
@@ -137,8 +138,12 @@ public class HoodieMetadataMergedLogRecordReader extends HoodieMergedLogRecordSc
* 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 class HoodieMetadataMergedLogRecordReader extends HoodieMergedLogRecordSc
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 f87e5a41b8..a5df6252e4 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 class TestHoodieLogFormat extends HoodieCommonTestHarness {
@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 class TestHoodieLogFormat extends HoodieCommonTestHarness {
.withSpillableMapBasePath(BASE_OUTPUT_PATH)
.withDiskMapType(diskMapType)
.withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
+ .withUseScanV2(useScanv2)
.build();
List<IndexedRecord> scannedRecords = new ArrayList<>();
@@ -872,7 +874,8 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
@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 class TestHoodieLogFormat extends HoodieCommonTestHarness {
.withSpillableMapBasePath(BASE_OUTPUT_PATH)
.withDiskMapType(diskMapType)
.withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
+ .withUseScanV2(useScanv2)
.build();
assertEquals(200, scanner.getTotalLogRecords());
Set<String> readKeys = new HashSet<>(200);
@@ -934,7 +938,8 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
@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 class TestHoodieLogFormat extends HoodieCommonTestHarness {
writer.appendBlock(dataBlock);
// Write 2
+ header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101");
List<IndexedRecord> records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
@@ -961,13 +967,16 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
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<IndexedRecord> records3 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List<IndexedRecord> copyOfRecords3 = records3.stream()
@@ -996,6 +1005,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.withSpillableMapBasePath(BASE_OUTPUT_PATH)
.withDiskMapType(diskMapType)
.withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
+ .withUseScanV2(useScanv2)
.build();
assertEquals(200, scanner.getTotalLogRecords(), "We read 200 records from 2 write batches");
Set<String> readKeys = new HashSet<>(200);
@@ -1011,7 +1021,8 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
@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 class TestHoodieLogFormat extends HoodieCommonTestHarness {
.withSpillableMapBasePath(BASE_OUTPUT_PATH)
.withDiskMapType(diskMapType)
.withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
+ .withUseScanV2(useScanv2)
.build();
assertEquals(200, scanner.getTotalLogRecords(), "We would read 200 records");
Set<String> readKeys = new HashSet<>(200);
@@ -1099,7 +1111,8 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
@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 class TestHoodieLogFormat extends HoodieCommonTestHarness {
dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records2, header);
writer.appendBlock(dataBlock);
- copyOfRecords1.addAll(copyOfRecords2);
- List<String> originalKeys =
- copyOfRecords1.stream().map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString())
- .collect(Collectors.toList());
-
// Delete 50 keys
List<DeleteRecord> 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<String> 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 class TestHoodieLogFormat extends HoodieCommonTestHarness {
.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 class TestHoodieLogFormat extends HoodieCommonTestHarness {
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 class TestHoodieLogFormat extends HoodieCommonTestHarness {
.withBasePath(basePath)
.withLogFilePaths(allLogFiles)
.withReaderSchema(schema)
- .withLatestInstantTime("101")
+ .withLatestInstantTime("103")
.withMaxMemorySizeInBytes(10240L)
.withReadBlocksLazily(readBlocksLazily)
.withReverseReader(false)
@@ -1209,7 +1223,25 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.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<Boolean> 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<String> 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 class TestHoodieLogFormat extends HoodieCommonTestHarness {
@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 class TestHoodieLogFormat extends HoodieCommonTestHarness {
// 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 class TestHoodieLogFormat extends HoodieCommonTestHarness {
.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 class TestHoodieLogFormat extends HoodieCommonTestHarness {
@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 class TestHoodieLogFormat extends HoodieCommonTestHarness {
// 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 class TestHoodieLogFormat extends HoodieCommonTestHarness {
.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 class TestHoodieLogFormat extends HoodieCommonTestHarness {
@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 class TestHoodieLogFormat extends HoodieCommonTestHarness {
// 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 class TestHoodieLogFormat extends HoodieCommonTestHarness {
.withSpillableMapBasePath(BASE_OUTPUT_PATH)
.withDiskMapType(diskMapType)
.withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
+ .withUseScanV2(useScanv2)
.build();
assertEquals(100, scanner.getTotalLogRecords(), "We still would read 100 records");
final List<String> readKeys = new ArrayList<>(100);
@@ -1535,7 +1573,8 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
@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 class TestHoodieLogFormat extends HoodieCommonTestHarness {
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 class TestHoodieLogFormat extends HoodieCommonTestHarness {
.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 class TestHoodieLogFormat extends HoodieCommonTestHarness {
@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 class TestHoodieLogFormat extends HoodieCommonTestHarness {
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 class TestHoodieLogFormat extends HoodieCommonTestHarness {
.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<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
+ Set<String> recordKeysOfFirstTwoBatches = records1.stream()
+ .map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)
+ .get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()).collect(Collectors.toSet());
+ Map<HoodieLogBlock.HeaderMetadataType, String> 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<IndexedRecord> 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<IndexedRecord> records3 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
+ Set<String> 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<IndexedRecord> 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<IndexedRecord> 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<IndexedRecord> 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<IndexedRecord> 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<IndexedRecord> 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<IndexedRecord> 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<String> 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<String> readKeys = new ArrayList<>();
+ scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
+ List<String> 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<String> validBlockInstants = scanner.getValidBlockInstants();
+ List<String> 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 class TestHoodieLogFormat extends HoodieCommonTestHarness {
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 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.withSpillableMapBasePath(BASE_OUTPUT_PATH)
.withDiskMapType(diskMapType)
.withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
+ .withUseScanV2(useScanv2)
.build();
assertEquals(Math.max(numRecordsInLog1, numRecordsInLog2), scanner.getNumMergedRecordsInLog(),
@@ -1788,39 +2022,42 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
@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 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
}
private static Stream<Arguments> 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<Arguments> 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 6c4d69a05b..d825c294cc 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 class TestHoodieLogFormatAppendFailure {
.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 1c8d5ece24..beca794dd6 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 class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
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 class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
.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 ea32a36d96..c8b6372630 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.CompactionOperation;
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 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
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<HoodieLogFile> 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<CompactionOperation> 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<String> logFilesCurrentlyPresent = new HashSet<>();
+ Set<String> baseFilesCurrentlyPresent = new HashSet<>();
+ Set<String> pendingCompactionFgIdsCurrentlyPresent = new HashSet<>();
+ Set<String> 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 2f284c5bef..2367ba3035 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.Collectors;
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 class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
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<String, List<String>> 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<String, List<String>>() {
+ {
+ 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 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
* 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<Pair<String, FileSlice>> 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 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
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<Pair<String, FileSlice>> 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 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
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 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
* 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 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
Set<Pair<String, CompactionOperation>> ops1 = view1.getPendingCompactionOperations().collect(Collectors.toSet());
Set<Pair<String, CompactionOperation>> 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<Pair<String, HoodieWriteStat>> 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 9fa96216f4..29fdbefced 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 class TestPriorityBasedFileSystemView {
});
}
+ @Test
+ public void testGetPendingLogCompactionOperations() {
+ Stream<Pair<String, CompactionOperation>> actual;
+ Stream<Pair<String, CompactionOperation>> expected = Collections.singleton(
+ (Pair<String, CompactionOperation>) 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 fb5f123e80..292cdc76b5 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 class CompactionTestUtils {
}
}).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 dc64856d3c..d9b3d4ecad 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 @@ public class HoodieCommonTestHarness {
}
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 c052b63ab5..cd437db8b3 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 @@ public class RawTripTestPayload implements HoodieRecordPayload<RawTripTestPayloa
}
}
+ 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 3622ed491b..14c5c16941 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 @@ package org.apache.hudi.sink.compact;
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 @@ public class CompactFunction extends ProcessFunction<CompactionPlanEvent, Compac
Collector<CompactionCommitEvent> collector,
HoodieWriteConfig writeConfig) throws IOException {
HoodieFlinkMergeOnReadTableCompactor<?> compactor = new HoodieFlinkMergeOnReadTableCompactor<>();
+ HoodieTableMetaClient metaClient = writeClient.getHoodieTable().getMetaClient();
+ String maxInstantTime = compactor.getMaxInstantTime(metaClient);
List<WriteStatus> 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 8dadd2e2dc..5f47219a52 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 @@ public class CompactionCommitSink extends CleanFunction<CompactionCommitEvent> {
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 cfe53c6039..b4b7906960 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.apache.flink.streaming.api.operators.ProcessOperator;
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 @@ public class HoodieFlinkCompactor {
.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 87c8379d6a..b4d0490aa1 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 @@ public class TestCompactionUtil {
*/
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 95945f38b2..de062bc251 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 1f1dd1b927..5ba62113a8 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 class HoodieRealtimePath extends Path {
}
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 b917f004bc..b1bd3df50f 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 @@ class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader
.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 bf4cbff666..782fb2d7a3 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.Arrays;
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 class HoodieRealtimeRecordReaderUtils {
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 class HoodieRealtimeRecordReaderUtils {
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 a4471845c3..134a326635 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 class InputFormatTestUtil {
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 class InputFormatTestUtil {
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 2648740f54..6c951868bb 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.HoodieTableFileSystemView;
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 @@ public class DFSHoodieDatasetInputReader extends DFSDeltaInputReader {
.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<HoodieRecord<? extends HoodieRecordPayload>> 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 f59a2ceba8..14539c45b0 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 @@ public class RequestHandler {
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 @@ public class RequestHandler {
writeValueAsString(ctx, dtos);
}, true));
+ app.get(RemoteHoodieTableFileSystemView.PENDING_LOG_COMPACTION_OPS, new ViewHandler(ctx -> {
+ metricsRegistry.add("PEDING_LOG_COMPACTION_OPS", 1);
+ List<CompactionOpDTO> 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<FileGroupDTO> 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 8b906351e7..caf1e3c939 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 class FileSliceHandler extends Handler {
.collect(Collectors.toList());
}
+ public List<CompactionOpDTO> getPendingLogCompactionOperations(String basePath) {
+ return viewManager.getFileSystemView(basePath).getPendingLogCompactionOperations()
+ .map(instantOp -> CompactionOpDTO.fromCompactionOperation(instantOp.getKey(), instantOp.getValue()))
+ .collect(Collectors.toList());
+ }
+
public List<FileGroupDTO> getAllFileGroups(String basePath, String partitionPath) {
return viewManager.getFileSystemView(basePath).getAllFileGroups(partitionPath).map(FileGroupDTO::fromFileGroup)
.collect(Collectors.toList());