-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-1399] support a independent clustering spark job to asynchronously clustering #2379
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
Merged
Merged
Changes from 4 commits
Commits
Show all changes
6 commits
Select commit
Hold shift + click to select a range
76dbf4a
[HUDI-1481] add structured streaming and delta streamer clustering …
lw309637554 80a946a
[HUDI-1399] support a independent clustering spark job to asynchronou…
lw309637554 9dfdb71
Merge remote-tracking branch 'upstream/master' into HUDI-1399
lw309637554 369dc27
[HUDI-1399] support a independent clustering spark job to asynchron…
lw309637554 152ea1c
[HUDI-1498] Read clustering plan from requested file for inflight ins…
1550ea5
[HUDI-1399] support a independent clustering spark job with schedul…
lw309637554 File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
159 changes: 159 additions & 0 deletions
159
hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,159 @@ | ||
| /* | ||
| * 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.utilities; | ||
|
|
||
| import com.beust.jcommander.JCommander; | ||
| import com.beust.jcommander.Parameter; | ||
| import org.apache.avro.Schema; | ||
| import org.apache.hadoop.fs.FileSystem; | ||
| import org.apache.hadoop.fs.Path; | ||
| import org.apache.hudi.client.SparkRDDWriteClient; | ||
| import org.apache.hudi.client.WriteStatus; | ||
| import org.apache.hudi.common.config.TypedProperties; | ||
| import org.apache.hudi.common.fs.FSUtils; | ||
| import org.apache.hudi.common.table.HoodieTableMetaClient; | ||
| import org.apache.hudi.common.table.TableSchemaResolver; | ||
| import org.apache.hudi.common.util.Option; | ||
| import org.apache.log4j.LogManager; | ||
| import org.apache.log4j.Logger; | ||
| import org.apache.spark.api.java.JavaRDD; | ||
| import org.apache.spark.api.java.JavaSparkContext; | ||
|
|
||
| import java.io.Serializable; | ||
| import java.util.ArrayList; | ||
| import java.util.List; | ||
|
|
||
| public class HoodieClusteringJob { | ||
lw309637554 marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
|
|
||
| private static final Logger LOG = LogManager.getLogger(HoodieClusteringJob.class); | ||
| private final Config cfg; | ||
| private transient FileSystem fs; | ||
| private TypedProperties props; | ||
| private final JavaSparkContext jsc; | ||
|
|
||
| public HoodieClusteringJob(JavaSparkContext jsc, Config cfg) { | ||
| this.cfg = cfg; | ||
| this.jsc = jsc; | ||
| this.props = cfg.propsFilePath == null | ||
| ? UtilHelpers.buildProperties(cfg.configs) | ||
| : readConfigFromFileSystem(jsc, cfg); | ||
| } | ||
|
|
||
| private TypedProperties readConfigFromFileSystem(JavaSparkContext jsc, Config cfg) { | ||
| final FileSystem fs = FSUtils.getFs(cfg.basePath, jsc.hadoopConfiguration()); | ||
|
|
||
| return UtilHelpers | ||
| .readConfig(fs, new Path(cfg.propsFilePath), cfg.configs) | ||
| .getConfig(); | ||
| } | ||
|
|
||
| public static class Config implements Serializable { | ||
| @Parameter(names = {"--base-path", "-sp"}, description = "Base path for the table", required = true) | ||
| public String basePath = null; | ||
| @Parameter(names = {"--table-name", "-tn"}, description = "Table name", required = true) | ||
| public String tableName = null; | ||
| @Parameter(names = {"--instant-time", "-it"}, description = "Clustering Instant time", required = true) | ||
lw309637554 marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| public String clusteringInstantTime = null; | ||
| @Parameter(names = {"--parallelism", "-pl"}, description = "Parallelism for hoodie insert", required = false) | ||
| public int parallelism = 1; | ||
| @Parameter(names = {"--spark-master", "-ms"}, description = "Spark master", required = false) | ||
| public String sparkMaster = null; | ||
| @Parameter(names = {"--spark-memory", "-sm"}, description = "spark memory to use", required = true) | ||
| public String sparkMemory = null; | ||
| @Parameter(names = {"--retry", "-rt"}, description = "number of retries", required = false) | ||
| public int retry = 0; | ||
|
|
||
| @Parameter(names = {"--schedule", "-sc"}, description = "Schedule clustering") | ||
| public Boolean runSchedule = false; | ||
| @Parameter(names = {"--help", "-h"}, help = true) | ||
| public Boolean help = false; | ||
|
|
||
| @Parameter(names = {"--props"}, description = "path to properties file on localfs or dfs, with configurations for " | ||
| + "hoodie client for clustering") | ||
| public String propsFilePath = null; | ||
|
|
||
| @Parameter(names = {"--hoodie-conf"}, description = "Any configuration that can be set in the properties file " | ||
| + "(using the CLI parameter \"--props\") can also be passed command line using this parameter. This can be repeated", | ||
| splitter = IdentitySplitter.class) | ||
| public List<String> configs = new ArrayList<>(); | ||
| } | ||
|
|
||
| public static void main(String[] args) { | ||
| final Config cfg = new Config(); | ||
| JCommander cmd = new JCommander(cfg, null, args); | ||
| if (cfg.help || args.length == 0) { | ||
| cmd.usage(); | ||
| System.exit(1); | ||
| } | ||
| final JavaSparkContext jsc = UtilHelpers.buildSparkContext("clustering-" + cfg.tableName, cfg.sparkMaster, cfg.sparkMemory); | ||
| HoodieClusteringJob clusteringJob = new HoodieClusteringJob(jsc, cfg); | ||
| int result = clusteringJob.cluster(cfg.retry); | ||
| String resultMsg = String.format("Clustering with basePath: %s, tableName: %s, runSchedule: %s, clusteringInstantTime: %s", | ||
| cfg.basePath, cfg.tableName, cfg.runSchedule, cfg.clusteringInstantTime); | ||
| if (result == -1) { | ||
| LOG.error(resultMsg + " failed"); | ||
| } else { | ||
| LOG.info(resultMsg + " success"); | ||
| } | ||
| jsc.stop(); | ||
| } | ||
|
|
||
| public int cluster(int retry) { | ||
lw309637554 marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| this.fs = FSUtils.getFs(cfg.basePath, jsc.hadoopConfiguration()); | ||
| int ret = -1; | ||
| try { | ||
| do { | ||
| if (cfg.runSchedule) { | ||
| LOG.info("Do schedule"); | ||
| ret = doSchedule(jsc); | ||
| } else { | ||
| LOG.info("Do cluster"); | ||
| ret = doCluster(jsc); | ||
| } | ||
| } while (ret != 0 && retry-- > 0); | ||
| } catch (Throwable t) { | ||
| LOG.error("Cluster failed", t); | ||
| } | ||
| return ret; | ||
| } | ||
|
|
||
| private String getSchemaFromLatestInstant() throws Exception { | ||
| HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.basePath, true); | ||
| TableSchemaResolver schemaUtil = new TableSchemaResolver(metaClient); | ||
| Schema schema = schemaUtil.getTableAvroSchema(false); | ||
lw309637554 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| return schema.toString(); | ||
| } | ||
|
|
||
| private int doCluster(JavaSparkContext jsc) throws Exception { | ||
| String schemaStr = getSchemaFromLatestInstant(); | ||
| SparkRDDWriteClient client = | ||
| UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props); | ||
| JavaRDD<WriteStatus> writeResponse = | ||
| (JavaRDD<WriteStatus>) client.cluster(cfg.clusteringInstantTime, true).getWriteStatuses(); | ||
| return UtilHelpers.handleErrors(jsc, cfg.clusteringInstantTime, writeResponse); | ||
| } | ||
|
|
||
| private int doSchedule(JavaSparkContext jsc) throws Exception { | ||
| String schemaStr = getSchemaFromLatestInstant(); | ||
| SparkRDDWriteClient client = | ||
| UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props); | ||
| return client.scheduleClusteringAtInstant(cfg.clusteringInstantTime, Option.empty()) ? 0 : -1; | ||
lw309637554 marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| } | ||
|
|
||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@lw309637554 Can you add this API to the HoodieTimeline ? You can name it
filterPendingExcludingCompactionAndReplacefor now.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@n3nash Hello, about this API in timeline or as a util have discussed with @satishkotha . Finally decided to put it here. Because here just need to exclude clustering of replace, not insert_overwrite of replace. Also check clustering need use metaclient, but timeline do not have metaclient object.