【Flink系列二十四】Flink HistoryServer 实现原理分析-源码解读
Flink系列二十四 Flink HistoryServer 实现原理
数据源头:Flink Runtime 对作业 ExecutionGraphInfo 进行归档
首先,作业停止或者故障时,调用 HistoryServerArchivist 进行归档
public interface HistoryServerArchivist {
/**
* Archives the given {@link ExecutionGraphInfo} on the history server.
*
* @param executionGraphInfo to store on the history server
* @return Future which is completed once the archiving has been completed.
*/
CompletableFuture<Acknowledge> archiveExecutionGraph(ExecutionGraphInfo executionGraphInfo);
}
调用的入口
protected CompletableFuture<CleanupJobState> jobReachedTerminalState(
ExecutionGraphInfo executionGraphInfo) {
}
...
private CompletableFuture<Acknowledge> archiveExecutionGraphToHistoryServer(
ExecutionGraphInfo executionGraphInfo) {
}
TaskManager 数据序列化
以 TaskManager 的数据组装为例
public class JobVertexTaskManagersHandler
extends AbstractAccessExecutionGraphHandler<
JobVertexTaskManagersInfo, JobVertexMessageParameters>
implements OnlyExecutionGraphJsonArchivist {
@Override
public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph)
throws IOException {
Collection<? extends AccessExecutionJobVertex> vertices = graph.getAllVertices().values();
List<ArchivedJson> archive = new ArrayList<>(vertices.size());
for (AccessExecutionJobVertex task : vertices) {
ResponseBody json = createJobVertexTaskManagersInfo(task, graph.getJobID(), null);
String path =
getMessageHeaders()
.getTargetRestEndpointURL()
.replace(':' + JobIDPathParameter.KEY, graph.getJobID().toString())
.replace(
':' + JobVertexIdPathParameter.KEY,
task.getJobVertexId().toString());
archive.add(new ArchivedJson(path, json));
}
return archive;
}
}
面向 Jackson 的数据结构
这里和实际的 JSON 数据对应上了,例如 taskmanager-id
而不是 taskmanagerid
org.apache.flink.runtime.rest.messages.JobVertexTaskManagersInfo.java
@JsonCreator
public TaskManagersInfo(
@JsonProperty(TASK_MANAGERS_FIELD_HOST) String host,
@JsonProperty(TASK_MANAGERS_FIELD_STATUS) ExecutionState status,
@JsonProperty(TASK_MANAGERS_FIELD_START_TIME) long startTime,
@JsonProperty(TASK_MANAGERS_FIELD_END_TIME) long endTime,
@JsonProperty(TASK_MANAGERS_FIELD_DURATION) long duration,
@JsonProperty(TASK_MANAGERS_FIELD_METRICS) IOMetricsInfo metrics,
@JsonProperty(TASK_MANAGERS_FIELD_STATUS_COUNTS)
Map<ExecutionState, Integer> statusCounts,
@JsonProperty(TASK_MANAGERS_FIELD_TASKMANAGER_ID) String taskmanagerId,
@JsonProperty(TASK_MANAGERS_FIELD_AGGREGATED)
AggregatedTaskDetailsInfo aggregated) {
this.host = checkNotNull(host);
this.status = checkNotNull(status);
this.startTime = startTime;
this.endTime = endTime;
this.duration = duration;
this.metrics = checkNotNull(metrics);
this.statusCounts = checkNotNull(statusCounts);
this.taskmanagerId = taskmanagerId;
this.aggregated = aggregated;
}
FsJobArchivist将数据序列化成JSON文本
org.apache.flink.runtime.history.FsJobArchivist#archiveJob
数据会写入: jobmanager.archive.fs.dir 对应的目录
public static Path archiveJob(
Path rootPath, JobID jobId, Collection<ArchivedJson> jsonToArchive) throws IOException {
try {
FileSystem fs = rootPath.getFileSystem();
Path path = new Path(rootPath, jobId.toString());
OutputStream out = fs.create(path, FileSystem.WriteMode.NO_OVERWRITE);
try (JsonGenerator gen = jacksonFactory.createGenerator(out, JsonEncoding.UTF8)) {
gen.writeStartObject();
gen.writeArrayFieldStart(ARCHIVE);
for (ArchivedJson archive : jsonToArchive) {
gen.writeStartObject();
gen.writeStringField(PATH, archive.getPath());
gen.writeStringField(JSON, archive.getJson());
gen.writeEndObject();
}
gen.writeEndArray();
gen.writeEndObject();
} catch (Exception e) {
fs.delete(path, false);
throw e;
}
LOG.info("Job {} has been archived at {}.", jobId, path);
return path;
} catch (IOException e) {
LOG.error("Failed to archive job.", e);
throw e;
}
}
查看归档的JSON数据
hdfs dfs -get hdfs://corp.slankka-hdfs.com/application/app-logs/flink/da12f990aba5bcdff710e96c5a409123
查看该数据,其实是一个JSON结构
{
"archive": [
{
"path": "/jobs/overview",
"json": "..." },
{
"path": "/jobs/da12f990aba5bcdff710e96c5a409123/config",
"json": "..."
},
{
"path": "/jobs/da12f990aba5bcdff710e96c5a409123/checkpoints",
"json": "..."
}
]
}
HistoryServer 启动后开始扫描归档的数据
public class HistoryServer {
void start() throws IOException, InterruptedException {
//...
executor.scheduleWithFixedDelay(
getArchiveFetchingRunnable(), 0, refreshIntervalMillis, TimeUnit.MILLISECONDS);
}
private Runnable getArchiveFetchingRunnable() {
return Runnables.withUncaughtExceptionHandler(
() -> archiveFetcher.fetchArchives(), FatalExitExceptionHandler.INSTANCE);
}
}
处理读取到的归档:下载到HistoryServer本地磁盘
可以发现,写入归档和处理归档的代码逻辑在同一个类文件内:FsJobArchivist
org.apache.flink.runtime.webmonitor.history.HistoryServerArchiveFetcher#processArchive
private void processArchive(String jobID, Path jobArchive) throws IOException {
for (ArchivedJson archive : FsJobArchivist.getArchivedJsons(jobArchive)) {
String path = archive.getPath();
String json = archive.getJson();
File target;
if (path.equals(JobsOverviewHeaders.URL)) {
target = new File(webOverviewDir, jobID + JSON_FILE_ENDING);
} else if (path.equals("/joboverview")) { // legacy path
LOG.debug("Migrating legacy archive {}", jobArchive);
json = convertLegacyJobOverview(json);
target = new File(webOverviewDir, jobID + JSON_FILE_ENDING);
} else {
// this implicitly writes into webJobDir
target = new File(webDir, path + JSON_FILE_ENDING);
}
...
}
}
上述逻辑会写入 historyserver.web.tmpdir 对应位置,如果未定义,则下载到 java.io.tmpdir
相对路径
HistoryServerArchiveFetcher(...) {
this.webDir = checkNotNull(webDir);
this.webJobDir = new File(webDir, "jobs");
Files.createDirectories(webJobDir.toPath());
this.webOverviewDir = new File(webDir, "overviews");
}
API 返回相应的文件
见 History Server Available Requests
本质上都是拉取以及解析对应的文件
router.addGet("/:*", new HistoryServerStaticFileServerHandler(webDir));
Router 相应的请求分为两类:
第一类
Job 归档的JSON文件
第二类:History Server 的 JobManager 以及 TaskManager 的日志链接
/jobs/<jobid>/jobmanager/log-url
/jobs/<jobid>/taskmanagers/<taskmanagerid>/log-url
有关Container日志的信息
- JobManager信息:暂无,可以通过
<jobId>
反查applicationId
直接调用Yarn REST API - TaskManager信息:可以通过下方链接文章的顺序获得,或者直接通过上述JSON获得 Vertex 中的TaskManager信息
-
A. 寻找 path=
/jobs/<jobid>/vertices/
找到所有的 vertexid -
B. 寻找 path=
/jobs/<jobid>/vertices/<vertexid>/taskmanagers
的值并建立映射关系
例如
{
"id": "cbc357ccb763df2852fee8c4fc7d55f2",
"name": "Source: Custom Source -> Process -> Map",
"now": 1731465012643,
"taskmanagers": [
{
"host": "kka136119:40079",
"status": "CANCELED",
"taskmanager-id": "container_e87_1724243239726_2160_01_000003"
},
{
"host": "kka129134:40853",
"status": "CANCELED",
"taskmanager-id": "container_e87_1724243239726_2160_01_000002"
}
...
]
}
通过API调用和这里JSON数据是一样的。
TaskManager hostName截断
因为HostNameSupplier 即便在开启了TaskManager反解析,即 jobmanager.retrieve-taskmanager-hostname
,也只会返回第一段。(默认开启,不建议关闭)
org.apache.flink.runtime.taskmanager.TaskManagerLocation.DefaultHostNameSupplier#getHostName
If the FQDN is the textual IP address, then the hostname is also the IP address
If the FQDN has only one segment (such as "localhost", or "host17"), then this is used as the hostname.
If the FQDN has multiple segments (such as "worker3.subgroup.company.net"), then the first segment (here "worker3") will be used as the hostname
如果对Yarn进行日志集成,则需要自行补全 HostName