[HUDI-1847] Adding inline scheduling support for spark datasource path for compaction and clustering (#4420)
- This adds support in spark-datasource to just schedule table services inline so that users can leverage async execution w/o the need for lock service providers.
This commit is contained in:
committed by
GitHub
parent
b3b44236fe
commit
0ababcfaa7
@@ -464,7 +464,7 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
|
||||
}
|
||||
|
||||
protected void runTableServicesInline(HoodieTable<T, I, K, O> table, HoodieCommitMetadata metadata, Option<Map<String, String>> extraMetadata) {
|
||||
if (config.areAnyTableServicesInline()) {
|
||||
if (config.areAnyTableServicesExecutedInline() || config.areAnyTableServicesScheduledInline()) {
|
||||
if (config.isMetadataTableEnabled()) {
|
||||
table.getHoodieView().sync();
|
||||
}
|
||||
@@ -472,19 +472,35 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
|
||||
if (config.inlineCompactionEnabled()) {
|
||||
runAnyPendingCompactions(table);
|
||||
metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT.key(), "true");
|
||||
inlineCompact(extraMetadata);
|
||||
inlineCompaction(extraMetadata);
|
||||
} else {
|
||||
metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT.key(), "false");
|
||||
}
|
||||
|
||||
// if just inline schedule is enabled
|
||||
if (!config.inlineCompactionEnabled() && config.scheduleInlineCompaction()
|
||||
&& !table.getActiveTimeline().getWriteTimeline().filterPendingCompactionTimeline().getInstants().findAny().isPresent()) {
|
||||
// proceed only if there are no pending compactions
|
||||
metadata.addMetadata(HoodieCompactionConfig.SCHEDULE_INLINE_COMPACT.key(), "true");
|
||||
inlineScheduleCompaction(extraMetadata);
|
||||
}
|
||||
|
||||
// Do an inline clustering if enabled
|
||||
if (config.inlineClusteringEnabled()) {
|
||||
runAnyPendingClustering(table);
|
||||
metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING.key(), "true");
|
||||
inlineCluster(extraMetadata);
|
||||
inlineClustering(extraMetadata);
|
||||
} else {
|
||||
metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING.key(), "false");
|
||||
}
|
||||
|
||||
// if just inline schedule is enabled
|
||||
if (!config.inlineClusteringEnabled() && config.scheduleInlineClustering()
|
||||
&& !table.getActiveTimeline().filterPendingReplaceTimeline().getInstants().findAny().isPresent()) {
|
||||
// proceed only if there are no pending clustering
|
||||
metadata.addMetadata(HoodieClusteringConfig.SCHEDULE_INLINE_CLUSTERING.key(), "true");
|
||||
inlineScheduleClustering(extraMetadata);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1005,9 +1021,10 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
|
||||
|
||||
/**
|
||||
* Performs a compaction operation on a table, serially before or after an insert/upsert action.
|
||||
* Scheduling and execution is done inline.
|
||||
*/
|
||||
protected Option<String> inlineCompact(Option<Map<String, String>> extraMetadata) {
|
||||
Option<String> compactionInstantTimeOpt = scheduleCompaction(extraMetadata);
|
||||
protected Option<String> inlineCompaction(Option<Map<String, String>> extraMetadata) {
|
||||
Option<String> compactionInstantTimeOpt = inlineScheduleCompaction(extraMetadata);
|
||||
compactionInstantTimeOpt.ifPresent(compactInstantTime -> {
|
||||
// inline compaction should auto commit as the user is never given control
|
||||
compact(compactInstantTime, true);
|
||||
@@ -1015,6 +1032,15 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
|
||||
return compactionInstantTimeOpt;
|
||||
}
|
||||
|
||||
/***
|
||||
* Schedules compaction inline.
|
||||
* @param extraMetadata extrametada to be used.
|
||||
* @return compaction instant if scheduled.
|
||||
*/
|
||||
protected Option<String> inlineScheduleCompaction(Option<Map<String, String>> extraMetadata) {
|
||||
return scheduleCompaction(extraMetadata);
|
||||
}
|
||||
|
||||
/**
|
||||
* Schedules a new clustering instant.
|
||||
* @param extraMetadata Extra Metadata to be stored
|
||||
@@ -1116,9 +1142,10 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
|
||||
|
||||
/**
|
||||
* Executes a clustering plan on a table, serially before or after an insert/upsert action.
|
||||
* Schedules and executes clustering inline.
|
||||
*/
|
||||
protected Option<String> inlineCluster(Option<Map<String, String>> extraMetadata) {
|
||||
Option<String> clusteringInstantOpt = scheduleClustering(extraMetadata);
|
||||
protected Option<String> inlineClustering(Option<Map<String, String>> extraMetadata) {
|
||||
Option<String> clusteringInstantOpt = inlineScheduleClustering(extraMetadata);
|
||||
clusteringInstantOpt.ifPresent(clusteringInstant -> {
|
||||
// inline cluster should auto commit as the user is never given control
|
||||
cluster(clusteringInstant, true);
|
||||
@@ -1126,6 +1153,15 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
|
||||
return clusteringInstantOpt;
|
||||
}
|
||||
|
||||
/**
|
||||
* Schedules clustering inline.
|
||||
* @param extraMetadata extrametadata to use.
|
||||
* @return clustering instant if scheduled.
|
||||
*/
|
||||
protected Option<String> inlineScheduleClustering(Option<Map<String, String>> extraMetadata) {
|
||||
return scheduleClustering(extraMetadata);
|
||||
}
|
||||
|
||||
protected void rollbackInflightClustering(HoodieInstant inflightInstant, HoodieTable<T, I, K, O> table) {
|
||||
String commitTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
table.scheduleRollback(context, commitTime, inflightInstant, false, config.shouldRollbackUsingMarkers());
|
||||
|
||||
@@ -25,6 +25,7 @@ import org.apache.hudi.common.config.HoodieConfig;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.engine.EngineType;
|
||||
import org.apache.hudi.common.util.TypeUtils;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||
import org.apache.hudi.table.action.cluster.ClusteringPlanPartitionFilterMode;
|
||||
@@ -177,6 +178,16 @@ public class HoodieClusteringConfig extends HoodieConfig {
|
||||
.withDocumentation("Determines how to handle updates, deletes to file groups that are under clustering."
|
||||
+ " Default strategy just rejects the update");
|
||||
|
||||
public static final ConfigProperty<String> SCHEDULE_INLINE_CLUSTERING = ConfigProperty
|
||||
.key("hoodie.clustering.schedule.inline")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("When set to true, clustering service will be attempted for inline scheduling after each write. Users have to ensure "
|
||||
+ "they have a separate job to run async clustering(execution) for the one scheduled by this writer. Users can choose to set both "
|
||||
+ "`hoodie.clustering.inline` and `hoodie.clustering.schedule.inline` to false and have both scheduling and execution triggered by any async process, on which "
|
||||
+ "case `hoodie.clustering.async.enabled` is expected to be set to true. But if `hoodie.clustering.inline` is set to false, and `hoodie.clustering.schedule.inline` "
|
||||
+ "is set to true, regular writers will schedule clustering inline, but users are expected to trigger async job for execution. If `hoodie.clustering.inline` is set "
|
||||
+ "to true, regular writers will do both scheduling and execution inline for clustering");
|
||||
|
||||
public static final ConfigProperty<String> ASYNC_CLUSTERING_ENABLE = ConfigProperty
|
||||
.key("hoodie.clustering.async.enabled")
|
||||
.defaultValue("false")
|
||||
@@ -505,6 +516,11 @@ public class HoodieClusteringConfig extends HoodieConfig {
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withScheduleInlineClustering(Boolean scheduleInlineClustering) {
|
||||
clusteringConfig.setValue(SCHEDULE_INLINE_CLUSTERING, String.valueOf(scheduleInlineClustering));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withInlineClusteringNumCommits(int numCommits) {
|
||||
clusteringConfig.setValue(INLINE_CLUSTERING_MAX_COMMITS, String.valueOf(numCommits));
|
||||
return this;
|
||||
@@ -562,6 +578,12 @@ public class HoodieClusteringConfig extends HoodieConfig {
|
||||
clusteringConfig.setDefaultValue(
|
||||
EXECUTION_STRATEGY_CLASS_NAME, getDefaultExecutionStrategyClassName(engineType));
|
||||
clusteringConfig.setDefaults(HoodieClusteringConfig.class.getName());
|
||||
|
||||
boolean inlineCluster = clusteringConfig.getBoolean(HoodieClusteringConfig.INLINE_CLUSTERING);
|
||||
boolean inlineClusterSchedule = clusteringConfig.getBoolean(HoodieClusteringConfig.SCHEDULE_INLINE_CLUSTERING);
|
||||
ValidationUtils.checkArgument(!(inlineCluster && inlineClusterSchedule), String.format("Either of inline clustering (%s) or "
|
||||
+ "schedule inline clustering (%s) can be enabled. Both can't be set to true at the same time. %s,%s", HoodieClusteringConfig.INLINE_CLUSTERING.key(),
|
||||
HoodieClusteringConfig.SCHEDULE_INLINE_CLUSTERING.key(), inlineCluster, inlineClusterSchedule));
|
||||
return clusteringConfig;
|
||||
}
|
||||
|
||||
|
||||
@@ -90,6 +90,16 @@ public class HoodieCompactionConfig extends HoodieConfig {
|
||||
.withDocumentation("When set to true, compaction service is triggered after each write. While being "
|
||||
+ " simpler operationally, this adds extra latency on the write path.");
|
||||
|
||||
public static final ConfigProperty<String> SCHEDULE_INLINE_COMPACT = ConfigProperty
|
||||
.key("hoodie.compact.schedule.inline")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("When set to true, compaction service will be attempted for inline scheduling after each write. Users have to ensure "
|
||||
+ "they have a separate job to run async compaction(execution) for the one scheduled by this writer. Users can choose to set both "
|
||||
+ "`hoodie.compact.inline` and `hoodie.compact.schedule.inline` to false and have both scheduling and execution triggered by any async process. "
|
||||
+ "But if `hoodie.compact.inline` is set to false, and `hoodie.compact.schedule.inline` is set to true, regular writers will schedule compaction inline, "
|
||||
+ "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_COMPACT_NUM_DELTA_COMMITS = ConfigProperty
|
||||
.key("hoodie.compact.inline.max.delta.commits")
|
||||
.defaultValue("5")
|
||||
@@ -537,6 +547,11 @@ public class HoodieCompactionConfig extends HoodieConfig {
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withScheduleInlineCompaction(Boolean scheduleAsyncCompaction) {
|
||||
compactionConfig.setValue(SCHEDULE_INLINE_COMPACT, String.valueOf(scheduleAsyncCompaction));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withInlineCompactionTriggerStrategy(CompactionTriggerStrategy compactionTriggerStrategy) {
|
||||
compactionConfig.setValue(INLINE_COMPACT_TRIGGER_STRATEGY, compactionTriggerStrategy.name());
|
||||
return this;
|
||||
@@ -700,6 +715,12 @@ public class HoodieCompactionConfig extends HoodieConfig {
|
||||
+ "missing data from few instants.",
|
||||
HoodieCompactionConfig.MIN_COMMITS_TO_KEEP.key(), minInstantsToKeep,
|
||||
HoodieCompactionConfig.CLEANER_COMMITS_RETAINED.key(), cleanerCommitsRetained));
|
||||
|
||||
boolean inlineCompact = compactionConfig.getBoolean(HoodieCompactionConfig.INLINE_COMPACT);
|
||||
boolean inlineCompactSchedule = compactionConfig.getBoolean(HoodieCompactionConfig.SCHEDULE_INLINE_COMPACT);
|
||||
ValidationUtils.checkArgument(!(inlineCompact && inlineCompactSchedule), String.format("Either of inline compaction (%s) or "
|
||||
+ "schedule inline compaction (%s) can be enabled. Both can't be set to true at the same time. %s, %s", HoodieCompactionConfig.INLINE_COMPACT.key(),
|
||||
HoodieCompactionConfig.SCHEDULE_INLINE_COMPACT.key(), inlineCompact, inlineCompactSchedule));
|
||||
return compactionConfig;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1140,6 +1140,10 @@ public class HoodieWriteConfig extends HoodieConfig {
|
||||
return getBoolean(HoodieCompactionConfig.INLINE_COMPACT);
|
||||
}
|
||||
|
||||
public boolean scheduleInlineCompaction() {
|
||||
return getBoolean(HoodieCompactionConfig.SCHEDULE_INLINE_COMPACT);
|
||||
}
|
||||
|
||||
public CompactionTriggerStrategy getInlineCompactTriggerStrategy() {
|
||||
return CompactionTriggerStrategy.valueOf(getString(HoodieCompactionConfig.INLINE_COMPACT_TRIGGER_STRATEGY));
|
||||
}
|
||||
@@ -1180,6 +1184,10 @@ public class HoodieWriteConfig extends HoodieConfig {
|
||||
return getBoolean(HoodieClusteringConfig.INLINE_CLUSTERING);
|
||||
}
|
||||
|
||||
public boolean scheduleInlineClustering() {
|
||||
return getBoolean(HoodieClusteringConfig.SCHEDULE_INLINE_CLUSTERING);
|
||||
}
|
||||
|
||||
public boolean isAsyncClusteringEnabled() {
|
||||
return getBoolean(HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE);
|
||||
}
|
||||
@@ -1859,11 +1867,11 @@ public class HoodieWriteConfig extends HoodieConfig {
|
||||
}
|
||||
|
||||
/**
|
||||
* Are any table services configured to run inline?
|
||||
* Are any table services configured to run inline for both scheduling and execution?
|
||||
*
|
||||
* @return True if any table services are configured to run inline, false otherwise.
|
||||
*/
|
||||
public Boolean areAnyTableServicesInline() {
|
||||
public Boolean areAnyTableServicesExecutedInline() {
|
||||
return inlineClusteringEnabled() || inlineCompactionEnabled() || isAutoClean();
|
||||
}
|
||||
|
||||
@@ -1876,6 +1884,10 @@ public class HoodieWriteConfig extends HoodieConfig {
|
||||
return isAsyncClusteringEnabled() || !inlineCompactionEnabled() || isAsyncClean();
|
||||
}
|
||||
|
||||
public Boolean areAnyTableServicesScheduledInline() {
|
||||
return scheduleInlineCompaction() || scheduleInlineClustering();
|
||||
}
|
||||
|
||||
public String getPreCommitValidators() {
|
||||
return getString(HoodiePreCommitValidatorConfig.VALIDATOR_CLASS_NAMES);
|
||||
}
|
||||
|
||||
@@ -167,7 +167,7 @@ public class TestHoodieWriteConfig {
|
||||
}
|
||||
});
|
||||
assertFalse(writeConfig.areAnyTableServicesAsync());
|
||||
assertTrue(writeConfig.areAnyTableServicesInline());
|
||||
assertTrue(writeConfig.areAnyTableServicesExecutedInline());
|
||||
assertEquals(HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME.defaultValue(), writeConfig.getLockProviderClass());
|
||||
|
||||
// 5. User override for the lock provider should always take the precedence
|
||||
|
||||
Reference in New Issue
Block a user