[
https://issues.apache.org/jira/browse/FLINK-9366?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16500018#comment-16500018
]
ASF GitHub Bot commented on FLINK-9366:
---------------------------------------
Github user zentol commented on a diff in the pull request:
https://github.com/apache/flink/pull/6107#discussion_r192692204
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/filecache/FileCache.java
---
@@ -302,6 +308,30 @@ public Path call() throws IOException {
}
}
+ /**
+ * Asynchronous file copy process.
+ */
+ private static class CopyFromDFSProcess implements Callable<Path> {
+
+ private final Path filePath;
+ private final Path cachedPath;
+ private boolean executable;
+
+ public CopyFromDFSProcess(DistributedCacheEntry e, Path
cachedPath) {
+ this.filePath = new Path(e.filePath);
+ this.executable = e.isExecutable;
+ this.cachedPath = cachedPath;
+ }
+
+ @Override
+ public Path call() throws IOException {
+ // let exceptions propagate. we can retrieve them later
from
+ // the future and report them upon access to the result
+ copy(filePath, cachedPath, this.executable);
--- End diff --
can you prefix this with `FileUtils`? Imo this makes the code more readable.
> Distribute Cache only works for client-accessible files
> -------------------------------------------------------
>
> Key: FLINK-9366
> URL: https://issues.apache.org/jira/browse/FLINK-9366
> Project: Flink
> Issue Type: Bug
> Components: Client, Local Runtime
> Affects Versions: 1.6.0
> Reporter: Chesnay Schepler
> Assignee: Dawid Wysakowicz
> Priority: Blocker
> Fix For: 1.6.0
>
>
> In FLINK-8620 the distributed cache was modified to the distribute files via
> the blob store, instead of downloading them from a distributed filesystem.
> Previously, taskmanagers would download requested files from the DFS. Now,
> they retrieve it form the blob store. This requires the client to
> preemptively upload all files used with distributed cache.
> As a result it is no longer possible to use the distributed cache for files
> that reside in a cluster-internal DFS, as the client cannot download it. This
> is a regression from the previous behavior and may break existing setups.
> [~aljoscha] [~dawidwys]
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)