[
https://issues.apache.org/jira/browse/FLINK-8620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16425174#comment-16425174
]
ASF GitHub Bot commented on FLINK-8620:
---------------------------------------
Github user zentol commented on a diff in the pull request:
https://github.com/apache/flink/pull/5580#discussion_r179056079
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/filecache/FileCache.java
---
@@ -262,106 +216,120 @@ private static Thread createShutdownHook(final
FileCache cache, final Logger log
);
}
+ public void releaseJob(JobID jobId, ExecutionAttemptID executionId) {
+ checkNotNull(jobId);
+
+ synchronized (lock) {
+ Set<ExecutionAttemptID> jobRefCounter =
jobRefHolders.get(jobId);
+
+ if (jobRefCounter == null || jobRefCounter.isEmpty()) {
+ LOG.warn("improper use of releaseJob() without
a matching number of createTmpFiles() calls for jobId " + jobId);
+ return;
+ }
+
+ jobRefCounter.remove(executionId);
+ if (jobRefCounter.isEmpty()) {
+ executorService.schedule(new
DeleteProcess(jobId), cleanupInterval, TimeUnit.SECONDS);
+ }
+ }
+ }
+
//
------------------------------------------------------------------------
// background processes
//
------------------------------------------------------------------------
/**
- * Asynchronous file copy process.
+ * Asynchronous file copy process from blob server.
*/
- private static class CopyProcess implements Callable<Path> {
+ private static class CopyFromBlobProcess implements Callable<Path> {
- private final Path filePath;
- private final Path cachedPath;
- private boolean executable;
+ private final PermanentBlobKey blobKey;
+ private final Path target;
+ private final boolean isDirectory;
+ private final boolean isExecutable;
+ private final JobID jobID;
+ private final PermanentBlobService blobService;
- public CopyProcess(DistributedCacheEntry e, Path cachedPath) {
- this.filePath = new Path(e.filePath);
- this.executable = e.isExecutable;
- this.cachedPath = cachedPath;
+ CopyFromBlobProcess(DistributedCacheEntry e, JobID jobID,
PermanentBlobService blobService, Path target) {
+ try {
+ this.isExecutable = e.isExecutable;
+ this.isDirectory = e.isZipped;
+ this.jobID = jobID;
+ this.blobService = blobService;
+ this.blobKey =
InstantiationUtil.deserializeObject(e.blobKey,
Thread.currentThread().getContextClassLoader());
+ this.target = target;
+ } catch (Exception ex) {
--- End diff --
the constructor signature should be modified instead
> 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)