[
https://issues.apache.org/jira/browse/FLINK-8620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16380017#comment-16380017
]
ASF GitHub Bot commented on FLINK-8620:
---------------------------------------
Github user dawidwys commented on a diff in the pull request:
https://github.com/apache/flink/pull/5580#discussion_r171181348
--- Diff:
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
---
@@ -1829,6 +1830,28 @@ public void registerCachedFile(String filePath,
String name) {
* @param executable flag indicating whether the file should be
executable
*/
public void registerCachedFile(String filePath, String name, boolean
executable) {
- this.cacheFile.add(new Tuple2<>(name, new
DistributedCache.DistributedCacheEntry(filePath, executable)));
+ registerCachedFile(filePath, name, executable, false);
}
+
+ /**
+ * Registers a file at the distributed cache under the given name. The
file will be accessible
+ * from any user-defined function in the (distributed) runtime under a
local path. If upload is true files will
+ * be distributed via {@link BlobServer} otherwise Files should be
local files (as long as all relevant workers
+ * have access to it), or files in a distributed file system. The
runtime will copy the files temporarily to a
+ * local cache, if needed.
+ *
+ * <p>The {@link org.apache.flink.api.common.functions.RuntimeContext}
can be obtained inside UDFs via
+ * {@link
org.apache.flink.api.common.functions.RichFunction#getRuntimeContext()} and
provides access
+ * {@link org.apache.flink.api.common.cache.DistributedCache} via
+ * {@link
org.apache.flink.api.common.functions.RuntimeContext#getDistributedCache()}.
+ *
+ * @param filePath The path of the file
+ * @param name The name under which the file is registered.
+ * @param executable flag indicating whether the file should be
executable
+ * @param upload flag indicating if the file should be distributed via
BlobServer
+ */
+ public void registerCachedFile(String filePath, String name, boolean
executable, boolean upload) {
--- End diff --
What behaviour do you suggest sending all local files through Blob and
serving files from DFS as before?
> Enable shipping custom artifacts to BlobStore and accessing them through
> DistributedCache
> -----------------------------------------------------------------------------------------
>
> Key: FLINK-8620
> URL: https://issues.apache.org/jira/browse/FLINK-8620
> Project: Flink
> Issue Type: New Feature
> Reporter: Dawid Wysakowicz
> Assignee: Dawid Wysakowicz
> Priority: Major
>
> We should be able to distribute custom files to taskmanagers. To do that we
> can store those files in BlobStore and later on access them in TaskManagers
> through DistributedCache.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)