flink-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From trohrm...@apache.org
Subject [3/3] flink git commit: [FLINK-7056][blob] add API to allow job-related BLOBs to be stored
Date Mon, 14 Aug 2017 09:18:09 GMT
[FLINK-7056][blob] add API to allow job-related BLOBs to be stored

[FLINK-7056][blob] refactor the new API for job-related BLOBs

For a cleaner API, instead of having a nullable jobId parameter, use two methods:
one for job-related BLOBs, another for job-unrelated ones.

This closes #4237.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/0a19c456
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/0a19c456
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/0a19c456

Branch: refs/heads/master
Commit: 0a19c456ac7781d94eb0aaaf8f2ac73d0157bacb
Parents: b7c1dfa
Author: Nico Kruber <nico@data-artisans.com>
Authored: Wed Jun 21 18:04:43 2017 +0200
Committer: Till Rohrmann <trohrmann@apache.org>
Committed: Mon Aug 14 11:06:24 2017 +0200

----------------------------------------------------------------------
 .../apache/flink/runtime/blob/BlobCache.java    | 167 +++++++--
 .../apache/flink/runtime/blob/BlobClient.java   | 361 +++++++++++++------
 .../apache/flink/runtime/blob/BlobServer.java   | 132 +++++--
 .../runtime/blob/BlobServerConnection.java      | 228 +++++++-----
 .../flink/runtime/blob/BlobServerProtocol.java  |  12 +-
 .../apache/flink/runtime/blob/BlobService.java  |  29 +-
 .../apache/flink/runtime/blob/BlobStore.java    |   6 +-
 .../apache/flink/runtime/blob/BlobUtils.java    | 151 +++++---
 .../org/apache/flink/runtime/blob/BlobView.java |   5 +-
 .../flink/runtime/blob/FileSystemBlobStore.java |  14 +-
 .../flink/runtime/blob/VoidBlobStore.java       |   7 +-
 .../apache/flink/runtime/client/JobClient.java  |   1 +
 .../apache/flink/runtime/jobgraph/JobGraph.java |   1 +
 .../runtime/blob/BlobCacheRetriesTest.java      | 103 ++++--
 .../runtime/blob/BlobCacheSuccessTest.java      | 100 ++++-
 .../flink/runtime/blob/BlobClientTest.java      |  36 +-
 .../flink/runtime/blob/BlobRecoveryITCase.java  |  34 +-
 .../runtime/blob/BlobServerDeleteTest.java      | 143 ++++++--
 .../flink/runtime/blob/BlobServerGetTest.java   | 133 ++++++-
 .../flink/runtime/blob/BlobServerPutTest.java   | 212 +++++++++--
 .../flink/runtime/blob/BlobUtilsTest.java       |  22 +-
 .../BlobLibraryCacheManagerTest.java            |  67 ++--
 .../BlobLibraryCacheRecoveryITCase.java         |   9 +-
 .../flink/runtime/jobmanager/JobSubmitTest.java |   7 +-
 24 files changed, 1503 insertions(+), 477 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/0a19c456/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java
index 3e19537..29f7706 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java
@@ -18,12 +18,15 @@
 
 package org.apache.flink.runtime.blob;
 
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.BlobServerOptions;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.util.FileUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -38,7 +41,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
 /**
  * The BLOB cache implements a local cache for content-addressable BLOBs.
  *
- * <p>When requesting BLOBs through the {@link BlobCache#getFile(BlobKey)} method, the
+ * <p>When requesting BLOBs through the {@link BlobCache#getFile} methods, the
  * BLOB cache will first attempt to serve the file from its local cache. Only if
  * the local cache does not contain the desired BLOB, the BLOB cache will try to
  * download it from a distributed file system (if available) or the BLOB
@@ -91,7 +94,7 @@ public final class BlobCache implements BlobService {
 
 		// configure and create the storage directory
 		String storageDirectory = blobClientConfig.getString(BlobServerOptions.STORAGE_DIRECTORY);
-		this.storageDir = BlobUtils.initStorageDirectory(storageDirectory);
+		this.storageDir = BlobUtils.initLocalStorageDirectory(storageDirectory);
 		LOG.info("Created BLOB cache storage directory " + storageDir);
 
 		// configure the number of fetch retries
@@ -110,19 +113,66 @@ public final class BlobCache implements BlobService {
 	}
 
 	/**
-	 * Returns local copy of the file for the BLOB with the given key. The method will first attempt to serve
-	 * the BLOB from its local cache. If the BLOB is not in the cache, the method will try to download it
-	 * from this cache's BLOB server.
+	 * Returns local copy of the (job-unrelated) file for the BLOB with the given key.
+	 * <p>
+	 * The method will first attempt to serve the BLOB from its local cache. If the BLOB is not in
+	 * the cache, the method will try to download it from this cache's BLOB server.
+	 *
+	 * @param key
+	 * 		The key of the desired BLOB.
 	 *
-	 * @param requiredBlob The key of the desired BLOB.
 	 * @return file referring to the local storage location of the BLOB.
-	 * @throws IOException Thrown if an I/O error occurs while downloading the BLOBs from the BLOB server.
+	 *
+	 * @throws IOException
+	 * 		Thrown if an I/O error occurs while downloading the BLOBs from the BLOB server.
 	 */
 	@Override
