1
0

[HUDI-3485] Adding scheduler pool configs for async clustering (#5043)

This commit is contained in:
Sivabalan Narayanan
2022-03-29 18:27:45 -07:00
committed by GitHub
parent 5c1b482a1b
commit 4fed8dd319
10 changed files with 72 additions and 19 deletions

View File

@@ -388,6 +388,14 @@ public class HoodieDeltaStreamer implements Serializable {
@Parameter(names = {"--retry-last-pending-inline-clustering", "-rc"}, description = "Retry last pending inline clustering plan before writing to sink.")
public Boolean retryLastPendingInlineClusteringJob = false;
@Parameter(names = {"--cluster-scheduling-weight"}, description = "Scheduling weight for clustering as defined in "
+ "https://spark.apache.org/docs/latest/job-scheduling.html")
public Integer clusterSchedulingWeight = 1;
@Parameter(names = {"--cluster-scheduling-minshare"}, description = "Minshare for clustering as defined in "
+ "https://spark.apache.org/docs/latest/job-scheduling.html")
public Integer clusterSchedulingMinShare = 0;
public boolean isAsyncCompactionEnabled() {
return continuousMode && !forceDisableCompaction
&& HoodieTableType.MERGE_ON_READ.equals(HoodieTableType.valueOf(tableType));
@@ -431,8 +439,10 @@ public class HoodieDeltaStreamer implements Serializable {
&& Objects.equals(commitOnErrors, config.commitOnErrors)
&& Objects.equals(deltaSyncSchedulingWeight, config.deltaSyncSchedulingWeight)
&& Objects.equals(compactSchedulingWeight, config.compactSchedulingWeight)
&& Objects.equals(clusterSchedulingWeight, config.clusterSchedulingWeight)
&& Objects.equals(deltaSyncSchedulingMinShare, config.deltaSyncSchedulingMinShare)
&& Objects.equals(compactSchedulingMinShare, config.compactSchedulingMinShare)
&& Objects.equals(clusterSchedulingMinShare, config.clusterSchedulingMinShare)
&& Objects.equals(forceDisableCompaction, config.forceDisableCompaction)
&& Objects.equals(checkpoint, config.checkpoint)
&& Objects.equals(initialCheckpointProvider, config.initialCheckpointProvider)
@@ -447,8 +457,8 @@ public class HoodieDeltaStreamer implements Serializable {
transformerClassNames, sourceLimit, operation, filterDupes,
enableHiveSync, maxPendingCompactions, maxPendingClustering, continuousMode,
minSyncIntervalSeconds, sparkMaster, commitOnErrors,
deltaSyncSchedulingWeight, compactSchedulingWeight, deltaSyncSchedulingMinShare,
compactSchedulingMinShare, forceDisableCompaction, checkpoint,
deltaSyncSchedulingWeight, compactSchedulingWeight, clusterSchedulingWeight, deltaSyncSchedulingMinShare,
compactSchedulingMinShare, clusterSchedulingMinShare, forceDisableCompaction, checkpoint,
initialCheckpointProvider, help);
}
@@ -478,8 +488,10 @@ public class HoodieDeltaStreamer implements Serializable {
+ ", commitOnErrors=" + commitOnErrors
+ ", deltaSyncSchedulingWeight=" + deltaSyncSchedulingWeight
+ ", compactSchedulingWeight=" + compactSchedulingWeight
+ ", clusterSchedulingWeight=" + clusterSchedulingWeight
+ ", deltaSyncSchedulingMinShare=" + deltaSyncSchedulingMinShare
+ ", compactSchedulingMinShare=" + compactSchedulingMinShare
+ ", clusterSchedulingMinShare=" + clusterSchedulingMinShare
+ ", forceDisableCompaction=" + forceDisableCompaction
+ ", checkpoint='" + checkpoint + '\''
+ ", initialCheckpointProvider='" + initialCheckpointProvider + '\''
@@ -762,7 +774,7 @@ public class HoodieDeltaStreamer implements Serializable {
if (asyncClusteringService.isPresent()) {
asyncClusteringService.get().updateWriteClient(writeClient);
} else {
asyncClusteringService = Option.ofNullable(new SparkAsyncClusteringService(writeClient));
asyncClusteringService = Option.ofNullable(new SparkAsyncClusteringService(new HoodieSparkEngineContext(jssc), writeClient));
HoodieTableMetaClient meta = HoodieTableMetaClient.builder()
.setConf(new Configuration(jssc.hadoopConfiguration()))
.setBasePath(cfg.targetBasePath)

View File

@@ -224,6 +224,8 @@ public class HoodieMultiTableDeltaStreamer {
tableConfig.compactSchedulingWeight = globalConfig.compactSchedulingWeight;
tableConfig.deltaSyncSchedulingMinShare = globalConfig.deltaSyncSchedulingMinShare;
tableConfig.deltaSyncSchedulingWeight = globalConfig.deltaSyncSchedulingWeight;
tableConfig.clusterSchedulingWeight = globalConfig.clusterSchedulingWeight;
tableConfig.clusterSchedulingMinShare = globalConfig.clusterSchedulingMinShare;
tableConfig.sparkMaster = globalConfig.sparkMaster;
}
}
@@ -377,6 +379,14 @@ public class HoodieMultiTableDeltaStreamer {
@Parameter(names = {"--checkpoint"}, description = "Resume Delta Streamer from this checkpoint.")
public String checkpoint = null;
@Parameter(names = {"--cluster-scheduling-weight"}, description = "Scheduling weight for clustering as defined in "
+ "https://spark.apache.org/docs/latest/job-scheduling.html")
public Integer clusterSchedulingWeight = 1;
@Parameter(names = {"--cluster-scheduling-minshare"}, description = "Minshare for clustering as defined in "
+ "https://spark.apache.org/docs/latest/job-scheduling.html")
public Integer clusterSchedulingMinShare = 0;
@Parameter(names = {"--help", "-h"}, help = true)
public Boolean help = false;
}

View File

@@ -35,6 +35,8 @@ import java.util.HashMap;
import java.util.Map;
import java.util.UUID;
import static org.apache.hudi.async.AsyncClusteringService.CLUSTERING_POOL_NAME;
/**
* Utility Class to generate Spark Scheduling allocation file. This kicks in only when user sets
* spark.scheduler.mode=FAIR at spark-submit time
@@ -61,13 +63,16 @@ public class SchedulerConfGenerator {
* @param compactionWeight Scheduling weight for compaction
* @param deltaSyncMinShare Minshare for delta sync
* @param compactionMinShare Minshare for compaction
* @param clusteringMinShare Scheduling weight for clustering
* @param clusteringWeight Minshare for clustering
* @return Spark scheduling configs
*/
private static String generateConfig(Integer deltaSyncWeight, Integer compactionWeight, Integer deltaSyncMinShare,
Integer compactionMinShare) {
Integer compactionMinShare, Integer clusteringWeight, Integer clusteringMinShare) {
return String.format(SPARK_SCHEDULING_PATTERN, DELTASYNC_POOL_NAME, SPARK_SCHEDULER_FAIR_MODE,
deltaSyncWeight.toString(), deltaSyncMinShare.toString(), COMPACT_POOL_NAME, SPARK_SCHEDULER_FAIR_MODE,
compactionWeight.toString(), compactionMinShare.toString());
compactionWeight.toString(), compactionMinShare.toString(), CLUSTERING_POOL_NAME, SPARK_SCHEDULER_FAIR_MODE,
clusteringWeight.toString(), clusteringMinShare.toString());
}
/**
@@ -84,7 +89,9 @@ public class SchedulerConfGenerator {
if (sparkSchedulerMode.isPresent() && SPARK_SCHEDULER_FAIR_MODE.equals(sparkSchedulerMode.get())
&& cfg.continuousMode && cfg.tableType.equals(HoodieTableType.MERGE_ON_READ.name())) {
String sparkSchedulingConfFile = generateAndStoreConfig(cfg.deltaSyncSchedulingWeight,
cfg.compactSchedulingWeight, cfg.deltaSyncSchedulingMinShare, cfg.compactSchedulingMinShare);
cfg.compactSchedulingWeight, cfg.deltaSyncSchedulingMinShare, cfg.compactSchedulingMinShare,
cfg.clusterSchedulingWeight, cfg.clusterSchedulingMinShare);
LOG.warn("Spark scheduling config file " + sparkSchedulingConfFile);
additionalSparkConfigs.put(SparkConfigs.SPARK_SCHEDULER_ALLOCATION_FILE_KEY(), sparkSchedulingConfFile);
} else {
LOG.warn("Job Scheduling Configs will not be in effect as spark.scheduler.mode "
@@ -100,14 +107,16 @@ public class SchedulerConfGenerator {
* @param compactionWeight Scheduling weight for compaction
* @param deltaSyncMinShare Minshare for delta sync
* @param compactionMinShare Minshare for compaction
* @param clusteringMinShare Scheduling weight for clustering
* @param clusteringWeight Minshare for clustering
* @return Return the absolute path of the tmp file which stores the spark schedule configs
* @throws IOException Throws an IOException when write configs to file failed
*/
private static String generateAndStoreConfig(Integer deltaSyncWeight, Integer compactionWeight,
Integer deltaSyncMinShare, Integer compactionMinShare) throws IOException {
Integer deltaSyncMinShare, Integer compactionMinShare, Integer clusteringWeight, Integer clusteringMinShare) throws IOException {
File tempConfigFile = File.createTempFile(UUID.randomUUID().toString(), ".xml");
BufferedWriter bw = new BufferedWriter(new FileWriter(tempConfigFile));
bw.write(generateConfig(deltaSyncWeight, compactionWeight, deltaSyncMinShare, compactionMinShare));
bw.write(generateConfig(deltaSyncWeight, compactionWeight, deltaSyncMinShare, compactionMinShare, clusteringWeight, clusteringMinShare));
bw.close();
LOG.info("Configs written to file" + tempConfigFile.getAbsolutePath());
return tempConfigFile.getAbsolutePath();