flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-6008) collection of BlobServer improvements
Date Fri, 17 Mar 2017 15:59:41 GMT

    [ https://issues.apache.org/jira/browse/FLINK-6008?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15930206#comment-15930206
] 

ASF GitHub Bot commented on FLINK-6008:
---------------------------------------

Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3512#discussion_r106677990
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java ---
    @@ -400,6 +418,47 @@ public void delete(BlobKey key) throws IOException {
     	}
     
     	/**
    +	 * Deletes the file associated with the given job and key if it exists in the local
    +	 * storage of the blob server.
    +	 *
    +	 * @param jobId     JobID of the file in the blob store
    +	 * @param key       String key of the file in the blob store
    +	 */
    +	@Override
    +	public void delete(JobID jobId, String key) {
    +		checkArgument(jobId != null, "Job id must not be null.");
    +		checkArgument(key != null, "BLOB name must not be null.");
    +
    +		final File localFile = BlobUtils.getStorageLocation(storageDir, jobId, key);
    +
    +		if (localFile.exists()) {
    +			if (!localFile.delete()) {
    +				LOG.warn("Failed to delete locally BLOB " + key + " at " + localFile.getAbsolutePath());
    +			}
    +		}
    +
    +		blobStore.delete(jobId, key);
    +	}
    +
    +	/**
    +	 * Deletes all files associated with the given job id from the storage.
    +	 *
    +	 * @param jobId     JobID of the files in the blob store
    +	 */
    +	@Override
    +	public void deleteAll(final JobID jobId) {
    +		checkArgument(jobId != null, "Job id must not be null.");
    +
    +		try {
    +			BlobUtils.deleteJobDirectory(storageDir, jobId);
    +		} catch (IOException e) {
    --- End diff --
    
    If we want to make sure we cleanup in any case, we can actually catch `Exception` here.


> collection of BlobServer improvements
> -------------------------------------
>
>                 Key: FLINK-6008
>                 URL: https://issues.apache.org/jira/browse/FLINK-6008
>             Project: Flink
>          Issue Type: Improvement
>          Components: Network
>    Affects Versions: 1.3.0
>            Reporter: Nico Kruber
>            Assignee: Nico Kruber
>
> The following things should be improved around the BlobServer/BlobCache:
> * update config uptions with non-deprecated ones, e.g. {{high-availability.cluster-id}}
and {{high-availability.storageDir}}
> * promote {{BlobStore#deleteAll(JobID)}} to the {{BlobService}}
> * extend the {{BlobService}} to work with {{NAME_ADDRESSABLE}} blobs (prepares FLINK-4399]
> * remove {{NAME_ADDRESSABLE}} blobs after job/task termination
> * do not fail the {{BlobServer}} when a delete operation fails
> * code style, like using {{Preconditions.checkArgument}}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Mime
View raw message