-	public File getFile(final BlobKey requiredBlob) throws IOException {
+	public File getFile(BlobKey key) throws IOException {
+		return getFileInternal(null, key);
+	}
+
+	/**
+	 * Returns local copy of the file for the BLOB with the given key.
+	 * <p>
+	 * The method will first attempt to serve the BLOB from its local cache. If the BLOB is not in
+	 * the cache, the method will try to download it from this cache's BLOB server.
+	 *
+	 * @param jobId
+	 * 		ID of the job this blob belongs to
+	 * @param key
+	 * 		The key of the desired BLOB.
+	 *
+	 * @return file referring to the local storage location of the BLOB.
+	 *
+	 * @throws IOException
+	 * 		Thrown if an I/O error occurs while downloading the BLOBs from the BLOB server.
+	 */
+	@Override
+	public File getFile(@Nonnull JobID jobId, BlobKey key) throws IOException {
+		checkNotNull(jobId);
+		return getFileInternal(jobId, key);
+	}
+
+	/**
+	 * Returns local copy of the file for the BLOB with the given key.
+	 * <p>
+	 * The method will first attempt to serve the BLOB from its local cache. If the BLOB is not in
+	 * the cache, the method will try to download it from this cache's BLOB server.
+	 *
+	 * @param jobId
+	 * 		ID of the job this blob belongs to (or <tt>null</tt> if job-unrelated)
+	 * @param requiredBlob
+	 * 		The key of the desired BLOB.
+	 *
+	 * @return file referring to the local storage location of the BLOB.
+	 *
+	 * @throws IOException
+	 * 		Thrown if an I/O error occurs while downloading the BLOBs from the BLOB server.
+	 */
+	private File getFileInternal(@Nullable JobID jobId, BlobKey requiredBlob) throws IOException {
 		checkArgument(requiredBlob != null, "BLOB key cannot be null.");
 
-		final File localJarFile = BlobUtils.getStorageLocation(storageDir, requiredBlob);
+		final File localJarFile = BlobUtils.getStorageLocation(storageDir, jobId, requiredBlob);
 
 		if (localJarFile.exists()) {
 			return localJarFile;
@@ -130,7 +180,7 @@ public final class BlobCache implements BlobService {
 
 		// first try the distributed blob store (if available)
 		try {
-			blobView.get(requiredBlob, localJarFile);
+			blobView.get(jobId, requiredBlob, localJarFile);
 		} catch (Exception e) {
 			LOG.info("Failed to copy from blob store. Downloading from BLOB server instead.", e);
 		}
@@ -141,14 +191,14 @@ public final class BlobCache implements BlobService {
 
 		// fallback: download from the BlobServer
 		final byte[] buf = new byte[BlobServerProtocol.BUFFER_SIZE];
-		LOG.info("Downloading {} from {}", requiredBlob, serverAddress);
+		LOG.info("Downloading {}/{} from {}", jobId, requiredBlob, serverAddress);
 
 		// loop over retries
 		int attempt = 0;
 		while (true) {
 			try (
 				final BlobClient bc = new BlobClient(serverAddress, blobClientConfig);
-				final InputStream is = bc.get(requiredBlob);
+				final InputStream is = bc.getInternal(jobId, requiredBlob);
 				final OutputStream os = new FileOutputStream(localJarFile)
 			) {
 				while (true) {
@@ -163,7 +213,7 @@ public final class BlobCache implements BlobService {
 				return localJarFile;
 			}
 			catch (Throwable t) {
-				String message = "Failed to fetch BLOB " + requiredBlob + " from " + serverAddress +
+				String message = "Failed to fetch BLOB " + jobId + "/" + requiredBlob + " from " + serverAddress +
 					" and store it under " + localJarFile.getAbsolutePath();
 				if (attempt < numFetchRetries) {
 					if (LOG.isDebugEnabled()) {
@@ -179,41 +229,110 @@ public final class BlobCache implements BlobService {
 
 				// retry
 				++attempt;
-				LOG.info("Downloading {} from {} (retry {})", requiredBlob, serverAddress, attempt);
+				LOG.info("Downloading {}/{} from {} (retry {})", jobId, requiredBlob, serverAddress, attempt);
 			}
 		} // end loop over retries
 	}
 
 	/**
-	 * Deletes the file associated with the given key from the BLOB cache.
-	 * @param key referring to the file to be deleted
+	 * Deletes the (job-unrelated) file associated with the blob key in this BLOB cache.
+	 *
+	 * @param key
+	 * 		blob key associated with the file to be deleted
+	 *
+	 * @throws IOException
 	 */
 	@Override
-	public void delete(BlobKey key) throws IOException{
-		final File localFile = BlobUtils.getStorageLocation(storageDir, key);
+	public void delete(BlobKey key) throws IOException {
+		deleteInternal(null, key);
+	}
 
+	/**
+	 * Deletes the file associated with the blob key in this BLOB cache.
+	 *
+	 * @param jobId
+	 * 		ID of the job this blob belongs to
+	 * @param key
+	 * 		blob key associated with the file to be deleted
+	 *
+	 * @throws IOException
+	 */
+	@Override
+	public void delete(@Nonnull JobID jobId, BlobKey key) throws IOException {
+		checkNotNull(jobId);
+		deleteInternal(jobId, key);
+	}
+
+	/**
+	 * Deletes the file associated with the blob key in this BLOB cache.
+	 *
+	 * @param jobId
+	 * 		ID of the job this blob belongs to (or <tt>null</tt> if job-unrelated)
+	 * @param key
+	 * 		blob key associated with the file to be deleted
+	 *
+	 * @throws IOException
+	 */
+	private void deleteInternal(@Nullable JobID jobId, BlobKey key) throws IOException{
+		final File localFile = BlobUtils.getStorageLocation(storageDir, jobId, key);
 		if (!localFile.delete() && localFile.exists()) {
 			LOG.warn("Failed to delete locally cached BLOB {} at {}", key, localFile.getAbsolutePath());
 		}
 	}
 
 	/**
-	 * Deletes the file associated with the given key from the BLOB cache and
+	 * Deletes the (job-unrelated) file associated with the given key from the BLOB cache and
 	 * BLOB server.
 	 *
-	 * @param key referring to the file to be deleted
+	 * @param key
+	 * 		referring to the file to be deleted
+	 *
 	 * @throws IOException
-	 *         thrown if an I/O error occurs while transferring the request to
-	 *         the BLOB server or if the BLOB server cannot delete the file
+	 * 		thrown if an I/O error occurs while transferring the request to the BLOB server or if the
+	 * 		BLOB server cannot delete the file
 	 */
 	public void deleteGlobal(BlobKey key) throws IOException {
+		deleteGlobalInternal(null, key);
+	}
+
+	/**
+	 * Deletes the file associated with the given key from the BLOB cache and BLOB server.
+	 *
+	 * @param jobId
+	 * 		ID of the job this blob belongs to
+	 * @param key
+	 * 		referring to the file to be deleted
+	 *
+	 * @throws IOException
+	 * 		thrown if an I/O error occurs while transferring the request to the BLOB server or if the
+	 * 		BLOB server cannot delete the file
+	 */
+	public void deleteGlobal(@Nonnull JobID jobId, BlobKey key) throws IOException {
+		checkNotNull(jobId);
+		deleteGlobalInternal(jobId, key);
+	}
+
+	/**
+	 * Deletes the file associated with the given key from the BLOB cache and
+	 * BLOB server.
+	 *
+	 * @param jobId
+	 * 		ID of the job this blob belongs to (or <tt>null</tt> if job-unrelated)
+	 * @param key
+	 * 		referring to the file to be deleted
+	 *
+	 * @throws IOException
+	 * 		thrown if an I/O error occurs while transferring the request to the BLOB server or if the
+	 * 		BLOB server cannot delete the file
+	 */
+	private void deleteGlobalInternal(@Nullable JobID jobId, BlobKey key) throws IOException {
 		// delete locally
-		delete(key);
+		deleteInternal(jobId, key);
 		// then delete on the BLOB server
 		// (don't use the distributed storage directly - this way the blob
 		// server is aware of the delete operation, too)
 		try (BlobClient bc = createClient()) {
-			bc.delete(key);
+			bc.deleteInternal(jobId, key);
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/0a19c456/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java
index 0882ec3..9a2f59e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.blob;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.BlobServerOptions;
 import org.apache.flink.configuration.Configuration;
@@ -29,6 +30,8 @@ import org.apache.flink.util.InstantiationUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
 import javax.net.ssl.SSLContext;
 import javax.net.ssl.SSLParameters;
 import javax.net.ssl.SSLSocket;
@@ -46,7 +49,8 @@ import java.util.Collections;
 import java.util.List;
 
 import static org.apache.flink.runtime.blob.BlobServerProtocol.BUFFER_SIZE;
-import static org.apache.flink.runtime.blob.BlobServerProtocol.CONTENT_ADDRESSABLE;
+import static org.apache.flink.runtime.blob.BlobServerProtocol.CONTENT_FOR_JOB;
+import static org.apache.flink.runtime.blob.BlobServerProtocol.CONTENT_NO_JOB;
 import static org.apache.flink.runtime.blob.BlobServerProtocol.DELETE_OPERATION;
 import static org.apache.flink.runtime.blob.BlobServerProtocol.GET_OPERATION;
 import static org.apache.flink.runtime.blob.BlobServerProtocol.PUT_OPERATION;
@@ -55,7 +59,7 @@ import static org.apache.flink.runtime.blob.BlobServerProtocol.RETURN_OKAY;
 import static org.apache.flink.runtime.blob.BlobUtils.readFully;
 import static org.apache.flink.runtime.blob.BlobUtils.readLength;
 import static org.apache.flink.runtime.blob.BlobUtils.writeLength;
-import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
  * The BLOB client can communicate with the BLOB server and either upload (PUT), download (GET),
@@ -75,6 +79,7 @@ public final class BlobClient implements Closeable {
 	 *        the network address of the BLOB server
 	 * @param clientConfig
 	 *        additional configuration like SSL parameters required to connect to the blob server
+	 *
 	 * @throws IOException
 	 *         thrown if the connection to the BLOB server could not be established
 	 */
@@ -130,22 +135,65 @@ public final class BlobClient implements Closeable {
 	// --------------------------------------------------------------------------------------------
 
 	/**
-	 * Downloads the BLOB identified by the given BLOB key from the BLOB server. If no such BLOB exists on the server, a
-	 * {@link FileNotFoundException} is thrown.
-	 * 
+	 * Downloads the (job-unrelated) BLOB identified by the given BLOB key from the BLOB server.
+	 *
 	 * @param blobKey
-	 *        the BLOB key identifying the BLOB to download
+	 * 		blob key associated with the requested file
+	 *
 	 * @return an input stream to read the retrieved data from
+	 *
+	 * @throws FileNotFoundException
+	 * 		if there is no such file;
 	 * @throws IOException
-	 *         thrown if an I/O error occurs during the download
+	 * 		if an I/O error occurs during the download
 	 */
 	public InputStream get(BlobKey blobKey) throws IOException {
+		return getInternal(null, blobKey);
+	}
+
+	/**
+	 * Downloads the BLOB identified by the given BLOB key from the BLOB server.
+	 *
+	 * @param jobId
+	 * 		ID of the job this blob belongs to
+	 * @param blobKey
+	 * 		blob key associated with the requested file
+	 *
+	 * @return an input stream to read the retrieved data from
+	 *
+	 * @throws FileNotFoundException
+	 * 		if there is no such file;
+	 * @throws IOException
+	 * 		if an I/O error occurs during the download
+	 */
+	public InputStream get(@Nonnull JobID jobId, BlobKey blobKey) throws IOException {
+		checkNotNull(jobId);
+		return getInternal(jobId, blobKey);
+	}
+
+	/**
+	 * Downloads the BLOB identified by the given BLOB key from the BLOB server.
+	 *
+	 * @param jobId
+	 * 		ID of the job this blob belongs to (or <tt>null</tt> if job-unrelated)
+	 * @param blobKey
+	 * 		blob key associated with the requested file
+	 *
+	 * @return an input stream to read the retrieved data from
+	 *
+	 * @throws FileNotFoundException
+	 * 		if there is no such file;
+	 * @throws IOException
+	 * 		if an I/O error occurs during the download
+	 */
+	InputStream getInternal(@Nullable JobID jobId, BlobKey blobKey) throws IOException {
 		if (this.socket.isClosed()) {
 			throw new IllegalStateException("BLOB Client is not connected. " +
 					"Client has been shut down or encountered an error before.");
 		}
 		if (LOG.isDebugEnabled()) {
-			LOG.debug(String.format("GET content addressable BLOB %s from %s", blobKey, socket.getLocalSocketAddress()));
+			LOG.debug("GET BLOB {}/{} from {}.", jobId, blobKey,
+				socket.getLocalSocketAddress());
 		}
 
 		try {
@@ -153,8 +201,8 @@ public final class BlobClient implements Closeable {
 			InputStream is = this.socket.getInputStream();
 
 			// Send GET header
-			sendGetHeader(os, null, blobKey);
-			receiveAndCheckResponse(is);
+			sendGetHeader(os, jobId, blobKey);
+			receiveAndCheckGetResponse(is);
 
 			return new BlobInputStream(is, blobKey);
 		}
@@ -169,29 +217,40 @@ public final class BlobClient implements Closeable {
 	 *
 	 * @param outputStream
 	 *        the output stream to write the header data to
-	 * @param jobID
-	 *        the job ID identifying the BLOB to download or <code>null</code> to indicate the BLOB key should be used
-	 *        to identify the BLOB on the server instead
+	 * @param jobId
+	 * 		ID of the job this blob belongs to (or <tt>null</tt> if job-unrelated)
 	 * @param blobKey
-	 *        the BLOB key to identify the BLOB to download if either the job ID or the regular key are
-	 *        <code>null</code>
+	 * 		blob key associated with the requested file
+	 *
 	 * @throws IOException
 	 *         thrown if an I/O error occurs while writing the header data to the output stream
 	 */
-	private void sendGetHeader(OutputStream outputStream, JobID jobID, BlobKey blobKey) throws IOException {
-		checkArgument(jobID == null);
+	private static void sendGetHeader(OutputStream outputStream, @Nullable JobID jobId, BlobKey blobKey) throws IOException {
+		checkNotNull(blobKey);
 
 		// Signal type of operation
 		outputStream.write(GET_OPERATION);
 
-		// Check if GET should be done in content-addressable manner
-		if (jobID == null) {
-			outputStream.write(CONTENT_ADDRESSABLE);
-			blobKey.writeToOutputStream(outputStream);
+		// Send job ID and key
+		if (jobId == null) {
+			outputStream.write(CONTENT_NO_JOB);
+		} else {
+			outputStream.write(CONTENT_FOR_JOB);
+			outputStream.write(jobId.getBytes());
 		}
+		blobKey.writeToOutputStream(outputStream);
 	}
 
-	private void receiveAndCheckResponse(InputStream is) throws IOException {
+	/**
+	 * Reads the response from the input stream and throws in case of errors
+	 *
+	 * @param is
+	 * 		stream to read from
+	 *
+	 * @throws IOException
+	 * 		if the response is an error or reading the response failed
+	 */
+	private static void receiveAndCheckGetResponse(InputStream is) throws IOException {
 		int response = is.read();
 		if (response < 0) {
 			throw new EOFException("Premature end of response");
@@ -211,82 +270,111 @@ public final class BlobClient implements Closeable {
 	// --------------------------------------------------------------------------------------------
 
 	/**
-	 * Uploads the data of the given byte array to the BLOB server in a content-addressable manner.
+	 * Uploads the data of the given byte array for the given job to the BLOB server.
 	 *
+	 * @param jobId
+	 * 		the ID of the job the BLOB belongs to (or <tt>null</tt> if job-unrelated)
 	 * @param value
-	 *        the buffer to upload
+	 * 		the buffer to upload
+	 *
 	 * @return the computed BLOB key identifying the BLOB on the server
+	 *
 	 * @throws IOException
-	 *         thrown if an I/O error occurs while uploading the data to the BLOB server
+	 * 		thrown if an I/O error occurs while uploading the data to the BLOB server
 	 */
-	public BlobKey put(byte[] value) throws IOException {
-		return put(value, 0, value.length);
+	@VisibleForTesting
+	public BlobKey put(@Nullable JobID jobId, byte[] value) throws IOException {
+		return put(jobId, value, 0, value.length);
 	}
 
 	/**
-	 * Uploads data from the given byte array to the BLOB server in a content-addressable manner.
+	 * Uploads data from the given byte array for the given job to the BLOB server.
 	 *
+	 * @param jobId
+	 * 		the ID of the job the BLOB belongs to (or <tt>null</tt> if job-unrelated)
 	 * @param value
-	 *        the buffer to upload data from
+	 * 		the buffer to upload data from
 	 * @param offset
-	 *        the read offset within the buffer
+	 * 		the read offset within the buffer
 	 * @param len
-	 *        the number of bytes to upload from the buffer
+	 * 		the number of bytes to upload from the buffer
+	 *
 	 * @return the computed BLOB key identifying the BLOB on the server
+	 *
 	 * @throws IOException
-	 *         thrown if an I/O error occurs while uploading the data to the BLOB server
+	 * 		thrown if an I/O error occurs while uploading the data to the BLOB server
 	 */
-	public BlobKey put(byte[] value, int offset, int len) throws IOException {
-		return putBuffer(null, value, offset, len);
+	@VisibleForTesting
+	public BlobKey put(@Nullable JobID jobId, byte[] value, int offset, int len) throws IOException {
+		return putBuffer(jobId, value, offset, len);
 	}
 
 	/**
-	 * Uploads the data from the given input stream to the BLOB server in a content-addressable manner.
+	 * Uploads the (job-unrelated) data from the given input stream to the BLOB server.
 	 *
 	 * @param inputStream
-	 *        the input stream to read the data from
+	 * 		the input stream to read the data from
+	 *
 	 * @return the computed BLOB key identifying the BLOB on the server
+	 *
 	 * @throws IOException
-	 *         thrown if an I/O error occurs while reading the data from the input stream or uploading the data to the
-	 *         BLOB server
+	 * 		thrown if an I/O error occurs while reading the data from the input stream or uploading the
+	 * 		data to the BLOB server
 	 */
 	public BlobKey put(InputStream inputStream) throws IOException {
 		return putInputStream(null, inputStream);
 	}
 
 	/**
+	 * Uploads the data from the given input stream for the given job to the BLOB server.
+	 *
+	 * @param jobId
+	 * 		ID of the job this blob belongs to
+	 * @param inputStream
+	 * 		the input stream to read the data from
+	 *
+	 * @return the computed BLOB key identifying the BLOB on the server
+	 *
+	 * @throws IOException
+	 * 		thrown if an I/O error occurs while reading the data from the input stream or uploading the
+	 * 		data to the BLOB server
+	 */
+	public BlobKey put(@Nonnull JobID jobId, InputStream inputStream) throws IOException {
+		checkNotNull(jobId);
+		return putInputStream(jobId, inputStream);
+	}
+
+	/**
 	 * Uploads data from the given byte buffer to the BLOB server.
 	 *
 	 * @param jobId
-	 *        the ID of the job the BLOB belongs to or <code>null</code> to store the BLOB in a content-addressable
-	 *        manner
+	 * 		the ID of the job the BLOB belongs to (or <tt>null</tt> if job-unrelated)
 	 * @param value
-	 *        the buffer to read the data from
+	 * 		the buffer to read the data from
 	 * @param offset
-	 *        the read offset within the buffer
+	 * 		the read offset within the buffer
 	 * @param len
-	 *        the number of bytes to read from the buffer
-	 * @return the computed BLOB key if the BLOB has been stored in a content-addressable manner, <code>null</code>
-	 *         otherwise
+	 * 		the number of bytes to read from the buffer
+	 *
+	 * @return the computed BLOB key of the uploaded BLOB
+	 *
 	 * @throws IOException
-	 *         thrown if an I/O error occurs while uploading the data to the BLOB server
+	 * 		thrown if an I/O error occurs while uploading the data to the BLOB server
 	 */
-	private BlobKey putBuffer(JobID jobId, byte[] value, int offset, int len) throws IOException {
+	private BlobKey putBuffer(@Nullable JobID jobId, byte[] value, int offset, int len) throws IOException {
 		if (this.socket.isClosed()) {
 			throw new IllegalStateException("BLOB Client is not connected. " +
 					"Client has been shut down or encountered an error before.");
 		}
+		checkNotNull(value);
 
 		if (LOG.isDebugEnabled()) {
-			if (jobId == null) {
-				LOG.debug(String.format("PUT content addressable BLOB buffer (%d bytes) to %s",
-						len, socket.getLocalSocketAddress()));
-			}
+			LOG.debug("PUT BLOB buffer ({} bytes) to {}.", len, socket.getLocalSocketAddress());
 		}
 
 		try {
 			final OutputStream os = this.socket.getOutputStream();
-			final MessageDigest md = jobId == null ? BlobUtils.createMessageDigest() : null;
+			final MessageDigest md = BlobUtils.createMessageDigest();
 
 			// Send the PUT header
 			sendPutHeader(os, jobId);
@@ -295,15 +383,15 @@ public final class BlobClient implements Closeable {
 			int remainingBytes = len;
 
 			while (remainingBytes > 0) {
+				// want a common code path for byte[] and InputStream at the BlobServer
+				// -> since for InputStream we don't know a total size beforehand, send lengths iteratively
 				final int bytesToSend = Math.min(BUFFER_SIZE, remainingBytes);
 				writeLength(bytesToSend, os);
 
 				os.write(value, offset, bytesToSend);
 
-				// Update the message digest if necessary
-				if (md != null) {
-					md.update(value, offset, bytesToSend);
-				}
+				// Update the message digest
+				md.update(value, offset, bytesToSend);
 
 				remainingBytes -= bytesToSend;
 				offset += bytesToSend;
@@ -313,7 +401,7 @@ public final class BlobClient implements Closeable {
 
 			// Receive blob key and compare
 			final InputStream is = this.socket.getInputStream();
-			return receivePutResponseAndCompare(is, md);
+			return receiveAndCheckPutResponse(is, md);
 		}
 		catch (Throwable t) {
 			BlobUtils.closeSilently(socket, LOG);
@@ -325,37 +413,36 @@ public final class BlobClient implements Closeable {
 	 * Uploads data from the given input stream to the BLOB server.
 	 *
 	 * @param jobId
-	 *        the ID of the job the BLOB belongs to or <code>null</code> to store the BLOB in a content-addressable
-	 *        manner
+	 * 		the ID of the job the BLOB belongs to (or <tt>null</tt> if job-unrelated)
 	 * @param inputStream
-	 *        the input stream to read the data from
-	 * @return he computed BLOB key if the BLOB has been stored in a content-addressable manner, <code>null</code>
-	 *         otherwise
+	 * 		the input stream to read the data from
+	 *
+	 * @return the computed BLOB key of the uploaded BLOB
+	 *
 	 * @throws IOException
-	 *         thrown if an I/O error occurs while uploading the data to the BLOB server
+	 * 		thrown if an I/O error occurs while uploading the data to the BLOB server
 	 */
-	private BlobKey putInputStream(JobID jobId, InputStream inputStream) throws IOException {
+	private BlobKey putInputStream(@Nullable JobID jobId, InputStream inputStream) throws IOException {
 		if (this.socket.isClosed()) {
 			throw new IllegalStateException("BLOB Client is not connected. " +
 					"Client has been shut down or encountered an error before.");
 		}
+		checkNotNull(inputStream);
 
 		if (LOG.isDebugEnabled()) {
-			if (jobId == null) {
-				LOG.debug(String.format("PUT content addressable BLOB stream to %s",
-						socket.getLocalSocketAddress()));
-			}
+			LOG.debug("PUT BLOB stream to {}.", socket.getLocalSocketAddress());
 		}
 
 		try {
 			final OutputStream os = this.socket.getOutputStream();
-			final MessageDigest md = jobId == null ? BlobUtils.createMessageDigest() : null;
+			final MessageDigest md = BlobUtils.createMessageDigest();
 			final byte[] xferBuf = new byte[BUFFER_SIZE];
 
 			// Send the PUT header
 			sendPutHeader(os, jobId);
 
 			while (true) {
+				// since we don't know a total size here, send lengths iteratively
 				final int read = inputStream.read(xferBuf);
 				if (read < 0) {
 					// we are done. send a -1 and be done
@@ -365,15 +452,13 @@ public final class BlobClient implements Closeable {
 				if (read > 0) {
 					writeLength(read, os);
 					os.write(xferBuf, 0, read);
-					if (md != null) {
-						md.update(xferBuf, 0, read);
-					}
+					md.update(xferBuf, 0, read);
 				}
 			}
 
 			// Receive blob key and compare
 			final InputStream is = this.socket.getInputStream();
-			return receivePutResponseAndCompare(is, md);
+			return receiveAndCheckPutResponse(is, md);
 		}
 		catch (Throwable t) {
 			BlobUtils.closeSilently(socket, LOG);
@@ -381,16 +466,25 @@ public final class BlobClient implements Closeable {
 		}
 	}
 
-	private BlobKey receivePutResponseAndCompare(InputStream is, MessageDigest md) throws IOException {
+	/**
+	 * Reads the response from the input stream and throws in case of errors
+	 *
+	 * @param is
+	 * 		stream to read from
+	 * @param md
+	 * 		message digest to check the response against
+	 *
+	 * @throws IOException
+	 * 		if the response is an error, the message digest does not match or reading the response
+	 * 		failed
+	 */
+	private static BlobKey receiveAndCheckPutResponse(InputStream is, MessageDigest md)
+			throws IOException {
 		int response = is.read();
 		if (response < 0) {
 			throw new EOFException("Premature end of response");
 		}
 		else if (response == RETURN_OKAY) {
-			if (md == null) {
-				// not content addressable
-				return null;
-			}
 
 			BlobKey remoteKey = BlobKey.readFromInputStream(is);
 			BlobKey localKey = new BlobKey(md.digest());
@@ -412,24 +506,24 @@ public final class BlobClient implements Closeable {
 
 	/**
 	 * Constructs and writes the header data for a PUT request to the given output stream.
-	 * NOTE: If the jobId and key are null, we send the data to the content addressable section.
 	 *
 	 * @param outputStream
-	 *        the output stream to write the PUT header data to
-	 * @param jobID
-	 *        the ID of job the BLOB belongs to or <code>null</code> to indicate the upload of a
-	 *        content-addressable BLOB
+	 * 		the output stream to write the PUT header data to
+	 * @param jobId
+	 * 		the ID of job the BLOB belongs to (or <tt>null</tt> if job-unrelated)
+	 *
 	 * @throws IOException
-	 *         thrown if an I/O error occurs while writing the header data to the output stream
+	 * 		thrown if an I/O error occurs while writing the header data to the output stream
 	 */
-	private void sendPutHeader(OutputStream outputStream, JobID jobID) throws IOException {
-		checkArgument(jobID == null);
-
+	private static void sendPutHeader(OutputStream outputStream, @Nullable JobID jobId) throws IOException {
 		// Signal type of operation
 		outputStream.write(PUT_OPERATION);
-
-		// Check if PUT should be done in content-addressable manner
-		outputStream.write(CONTENT_ADDRESSABLE);
+		if (jobId == null) {
+			outputStream.write(CONTENT_NO_JOB);
+		} else {
+			outputStream.write(CONTENT_FOR_JOB);
+			outputStream.write(jobId.getBytes());
+		}
 	}
 
 	// --------------------------------------------------------------------------------------------
@@ -437,16 +531,50 @@ public final class BlobClient implements Closeable {
 	// --------------------------------------------------------------------------------------------
 
 	/**
-	 * Deletes the BLOB identified by the given BLOB key from the BLOB server.
+	 * Deletes the (job-unrelated) BLOB identified by the given BLOB key from the BLOB server.
+	 *
+	 * @param key
+	 * 		the key to identify the BLOB
 	 *
-	 * @param blobKey
-	 *        the key to identify the BLOB
 	 * @throws IOException
-	 *         thrown if an I/O error occurs while transferring the request to
-	 *         the BLOB server or if the BLOB server cannot delete the file
+	 * 		thrown if an I/O error occurs while transferring the request to the BLOB server or if the
+	 * 		BLOB server cannot delete the file
 	 */
-	public void delete(BlobKey blobKey) throws IOException {
-		checkArgument(blobKey != null, "BLOB key must not be null.");
+	public void delete(BlobKey key) throws IOException {
+		deleteInternal(null, key);
+	}
+
+	/**
+	 * Deletes the BLOB identified by the given BLOB key and job ID from the BLOB server.
+	 *
+	 * @param jobId
+	 * 		the ID of job the BLOB belongs to
+	 * @param key
+	 * 		the key to identify the BLOB
+	 *
+	 * @throws IOException
+	 * 		thrown if an I/O error occurs while transferring the request to the BLOB server or if the
+	 * 		BLOB server cannot delete the file
+	 */
+	public void delete(@Nonnull JobID jobId, BlobKey key) throws IOException {
+		checkNotNull(jobId);
+		deleteInternal(jobId, key);
+	}
+
+	/**
+	 * Deletes the BLOB identified by the given BLOB key and job ID from the BLOB server.
+	 *
+	 * @param jobId
+	 * 		the ID of job the BLOB belongs to (or <tt>null</tt> if job-unrelated)
+	 * @param key
+	 * 		the key to identify the BLOB
+	 *
+	 * @throws IOException
+	 * 		thrown if an I/O error occurs while transferring the request to the BLOB server or if the
+	 * 		BLOB server cannot delete the file
+	 */
+	public void deleteInternal(@Nullable JobID jobId, BlobKey key) throws IOException {
+		checkNotNull(key);
 
 		try {
 			final OutputStream outputStream = this.socket.getOutputStream();
@@ -456,20 +584,16 @@ public final class BlobClient implements Closeable {
 			outputStream.write(DELETE_OPERATION);
 
 			// delete blob key
-			outputStream.write(CONTENT_ADDRESSABLE);
-			blobKey.writeToOutputStream(outputStream);
-
-			int response = inputStream.read();
-			if (response < 0) {
-				throw new EOFException("Premature end of response");
-			}
-			if (response == RETURN_ERROR) {
-				Throwable cause = readExceptionFromStream(inputStream);
-				throw new IOException("Server side error: " + cause.getMessage(), cause);
-			}
-			else if (response != RETURN_OKAY) {
-				throw new IOException("Unrecognized response");
+			if (jobId == null) {
+				outputStream.write(CONTENT_NO_JOB);
+			} else {
+				outputStream.write(CONTENT_FOR_JOB);
+				outputStream.write(jobId.getBytes());
 			}
+			key.writeToOutputStream(outputStream);
+
+			// the response is the same as for a GET request
+			receiveAndCheckGetResponse(inputStream);
 		}
 		catch (Throwable t) {
 			BlobUtils.closeSilently(socket, LOG);
@@ -479,11 +603,18 @@ public final class BlobClient implements Closeable {
 
 	/**
 	 * Uploads the JAR files to a {@link BlobServer} at the given address.
+	 * <p>
+	 * TODO: add jobId to signature after adapting the BlobLibraryCacheManager
+	 *
+	 * @param serverAddress
+	 * 		Server address of the {@link BlobServer}
+	 * @param clientConfig
+	 * 		Any additional configuration for the blob client
+	 * @param jars
+	 * 		List of JAR files to upload
 	 *
-	 * @param serverAddress Server address of the {@link BlobServer}
-	 * @param clientConfig Any additional configuration for the blob client
-	 * @param jars List of JAR files to upload
-	 * @throws IOException Thrown if the upload fails
+	 * @throws IOException
+	 * 		if the upload fails
 	 */
 	public static List<BlobKey> uploadJarFiles(
 			InetSocketAddress serverAddress,
@@ -500,7 +631,7 @@ public final class BlobClient implements Closeable {
 					FSDataInputStream is = null;
 					try {
 						is = fs.open(jar);
-						final BlobKey key = blobClient.put(is);
+						final BlobKey key = blobClient.putInputStream(null, is);
 						blobKeys.add(key);
 					} finally {
 						if (is != null) {

http://git-wip-us.apache.org/repos/asf/flink/blob/0a19c456/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
index add9f7f..43a060a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.blob;
 
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.BlobServerOptions;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
@@ -28,6 +29,8 @@ import org.apache.flink.util.NetUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
 import javax.net.ssl.SSLContext;
 import java.io.File;
 import java.io.FileNotFoundException;
@@ -40,7 +43,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
@@ -58,7 +61,7 @@ public class BlobServer extends Thread implements BlobService {
 	private static final Logger LOG = LoggerFactory.getLogger(BlobServer.class);
 
 	/** Counter to generate unique names for temporary files. */
-	private final AtomicInteger tempFileCounter = new AtomicInteger(0);
+	private final AtomicLong tempFileCounter = new AtomicLong(0);
 
 	/** The server socket listening for incoming connections. */
 	private final ServerSocket serverSocket;
@@ -110,7 +113,7 @@ public class BlobServer extends Thread implements BlobService {
 
 		// configure and create the storage directory
 		String storageDirectory = config.getString(BlobServerOptions.STORAGE_DIRECTORY);
-		this.storageDir = BlobUtils.initStorageDirectory(storageDirectory);
+		this.storageDir = BlobUtils.initLocalStorageDirectory(storageDirectory);
 		LOG.info("Created BLOB server storage directory {}", storageDir);
 
 		// configure the maximum number of concurrent connections
@@ -189,11 +192,12 @@ public class BlobServer extends Thread implements BlobService {
 	 *
 	 * <p><strong>This is only called from the {@link BlobServerConnection}</strong>
 	 *
+	 * @param jobId ID of the job this blob belongs to (or <tt>null</tt> if job-unrelated)
 	 * @param key identifying the file
 	 * @return file handle to the file
 	 */
-	File getStorageLocation(BlobKey key) {
-		return BlobUtils.getStorageLocation(storageDir, key);
+	File getStorageLocation(JobID jobId, BlobKey key) {
+		return BlobUtils.getStorageLocation(storageDir, jobId, key);
 	}
 
 	/**
@@ -333,20 +337,69 @@ public class BlobServer extends Thread implements BlobService {
 	}
 
 	/**
-	 * Method which retrieves the local path of a file associated with a blob key. The blob server
-	 * looks the blob key up in its local storage. If the file exists, it is returned. If the
-	 * file does not exist, it is retrieved from the HA blob store (if available) or a
-	 * FileNotFoundException is thrown.
+	 * Retrieves the local path of a (job-unrelated) file associated with a job and a blob key.
+	 * <p>
+	 * The blob server looks the blob key up in its local storage. If the file exists, it is
+	 * returned. If the file does not exist, it is retrieved from the HA blob store (if available)
+	 * or a {@link FileNotFoundException} is thrown.
 	 *
-	 * @param requiredBlob blob key associated with the requested file
-	 * @return file referring to the local storage location of the BLOB.
-	 * @throws IOException Thrown if the file retrieval failed.
+	 * @param key
+	 * 		blob key associated with the requested file
+	 *
+	 * @return file referring to the local storage location of the BLOB
+	 *
+	 * @throws IOException
+	 * 		Thrown if the file retrieval failed.
+	 */
+	@Override
+	public File getFile(BlobKey key) throws IOException {
+		return getFileInternal(null, key);
+	}
+
+	/**
+	 * Retrieves the local path of a file associated with a job and a blob key.
+	 * <p>
+	 * The blob server looks the blob key up in its local storage. If the file exists, it is
+	 * returned. If the file does not exist, it is retrieved from the HA blob store (if available)
+	 * or a {@link FileNotFoundException} is thrown.
+	 *
+	 * @param jobId
+	 * 		ID of the job this blob belongs to
+	 * @param key
+	 * 		blob key associated with the requested file
+	 *
+	 * @return file referring to the local storage location of the BLOB
+	 *
+	 * @throws IOException
+	 * 		Thrown if the file retrieval failed.
 	 */
 	@Override
-	public File getFile(BlobKey requiredBlob) throws IOException {
+	public File getFile(@Nonnull JobID jobId, BlobKey key) throws IOException {
+		checkNotNull(jobId);
+		return getFileInternal(jobId, key);
+	}
+
+	/**
+	 * Retrieves the local path of a file associated with a job and a blob key.
+	 * <p>
+	 * The blob server looks the blob key up in its local storage. If the file exists, it is
+	 * returned. If the file does not exist, it is retrieved from the HA blob store (if available)
+	 * or a {@link FileNotFoundException} is thrown.
+	 *
+	 * @param jobId
+	 * 		ID of the job this blob belongs to (or <tt>null</tt> if job-unrelated)
+	 * @param requiredBlob
+	 * 		blob key associated with the requested file
+	 *
+	 * @return file referring to the local storage location of the BLOB
+	 *
+	 * @throws IOException
+	 * 		Thrown if the file retrieval failed.
+	 */
+	private File getFileInternal(@Nullable JobID jobId, BlobKey requiredBlob) throws IOException {
 		checkArgument(requiredBlob != null, "BLOB key cannot be null.");
 
-		final File localFile = BlobUtils.getStorageLocation(storageDir, requiredBlob);
+		final File localFile = BlobUtils.getStorageLocation(storageDir, jobId, requiredBlob);
 
 		if (localFile.exists()) {
 			return localFile;
@@ -354,10 +407,11 @@ public class BlobServer extends Thread implements BlobService {
 		else {
 			try {
 				// Try the blob store
-				blobStore.get(requiredBlob, localFile);
+				blobStore.get(jobId, requiredBlob, localFile);
 			}
 			catch (Exception e) {
-				throw new IOException("Failed to copy from blob store.", e);
+				throw new IOException(
+					"Failed to copy BLOB " + requiredBlob + " from blob store to " + localFile, e);
 			}
 
 			if (localFile.exists()) {
@@ -371,24 +425,58 @@ public class BlobServer extends Thread implements BlobService {
 	}
 
 	/**
-	 * This method deletes the file associated to the blob key if it exists in the local storage
-	 * of the blob server.
+	 * Deletes the (job-unrelated) file associated with the blob key in both the local storage as
+	 * well as in the HA store of the blob server.
+	 *
+	 * @param key
+	 * 		blob key associated with the file to be deleted
 	 *
-	 * @param key associated with the file to be deleted
 	 * @throws IOException
 	 */
 	@Override
 	public void delete(BlobKey key) throws IOException {
-		final File localFile = BlobUtils.getStorageLocation(storageDir, key);
+		deleteInternal(null, key);
+	}
+
+	/**
+	 * Deletes the file associated with the blob key in both the local storage as well as in the HA
+	 * store of the blob server.
+	 *
+	 * @param jobId
+	 * 		ID of the job this blob belongs to
+	 * @param key
+	 * 		blob key associated with the file to be deleted
+	 *
+	 * @throws IOException
+	 */
+	@Override
+	public void delete(@Nonnull JobID jobId, BlobKey key) throws IOException {
+		checkNotNull(jobId);
+		deleteInternal(jobId, key);
+	}
+
+	/**
+	 * Deletes the file associated with the blob key in both the local storage as well as in the HA
+	 * store of the blob server.
+	 *
+	 * @param jobId
+	 * 		ID of the job this blob belongs to (or <tt>null</tt> if job-unrelated)
+	 * @param key
+	 * 		blob key associated with the file to be deleted
+	 *
+	 * @throws IOException
+	 */
+	void deleteInternal(@Nullable JobID jobId, BlobKey key) throws IOException {
+		final File localFile = BlobUtils.getStorageLocation(storageDir, jobId, key);
 
 		readWriteLock.writeLock().lock();
 
 		try {
 			if (!localFile.delete() && localFile.exists()) {
-				LOG.warn("Failed to delete locally BLOB " + key + " at " + localFile.getAbsolutePath());
+				LOG.warn("Failed to locally delete BLOB " + key + " at " + localFile.getAbsolutePath());
 			}
 
-			blobStore.delete(key);
+			blobStore.delete(jobId, key);
 		} finally {
 			readWriteLock.writeLock().unlock();
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/0a19c456/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java
index 181211d..f1054c0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java
@@ -39,7 +39,8 @@ import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
 
 import static org.apache.flink.runtime.blob.BlobServerProtocol.BUFFER_SIZE;
-import static org.apache.flink.runtime.blob.BlobServerProtocol.CONTENT_ADDRESSABLE;
+import static org.apache.flink.runtime.blob.BlobServerProtocol.CONTENT_FOR_JOB;
+import static org.apache.flink.runtime.blob.BlobServerProtocol.CONTENT_NO_JOB;
 import static org.apache.flink.runtime.blob.BlobServerProtocol.DELETE_OPERATION;
 import static org.apache.flink.runtime.blob.BlobServerProtocol.GET_OPERATION;
 import static org.apache.flink.runtime.blob.BlobServerProtocol.PUT_OPERATION;
@@ -49,6 +50,7 @@ import static org.apache.flink.runtime.blob.BlobUtils.closeSilently;
 import static org.apache.flink.runtime.blob.BlobUtils.readFully;
 import static org.apache.flink.runtime.blob.BlobUtils.readLength;
 import static org.apache.flink.runtime.blob.BlobUtils.writeLength;
+import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
  * A BLOB connection handles a series of requests from a particular BLOB client.
@@ -83,12 +85,8 @@ class BlobServerConnection extends Thread {
 		super("BLOB connection for " + clientSocket.getRemoteSocketAddress());
 		setDaemon(true);
 
-		if (blobServer == null) {
-			throw new NullPointerException();
-		}
-
 		this.clientSocket = clientSocket;
-		this.blobServer = blobServer;
+		this.blobServer = checkNotNull(blobServer);
 		this.blobStore = blobServer.getBlobStore();
 
 		ReadWriteLock readWriteLock = blobServer.getReadWriteLock();
@@ -167,15 +165,16 @@ class BlobServerConnection extends Thread {
 
 	/**
 	 * Handles an incoming GET request from a BLOB client.
-	 * 
+	 *
 	 * @param inputStream
-	 *        the input stream to read incoming data from
+	 * 		the input stream to read incoming data from
 	 * @param outputStream
-	 *        the output stream to send data back to the client
+	 * 		the output stream to send data back to the client
 	 * @param buf
-	 *        an auxiliary buffer for data serialization/deserialization
+	 * 		an auxiliary buffer for data serialization/deserialization
+	 *
 	 * @throws IOException
-	 *         thrown if an I/O error occurs while reading/writing data from/to the respective streams
+	 * 		thrown if an I/O error occurs while reading/writing data from/to the respective streams
 	 */
 	private void get(InputStream inputStream, OutputStream outputStream, byte[] buf) throws IOException {
 		/*
@@ -187,25 +186,36 @@ class BlobServerConnection extends Thread {
 		 * so a local cache makes more sense.
 		 */
 
-		File blobFile;
-		int contentAddressable = -1;
-		JobID jobId = null;
-		BlobKey blobKey = null;
+		final File blobFile;
+		final JobID jobId;
+		final BlobKey blobKey;
 
 		try {
-			contentAddressable = inputStream.read();
+			final int mode = inputStream.read();
 
-			if (contentAddressable < 0) {
+			if (mode < 0) {
 				throw new EOFException("Premature end of GET request");
 			}
-			if (contentAddressable == CONTENT_ADDRESSABLE) {
-				blobKey = BlobKey.readFromInputStream(inputStream);
-				blobFile = blobServer.getStorageLocation(blobKey);
+
+			// Receive the job ID and key
+			if (mode == CONTENT_NO_JOB) {
+				jobId = null;
+			} else if (mode == CONTENT_FOR_JOB) {
+				byte[] jidBytes = new byte[JobID.SIZE];
+				readFully(inputStream, jidBytes, 0, JobID.SIZE, "JobID");
+				jobId = JobID.fromByteArray(jidBytes);
+			} else {
+				throw new IOException("Unknown type of BLOB addressing: " + mode + '.');
 			}
-			else {
-				throw new IOException("Unknown type of BLOB addressing: " + contentAddressable + '.');
+			blobKey = BlobKey.readFromInputStream(inputStream);
+
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("Received GET request for BLOB {}/{} from {}.", jobId,
+					blobKey, clientSocket.getInetAddress());
 			}
 
+			blobFile = blobServer.getStorageLocation(jobId, blobKey);
+
 			// up to here, an error can give a good message
 		}
 		catch (Throwable t) {
@@ -214,7 +224,7 @@ class BlobServerConnection extends Thread {
 				writeErrorToStream(outputStream, t);
 			}
 			catch (IOException e) {
-				// since we are in an exception case, it means not much that we could not send the error
+				// since we are in an exception case, it means that we could not send the error
 				// ignore this
 			}
 			clientSocket.close();
@@ -224,6 +234,7 @@ class BlobServerConnection extends Thread {
 		readLock.lock();
 
 		try {
+			// copy the file to local store if it does not exist yet
 			try {
 				if (!blobFile.exists()) {
 					// first we have to release the read lock in order to acquire the write lock
@@ -232,9 +243,9 @@ class BlobServerConnection extends Thread {
 
 					try {
 						if (blobFile.exists()) {
-							LOG.debug("Blob file {} has downloaded from the BlobStore by a different connection.", blobFile);
+							LOG.debug("Blob file {} has been downloaded from the (distributed) blob store by a different connection.", blobFile);
 						} else {
-							blobStore.get(blobKey, blobFile);
+							blobStore.get(jobId, blobKey, blobFile);
 						}
 					} finally {
 						writeLock.unlock();
@@ -248,6 +259,7 @@ class BlobServerConnection extends Thread {
 					}
 				}
 
+				// enforce a 2GB max for now (otherwise the protocol's length field needs to be increased)
 				if (blobFile.length() > Integer.MAX_VALUE) {
 					throw new IOException("BLOB size exceeds the maximum size (2 GB).");
 				}
@@ -259,7 +271,7 @@ class BlobServerConnection extends Thread {
 					writeErrorToStream(outputStream, t);
 				}
 				catch (IOException e) {
-					// since we are in an exception case, it means not much that we could not send the error
+					// since we are in an exception case, it means that we could not send the error
 					// ignore this
 				}
 				clientSocket.close();
@@ -294,59 +306,48 @@ class BlobServerConnection extends Thread {
 
 	/**
 	 * Handles an incoming PUT request from a BLOB client.
-	 * 
-	 * @param inputStream The input stream to read incoming data from.
-	 * @param outputStream The output stream to send data back to the client.
-	 * @param buf An auxiliary buffer for data serialization/deserialization.
+	 *
+	 * @param inputStream
+	 * 		The input stream to read incoming data from
+	 * @param outputStream
+	 * 		The output stream to send data back to the client
+	 * @param buf
+	 * 		An auxiliary buffer for data serialization/deserialization
+	 *
+	 * @throws IOException
+	 * 		thrown if an I/O error occurs while reading/writing data from/to the respective streams
 	 */
 	private void put(InputStream inputStream, OutputStream outputStream, byte[] buf) throws IOException {
-		JobID jobID = null;
-		MessageDigest md = null;
-
 		File incomingFile = null;
-		FileOutputStream fos = null;
 
 		try {
-			final int contentAddressable = inputStream.read();
-			if (contentAddressable < 0) {
+			final int mode = inputStream.read();
+
+			if (mode < 0) {
 				throw new EOFException("Premature end of PUT request");
 			}
 
-			if (contentAddressable == CONTENT_ADDRESSABLE) {
-				md = BlobUtils.createMessageDigest();
-			}
-			else {
+			// Receive the job ID and key
+			final JobID jobId;
+			if (mode == CONTENT_NO_JOB) {
+				jobId = null;
+			} else if (mode == CONTENT_FOR_JOB) {
+				byte[] jidBytes = new byte[JobID.SIZE];
+				readFully(inputStream, jidBytes, 0, JobID.SIZE, "JobID");
+				jobId = JobID.fromByteArray(jidBytes);
+			} else {
 				throw new IOException("Unknown type of BLOB addressing.");
 			}
 
 			if (LOG.isDebugEnabled()) {
-				LOG.debug("Received PUT request for content addressable BLOB");
+				LOG.debug("Received PUT request for BLOB of job {} with from {}.", jobId,
+					clientSocket.getInetAddress());
 			}
 
 			incomingFile = blobServer.createTemporaryFilename();
-			fos = new FileOutputStream(incomingFile);
+			BlobKey blobKey = readFileFully(inputStream, incomingFile, buf);
 
-			while (true) {
-				final int bytesExpected = readLength(inputStream);
-				if (bytesExpected == -1) {
-					// done
-					break;
-				}
-				if (bytesExpected > BUFFER_SIZE) {
-					throw new IOException("Unexpected number of incoming bytes: " + bytesExpected);
-				}
-
-				readFully(inputStream, buf, 0, bytesExpected, "buffer");
-				fos.write(buf, 0, bytesExpected);
-
-				if (md != null) {
-					md.update(buf, 0, bytesExpected);
-				}
-			}
-			fos.close();
-
-			BlobKey blobKey = new BlobKey(md.digest());
-			File storageFile = blobServer.getStorageLocation(blobKey);
+			File storageFile = blobServer.getStorageLocation(jobId, blobKey);
 
 			writeLock.lock();
 
@@ -369,13 +370,15 @@ class BlobServerConnection extends Thread {
 
 					// only the one moving the incoming file to its final destination is allowed to upload the
 					// file to the blob store
-					blobStore.put(storageFile, blobKey);
+					blobStore.put(storageFile, jobId, blobKey);
+				} else {
+					LOG.warn("File upload for an existing file with key {} for job {}. This may indicate a duplicate upload or a hash collision. Ignoring newest upload.", blobKey, jobId);
 				}
 			} catch(IOException ioe) {
 				// we failed to either create the local storage file or to upload it --> try to delete the local file
 				// while still having the write lock
-				if (storageFile.exists() && !storageFile.delete()) {
-					LOG.warn("Could not delete the storage file.");
+				if (!storageFile.delete() && storageFile.exists()) {
+					LOG.warn("Could not delete the storage file with key {} and job {}.", blobKey, jobId);
 				}
 
 				throw ioe;
@@ -403,15 +406,8 @@ class BlobServerConnection extends Thread {
 			clientSocket.close();
 		}
 		finally {
-			if (fos != null) {
-				try {
-					fos.close();
-				} catch (Throwable t) {
-					LOG.warn("Cannot close stream to BLOB staging file", t);
-				}
-			}
 			if (incomingFile != null) {
-				if (!incomingFile.delete()) {
+				if (!incomingFile.delete() && incomingFile.exists()) {
 					LOG.warn("Cannot delete BLOB server staging file " + incomingFile.getAbsolutePath());
 				}
 			}
@@ -419,27 +415,87 @@ class BlobServerConnection extends Thread {
 	}
 
 	/**
+	 * Reads a full file from <tt>inputStream</tt> into <tt>incomingFile</tt> returning its checksum.
+	 *
+	 * @param inputStream
+	 * 		stream to read from
+	 * @param incomingFile
+	 * 		file to write to
+	 * @param buf
+	 * 		An auxiliary buffer for data serialization/deserialization
+	 *
+	 * @return the received file's content hash as a BLOB key
+	 *
+	 * @throws IOException
+	 * 		thrown if an I/O error occurs while reading/writing data from/to the respective streams
+	 */
+	private static BlobKey readFileFully(
+			final InputStream inputStream, final File incomingFile, final byte[] buf)
+			throws IOException {
+		MessageDigest md = BlobUtils.createMessageDigest();
+		FileOutputStream fos = new FileOutputStream(incomingFile);
+
+		try {
+			while (true) {
+				final int bytesExpected = readLength(inputStream);
+				if (bytesExpected == -1) {
+					// done
+					break;
+				}
+				if (bytesExpected > BUFFER_SIZE) {
+					throw new IOException(
+						"Unexpected number of incoming bytes: " + bytesExpected);
+				}
+
+				readFully(inputStream, buf, 0, bytesExpected, "buffer");
+				fos.write(buf, 0, bytesExpected);
+
+				md.update(buf, 0, bytesExpected);
+			}
+			return new BlobKey(md.digest());
+		} finally {
+			try {
+				fos.close();
+			} catch (Throwable t) {
+				LOG.warn("Cannot close stream to BLOB staging file", t);
+			}
+		}
+	}
+
+	/**
 	 * Handles an incoming DELETE request from a BLOB client.
-	 * 
-	 * @param inputStream The input stream to read the request from.
-	 * @param outputStream The output stream to write the response to.
-	 * @throws java.io.IOException Thrown if an I/O error occurs while reading the request data from the input stream.
+	 *
+	 * @param inputStream
+	 * 		The input stream to read the request from.
+	 * @param outputStream
+	 * 		The output stream to write the response to.
+	 *
+	 * @throws IOException
+	 * 		Thrown if an I/O error occurs while reading the request data from the input stream.
 	 */
 	private void delete(InputStream inputStream, OutputStream outputStream) throws IOException {
 
 		try {
-			int type = inputStream.read();
-			if (type < 0) {
+			final int mode = inputStream.read();
+
+			if (mode < 0) {
 				throw new EOFException("Premature end of DELETE request");
 			}
 
-			if (type == CONTENT_ADDRESSABLE) {
-				BlobKey key = BlobKey.readFromInputStream(inputStream);
-				blobServer.delete(key);
-			}
-			else {
-				throw new IOException("Unrecognized addressing type: " + type);
+			// Receive the job ID and key
+			final JobID jobId;
+			if (mode == CONTENT_NO_JOB) {
+				jobId = null;
+			} else if (mode == CONTENT_FOR_JOB) {
+				byte[] jidBytes = new byte[JobID.SIZE];
+				readFully(inputStream, jidBytes, 0, JobID.SIZE, "JobID");
+				jobId = JobID.fromByteArray(jidBytes);
+			} else {
+				throw new IOException("Unknown type of BLOB addressing.");
 			}
+			BlobKey key = BlobKey.readFromInputStream(inputStream);
+
+			blobServer.deleteInternal(jobId, key);
 
 			outputStream.write(RETURN_OKAY);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/0a19c456/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerProtocol.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerProtocol.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerProtocol.java
index d8ac833..681fc81 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerProtocol.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerProtocol.java
@@ -42,12 +42,20 @@ public class BlobServerProtocol {
 	static final byte RETURN_ERROR = 1;
 
 	/**
-	 * Internal code to identify a reference via content hash as the key.
+	 * Internal code to identify a job-unrelated reference via content hash as the key.
 	 * <p>
 	 * Note: previously, there was also <tt>NAME_ADDRESSABLE</tt> (code <tt>1</tt>) and
 	 * <tt>JOB_ID_SCOPE</tt> (code <tt>2</tt>).
 	 */
-	static final byte CONTENT_ADDRESSABLE = 0;
+	static final byte CONTENT_NO_JOB = 0;
+
+	/**
+	 * Internal code to identify a job-related reference via content hash as the key.
+	 * <p>
+	 * Note: previously, there was also <tt>NAME_ADDRESSABLE</tt> (code <tt>1</tt>) and
+	 * <tt>JOB_ID_SCOPE</tt> (code <tt>2</tt>).
+	 */
+	static final byte CONTENT_FOR_JOB = 3;
 
 	// --------------------------------------------------------------------------------------------
 

http://git-wip-us.apache.org/repos/asf/flink/blob/0a19c456/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobService.java
index 1e56f26..a78c88c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobService.java
@@ -18,6 +18,9 @@
 
 package org.apache.flink.runtime.blob;
 
+import org.apache.flink.api.common.JobID;
+
+import javax.annotation.Nonnull;
 import java.io.Closeable;
 import java.io.File;
 import java.io.IOException;
@@ -28,7 +31,8 @@ import java.io.IOException;
 public interface BlobService extends Closeable {
 
 	/**
-	 * Returns the path to a local copy of the file associated with the provided blob key.
+	 * Returns the path to a local copy of the (job-unrelated) file associated with the provided
+	 * blob key.
 	 *
 	 * @param key blob key associated with the requested file
 	 * @return The path to the file.
@@ -37,9 +41,19 @@ public interface BlobService extends Closeable {
 	 */
 	File getFile(BlobKey key) throws IOException;
 
+	/**
+	 * Returns the path to a local copy of the file associated with the provided job ID and blob key.
+	 *
+	 * @param jobId ID of the job this blob belongs to
+	 * @param key blob key associated with the requested file
+	 * @return The path to the file.
+	 * @throws java.io.FileNotFoundException when the path does not exist;
+	 * @throws IOException if any other error occurs when retrieving the file
+	 */
+	File getFile(@Nonnull JobID jobId, BlobKey key) throws IOException;
 
 	/**
-	 * Deletes the file associated with the provided blob key.
+	 * Deletes the (job-unrelated) file associated with the provided blob key.
 	 *
 	 * @param key associated with the file to be deleted
 	 * @throws IOException
@@ -47,10 +61,19 @@ public interface BlobService extends Closeable {
 	void delete(BlobKey key) throws IOException;
 
 	/**
+	 * Deletes the file associated with the provided job ID and blob key.
+	 *
+	 * @param jobId ID of the job this blob belongs to
+	 * @param key associated with the file to be deleted
+	 * @throws IOException
+	 */
+	void delete(@Nonnull JobID jobId, BlobKey key) throws IOException;
+
+	/**
 	 * Returns the port of the blob service.
 	 * @return the port of the blob service.
 	 */
 	int getPort();
-	
+
 	BlobClient createClient() throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/0a19c456/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java
index 1e8b73a..d2ea8ca 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java
@@ -32,19 +32,21 @@ public interface BlobStore extends BlobView {
 	 * Copies the local file to the blob store.
 	 *
 	 * @param localFile The file to copy
+	 * @param jobId ID of the job this blob belongs to (or <tt>null</tt> if job-unrelated)
 	 * @param blobKey   The ID for the file in the blob store
 	 * @throws IOException If the copy fails
 	 */
-	void put(File localFile, BlobKey blobKey) throws IOException;
+	void put(File localFile, JobID jobId, BlobKey blobKey) throws IOException;
 
 	/**
 	 * Tries to delete a blob from storage.
 	 *
 	 * <p>NOTE: This also tries to delete any created directories if empty.</p>
 	 *
+	 * @param jobId ID of the job this blob belongs to (or <tt>null</tt> if job-unrelated)
 	 * @param blobKey The blob ID
 	 */
-	void delete(BlobKey blobKey);
+	void delete(JobID jobId, BlobKey blobKey);
 
 	/**
 	 * Tries to delete all blobs for the given job from storage.

http://git-wip-us.apache.org/repos/asf/flink/blob/0a19c456/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java
index e8f3fe5..9a13412 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java
@@ -28,6 +28,8 @@ import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
 import org.apache.flink.util.StringUtils;
 import org.slf4j.Logger;
 
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
 import java.io.EOFException;
 import java.io.File;
 import java.io.IOException;
@@ -62,6 +64,11 @@ public class BlobUtils {
 	private static final String JOB_DIR_PREFIX = "job_";
 
 	/**
+	 * The prefix of all job-unrelated directories created by the BLOB server.
+	 */
+	private static final String NO_JOB_DIR_PREFIX = "no_job";
+
+	/**
 	 * Creates a BlobStore based on the parameters set in the configuration.
 	 *
 	 * @param config
@@ -116,26 +123,29 @@ public class BlobUtils {
 	}
 
 	/**
-	 * Creates a storage directory for a blob service.
+	 * Creates a local storage directory for a blob service under the given parent directory.
 	 *
-	 * @return the storage directory used by a BLOB service
+	 * @param basePath
+	 * 		base path, i.e. parent directory, of the storage directory to use (if <tt>null</tt> or
+	 * 		empty, the path in <tt>java.io.tmpdir</tt> will be used)
+	 *
+	 * @return a new local storage directory
 	 *
 	 * @throws IOException
-	 * 		thrown if the (local or distributed) file storage cannot be created or
-	 * 		is not usable
+	 * 		thrown if the local file storage cannot be created or is not usable
 	 */
-	static File initStorageDirectory(String storageDirectory) throws
-		IOException {
+	static File initLocalStorageDirectory(String basePath) throws IOException {
 		File baseDir;
-		if (StringUtils.isNullOrWhitespaceOnly(storageDirectory)) {
+		if (StringUtils.isNullOrWhitespaceOnly(basePath)) {
 			baseDir = new File(System.getProperty("java.io.tmpdir"));
 		}
 		else {
-			baseDir = new File(storageDirectory);
+			baseDir = new File(basePath);
 		}
 
 		File storageDir;
 
+		// NOTE: although we will be using UUIDs, there may be collisions
 		final int MAX_ATTEMPTS = 10;
 		for(int attempt = 0; attempt < MAX_ATTEMPTS; attempt++) {
 			storageDir = new File(baseDir, String.format(
@@ -143,7 +153,7 @@ public class BlobUtils {
 
 			// Create the storage dir if it doesn't exist. Only return it when the operation was
 			// successful.
-			if (!storageDir.exists() && storageDir.mkdirs()) {
+			if (storageDir.mkdirs()) {
 				return storageDir;
 			}
 		}
@@ -153,46 +163,108 @@ public class BlobUtils {
 	}
 
 	/**
-	 * Returns the BLOB service's directory for incoming files. The directory is created if it did
-	 * not exist so far.
+	 * Returns the BLOB service's directory for incoming (job-unrelated) files. The directory is
+	 * created if it does not exist yet.
+	 *
+	 * @param storageDir
+	 * 		storage directory used be the BLOB service
 	 *
-	 * @return the BLOB server's directory for incoming files
+	 * @return the BLOB service's directory for incoming files
 	 */
 	static File getIncomingDirectory(File storageDir) {
 		final File incomingDir = new File(storageDir, "incoming");
 
-		if (!incomingDir.mkdirs() && !incomingDir.exists()) {
-			throw new RuntimeException("Cannot create directory for incoming files " + incomingDir.getAbsolutePath());
-		}
+		mkdirTolerateExisting(incomingDir, "incoming");
 
 		return incomingDir;
 	}
 
 	/**
-	 * Returns the BLOB service's directory for cached files. The directory is created if it did
-	 * not exist so far.
+	 * Makes sure a given directory exists by creating it if necessary.
 	 *
-	 * @return the BLOB server's directory for cached files
+	 * @param dir
+	 * 		directory to create
+	 * @param dirType
+	 * 		the type of the directory (included in error message if something fails)
 	 */
-	private static File getCacheDirectory(File storageDir) {
-		final File cacheDirectory = new File(storageDir, "cache");
-
-		if (!cacheDirectory.mkdirs() && !cacheDirectory.exists()) {
-			throw new RuntimeException("Could not create cache directory '" + cacheDirectory.getAbsolutePath() + "'.");
+	private static void mkdirTolerateExisting(final File dir, final String dirType) {
+		// note: thread-safe create should try to mkdir first and then ignore the case that the
+		//       directory already existed
+		if (!dir.mkdirs() && !dir.exists()) {
+			throw new RuntimeException(
+				"Cannot create " + dirType + " directory '" + dir.getAbsolutePath() + "'.");
 		}
-
-		return cacheDirectory;
 	}
 
 	/**
 	 * Returns the (designated) physical storage location of the BLOB with the given key.
 	 *
+	 * @param storageDir
+	 * 		storage directory used be the BLOB service
 	 * @param key
-	 *        the key identifying the BLOB
+	 * 		the key identifying the BLOB
+	 * @param jobId
+	 * 		ID of the job for the incoming files (or <tt>null</tt> if job-unrelated)
+	 *
 	 * @return the (designated) physical storage location of the BLOB
 	 */
-	static File getStorageLocation(File storageDir, BlobKey key) {
-		return new File(getCacheDirectory(storageDir), BLOB_FILE_PREFIX + key.toString());
+	static File getStorageLocation(
+			@Nonnull File storageDir, @Nullable JobID jobId, @Nonnull BlobKey key) {
+		File file = new File(getStorageLocationPath(storageDir.getAbsolutePath(), jobId, key));
+
+		mkdirTolerateExisting(file.getParentFile(), "cache");
+
+		return file;
+	}
+
+	/**
+	 * Returns the BLOB server's storage directory for BLOBs belonging to the job with the given ID
+	 * <em>without</em> creating the directory.
+	 *
+	 * @param storageDir
+	 * 		storage directory used be the BLOB service
+	 * @param jobId
+	 * 		the ID of the job to return the storage directory for
+	 *
+	 * @return the storage directory for BLOBs belonging to the job with the given ID
+	 */
+	static String getStorageLocationPath(@Nonnull String storageDir, @Nullable JobID jobId) {
+		if (jobId == null) {
+			// format: $base/no_job
+			return String.format("%s/%s", storageDir, NO_JOB_DIR_PREFIX);
+		} else {
+			// format: $base/job_$jobId
+			return String.format("%s/%s%s", storageDir, JOB_DIR_PREFIX, jobId.toString());
+		}
+	}
+
+	/**
+	 * Returns the path for the given blob key.
+	 * <p>
+	 * The returned path can be used with the (local or HA) BLOB store file system back-end for
+	 * recovery purposes and follows the same scheme as {@link #getStorageLocation(File, JobID,
+	 * BlobKey)}.
+	 *
+	 * @param storageDir
+	 * 		storage directory used be the BLOB service
+	 * @param key
+	 * 		the key identifying the BLOB
+	 * @param jobId
+	 * 		ID of the job for the incoming files
+	 *
+	 * @return the path to the given BLOB
+	 */
+	static String getStorageLocationPath(
+			@Nonnull String storageDir, @Nullable JobID jobId, @Nonnull BlobKey key) {
+		if (jobId == null) {
+			// format: $base/no_job/blob_$key
+			return String.format("%s/%s/%s%s",
+				storageDir, NO_JOB_DIR_PREFIX, BLOB_FILE_PREFIX, key.toString());
+		} else {
+			// format: $base/job_$jobId/blob_$key
+			return String.format("%s/%s%s/%s%s",
+				storageDir, JOB_DIR_PREFIX, jobId.toString(), BLOB_FILE_PREFIX, key.toString());
+		}
 	}
 
 	/**
@@ -200,6 +272,7 @@ public class BlobUtils {
 	 *
 	 * @return a new instance of the message digest to use for the BLOB key computation
 	 */
+	@Nonnull
 	static MessageDigest createMessageDigest() {
 		try {
 			return MessageDigest.getInstance(HASHING_ALGORITHM);
@@ -333,28 +406,6 @@ public class BlobUtils {
 	}
 
 	/**
-	 * Returns the path for the given blob key.
-	 *
-	 * <p>The returned path can be used with the state backend for recovery purposes.
-	 *
-	 * <p>This follows the same scheme as {@link #getStorageLocation(File, BlobKey)}
-	 * and is used for HA.
-	 */
-	static String getRecoveryPath(String basePath, BlobKey blobKey) {
-		// format: $base/cache/blob_$key
-		return String.format("%s/cache/%s%s", basePath, BLOB_FILE_PREFIX, blobKey.toString());
-	}
-
-	/**
-	 * Returns the path for the given job ID.
-	 *
-	 * <p>The returned path can be used with the state backend for recovery purposes.
-	 */
-	static String getRecoveryPath(String basePath, JobID jobId) {
-		return String.format("%s/%s%s", basePath, JOB_DIR_PREFIX, jobId.toString());
-	}
-
-	/**
 	 * Private constructor to prevent instantiation.
 	 */
 	private BlobUtils() {

http://git-wip-us.apache.org/repos/asf/flink/blob/0a19c456/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobView.java
index 2e2e4a7..8916d95 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobView.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobView.java
@@ -18,6 +18,8 @@
 
 package org.apache.flink.runtime.blob;
 
+import org.apache.flink.api.common.JobID;
+
 import java.io.File;
 import java.io.IOException;
 
@@ -29,9 +31,10 @@ public interface BlobView {
 	/**
 	 * Copies a blob to a local file.
 	 *
+	 * @param jobId     ID of the job this blob belongs to (or <tt>null</tt> if job-unrelated)
 	 * @param blobKey   The blob ID
 	 * @param localFile The local file to copy to
 	 * @throws IOException If the copy fails
 	 */
-	void get(BlobKey blobKey, File localFile) throws IOException;
+	void get(JobID jobId, BlobKey blobKey, File localFile) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/0a19c456/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java
index 5f8058b..83abecb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java
@@ -64,8 +64,8 @@ public class FileSystemBlobStore implements BlobStoreService {
 	// - Put ------------------------------------------------------------------
 
 	@Override
-	public void put(File localFile, BlobKey blobKey) throws IOException {
-		put(localFile, BlobUtils.getRecoveryPath(basePath, blobKey));
+	public void put(File localFile, JobID jobId, BlobKey blobKey) throws IOException {
+		put(localFile, BlobUtils.getStorageLocationPath(basePath, jobId, blobKey));
 	}
 
 	private void put(File fromFile, String toBlobPath) throws IOException {
@@ -78,8 +78,8 @@ public class FileSystemBlobStore implements BlobStoreService {
 	// - Get ------------------------------------------------------------------
 
 	@Override
-	public void get(BlobKey blobKey, File localFile) throws IOException {
-		get(BlobUtils.getRecoveryPath(basePath, blobKey), localFile);
+	public void get(JobID jobId, BlobKey blobKey, File localFile) throws IOException {
+		get(BlobUtils.getStorageLocationPath(basePath, jobId, blobKey), localFile);
 	}
 
 	private void get(String fromBlobPath, File toFile) throws IOException {
@@ -112,13 +112,13 @@ public class FileSystemBlobStore implements BlobStoreService {
 	// - Delete ---------------------------------------------------------------
 
 	@Override
-	public void delete(BlobKey blobKey) {
-		delete(BlobUtils.getRecoveryPath(basePath, blobKey));
+	public void delete(JobID jobId, BlobKey blobKey) {
+		delete(BlobUtils.getStorageLocationPath(basePath, jobId, blobKey));
 	}
 
 	@Override
 	public void deleteAll(JobID jobId) {
-		delete(BlobUtils.getRecoveryPath(basePath, jobId));
+		delete(BlobUtils.getStorageLocationPath(basePath, jobId));
 	}
 
 	private void delete(String blobPath) {

http://git-wip-us.apache.org/repos/asf/flink/blob/0a19c456/flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java
index 6e2bb53..95be569 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java
@@ -29,16 +29,15 @@ import java.io.IOException;
 public class VoidBlobStore implements BlobStoreService {
 
 	@Override
-	public void put(File localFile, BlobKey blobKey) throws IOException {
+	public void put(File localFile, JobID jobId, BlobKey blobKey) throws IOException {
 	}
 
-
 	@Override
-	public void get(BlobKey blobKey, File localFile) throws IOException {
+	public void get(JobID jobId, BlobKey blobKey, File localFile) throws IOException {
 	}
 
 	@Override
-	public void delete(BlobKey blobKey) {
+	public void delete(JobID jobId, BlobKey blobKey) {
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/0a19c456/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java
index e3657ff..9cc6210 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java
@@ -234,6 +234,7 @@ public class JobClient {
 			int pos = 0;
 			for (BlobKey blobKey : props.requiredJarFiles()) {
 				try {
+					// TODO: make use of job-related BLOBs after adapting the BlobLibraryCacheManager
 					allURLs[pos++] = blobClient.getFile(blobKey).toURI().toURL();
 				} catch (Exception e) {
 					try {

http://git-wip-us.apache.org/repos/asf/flink/blob/0a19c456/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
index 1c68515..6b92d79 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
@@ -535,6 +535,7 @@ public class JobGraph implements Serializable {
 			InetSocketAddress blobServerAddress,
 			Configuration blobClientConfig) throws IOException {
 		if (!userJars.isEmpty()) {
+			// TODO: make use of job-related BLOBs after adapting the BlobLibraryCacheManager
 			List<BlobKey> blobKeys = BlobClient.uploadJarFiles(blobServerAddress, blobClientConfig, userJars);
 
 			for (BlobKey blobKey : blobKeys) {

http://git-wip-us.apache.org/repos/asf/flink/blob/0a19c456/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheRetriesTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheRetriesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheRetriesTest.java
index fe763fa..8c575a9 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheRetriesTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheRetriesTest.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.blob;
 
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.BlobServerOptions;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.HighAvailabilityOptions;
@@ -25,6 +26,7 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 
+import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.net.InetSocketAddress;
@@ -41,7 +43,8 @@ public class BlobCacheRetriesTest {
 	public TemporaryFolder temporaryFolder = new TemporaryFolder();
 
 	/**
-	 * A test where the connection fails twice and then the get operation succeeds.
+	 * A test where the connection fails twice and then the get operation succeeds
+	 * (job-unrelated blob).
 	 */
 	@Test
 	public void testBlobFetchRetries() throws IOException {
@@ -49,15 +52,41 @@ public class BlobCacheRetriesTest {
 		config.setString(BlobServerOptions.STORAGE_DIRECTORY,
 			temporaryFolder.newFolder().getAbsolutePath());
 
-		testBlobFetchRetries(config, new VoidBlobStore());
+		testBlobFetchRetries(config, new VoidBlobStore(), null);
+	}
+
+	/**
+	 * A test where the connection fails twice and then the get operation succeeds
+	 * (job-related blob).
+	 */
+	@Test
+	public void testBlobForJobFetchRetries() throws IOException {
+		final Configuration config = new Configuration();
+		config.setString(BlobServerOptions.STORAGE_DIRECTORY,
+			temporaryFolder.newFolder().getAbsolutePath());
+
+		testBlobFetchRetries(config, new VoidBlobStore(), new JobID());
+	}
+
+	/**
+	 * A test where the connection fails twice and then the get operation succeeds
+	 * (with high availability set, job-unrelated blob).
+	 */
+	@Test
+	public void testBlobNoJobFetchRetriesHa() throws IOException {
+		testBlobFetchRetriesHa(null);
 	}
 
 	/**
 	 * A test where the connection fails twice and then the get operation succeeds
-	 * (with high availability set).
+	 * (with high availability set, job-related job).
 	 */
 	@Test
 	public void testBlobFetchRetriesHa() throws IOException {
+		testBlobFetchRetriesHa(new JobID());
+	}
+
+	private void testBlobFetchRetriesHa(final JobID jobId) throws IOException {
 		final Configuration config = new Configuration();
 		config.setString(BlobServerOptions.STORAGE_DIRECTORY,
 			temporaryFolder.newFolder().getAbsolutePath());
@@ -70,7 +99,7 @@ public class BlobCacheRetriesTest {
 		try {
 			blobStoreService = BlobUtils.createBlobStoreFromConfig(config);
 
-			testBlobFetchRetries(config, blobStoreService);
+			testBlobFetchRetries(config, blobStoreService, jobId);
 		} finally {
 			if (blobStoreService != null) {
 				blobStoreService.closeAndCleanupAllData();
@@ -86,7 +115,9 @@ public class BlobCacheRetriesTest {
 	 * 		configuration to use (the BlobCache will get some additional settings
 	 * 		set compared to this one)
 	 */
-	private void testBlobFetchRetries(final Configuration config, final BlobStore blobStore) throws IOException {
+	private static void testBlobFetchRetries(
+			final Configuration config, final BlobStore blobStore, final JobID jobId)
+			throws IOException {
 		final byte[] data = new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 0};
 
 		BlobServer server = null;
@@ -104,7 +135,7 @@ public class BlobCacheRetriesTest {
 			try {
 				blobClient = new BlobClient(serverAddress, config);
 
-				key = blobClient.put(data);
+				key = blobClient.put(jobId, data);
 			}
 			finally {
 				if (blobClient != null) {
@@ -115,16 +146,13 @@ public class BlobCacheRetriesTest {
 			cache = new BlobCache(serverAddress, config, new VoidBlobStore());
 
 			// trigger a download - it should fail the first two times, but retry, and succeed eventually
-			URL url = cache.getFile(key).toURI().toURL();
-			InputStream is = url.openStream();
-			try {
+			File file = jobId == null ? cache.getFile(key) : cache.getFile(jobId, key);
+			URL url = file.toURI().toURL();
+			try (InputStream is = url.openStream()) {
 				byte[] received = new byte[data.length];
 				assertEquals(data.length, is.read(received));
 				assertArrayEquals(data, received);
 			}
-			finally {
-				is.close();
-			}
 		} finally {
 			if (cache != null) {
 				cache.close();
@@ -136,23 +164,50 @@ public class BlobCacheRetriesTest {
 	}
 
 	/**
-	 * A test where the connection fails too often and eventually fails the GET request.
+	 * A test where the connection fails too often and eventually fails the GET request
+	 * (job-unrelated blob).
+	 */
+	@Test
+	public void testBlobNoJobFetchWithTooManyFailures() throws IOException {
+		final Configuration config = new Configuration();
+		config.setString(BlobServerOptions.STORAGE_DIRECTORY,
+			temporaryFolder.newFolder().getAbsolutePath());
+
+		testBlobFetchWithTooManyFailures(config, new VoidBlobStore(), null);
+	}
+
+	/**
+	 * A test where the connection fails too often and eventually fails the GET request (job-related
+	 * blob).
 	 */
 	@Test
-	public void testBlobFetchWithTooManyFailures() throws IOException {
+	public void testBlobForJobFetchWithTooManyFailures() throws IOException {
 		final Configuration config = new Configuration();
 		config.setString(BlobServerOptions.STORAGE_DIRECTORY,
 			temporaryFolder.newFolder().getAbsolutePath());
 
-		testBlobFetchWithTooManyFailures(config, new VoidBlobStore());
+		testBlobFetchWithTooManyFailures(config, new VoidBlobStore(), new JobID());
+	}
+
+	/**
+	 * A test where the connection fails twice and then the get operation succeeds
+	 * (with high availability set, job-unrelated blob).
+	 */
+	@Test
+	public void testBlobNoJobFetchWithTooManyFailuresHa() throws IOException {
+		testBlobFetchWithTooManyFailuresHa(null);
 	}
 
 	/**
 	 * A test where the connection fails twice and then the get operation succeeds
-	 * (with high availability set).
+	 * (with high availability set, job-related blob).
 	 */
 	@Test
-	public void testBlobFetchWithTooManyFailuresHa() throws IOException {
+	public void testBlobForJobFetchWithTooManyFailuresHa() throws IOException {
+		testBlobFetchWithTooManyFailuresHa(new JobID());
+	}
+
+	private void testBlobFetchWithTooManyFailuresHa(final JobID jobId) throws IOException {
 		final Configuration config = new Configuration();
 		config.setString(BlobServerOptions.STORAGE_DIRECTORY,
 			temporaryFolder.newFolder().getAbsolutePath());
@@ -165,7 +220,7 @@ public class BlobCacheRetriesTest {
 		try {
 			blobStoreService = BlobUtils.createBlobStoreFromConfig(config);
 
-			testBlobFetchWithTooManyFailures(config, blobStoreService);
+			testBlobFetchWithTooManyFailures(config, blobStoreService, jobId);
 		} finally {
 			if (blobStoreService != null) {
 				blobStoreService.closeAndCleanupAllData();
@@ -181,7 +236,9 @@ public class BlobCacheRetriesTest {
 	 * 		configuration to use (the BlobCache will get some additional settings
 	 * 		set compared to this one)
 	 */
-	private void testBlobFetchWithTooManyFailures(final Configuration config, final BlobStore blobStore) throws IOException {
+	private static void testBlobFetchWithTooManyFailures(
+		final Configuration config, final BlobStore blobStore, final JobID jobId)
+			throws IOException {
 		final byte[] data = new byte[] { 1, 2, 3, 4, 5, 6, 7, 8, 9, 0 };
 
 		BlobServer server = null;
@@ -199,7 +256,7 @@ public class BlobCacheRetriesTest {
 			try {
 				blobClient = new BlobClient(serverAddress, config);
 
-				key = blobClient.put(data);
+				key = blobClient.put(jobId, data);
 			}
 			finally {
 				if (blobClient != null) {
@@ -211,7 +268,11 @@ public class BlobCacheRetriesTest {
 
 			// trigger a download - it should fail eventually
 			try {
-				cache.getFile(key);
+				if (jobId == null) {
+					cache.getFile(key);
+				} else {
+					cache.getFile(jobId, key);
+				}
 				fail("This should fail");
 			}
 			catch (IOException e) {


Mime
View raw message