BlobServer 源码解析
本文聚焦 BlobServer 这个类在 Flink Runtime 中的职责、生命周期与核心实现路径:它如何在 JobManager 进程内被创建并监听端口、如何用"一个监听线程 + 多个连接线程"处理 PUT/GET 请求、以及在 HA 模式下如何与 BlobStore 协作实现"永久 BLOB 可恢复"。
源码入口:
- <flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java>
一、主题与核心职责
BlobServer 是 JobManager 进程内的 BLOB 文件服务端,实现了"上传/下载作业相关文件(jar、依赖、文件分发产物等)+ 本地缓存 +(可选)HA 持久化 + 清理"的一揽子能力。
- 网络服务端 :绑定端口,监听客户端连接;每个连接由一个
BlobServerConnection线程处理连续请求。 - 本地存储管理:创建/维护本地目录结构(incoming + job/no_job 目录),支持并发读写的锁保护。
- PUT(上传) :接收字节流/文件块,落盘到临时文件,计算 digest,生成
BlobKey,再原子移动到最终存储位置。 - GET(下载) :优先从本地缓存读取;若是永久 BLOB 且本地不存在,会从
BlobStore拉回并缓存到本地后再返回。 - Transient TTL 清理:维护 transient BLOB 的过期时间表,后台定时任务扫描并删除过期文件。
- 作业级清理能力 :实现
LocallyCleanableResource/GloballyCleanableResource,支持"只清本地"与"本地 + BlobStore 全量清理"。
类签名(职责一眼看全):
java
// flink-runtime/.../blob/BlobServer.java
public class BlobServer extends Thread
implements BlobService,
BlobWriter,
PermanentBlobService,
TransientBlobService,
LocallyCleanableResource,
GloballyCleanableResource {
二、它在 JobManager 启动链路中的位置
BlobServer 属于 JobManager 启动的"地基服务组件",在 ClusterEntrypoint.initializeServices 阶段创建并启动,后续 Dispatcher/ResourceManager/WebRestEndpoint 才会被拉起。
在本仓库的 JobManager 启动解析中,对应的是"初始化 BlobServer 并启动"这一步:
BlobUtils.createBlobServer(configuration, workingDirectory.getBlobStorageDirectory(), haServices.createBlobStore())blobServer.start()- 把实际绑定的端口写回
configuration(供其他组件使用)
BlobUtils.createBlobServer(...) 的核心逻辑就是"决定本地 storageDirectory,然后 new BlobServer(...)":
java
// flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java
public static BlobServer createBlobServer(
Configuration configuration,
Reference<File> fallbackStorageDirectory,
BlobStore blobStore)
throws IOException {
final Reference<File> storageDirectory =
createBlobStorageDirectory(configuration, fallbackStorageDirectory);
return new BlobServer(configuration, storageDirectory, blobStore);
}
三、关键依赖与数据结构
3.1 本地目录结构:incoming + job/no_job
BlobServer 的存储根目录来自 BlobUtils.createBlobStorageDirectory(...) 的选择结果:
BlobServerOptions.STORAGE_DIRECTORY配置了就用配置目录,并创建blobStore-<uuid>子目录(owned)- 没配则使用 fallback 目录(通常来自 JobManager working directory)
在根目录下,BlobUtils 约定了两类 job scope:
- job 无关:
$base/no_job - job 相关:
$base/job_<jobId>
上传阶段会先写入:
$base/incoming/temp-xxxxxxxx临时文件(用于落盘与校验)
3.2 并发控制:ReadWriteLock + 活跃连接上限
BlobServer 处理并发的核心是两套机制:
- 文件读写锁 :
ReadWriteLock readWriteLock- GET 本地读:读锁
- PUT/DELETE/cleanup:写锁
- 特殊点:永久 BLOB 的 GET 在"需要从 BlobStore 下载"时,会释放读锁 → 下载到临时文件 → 再拿写锁原子 move 到最终位置
- 连接并发上限 :
maxConnections+activeConnections- 监听线程 accept 到 socket 后,如果活跃连接数达到上限会
wait(),直到连接线程结束并notifyAll()
- 监听线程 accept 到 socket 后,如果活跃连接数达到上限会
四、网络模型:一个 listener 线程 + 多个 connection 线程
BlobServer 自己就是一个 daemon 线程,职责是"accept socket + 限流 + 交给连接线程":
- BlobServer.run: <flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java>
核心代码(listener 循环 + 并发连接限流):
java
// flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
@Override
public void run() {
try {
while (!this.shutdownRequested.get()) {
BlobServerConnection conn =
new BlobServerConnection(NetUtils.acceptWithoutTimeout(serverSocket), this);
try {
synchronized (activeConnections) {
while (activeConnections.size() >= maxConnections) {
activeConnections.wait(2000);
}
activeConnections.add(conn);
}
conn.start();
conn = null;
} finally {
if (conn != null) {
conn.close();
synchronized (activeConnections) {
activeConnections.remove(conn);
}
}
}
}
} catch (Throwable t) {
if (!this.shutdownRequested.get()) {
LOG.error("BLOB server stopped working. Shutting down", t);
close();
}
}
}
连接线程由 BlobServerConnection 表示,一个连接可以承载多个请求(循环读取 operation):
- BlobServerConnection.run: <flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java>
核心代码(同一 TCP 连接上循环处理 PUT/GET):
java
// flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java
@Override
public void run() {
try {
final InputStream inputStream = this.clientSocket.getInputStream();
final OutputStream outputStream = this.clientSocket.getOutputStream();
while (true) {
final int operation = inputStream.read();
if (operation < 0) {
return;
}
switch (operation) {
case PUT_OPERATION:
put(inputStream, outputStream, new byte[BUFFER_SIZE]);
break;
case GET_OPERATION:
get(inputStream, outputStream, new byte[BUFFER_SIZE]);
break;
default:
throw new IOException("Unknown operation " + operation);
}
}
} finally {
closeSilently(clientSocket, LOG);
blobServer.unregisterConnection(this);
}
}
协议常量定义在:
- <flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerProtocol.java>
协议核心只有两类操作:
- PUT:
PUT_OPERATION = 0 - GET:
GET_OPERATION = 1
五、GET:本地优先,永久 BLOB 支持从 BlobStore 回源
从"请求处理"视角,GET 的关键链路是:
- 连接线程解析 header:job scope(job/no_job)+
BlobKey - 加读锁,调用
blobServer.getFileInternal(jobId, blobKey) - 找到本地文件就直接返回;若是永久 BLOB 且本地缺失,则从
BlobStore.get(...)下载回源并缓存 - 把文件长度与内容写回客户端
- 若是 transient BLOB 且客户端返回 OK,会删除本地文件("读完即删"的语义)
对应代码:
- 连接侧 GET 处理(BlobServerConnection.get):<flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java>
- 服务侧本地/回源逻辑(BlobServer.getFileInternal):<flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java>
连接侧 GET:解析 job scope + key,然后通过 blobServer.getFileInternal(...) 找到可发送的本地文件:
java
// flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java
final int mode = inputStream.read();
if (mode == JOB_UNRELATED_CONTENT) {
jobId = null;
} else if (mode == JOB_RELATED_CONTENT) {
byte[] jidBytes = new byte[JobID.SIZE];
readFully(inputStream, jidBytes, 0, JobID.SIZE, "JobID");
jobId = JobID.fromByteArray(jidBytes);
} else {
throw new IOException("Unknown type of BLOB addressing: " + mode + '.');
}
blobKey = BlobKey.readFromInputStream(inputStream);
readLock.lock();
try {
blobFile = blobServer.getFileInternal(jobId, blobKey);
outputStream.write(RETURN_OKAY);
writeLength((int) blobFile.length(), outputStream);
// ... stream file bytes
} finally {
readLock.unlock();
}
getFileInternal 的分支要点:
- 本地命中 :
- 直接返回文件
- 若 key 是 transient,则刷新 TTL:
blobExpiryTimes.put(..., now + cleanupInterval)
- 本地未命中 + 永久 BLOB :
- 从
blobStore.get(jobId, blobKey, incomingFile)下载到临时文件 - 加写锁,把临时文件 move 到最终位置(原子化落地 + 避免并发重复写)
- 从
- 本地未命中 + transient :
- 不会去
BlobStore回源,直接抛FileNotFoundException
- 不会去
服务侧 getFileInternal(...):本地命中即返回;永久 BLOB 未命中则从 BlobStore 下载回源并缓存:
java
// flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
final File localFile = BlobUtils.getStorageLocation(storageDir.deref(), jobId, blobKey);
if (localFile.exists()) {
if (blobKey instanceof TransientBlobKey) {
blobExpiryTimes.put(
Tuple2.of(jobId, (TransientBlobKey) blobKey),
System.currentTimeMillis() + cleanupInterval);
}
return localFile;
} else if (blobKey instanceof PermanentBlobKey) {
readWriteLock.readLock().unlock();
File incomingFile = null;
try {
incomingFile = createTemporaryFilename();
blobStore.get(jobId, blobKey, incomingFile);
readWriteLock.writeLock().lock();
try {
BlobUtils.moveTempFileToStore(incomingFile, jobId, blobKey, localFile, LOG, null);
} finally {
readWriteLock.writeLock().unlock();
}
return localFile;
} finally {
if (incomingFile != null) {
incomingFile.delete();
}
readWriteLock.readLock().lock();
}
}
throw new FileNotFoundException("Local file " + localFile + " does not exist and failed to copy from blob store.");
六、PUT:落盘 + digest + 生成 BlobKey +(永久)写入 BlobStore
PUT 有两条入口:
- 写 byte[]:
putBuffer(...) - 写 InputStream:
putInputStream(...)
共同逻辑是:
- 写入
$base/incoming/temp-xxxx临时文件(同时计算 digest) - 调用
moveTempFileToStore(incomingFile, jobId, digest, blobType) - 生成
BlobKey,并把临时文件 move 到最终位置 - 若是永久 BLOB:move 时会把内容同步写入
BlobStore(用于 HA 恢复) - 若是 transient BLOB:只落本地,并登记 TTL
对应代码:
- BlobServer.putBuffer: <flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java>
- BlobServer.putInputStream: <flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java>
- BlobServer.moveTempFileToStore: <flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java>
连接侧 PUT:读取 job scope + blobType,写入临时文件并计算 digest,最后让服务端生成 key 并落地:
java
// flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java
final int mode = inputStream.read();
final JobID jobId;
if (mode == JOB_UNRELATED_CONTENT) {
jobId = null;
} else if (mode == JOB_RELATED_CONTENT) {
byte[] jidBytes = new byte[JobID.SIZE];
readFully(inputStream, jidBytes, 0, JobID.SIZE, "JobID");
jobId = JobID.fromByteArray(jidBytes);
} else {
throw new IOException("Unknown type of BLOB addressing.");
}
final int read = inputStream.read();
final BlobKey.BlobType blobType =
read == TRANSIENT_BLOB.ordinal() ? TRANSIENT_BLOB : PERMANENT_BLOB;
incomingFile = blobServer.createTemporaryFilename();
byte[] digest = readFileFully(inputStream, incomingFile, buf);
BlobKey blobKey = blobServer.moveTempFileToStore(incomingFile, jobId, digest, blobType);
outputStream.write(RETURN_OKAY);
blobKey.writeToOutputStream(outputStream);
moveTempFileToStore 的关键点:
- key 生成 :
BlobKey.createKey(blobType, digest)(digest + 唯一成分,避免冲突) - 写锁内检查并落地 :
if (!storageFile.exists()) { moveTempFileToStore(...) } - 永久 BLOB 的 HA 写入 :调用
BlobUtils.moveTempFileToStore(..., blobStore)(仅永久 BLOB 传 blobStore) - transient TTL :在写锁内登记
blobExpiryTimes
服务侧 moveTempFileToStore(...):在写锁内完成"检查冲突 + 原子落地 +(可选)写入 BlobStore":
java
// flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
while (true) {
BlobKey blobKey = BlobKey.createKey(blobType, digest);
File storageFile = BlobUtils.getStorageLocation(storageDir.deref(), jobId, blobKey);
readWriteLock.writeLock().lock();
try {
if (!storageFile.exists()) {
BlobUtils.moveTempFileToStore(
incomingFile,
jobId,
blobKey,
storageFile,
LOG,
blobKey instanceof PermanentBlobKey ? blobStore : null);
if (blobKey instanceof TransientBlobKey) {
blobExpiryTimes.put(
Tuple2.of(jobId, (TransientBlobKey) blobKey),
System.currentTimeMillis() + cleanupInterval);
}
return blobKey;
}
} finally {
readWriteLock.writeLock().unlock();
}
}
连接侧 PUT 的网络协议/落盘过程见:
- BlobServerConnection.put: <flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java>
七、Transient TTL:为什么要有 blobExpiryTimes + cleanupTimer
BlobServer 用一个 ConcurrentHashMap<Tuple2<JobID, TransientBlobKey>, Long> 维护 transient BLOB 的过期时间:
- 每次 GET 命中本地 transient 文件,会刷新 TTL
- 每次 PUT transient 成功,也会登记 TTL
后台 Timer 周期性执行 TransientBlobCleanupTask:
- 扫描
blobExpiryTimes,若now >= expiryTime则回调deleteInternal(jobId, key) - 该任务容忍文件已不存在(例如被客户端 GET 成功后删除)
对应代码:
- BlobServer 构造函数中启动定时任务: <flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java>
- TransientBlobCleanupTask: <flink-runtime/src/main/java/org/apache/flink/runtime/blob/TransientBlobCleanupTask.java>
构造函数里注册定时任务(方法签名 + 关键内部调用):
java
// flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
public BlobServer(Configuration config, Reference<File> storageDir, BlobStore blobStore)
throws IOException {
// ...
this.cleanupTimer = new Timer(true);
this.cleanupInterval = config.get(BlobServerOptions.CLEANUP_INTERVAL) * 1000;
this.cleanupTimer.schedule(
new TransientBlobCleanupTask(blobExpiryTimes, this::deleteInternal, LOG),
cleanupInterval,
cleanupInterval);
// ...
registerBlobExpiryTimes();
}
清理任务的核心逻辑(方法签名 + 关键内部调用):
java
// flink-runtime/src/main/java/org/apache/flink/runtime/blob/TransientBlobCleanupTask.java
@Override
public void run() {
final long currentTimeMillis = System.currentTimeMillis();
Set<Map.Entry<Tuple2<JobID, TransientBlobKey>, Long>> entries =
new HashSet<>(blobExpiryTimes.entrySet());
for (Map.Entry<Tuple2<JobID, TransientBlobKey>, Long> entry : entries) {
if (currentTimeMillis >= entry.getValue()) {
JobID jobId = entry.getKey().f0;
TransientBlobKey blobKey = entry.getKey().f1;
cleanupCallback.accept(jobId, blobKey);
}
}
}
此外,registerBlobExpiryTimes() 会在启动时把存量 transient 文件注册进 map(统一给一个 now+interval 的初始 TTL),避免重启后 transient 文件"永不过期":
- BlobServer.registerBlobExpiryTimes: <flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java>
启动时补登记存量 transient(方法签名 + 关键内部调用):
java
// flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
private void registerBlobExpiryTimes() throws IOException {
if (storageDir.deref().exists()) {
final Collection<BlobUtils.TransientBlob> transientBlobs =
BlobUtils.listTransientBlobsInDirectory(storageDir.deref().toPath());
final long expiryTime = System.currentTimeMillis() + cleanupInterval;
for (BlobUtils.TransientBlob transientBlob : transientBlobs) {
blobExpiryTimes.put(
Tuple2.of(transientBlob.getJobId(), transientBlob.getBlobKey()),
expiryTime);
}
}
}
八、清理语义:localCleanup vs globalCleanup
BlobServer 把"清理"做成了两个层次:
- localCleanupAsync(jobId) :只删本地
job_<jobId>目录,不动BlobStore(为恢复留后路)- BlobServer.localCleanupAsync: <flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java>
- globalCleanupAsync(jobId) :本地清理 +
blobStore.deleteAll(jobId)(彻底删除)- BlobServer.globalCleanupAsync: <flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java>
两者都通过 runAsyncWithWriteLock(...) 保证清理与并发 GET/PUT/DELETE 的互斥:
- BlobServer.runAsyncWithWriteLock: <flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java>
localCleanup:只删本地(方法签名 + 关键内部调用):
java
// flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
@Override
public CompletableFuture<Void> localCleanupAsync(JobID jobId, Executor cleanupExecutor) {
checkNotNull(jobId);
return runAsyncWithWriteLock(() -> internalLocalCleanup(jobId), cleanupExecutor);
}
@GuardedBy("readWriteLock")
private void internalLocalCleanup(JobID jobId) throws IOException {
final File jobDir =
new File(BlobUtils.getStorageLocationPath(storageDir.deref().getAbsolutePath(), jobId));
FileUtils.deleteDirectory(jobDir);
// ...
}
globalCleanup:本地 + BlobStore(方法签名 + 关键内部调用):
java
// flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
@Override
public CompletableFuture<Void> globalCleanupAsync(JobID jobId, Executor executor) {
checkNotNull(jobId);
return runAsyncWithWriteLock(
() -> {
IOException exception = null;
try {
internalLocalCleanup(jobId);
} catch (IOException e) {
exception = e;
}
if (!blobStore.deleteAll(jobId)) {
exception =
ExceptionUtils.firstOrSuppressed(
new IOException(
"Error while cleaning up the BlobStore for job " + jobId),
exception);
}
if (exception != null) {
throw new IOException(exception);
}
},
executor);
}
写锁保护的异步执行器(方法签名 + 关键内部调用):
java
// flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
private CompletableFuture<Void> runAsyncWithWriteLock(
ThrowingRunnable<IOException> runnable, Executor executor) {
return CompletableFuture.runAsync(
() -> {
readWriteLock.writeLock().lock();
try {
runnable.run();
} catch (IOException e) {
throw new CompletionException(e);
} finally {
readWriteLock.writeLock().unlock();
}
},
executor);
}
九、关闭流程:close() 做了什么
close() 的目标是"停止对外服务 + 终止连接 + 删除本地目录(如果 owned)+ 移除 shutdown hook":
- cancel cleanup timer
- CAS 设置
shutdownRequested - 关闭
serverSocket,打断监听线程并join() - 关闭所有
activeConnections - 删除
storageDir(仅当Reference.owned) - 移除 shutdown hook,避免资源泄漏
对应代码:
- BlobServer.close: <flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java>
关闭流程的核心逻辑(方法签名 + 关键内部调用):
java
// flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
@Override
public void close() throws IOException {
cleanupTimer.cancel();
if (shutdownRequested.compareAndSet(false, true)) {
Exception exception = null;
if (serverSocket != null) {
try {
this.serverSocket.close();
} catch (IOException ioe) {
exception = ioe;
}
}
interrupt();
try {
join();
} catch (InterruptedException ie) {
Thread.currentThread().interrupt();
}
synchronized (activeConnections) {
if (!activeConnections.isEmpty()) {
for (BlobServerConnection conn : activeConnections) {
conn.close();
}
activeConnections.clear();
}
}
try {
storageDir.owned().ifPresent(FunctionUtils.uncheckedConsumer(FileUtils::deleteDirectory));
} catch (Exception e) {
exception = ExceptionUtils.firstOrSuppressed(e, exception);
}
ShutdownHookUtil.removeShutdownHook(shutdownHook, getClass().getSimpleName(), LOG);
ExceptionUtils.tryRethrowIOException(exception);
}
}
十、回到主题:一句话总结 BlobServer 的定位
BlobServer是 JobManager 的文件分发服务端:对外提供 PUT/GET,内部实现"本地缓存 +(可选)HA 持久化 + transient TTL + 作业级清理"。- 它在启动链路里属于地基服务:被创建并启动后,核心组件(Dispatcher/RM/Web)才开始启动;并且 HA 模式下它与
BlobStore一起保障"永久 BLOB 可恢复"。