本文主要分析提交一個flink on yarn的任務的流程,以job模式爲例子
- CliFront作爲提交命令行的入口
- 一個命令行runJob的整體調用鏈路如下
- run -> runProgram -> executeProgram -> ClusterClient.run
其中yarn相關的流程就在runProgram方法中
// 獲取激活的customCommandLine
final CustomCommandLine<?> customCommandLine = getActiveCustomCommandLine(commandLine);
try {
runProgram(customCommandLine, commandLine, runOptions, program);
} finally {
program.deleteExtractedLibraries();
}
private <T> void runProgram(
CustomCommandLine<T> customCommandLine,
CommandLine commandLine,
RunOptions runOptions,
PackagedProgram program) throws ProgramInvocationException, FlinkException {
// 從customCommandLine獲取到ClusterDescriptor
final ClusterDescriptor<T> clusterDescriptor = customCommandLine.createClusterDescriptor(commandLine);
...
// 集羣部署
client = clusterDescriptor.deployJobCluster(
clusterSpecification,
jobGraph,
runOptions.getDetachedMode());
// 執行任務提交邏輯
executeProgram(program, client, userParallelism);
....
}
可以看到集羣部署的邏輯是通過customCommandLine.createClusterDescriptor
獲取到的ClusterDescriptor來實現的。所以我們首先看一下獲取customCommandLine的邏輯
public CustomCommandLine<?> getActiveCustomCommandLine(CommandLine commandLine) {
for (CustomCommandLine<?> cli : customCommandLines) {
if (cli.isActive(commandLine)) {
return cli;
}
}
throw new IllegalStateException("No command-line ran.");
}
就是遍歷所有的CustomCommandLine,然後選取被激活的頭一個,看一下customCommandLines有哪些,初始化customCommandLines的邏輯就是在CliFrontend.loadCustomCommandLines
方法
public static List<CustomCommandLine<?>> loadCustomCommandLines(Configuration configuration, String configurationDirectory) {
List<CustomCommandLine<?>> customCommandLines = new ArrayList<>(2);
// Command line interface of the YARN session, with a special initialization here
// to prefix all options with y/yarn.
// Tips: DefaultCLI must be added at last, because getActiveCustomCommandLine(..) will get the
// active CustomCommandLine in order and DefaultCLI isActive always return true.
final String flinkYarnSessionCLI = "org.apache.flink.yarn.cli.FlinkYarnSessionCli";
try {
customCommandLines.add(
loadCustomCommandLine(flinkYarnSessionCLI,
configuration,
configurationDirectory,
"y",
"yarn"));
} catch (NoClassDefFoundError | Exception e) {
LOG.warn("Could not load CLI class {}.", flinkYarnSessionCLI, e);
}
customCommandLines.add(new DefaultCLI(configuration));
return customCommandLines;
}
實際上就兩個customCommandLine,一個DefaultCLI,一個FlinkYarnSessionCli。我們直接看FlinkYarnSessionCli的isActive方法
@Override
public boolean isActive(CommandLine commandLine) {
String jobManagerOption = commandLine.getOptionValue(addressOption.getOpt(), null);
boolean yarnJobManager = ID.equals(jobManagerOption);
boolean yarnAppId = commandLine.hasOption(applicationId.getOpt());
return yarnJobManager || yarnAppId || (isYarnPropertiesFileMode(commandLine) && yarnApplicationIdFromYarnProperties != null);
}
可以看到其中一個條件是-m選項爲 private static final String ID = "yarn-cluster";
即可。
我們看一下FlinkYarnSessionCli的createClusterDescriptor方法,最後返回的是一個YarnClusterDescriptor對象。所以我們直接看它的deployJobCluster方法,該方法最後調用的是AbstractYarnClusterDescriptor.startAppMaster
public ApplicationReport startAppMaster(
Configuration configuration,
String applicationName,
String yarnClusterEntrypoint,
JobGraph jobGraph,
YarnClient yarnClient,
YarnClientApplication yarnApplication,
ClusterSpecification clusterSpecification) throws Exception {
...
// 構建集羣的啓動方法
final ContainerLaunchContext amContainer = setupApplicationMasterContainer(
yarnClusterEntrypoint,
hasLogback,
hasLog4j,
hasKrb5,
clusterSpecification.getMasterMemoryMB());
...
// set classpath from YARN configuration
Utils.setupYarnClassPath(yarnConfiguration, appMasterEnv);
amContainer.setEnvironment(appMasterEnv);
// Set up resource type requirements for ApplicationMaster
Resource capability = Records.newRecord(Resource.class);
capability.setMemory(clusterSpecification.getMasterMemoryMB());
capability.setVirtualCores(flinkConfiguration.getInteger(YarnConfigOptions.APP_MASTER_VCORES));
final String customApplicationName = customName != null ? customName : applicationName;
appContext.setApplicationName(customApplicationName);
appContext.setApplicationType(applicationType != null ? applicationType : "Apache Flink");
// 設置ApplicationMaster的啓動方法
appContext.setAMContainerSpec(amContainer);
appContext.setResource(capability);
if (yarnQueue != null) {
appContext.setQueue(yarnQueue);
}
setApplicationNodeLabel(appContext);
setApplicationTags(appContext);
// add a hook to clean up in case deployment fails
Thread deploymentFailureHook = new DeploymentFailureHook(yarnApplication, yarnFilesDir);
Runtime.getRuntime().addShutdownHook(deploymentFailureHook);
LOG.info("Submitting application master " + appId);
// 提交Application
yarnClient.submitApplication(appContext);
...
}
startAppMaster也是一堆邏輯,我們直接看主要的,主要邏輯就是構建ContainerLaunchContext
,然後將ContainerLaunchContext作爲提交參數之一提交一個yarn app。看一下構造ContainerLaunchContext的方法邏輯
protected ContainerLaunchContext setupApplicationMasterContainer(
String yarnClusterEntrypoint,
boolean hasLogback,
boolean hasLog4j,
boolean hasKrb5,
int jobManagerMemoryMb) {
// ------------------ Prepare Application Master Container ------------------------------
// respect custom JVM options in the YAML file
String javaOpts = flinkConfiguration.getString(CoreOptions.FLINK_JVM_OPTIONS);
if (flinkConfiguration.getString(CoreOptions.FLINK_JM_JVM_OPTIONS).length() > 0) {
javaOpts += " " + flinkConfiguration.getString(CoreOptions.FLINK_JM_JVM_OPTIONS);
}
//applicable only for YarnMiniCluster secure test run
//krb5.conf file will be available as local resource in JM/TM container
if (hasKrb5) {
javaOpts += " -Djava.security.krb5.conf=krb5.conf";
}
// Set up the container launch context for the application master
ContainerLaunchContext amContainer = Records.newRecord(ContainerLaunchContext.class);
final Map<String, String> startCommandValues = new HashMap<>();
startCommandValues.put("java", "$JAVA_HOME/bin/java");
int heapSize = Utils.calculateHeapSize(jobManagerMemoryMb, flinkConfiguration);
String jvmHeapMem = String.format("-Xms%sm -Xmx%sm", heapSize, heapSize);
startCommandValues.put("jvmmem", jvmHeapMem);
startCommandValues.put("jvmopts", javaOpts);
String logging = "";
if (hasLogback || hasLog4j) {
logging = "-Dlog.file=\"" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.log\"";
if (hasLogback) {
logging += " -Dlogback.configurationFile=file:" + CONFIG_FILE_LOGBACK_NAME;
}
if (hasLog4j) {
logging += " -Dlog4j.configuration=file:" + CONFIG_FILE_LOG4J_NAME;
}
}
startCommandValues.put("logging", logging);
// 將yarnClusterEntrypoint作爲Container啓動命令的入口類
startCommandValues.put("class", yarnClusterEntrypoint);
startCommandValues.put("redirects",
"1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.out " +
"2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.err");
startCommandValues.put("args", "");
final String commandTemplate = flinkConfiguration
.getString(ConfigConstants.YARN_CONTAINER_START_COMMAND_TEMPLATE,
ConfigConstants.DEFAULT_YARN_CONTAINER_START_COMMAND_TEMPLATE);
// 根據參數構建啓動command
final String amCommand =
BootstrapTools.getStartCommand(commandTemplate, startCommandValues);
// 設置command
amContainer.setCommands(Collections.singletonList(amCommand));
LOG.debug("Application Master start command: " + amCommand);
return amContainer;
}
最後我們看一下yarnClusterEntrypoint的來源,其實就是YarnSessionClusterEntrypoint(session模式)和YarnJobClusterEntrypoint(job模式)的兩個類名。所以這個向yarn提交的AppMaster的邏輯實際上就是執行了YarnJobClusterEntrypoint/YarnSessionClusterEntrypoint的main方法。我們直接看YarnJobClusterEntrypoint的main方法。
public static void main(String[] args) {
// startup checks and logging
EnvironmentInformation.logEnvironmentInfo(LOG, YarnJobClusterEntrypoint.class.getSimpleName(), args);
SignalHandler.register(LOG);
JvmShutdownSafeguard.installAsShutdownHook(LOG);
Map<String, String> env = System.getenv();
final String workingDirectory = env.get(ApplicationConstants.Environment.PWD.key());
Preconditions.checkArgument(
workingDirectory != null,
"Working directory variable (%s) not set",
ApplicationConstants.Environment.PWD.key());
try {
YarnEntrypointUtils.logYarnEnvironmentInformation(env, LOG);
} catch (IOException e) {
LOG.warn("Could not log YARN environment information.", e);
}
Configuration configuration = YarnEntrypointUtils.loadConfiguration(workingDirectory, env, LOG);
YarnJobClusterEntrypoint yarnJobClusterEntrypoint = new YarnJobClusterEntrypoint(
configuration,
workingDirectory);
// 啓動邏輯
ClusterEntrypoint.runClusterEntrypoint(yarnJobClusterEntrypoint);
}
最後調用了ClusterEntrypoint.runCluster
方法
private void runCluster(Configuration configuration) throws Exception {
synchronized (lock) {
initializeServices(configuration);
// write host information into configuration
configuration.setString(JobManagerOptions.ADDRESS, commonRpcService.getAddress());
configuration.setInteger(JobManagerOptions.PORT, commonRpcService.getPort());
// 構建啓動集羣相關組件的factory
final DispatcherResourceManagerComponentFactory<?> dispatcherResourceManagerComponentFactory = createDispatcherResourceManagerComponentFactory(configuration);
// 調用factory的create方法
clusterComponent = dispatcherResourceManagerComponentFactory.create(
configuration,
commonRpcService,
haServices,
blobServer,
heartbeatServices,
metricRegistry,
archivedExecutionGraphStore,
new RpcMetricQueryServiceRetriever(metricRegistry.getMetricQueryServiceRpcService()),
this);
clusterComponent.getShutDownFuture().whenComplete(
(ApplicationStatus applicationStatus, Throwable throwable) -> {
if (throwable != null) {
shutDownAsync(
ApplicationStatus.UNKNOWN,
ExceptionUtils.stringifyException(throwable),
false);
} else {
// This is the general shutdown path. If a separate more specific shutdown was
// already triggered, this will do nothing
shutDownAsync(
applicationStatus,
null,
true);
}
});
}
}
DispatcherResourceManagerComponentFactory
有兩個具體子類, JobDispatcherResourceManagerComponentFactory
和SessionDispatcherResourceManagerComponentFactory
,分別代表了session和job模式。然後createDispatcherResourceManagerComponentFactory
方法在各個ClusterEntrypoint的子類均有實現,實際差別就是構建的ResourceManagerFactory
類有區別。即資源申請方式有區別。
接下來看一下各個組件的構建
public DispatcherResourceManagerComponent<T> create(
Configuration configuration,
RpcService rpcService,
HighAvailabilityServices highAvailabilityServices,
BlobServer blobServer,
HeartbeatServices heartbeatServices,
MetricRegistry metricRegistry,
ArchivedExecutionGraphStore archivedExecutionGraphStore,
MetricQueryServiceRetriever metricQueryServiceRetriever,
FatalErrorHandler fatalErrorHandler) throws Exception {
// 選主服務,爲了HA
LeaderRetrievalService dispatcherLeaderRetrievalService = null;
LeaderRetrievalService resourceManagerRetrievalService = null;
// webUI後端服務
WebMonitorEndpoint<U> webMonitorEndpoint = null;
// 資源管理服務
ResourceManager<?> resourceManager = null;
// metric服務
JobManagerMetricGroup jobManagerMetricGroup = null;
// 對外交互的dispatcher
T dispatcher = null;
try {
dispatcherLeaderRetrievalService = highAvailabilityServices.getDispatcherLeaderRetriever();
resourceManagerRetrievalService = highAvailabilityServices.getResourceManagerLeaderRetriever();
final LeaderGatewayRetriever<DispatcherGateway> dispatcherGatewayRetriever = new RpcGatewayRetriever<>(
rpcService,
DispatcherGateway.class,
DispatcherId::fromUuid,
10,
Time.milliseconds(50L));
final LeaderGatewayRetriever<ResourceManagerGateway> resourceManagerGatewayRetriever = new RpcGatewayRetriever<>(
rpcService,
ResourceManagerGateway.class,
ResourceManagerId::fromUuid,
10,
Time.milliseconds(50L));
final ExecutorService executor = WebMonitorEndpoint.createExecutorService(
configuration.getInteger(RestOptions.SERVER_NUM_THREADS),
configuration.getInteger(RestOptions.SERVER_THREAD_PRIORITY),
"DispatcherRestEndpoint");
final long updateInterval = configuration.getLong(MetricOptions.METRIC_FETCHER_UPDATE_INTERVAL);
final MetricFetcher metricFetcher = updateInterval == 0
? VoidMetricFetcher.INSTANCE
: MetricFetcherImpl.fromConfiguration(
configuration,
metricQueryServiceRetriever,
dispatcherGatewayRetriever,
executor);
webMonitorEndpoint = restEndpointFactory.createRestEndpoint(
configuration,
dispatcherGatewayRetriever,
resourceManagerGatewayRetriever,
blobServer,
executor,
metricFetcher,
highAvailabilityServices.getWebMonitorLeaderElectionService(),
fatalErrorHandler);
log.debug("Starting Dispatcher REST endpoint.");
webMonitorEndpoint.start();
final String hostname = getHostname(rpcService);
jobManagerMetricGroup = MetricUtils.instantiateJobManagerMetricGroup(
metricRegistry,
hostname,
ConfigurationUtils.getSystemResourceMetricsProbingInterval(configuration));
// 構建resourceManager
resourceManager = resourceManagerFactory.createResourceManager(
configuration,
ResourceID.generate(),
rpcService,
highAvailabilityServices,
heartbeatServices,
metricRegistry,
fatalErrorHandler,
new ClusterInformation(hostname, blobServer.getPort()),
webMonitorEndpoint.getRestBaseUrl(),
jobManagerMetricGroup);
final HistoryServerArchivist historyServerArchivist = HistoryServerArchivist.createHistoryServerArchivist(configuration, webMonitorEndpoint);
// 構建dispatcher
dispatcher = dispatcherFactory.createDispatcher(
configuration,
rpcService,
highAvailabilityServices,
resourceManagerGatewayRetriever,
blobServer,
heartbeatServices,
jobManagerMetricGroup,
metricRegistry.getMetricQueryServiceGatewayRpcAddress(),
archivedExecutionGraphStore,
fatalErrorHandler,
historyServerArchivist);
log.debug("Starting ResourceManager.");
resourceManager.start();
resourceManagerRetrievalService.start(resourceManagerGatewayRetriever);
log.debug("Starting Dispatcher.");
dispatcher.start();
dispatcherLeaderRetrievalService.start(dispatcherGatewayRetriever);
return createDispatcherResourceManagerComponent(
dispatcher,
resourceManager,
dispatcherLeaderRetrievalService,
resourceManagerRetrievalService,
webMonitorEndpoint,
jobManagerMetricGroup);
} catch (Exception exception) {
// clean up all started components
if (dispatcherLeaderRetrievalService != null) {
try {
dispatcherLeaderRetrievalService.stop();
} catch (Exception e) {
exception = ExceptionUtils.firstOrSuppressed(e, exception);
}
}
if (resourceManagerRetrievalService != null) {
try {
resourceManagerRetrievalService.stop();
} catch (Exception e) {
exception = ExceptionUtils.firstOrSuppressed(e, exception);
}
}
final Collection<CompletableFuture<Void>> terminationFutures = new ArrayList<>(3);
if (webMonitorEndpoint != null) {
terminationFutures.add(webMonitorEndpoint.closeAsync());
}
if (resourceManager != null) {
terminationFutures.add(resourceManager.closeAsync());
}
if (dispatcher != null) {
terminationFutures.add(dispatcher.closeAsync());
}
final FutureUtils.ConjunctFuture<Void> terminationFuture = FutureUtils.completeAll(terminationFutures);
try {
terminationFuture.get();
} catch (Exception e) {
exception = ExceptionUtils.firstOrSuppressed(e, exception);
}
if (jobManagerMetricGroup != null) {
jobManagerMetricGroup.close();
}
throw new FlinkException("Could not create the DispatcherResourceManagerComponent.", exception);
}
}
到此爲止集羣就已經啓動完畢了,接下來我們回到deployJobCluster方法來看如何構建ClusterClient了
protected ClusterClient<ApplicationId> deployInternal(
ClusterSpecification clusterSpecification,
String applicationName,
String yarnClusterEntrypoint,
@Nullable JobGraph jobGraph,
boolean detached) throws Exception {
// the Flink cluster is deployed in YARN. Represent cluster
return createYarnClusterClient(
this,
validClusterSpecification.getNumberTaskManagers(),
validClusterSpecification.getSlotsPerTaskManager(),
report,
flinkConfiguration,
true);
}
@Override
protected ClusterClient<ApplicationId> createYarnClusterClient(
AbstractYarnClusterDescriptor descriptor,
int numberTaskManagers,
int slotsPerTaskManager,
ApplicationReport report,
Configuration flinkConfiguration,
boolean perJobCluster) throws Exception {
return new RestClusterClient<>(
flinkConfiguration,
report.getApplicationId());
}
RestClusterClient(
Configuration configuration,
@Nullable RestClient restClient,
T clusterId,
WaitStrategy waitStrategy,
@Nullable LeaderRetrievalService webMonitorRetrievalService) throws Exception {
super(configuration);
this.restClusterClientConfiguration = RestClusterClientConfiguration.fromConfiguration(configuration);
if (restClient != null) {
this.restClient = restClient;
} else {
// 構建RestClient
this.restClient = new RestClient(restClusterClientConfiguration.getRestClientConfiguration(), executorService);
}
this.waitStrategy = Preconditions.checkNotNull(waitStrategy);
this.clusterId = Preconditions.checkNotNull(clusterId);
if (webMonitorRetrievalService == null) {
this.webMonitorRetrievalService = highAvailabilityServices.getWebMonitorLeaderRetriever();
} else {
this.webMonitorRetrievalService = webMonitorRetrievalService;
}
this.dispatcherRetrievalService = highAvailabilityServices.getDispatcherLeaderRetriever();
this.retryExecutorService = Executors.newSingleThreadScheduledExecutor(new ExecutorThreadFactory("Flink-RestClusterClient-Retry"));
startLeaderRetrievers();
}
RestClient實際上就是一個基於netty實現的http client。到此爲止集羣以及能對該集羣發起請求的client已經構造好了。接下來就是提交作業了
最後調用RestClusterClient.submitJob
方法
@Override
public JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException {
log.info("Submitting job {} (detached: {}).", jobGraph.getJobID(), isDetached());
final CompletableFuture<JobSubmissionResult> jobSubmissionFuture = submitJob(jobGraph);
if (isDetached()) {
try {
return jobSubmissionFuture.get();
} catch (Exception e) {
throw new ProgramInvocationException("Could not submit job",
jobGraph.getJobID(), ExceptionUtils.stripExecutionException(e));
}
} else {
final CompletableFuture<JobResult> jobResultFuture = jobSubmissionFuture.thenCompose(
ignored -> requestJobResult(jobGraph.getJobID()));
final JobResult jobResult;
try {
jobResult = jobResultFuture.get();
} catch (Exception e) {
throw new ProgramInvocationException("Could not retrieve the execution result.",
jobGraph.getJobID(), ExceptionUtils.stripExecutionException(e));
}
try {
this.lastJobExecutionResult = jobResult.toJobExecutionResult(classLoader);
return lastJobExecutionResult;
} catch (JobExecutionException e) {
throw new ProgramInvocationException("Job failed.", jobGraph.getJobID(), e);
} catch (IOException | ClassNotFoundException e) {
throw new ProgramInvocationException("Job failed.", jobGraph.getJobID(), e);
}
}
}
最後調用的url其實是到了JobSubmitHandler的處理邏輯裏,如何最後調用的就是DispatcherGateway.submitJob
方法了,接下來的邏輯就和上文類似了
@Override
protected CompletableFuture<JobSubmitResponseBody> handleRequest(@Nonnull HandlerRequest<JobSubmitRequestBody, EmptyMessageParameters> request, @Nonnull DispatcherGateway gateway) throws RestHandlerException {
final Collection<File> uploadedFiles = request.getUploadedFiles();
final Map<String, Path> nameToFile = uploadedFiles.stream().collect(Collectors.toMap(
File::getName,
Path::fromLocalFile
));
if (uploadedFiles.size() != nameToFile.size()) {
throw new RestHandlerException(
String.format("The number of uploaded files was %s than the expected count. Expected: %s Actual %s",
uploadedFiles.size() < nameToFile.size() ? "lower" : "higher",
nameToFile.size(),
uploadedFiles.size()),
HttpResponseStatus.BAD_REQUEST
);
}
final JobSubmitRequestBody requestBody = request.getRequestBody();
if (requestBody.jobGraphFileName == null) {
throw new RestHandlerException(
String.format("The %s field must not be omitted or be null.",
JobSubmitRequestBody.FIELD_NAME_JOB_GRAPH),
HttpResponseStatus.BAD_REQUEST);
}
// 從request body裏反序列化出JobGraph
CompletableFuture<JobGraph> jobGraphFuture = loadJobGraph(requestBody, nameToFile);
Collection<Path> jarFiles = getJarFilesToUpload(requestBody.jarFileNames, nameToFile);
Collection<Tuple2<String, Path>> artifacts = getArtifactFilesToUpload(requestBody.artifactFileNames, nameToFile);
// 將job的資源文件上傳blobServer
CompletableFuture<JobGraph> finalizedJobGraphFuture = uploadJobGraphFiles(gateway, jobGraphFuture, jarFiles, artifacts, configuration);
// 調用dispatcher的submitJob方法
CompletableFuture<Acknowledge> jobSubmissionFuture = finalizedJobGraphFuture.thenCompose(jobGraph -> gateway.submitJob(jobGraph, timeout));
return jobSubmissionFuture.thenCombine(jobGraphFuture,
(ack, jobGraph) -> new JobSubmitResponseBody("/jobs/" + jobGraph.getJobID()));
}
關於on k8s
從yarn的流程我們也可以比較容易的遷移到k8s的流程,
- 實現一份k8s的CustomCommandLine,用來構建返回k8s相關的集羣createClusterDescriptor
- 實現k8s的ClusterDescriptor,用於deploy集羣
- 實現基於k8s的ResourceManager
k8s上大致要做的應該就是實現JobManager/TaskManger相應的operator