[
https://issues.apache.org/jira/browse/FLINK-5815?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15885751#comment-15885751
]
ASF GitHub Bot commented on FLINK-5815:
---------------------------------------
Github user tillrohrmann commented on a diff in the pull request:
https://github.com/apache/flink/pull/3388#discussion_r103197159
--- Diff: flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java ---
@@ -136,21 +137,36 @@ public static Path setupLocalResource(
Path homedir) throws IOException {
// copy resource to HDFS
- String suffix = ".flink/" + appId + "/" +
localRsrcPath.getName();
-
- Path dst = new Path(homedir, suffix);
+ Path dst = getRemoteResourceRoot(appId, localRsrcPath, homedir);
LOG.info("Copying from " + localRsrcPath + " to " + dst);
fs.copyFromLocalFile(localRsrcPath, dst);
registerLocalResource(fs, dst, appMasterJar);
return dst;
}
+ public static Path getRemoteResourceRoot(
--- End diff --
Maybe it's better called `getRemoteResourcePath`
> Add resource files configuration for Yarn Mode
> ----------------------------------------------
>
> Key: FLINK-5815
> URL: https://issues.apache.org/jira/browse/FLINK-5815
> Project: Flink
> Issue Type: Improvement
> Components: Client, YARN
> Affects Versions: 1.3.0
> Reporter: Wenlong Lyu
> Assignee: Wenlong Lyu
>
> Currently in flink, when we want to setup a resource file to distributed
> cache, we need to make the file accessible remotely by a url, which is often
> difficult to maintain a service like that. What's more, when we want do add
> some extra jar files to job classpath, we need to copy the jar files to blob
> server when submitting the jobgraph. In yarn, especially in flip-6, the blob
> server is not running yet when we try to start a flink job.
> Yarn has a efficient distributed cache implementation for application running
> on it, what's more we can be easily share the files stored in hdfs in
> different application by distributed cache without extra IO operations.
> I suggest to introduce -yfiles, -ylibjars -yarchives options to FlinkYarnCLI
> to enable yarn user setup their job resource files by yarn distributed cache.
> The options is compatible with what is used in mapreduce, which make it easy
> to use for yarn user who generally has experience on using mapreduce.
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)