-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-4766] Strengthen flink clustering job #6566
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,14 +18,24 @@ | |
|
|
||
| package org.apache.hudi.sink.clustering; | ||
|
|
||
| import org.apache.hadoop.fs.Path; | ||
| import org.apache.hudi.client.clustering.plan.strategy.FlinkSizeBasedClusteringPlanStrategy; | ||
| import org.apache.hudi.common.config.DFSPropertiesConfiguration; | ||
| import org.apache.hudi.common.config.TypedProperties; | ||
| import org.apache.hudi.common.table.HoodieTableConfig; | ||
| import org.apache.hudi.common.util.ValidationUtils; | ||
| import org.apache.hudi.configuration.FlinkOptions; | ||
| import org.apache.hudi.configuration.HadoopConfigurations; | ||
| import org.apache.hudi.util.StreamerUtil; | ||
|
|
||
| import com.beust.jcommander.Parameter; | ||
| import org.apache.flink.configuration.Configuration; | ||
|
|
||
| import java.util.ArrayList; | ||
| import java.util.HashMap; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
|
|
||
| /** | ||
| * Configurations for Hoodie Flink clustering. | ||
| */ | ||
|
|
@@ -69,13 +79,14 @@ public class FlinkClusteringConfig extends Configuration { | |
| required = false) | ||
| public Integer archiveMaxCommits = 30; | ||
|
|
||
| @Parameter(names = {"--schedule", "-sc"}, description = "Not recommended. Schedule the clustering plan in this job.\n" | ||
| + "There is a risk of losing data when scheduling clustering outside the writer job.\n" | ||
| + "Scheduling clustering in the writer job and only let this job do the clustering execution is recommended.\n" | ||
| + "Default is true", required = false) | ||
| public Boolean schedule = true; | ||
| @Parameter(names = {"--schedule", "-sc"}, description = "Schedule the clustering plan in this job.\n" | ||
| + "Default is false", required = false) | ||
| public Boolean schedule = false; | ||
|
|
||
| @Parameter(names = {"--instant-time", "-it"}, description = "Clustering Instant time") | ||
| public String clusteringInstantTime = null; | ||
|
|
||
| @Parameter(names = {"--clean-async-enabled"}, description = "Whether to cleanup the old commits immediately on new commits, enabled by default", required = false) | ||
| @Parameter(names = {"--clean-async-enabled"}, description = "Whether to cleanup the old commits immediately on new commits, disabled by default", required = false) | ||
| public Boolean cleanAsyncEnable = false; | ||
|
|
||
| @Parameter(names = {"--plan-strategy-class"}, description = "Config to provide a strategy class to generator clustering plan", required = false) | ||
|
|
@@ -85,10 +96,10 @@ public class FlinkClusteringConfig extends Configuration { | |
| public String planPartitionFilterMode = "NONE"; | ||
|
|
||
| @Parameter(names = {"--target-file-max-bytes"}, description = "Each group can produce 'N' (CLUSTERING_MAX_GROUP_SIZE/CLUSTERING_TARGET_FILE_SIZE) output file groups, default 1 GB", required = false) | ||
| public Integer targetFileMaxBytes = 1024 * 1024 * 1024; | ||
| public Long targetFileMaxBytes = 1024 * 1024 * 1024L; | ||
|
|
||
| @Parameter(names = {"--small-file-limit"}, description = "Files smaller than the size specified here are candidates for clustering, default 600 MB", required = false) | ||
| public Integer smallFileLimit = 600; | ||
| public Long smallFileLimit = 600L; | ||
|
|
||
| @Parameter(names = {"--skip-from-latest-partitions"}, description = "Number of partitions to skip from latest when choosing partitions to create ClusteringPlan, default 0", required = false) | ||
| public Integer skipFromLatestPartitions = 0; | ||
|
|
@@ -116,14 +127,42 @@ public class FlinkClusteringConfig extends Configuration { | |
| description = "Min clustering interval of async clustering service, default 10 minutes") | ||
| public Integer minClusteringIntervalSeconds = 600; | ||
|
|
||
| @Parameter(names = {"--hoodie-conf"}, description = "Any configuration that can be set in the properties file " | ||
| + "(using the CLI parameter \"--props\") can also be passed through command line using this parameter.") | ||
| public List<String> configs = new ArrayList<>(); | ||
voonhous marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
|
||
| @Parameter(names = {"--props"}, description = "Path to properties file on localfs or dfs, with configurations for " | ||
| + "hoodie client, schema provider, key generator and data source. For hoodie client props, sane defaults are " | ||
| + "used, but recommend use to provide basic things like metrics endpoints, hive configs etc. For sources, refer" | ||
| + "to individual classes, for supported properties.") | ||
| public String propsFilePath = ""; | ||
|
|
||
| public static TypedProperties buildProperties(List<String> props) { | ||
| TypedProperties properties = DFSPropertiesConfiguration.getGlobalProps(); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We can also move this clazz into
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Understood. I rewrote this method as the one in StreamerUtil is buggy and will ignore parameters passed in by |
||
| props.forEach(x -> { | ||
| String[] kv = x.split("="); | ||
| ValidationUtils.checkArgument(kv.length == 2); | ||
| properties.setProperty(kv[0], kv[1]); | ||
| }); | ||
| return properties; | ||
| } | ||
|
|
||
| public static TypedProperties getProps(FlinkClusteringConfig cfg) { | ||
| return cfg.propsFilePath.isEmpty() | ||
| ? buildProperties(cfg.configs) | ||
| : StreamerUtil.readConfig(HadoopConfigurations.getHadoopConf(cfg), | ||
| new Path(cfg.propsFilePath), cfg.configs).getProps(); | ||
| } | ||
|
|
||
| /** | ||
| * Transforms a {@code FlinkClusteringConfig.config} into {@code Configuration}. | ||
| * The latter is more suitable for the table APIs. It reads all the properties | ||
| * in the properties file (set by `--props` option) and cmd line options | ||
| * (set by `--hoodie-conf` option). | ||
| */ | ||
| public static Configuration toFlinkConfig(FlinkClusteringConfig config) { | ||
| Configuration conf = new Configuration(); | ||
| Map<String, String> propsMap = new HashMap<String, String>((Map) getProps(config)); | ||
| org.apache.flink.configuration.Configuration conf = fromMap(propsMap); | ||
|
|
||
| conf.setString(FlinkOptions.PATH, config.path); | ||
| conf.setInteger(FlinkOptions.ARCHIVE_MAX_COMMITS, config.archiveMaxCommits); | ||
|
|
@@ -134,8 +173,8 @@ public static Configuration toFlinkConfig(FlinkClusteringConfig config) { | |
| conf.setInteger(FlinkOptions.CLUSTERING_TASKS, config.clusteringTasks); | ||
| conf.setString(FlinkOptions.CLUSTERING_PLAN_STRATEGY_CLASS, config.planStrategyClass); | ||
| conf.setString(FlinkOptions.CLUSTERING_PLAN_PARTITION_FILTER_MODE_NAME, config.planPartitionFilterMode); | ||
| conf.setInteger(FlinkOptions.CLUSTERING_PLAN_STRATEGY_TARGET_FILE_MAX_BYTES, config.targetFileMaxBytes); | ||
| conf.setInteger(FlinkOptions.CLUSTERING_PLAN_STRATEGY_SMALL_FILE_LIMIT, config.smallFileLimit); | ||
| conf.setLong(FlinkOptions.CLUSTERING_PLAN_STRATEGY_TARGET_FILE_MAX_BYTES, config.targetFileMaxBytes); | ||
| conf.setLong(FlinkOptions.CLUSTERING_PLAN_STRATEGY_SMALL_FILE_LIMIT, config.smallFileLimit); | ||
| conf.setInteger(FlinkOptions.CLUSTERING_PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST, config.skipFromLatestPartitions); | ||
| conf.setString(FlinkOptions.CLUSTERING_SORT_COLUMNS, config.sortColumns); | ||
| conf.setInteger(FlinkOptions.CLUSTERING_MAX_NUM_GROUPS, config.maxNumGroups); | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.