-
Notifications
You must be signed in to change notification settings - Fork 2.1k
[FLINK-37630] Support flink cdc pipeline Yarn session mode #3989
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
base: master
Are you sure you want to change the base?
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 |
---|---|---|
@@ -0,0 +1,77 @@ | ||
/* | ||
* 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.flink.cdc.composer.flink.deployment; | ||
|
||
import org.apache.flink.cdc.common.utils.Preconditions; | ||
import org.apache.flink.cdc.composer.PipelineDeploymentExecutor; | ||
import org.apache.flink.core.fs.FileStatus; | ||
import org.apache.flink.core.fs.Path; | ||
|
||
import java.io.FileNotFoundException; | ||
import java.io.IOException; | ||
import java.util.Arrays; | ||
import java.util.Optional; | ||
|
||
/** Abstract Deployment Executor. */ | ||
public abstract class AbstractDeploymentExecutor implements PipelineDeploymentExecutor { | ||
private static final String FLINK_CDC_HOME_ENV_VAR = "FLINK_CDC_HOME"; | ||
private static final String FLINK_CDC_DIST_JAR_PATTERN = | ||
"^flink-cdc-dist-(\\d+(\\.\\d+)*)(-SNAPSHOT)?\\.jar$"; | ||
private static final String CDC_MAIN_CLASS = "org.apache.flink.cdc.cli.CliExecutor"; | ||
|
||
/** Get the Flink CDC dist jar from FLINK_CDC_HOME. */ | ||
public String getFlinkCDCDistJarFromEnv() throws IOException { | ||
String flinkCDCHomeFromEnvVar = System.getenv(FLINK_CDC_HOME_ENV_VAR); | ||
Preconditions.checkNotNull( | ||
flinkCDCHomeFromEnvVar, | ||
"FLINK_CDC_HOME is not correctly set in environment variable, current FLINK_CDC_HOME is: " | ||
+ flinkCDCHomeFromEnvVar); | ||
Path flinkCDCLibPath = new Path(flinkCDCHomeFromEnvVar, "lib"); | ||
if (!flinkCDCLibPath.getFileSystem().exists(flinkCDCLibPath) | ||
|| !flinkCDCLibPath.getFileSystem().getFileStatus(flinkCDCLibPath).isDir()) { | ||
throw new RuntimeException( | ||
"Flink cdc home lib is not file or not directory: " | ||
+ flinkCDCLibPath.makeQualified(flinkCDCLibPath.getFileSystem())); | ||
} | ||
|
||
FileStatus[] fileStatuses = flinkCDCLibPath.getFileSystem().listStatus(flinkCDCLibPath); | ||
Optional<Path> distJars = | ||
Arrays.stream(fileStatuses) | ||
.filter(status -> !status.isDir()) | ||
.map(FileStatus::getPath) | ||
.filter(path -> path.getName().matches(FLINK_CDC_DIST_JAR_PATTERN)) | ||
.findFirst(); | ||
|
||
if (distJars.isPresent()) { | ||
Path path = distJars.get().makeQualified(distJars.get().getFileSystem()); | ||
return path.toString(); | ||
} else { | ||
throw new FileNotFoundException( | ||
"Failed to fetch Flink CDC dist jar from path: " + flinkCDCLibPath); | ||
} | ||
} | ||
|
||
/** | ||
* Get the main class of Flink CDC. | ||
* | ||
* @return the main class of Flink CDC. | ||
*/ | ||
public String getCDCMainClass() { | ||
return CDC_MAIN_CLASS; | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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.flink.cdc.composer.flink.deployment; | ||
|
||
import org.apache.flink.cdc.composer.PipelineExecution; | ||
import org.apache.flink.client.deployment.ClusterSpecification; | ||
import org.apache.flink.client.deployment.application.ApplicationConfiguration; | ||
import org.apache.flink.client.program.ClusterClient; | ||
import org.apache.flink.client.program.ClusterClientProvider; | ||
import org.apache.flink.client.program.PackagedProgram; | ||
import org.apache.flink.client.program.PackagedProgramUtils; | ||
import org.apache.flink.configuration.Configuration; | ||
import org.apache.flink.configuration.PipelineOptions; | ||
import org.apache.flink.core.fs.FSDataInputStream; | ||
import org.apache.flink.core.fs.FileSystem; | ||
import org.apache.flink.core.fs.Path; | ||
import org.apache.flink.runtime.jobgraph.JobGraph; | ||
import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; | ||
import org.apache.flink.yarn.YarnClusterClientFactory; | ||
import org.apache.flink.yarn.YarnClusterDescriptor; | ||
import org.apache.flink.yarn.configuration.YarnConfigOptions; | ||
import org.apache.flink.yarn.configuration.YarnLogConfigUtil; | ||
|
||
import org.apache.flink.shaded.guava31.com.google.common.base.Joiner; | ||
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; | ||
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.yaml.YAMLFactory; | ||
|
||
import org.apache.commons.cli.CommandLine; | ||
import org.apache.hadoop.yarn.api.records.ApplicationId; | ||
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import java.io.File; | ||
import java.util.ArrayList; | ||
import java.util.Collections; | ||
import java.util.List; | ||
import java.util.stream.Collectors; | ||
|
||
/** Deploy flink cdc job by yarn session mode. */ | ||
public class YarnSessionDeploymentExecutor extends AbstractDeploymentExecutor { | ||
private static final Logger LOG = LoggerFactory.getLogger(YarnSessionDeploymentExecutor.class); | ||
|
||
@Override | ||
public PipelineExecution.ExecutionInfo deploy( | ||
CommandLine commandLine, | ||
Configuration flinkConfig, | ||
List<Path> additionalJars, | ||
Path flinkHome) | ||
throws Exception { | ||
LOG.info("Submitting flink job in 'Flink Yarn Session Mode'."); | ||
if (flinkConfig.get(PipelineOptions.JARS) == null) { | ||
flinkConfig.set( | ||
PipelineOptions.JARS, Collections.singletonList(getFlinkCDCDistJarFromEnv())); | ||
} | ||
flinkConfig.set( | ||
YarnConfigOptions.SHIP_FILES, | ||
additionalJars.stream().map(Path::toString).collect(Collectors.toList())); | ||
|
||
ObjectMapper mapper = new ObjectMapper(new YAMLFactory()); | ||
|
||
Path pipelinePath = new Path(commandLine.getArgList().get(0)); | ||
FileSystem fileSystem = FileSystem.get(pipelinePath.toUri()); | ||
FSDataInputStream pipelineInStream = fileSystem.open(pipelinePath); | ||
|
||
flinkConfig.set( | ||
ApplicationConfiguration.APPLICATION_ARGS, | ||
Collections.singletonList(mapper.readTree(pipelineInStream).toString())); | ||
YarnLogConfigUtil.setLogConfigFileInConfig( | ||
flinkConfig, Joiner.on(File.separator).join(flinkHome, "conf")); | ||
|
||
flinkConfig.set(ApplicationConfiguration.APPLICATION_MAIN_CLASS, getCDCMainClass()); | ||
final YarnClusterClientFactory yarnClusterClientFactory = new YarnClusterClientFactory(); | ||
final YarnClusterDescriptor descriptor = | ||
yarnClusterClientFactory.createClusterDescriptor(flinkConfig); | ||
ClusterSpecification specification = | ||
yarnClusterClientFactory.getClusterSpecification(flinkConfig); | ||
|
||
ClusterClient<ApplicationId> client = null; | ||
try { | ||
// If applicationId is passed, we get the state of yarn; if not, we create a session | ||
// cluster. | ||
String applicationId = flinkConfig.get(YarnConfigOptions.APPLICATION_ID); | ||
if (applicationId != null) { | ||
Comment on lines
+96
to
+99
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. It seems that we cannot determine whether the application_id does not exist or is a misspelled application_id. If it is a misspelled, a new session cluster will be generated each time it is provided 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. Maybe we can add a simple validate method. |
||
FinalApplicationStatus applicationStatus = | ||
descriptor | ||
.getYarnClient() | ||
.getApplicationReport(ApplicationId.fromString(applicationId)) | ||
.getFinalApplicationStatus(); | ||
if (FinalApplicationStatus.UNDEFINED.equals(applicationStatus)) { | ||
// applicationId is running. | ||
client = | ||
descriptor | ||
.retrieve(ApplicationId.fromString(applicationId)) | ||
.getClusterClient(); | ||
} | ||
} else { | ||
ClusterClientProvider<ApplicationId> clusterClientProvider = | ||
descriptor.deploySessionCluster(specification); | ||
client = clusterClientProvider.getClusterClient(); | ||
applicationId = String.valueOf(client.getClusterId()); | ||
} | ||
LOG.info("Deployment Flink CDC From application ID {}", applicationId); | ||
if (client == null) { | ||
throw new RuntimeException("Failed to get yarn session cluster client"); | ||
} | ||
client.submitJob(getJobGraph(flinkConfig, 1)); | ||
|
||
return new PipelineExecution.ExecutionInfo(applicationId, "submit job successful"); | ||
} catch (Exception e) { | ||
if (client != null) { | ||
client.shutDownCluster(); | ||
} | ||
throw new RuntimeException("Failed to yarn session deploy Flink CDC job", e); | ||
} finally { | ||
descriptor.close(); | ||
if (client != null) { | ||
client.close(); | ||
} | ||
} | ||
} | ||
|
||
/** Get jobGraph from configuration. */ | ||
private JobGraph getJobGraph(Configuration configuration, int parallelism) throws Exception { | ||
SavepointRestoreSettings savepointRestoreSettings = | ||
SavepointRestoreSettings.fromConfiguration(configuration); | ||
PackagedProgram.Builder builder = | ||
PackagedProgram.newBuilder() | ||
.setSavepointRestoreSettings(savepointRestoreSettings) | ||
.setEntryPointClassName( | ||
configuration | ||
.getOptional( | ||
ApplicationConfiguration.APPLICATION_MAIN_CLASS) | ||
.get()) | ||
.setArguments( | ||
configuration | ||
.getOptional(ApplicationConfiguration.APPLICATION_ARGS) | ||
.orElse(new ArrayList<>()) | ||
.toArray(new String[] {})) | ||
.setJarFile( | ||
new File( | ||
configuration | ||
.getOptional(PipelineOptions.JARS) | ||
.orElse(new ArrayList<>()) | ||
.get(0))); | ||
PackagedProgram program = builder.build(); | ||
return PackagedProgramUtils.createJobGraph( | ||
program, configuration, parallelism, null, false); | ||
} | ||
} |
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.
If "JARS" not null, do we need to add the flink cdc dist into JARS?
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.
As application mode it seem this.