背景
在1.9.1版本中分布式缓存并未拷贝HDFS下的文件到TM,运行时抛出如下异常。
升级到1.10.1版本,能正常使用。借此,学习下Flink 分布式缓存相关知识。
定义
官网对 distributed cache 的定义:
Flink offers a distributed cache, similar to Apache Hadoop, to make files locally accessible to parallel instances of user functions. This functionality can be used to share files that contain static external data such as dictionaries or machine-learned regression models.
The cache works as follows. A program registers a file or directory of a local or remote filesystem such as HDFS or S3 under a specific name in its ExecutionEnvironment as a cached file. When the program is executed, Flink automatically copies the file or directory to the local filesystem of all workers. A user function can look up the file or directory under the specified name and access it from the worker’s local filesystem.
意思是通过Flink程序注册一个本地或者Hdfs文件,程序在运行时,Flink会自动将该文件拷贝到每个tm中,每个函数可以通过注册的名称获取该文件。
使用
官网给出的使用案例:
final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
// register a file from HDFS
env.registerCachedFile("hdfs:///path/to/your/file", "hdfsFile")
// register a local executable file (script, executable, ...)
env.registerCachedFile("file:///path/to/exec/file", "localExecFile", true)
// define your program and execute
...
DataStream<String> input = ...
DataStream<Integer> result = input.map(new MyMapper());
...
env.execute();
---------------------------------------------------------------
// extend a RichFunction to have access to the RuntimeContext
public final class MyMapper extends RichMapFunction<String, Integer> {
@Override
public void open(Configuration config) {
// access cached file via RuntimeContext and DistributedCache
File myFile = getRuntimeContext().getDistributedCache().getFile("hdfsFile");
// read the file (or navigate the directory)
...
}
@Override
public Integer map(String value) throws Exception {
// use content of cached file
...
}
}
实现流程
参考flink1.10.1版本的源码,了解实现流程。
- 将分布式文件地址及注册名称写入StreamExecutionEnvironment的cacheFile中。
protected final List<Tuple2<String, DistributedCache.DistributedCacheEntry>> cacheFile = new ArrayList<>();
public void registerCachedFile(String filePath, String name, boolean executable) {
this.cacheFile.add(new Tuple2<>(name, new DistributedCache.DistributedCacheEntry(filePath, executable)));
}
- 在生成StreamGraph时将该cacheFile传递给StreamGraph的 userArtifacts。
StreamGraphGenerator-->StreamGraph
org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#getStreamGraphGenerator
private StreamGraphGenerator getStreamGraphGenerator() {
if (transformations.size() <= 0) {
throw new IllegalStateException("No operators defined in streaming topology. Cannot execute.");
}
return new StreamGraphGenerator(transformations, config, checkpointCfg)
.setStateBackend(defaultStateBackend)
.setChaining(isChainingEnabled)
.setUserArtifacts(cacheFile) // note:传递cacheFile
.setTimeCharacteristic(timeCharacteristic)
.setDefaultBufferTimeout(bufferTimeout);
}
org.apache.flink.streaming.api.graph.StreamGraphGenerator#generate
public StreamGraph generate() {
streamGraph = new StreamGraph(executionConfig, checkpointConfig, savepointRestoreSettings);
streamGraph.setStateBackend(stateBackend);
streamGraph.setChaining(chaining);
streamGraph.setScheduleMode(scheduleMode);
streamGraph.setUserArtifacts(userArtifacts); // note:传递userArtifacts
streamGraph.setTimeCharacteristic(timeCharacteristic);
streamGraph.setJobName(jobName);
streamGraph.setBlockingConnectionsBetweenChains(blockingConnectionsBetweenChains);
alreadyTransformed = new HashMap<>();
for (Transformation> transformation: transformations) {
transform(transformation);
}
final StreamGraph builtStreamGraph = streamGraph;
alreadyTransformed.clear();
alreadyTransformed = null;
streamGraph = null;
return builtStreamGraph;
}
3. 在生成JobGraph时将StreamGraph的userArtifacts 传递给JobGraph的userArtifacts。如果缓存文件为本地文件夹则会将该文件夹压缩为.zip格式存储在客户端的临时文件夹中,并使用新的存储路径。
org.apache.flink.streaming.api.graph.StreamingJobGraphGenerator#createJobGraph()
private JobGraph createJobGraph() {
...
JobGraphGenerator.addUserArtifactEntries(streamGraph.getUserArtifacts(), jobGraph);
...
return jobGraph;
}
public static void addUserArtifactEntries(Collection<Tuple2<String, DistributedCache.DistributedCacheEntry>> userArtifacts, JobGraph jobGraph) {
if (userArtifacts != null && !userArtifacts.isEmpty()) {
try {
java.nio.file.Path tmpDir = Files.createTempDirectory("flink-distributed-cache-" + jobGraph.getJobID());
for (Tuple2<String, DistributedCache.DistributedCacheEntry> originalEntry : userArtifacts) {
Path filePath = new Path(originalEntry.f1.filePath);
boolean isLocalDir = false;
try {
FileSystem sourceFs = filePath.getFileSystem();
isLocalDir = !sourceFs.isDistributedFS() && sourceFs.getFileStatus(filePath).isDir();
} catch (IOException ioe) {
LOG.warn("Could not determine whether {} denotes a local path.", filePath, ioe);
}
// zip local directories because we only support file uploads
DistributedCache.DistributedCacheEntry entry;
if (isLocalDir) {
// note: 压缩本地文件夹,返回zip文件路径
Path zip = FileUtils.compressDirectory(filePath, new Path(tmpDir.toString(), filePath.getName() + ".zip"));
entry = new DistributedCache.DistributedCacheEntry(zip.toString(), originalEntry.f1.isExecutable, true);
} else {
entry = new DistributedCache.DistributedCacheEntry(filePath.toString(), originalEntry.f1.isExecutable, false);
}
jobGraph.addUserArtifact(originalEntry.f0, entry);
}
} catch (IOException ioe) {
throw new FlinkRuntimeException("Could not compress distributed-cache artifacts.", ioe);
}
}
}
4. yarnPerjob 模式部署jobGraph时,如果是本地文件则上传本地zip,返回该文件所在的hdfs路径。如果缓存文件为hdfs已存在路径,则直接写入配置文件。
org.apache.flink.yarn.YarnClusterDescriptor#startAppMaster
// only for per job mode
if (jobGraph != null) {
for (Map.Entry<String, DistributedCache.DistributedCacheEntry> entry : jobGraph.getUserArtifacts().entrySet()) {
org.apache.flink.core.fs.Path path = new org.apache.flink.core.fs.Path(entry.getValue().filePath);
// only upload local files
// note: 上传本地文件,返回hdfs中的路径存储在jobGraph的userArtifacts
if (!path.getFileSystem().isDistributedFS()) {
Path localPath = new Path(path.getPath());
Tuple2<Path, Long> remoteFileInfo =
Utils.uploadLocalFileToRemote(fs, appId.toString(), localPath, homeDir, entry.getKey());
jobGraph.setUserArtifactRemotePath(entry.getKey(), remoteFileInfo.f0.toString());
}
}
// 将分布式缓存文件信息写入到Configuration中
jobGraph.writeUserArtifactEntriesToConfiguration();
}
DistributedCache#writeFileInfoToConfig
public static void writeFileInfoToConfig(String name, DistributedCacheEntry e, Configuration conf) {
int num = conf.getInteger(CACHE_FILE_NUM, 0) + 1;
conf.setInteger(CACHE_FILE_NUM, num);
conf.setString(CACHE_FILE_NAME + num, name);
// note: DISTRIBUTED_CACHE_FILE_PATH_0
conf.setString(CACHE_FILE_PATH + num, e.filePath);
conf.setBoolean(CACHE_FILE_EXE + num, e.isExecutable || new File(e.filePath).canExecute());
conf.setBoolean(CACHE_FILE_DIR + num, e.isZipped || new File(e.filePath).isDirectory());
if (e.blobKey != null) {
conf.setBytes(CACHE_FILE_BLOB_KEY + num, e.blobKey);
}
}
- Task执行时,会先读取缓存文件中,并传递给RuntimeEnvironment,这样便可以根据注册名称获取文件。
- 从config文件中读取缓存文件路径。
- 创建临时文件,将缓存文件从hdfs异步拷贝到当前TM,并将拷贝后的本地路径存储在内存中。临时文件夹格式flink-dist-cache-uuid/jobId/。
org.apache.flink.runtime.taskmanager.Task#doRun
private void doRun() {
.......
// all resource acquisitions and registrations from here on
// need to be undone in the end
Map<String, Future<Path>> distributedCacheEntries = new HashMap<>();
// next, kick off the background copying of files for the distributed cache
try {
for (Map.Entry<String, DistributedCache.DistributedCacheEntry> entry :
DistributedCache.readFileInfoFromConfig(jobConfiguration)) {
LOG.info("Obtaining local cache file for '{}'.", entry.getKey());
Future<Path> cp = fileCache.createTmpFile(entry.getKey(), entry.getValue(), jobId, executionId);
distributedCacheEntries.put(entry.getKey(), cp);
}
}
catch (Exception e) {
throw new Exception(
String.format("Exception while adding files to distributed cache of task %s (%s).", taskNameWithSubtask, executionId), e);
}
Environment env = new RuntimeEnvironment(
jobId,
vertexId,
executionId,
executionConfig,
taskInfo,
jobConfiguration,
taskConfiguration,
userCodeClassLoader,
memoryManager,
ioManager,
broadcastVariableManager,
taskStateManager,
aggregateManager,
accumulatorRegistry,
kvStateRegistry,
inputSplitProvider,
distributedCacheEntries, // note:
consumableNotifyingPartitionWriters,
inputGates,
taskEventDispatcher,
checkpointResponder,
taskManagerConfig,
metrics,
this);
}
public Future<Path> createTmpFile(String name, DistributedCacheEntry entry, JobID jobID, ExecutionAttemptID executionId) throws Exception {
synchronized (lock) {
Map<String, Future<Path>> jobEntries = entries.computeIfAbsent(jobID, k -> new HashMap<>());
// register reference holder
final Set<ExecutionAttemptID> refHolders = jobRefHolders.computeIfAbsent(jobID, id -> new HashSet<>());
refHolders.add(executionId);
Future<Path> fileEntry = jobEntries.get(name);
if (fileEntry != null) {
// file is already in the cache. return a future that
// immediately returns the file
return fileEntry;
} else {
// need to copy the file
// create the target path
File tempDirToUse = new File(storageDirectories[nextDirectory++], jobID.toString());
if (nextDirectory >= storageDirectories.length) {
nextDirectory = 0;
}
// kick off the copying
Callable<Path> cp;
if (entry.blobKey != null) {
cp = new CopyFromBlobProcess(entry, jobID, blobService, new Path(tempDirToUse.getAbsolutePath()));
} else {
## note: 从hdfs异步拷贝到TM内部文件夹
cp = new CopyFromDFSProcess(entry, new Path(tempDirToUse.getAbsolutePath()));
}
FutureTask<Path> copyTask = new FutureTask<>(cp);
executorService.submit(copyTask);
// store our entry
jobEntries.put(name, copyTask);
return copyTask;
}
}
}
- 算子在open函数中,读取缓存文件。
org.apache.flink.api.common.cache.DistributedCache#getFile
public File getFile(String name) {
// note: Map<String, Future<Path>> distributedCacheEntries
Future<Path> future = cacheCopyTasks.get(name);
try {
final Path path = future.get();
URI tmp = path.makeQualified(path.getFileSystem()).toUri();
return new File(tmp);
}
catch (ExecutionException e) {
throw new RuntimeException("An error occurred while copying the file.", e.getCause());
}
catch (Exception e) {
throw new RuntimeException("Error while getting the file registered under '" + name +
"' from the distributed cache", e);
}
}