From 6e4a213e077116282cbe2729274bdf50a3d02015 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Thu, 5 Oct 2017 09:33:27 +0200 Subject: [PATCH 1/6] [FLINK-7140][blob] add an additional random component into the BlobKey This should guard us from uploading (and deleting) the same file more than once and also from hash collisions. --- .../flink/runtime/blob/AbstractBlobCache.java | 18 +++- .../apache/flink/runtime/blob/BlobClient.java | 12 ++- .../apache/flink/runtime/blob/BlobKey.java | 100 ++++++++++++++++-- .../apache/flink/runtime/blob/BlobServer.java | 66 +++++++++--- .../runtime/blob/BlobServerConnection.java | 18 ++-- .../apache/flink/runtime/blob/BlobUtils.java | 12 +-- .../flink/runtime/blob/PermanentBlobKey.java | 12 +++ .../flink/runtime/blob/TransientBlobKey.java | 12 +++ .../handler/legacy/TaskManagerLogHandler.java | 3 + .../runtime/blob/BlobCacheDeleteTest.java | 24 +++-- .../flink/runtime/blob/BlobCacheGetTest.java | 33 +++--- .../flink/runtime/blob/BlobCachePutTest.java | 37 +++++-- .../runtime/blob/BlobCacheRecoveryTest.java | 12 +-- .../flink/runtime/blob/BlobClientTest.java | 53 +++++----- .../flink/runtime/blob/BlobKeyTest.java | 84 ++++++++++++--- .../runtime/blob/BlobServerDeleteTest.java | 21 ++-- .../flink/runtime/blob/BlobServerGetTest.java | 19 ++-- .../flink/runtime/blob/BlobServerPutTest.java | 33 ++++-- .../runtime/blob/BlobServerRecoveryTest.java | 7 +- 19 files changed, 405 insertions(+), 171 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/AbstractBlobCache.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/AbstractBlobCache.java index dc031e0c0c63c..729ac9ab9fb0c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/AbstractBlobCache.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/AbstractBlobCache.java @@ -159,8 +159,13 @@ protected File getFileInternal(@Nullable JobID jobId, BlobKey blobKey) throws IO try { if (blobView.get(jobId, blobKey, incomingFile)) { // now move the temp file to our local cache atomically - BlobUtils.moveTempFileToStore( - incomingFile, jobId, blobKey, localFile, readWriteLock.writeLock(), log, null); + readWriteLock.writeLock().lock(); + try { + BlobUtils.moveTempFileToStore( + incomingFile, jobId, blobKey, localFile, log, null); + } finally { + readWriteLock.writeLock().unlock(); + } return localFile; } @@ -172,8 +177,13 @@ protected File getFileInternal(@Nullable JobID jobId, BlobKey blobKey) throws IO BlobClient.downloadFromBlobServer( jobId, blobKey, incomingFile, serverAddress, blobClientConfig, numFetchRetries); - BlobUtils.moveTempFileToStore( - incomingFile, jobId, blobKey, localFile, readWriteLock.writeLock(), log, null); + readWriteLock.writeLock().lock(); + try { + BlobUtils.moveTempFileToStore( + incomingFile, jobId, blobKey, localFile, log, null); + } finally { + readWriteLock.writeLock().unlock(); + } return localFile; } finally { 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 3154f69d567d9..fbcce58d246ce 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 @@ -47,9 +47,11 @@ import java.net.Socket; import java.security.MessageDigest; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; +import static org.apache.flink.runtime.blob.BlobKey.BlobType.PERMANENT_BLOB; import static org.apache.flink.runtime.blob.BlobServerProtocol.BUFFER_SIZE; import static org.apache.flink.runtime.blob.BlobServerProtocol.GET_OPERATION; import static org.apache.flink.runtime.blob.BlobServerProtocol.JOB_RELATED_CONTENT; @@ -57,7 +59,6 @@ import static org.apache.flink.runtime.blob.BlobServerProtocol.PUT_OPERATION; import static org.apache.flink.runtime.blob.BlobServerProtocol.RETURN_ERROR; import static org.apache.flink.runtime.blob.BlobServerProtocol.RETURN_OKAY; -import static org.apache.flink.runtime.blob.BlobKey.BlobType.PERMANENT_BLOB; 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; @@ -496,13 +497,16 @@ private static BlobKey receiveAndCheckPutResponse( else if (response == RETURN_OKAY) { BlobKey remoteKey = BlobKey.readFromInputStream(is); - BlobKey localKey = BlobKey.createKey(blobType, md.digest()); + byte[] localHash = md.digest(); - if (!localKey.equals(remoteKey)) { + if (blobType != remoteKey.getType()) { + throw new IOException("Detected data corruption during transfer"); + } + if (!Arrays.equals(localHash, remoteKey.getHash())) { throw new IOException("Detected data corruption during transfer"); } - return localKey; + return remoteKey; } else if (response == RETURN_ERROR) { Throwable cause = readExceptionFromStream(is); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobKey.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobKey.java index 0aa45e169bed7..ef2d64db3c0f1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobKey.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobKey.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.blob; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.util.AbstractID; import org.apache.flink.util.StringUtils; import java.io.EOFException; @@ -41,7 +42,7 @@ abstract class BlobKey implements Serializable, Comparable { private static final long serialVersionUID = 3847117712521785209L; /** Size of the internal BLOB key in bytes. */ - private static final int SIZE = 20; + public static final int SIZE = 20; /** The byte buffer storing the actual key data. */ private final byte[] key; @@ -67,6 +68,11 @@ enum BlobType { TRANSIENT_BLOB } + /** + * Random component of the key. + */ + private final AbstractID random; + /** * Constructs a new BLOB key. * @@ -76,6 +82,7 @@ enum BlobType { protected BlobKey(BlobType type) { this.type = checkNotNull(type); this.key = new byte[SIZE]; + this.random = new AbstractID(); } /** @@ -87,13 +94,33 @@ protected BlobKey(BlobType type) { * the actual key data */ protected BlobKey(BlobType type, byte[] key) { + if (key == null || key.length != SIZE) { + throw new IllegalArgumentException("BLOB key must have a size of " + SIZE + " bytes"); + } + this.type = checkNotNull(type); + this.key = key; + this.random = new AbstractID(); + } + /** + * Constructs a new BLOB key from the given byte array. + * + * @param type + * whether the referenced BLOB is permanent or transient + * @param key + * the actual key data + * @param random + * the random component of the key + */ + protected BlobKey(BlobType type, byte[] key, byte[] random) { if (key == null || key.length != SIZE) { throw new IllegalArgumentException("BLOB key must have a size of " + SIZE + " bytes"); } + this.type = checkNotNull(type); this.key = key; + this.random = new AbstractID(random); } /** @@ -107,10 +134,10 @@ protected BlobKey(BlobType type, byte[] key) { @VisibleForTesting static BlobKey createKey(BlobType type) { if (type == PERMANENT_BLOB) { - return new PermanentBlobKey(); - } else { + return new PermanentBlobKey(); + } else { return new TransientBlobKey(); - } + } } /** @@ -125,10 +152,30 @@ static BlobKey createKey(BlobType type) { */ static BlobKey createKey(BlobType type, byte[] key) { if (type == PERMANENT_BLOB) { - return new PermanentBlobKey(key); - } else { + return new PermanentBlobKey(key); + } else { return new TransientBlobKey(key); - } + } + } + + /** + * Returns the right {@link BlobKey} subclass for the given parameters. + * + * @param type + * whether the referenced BLOB is permanent or transient + * @param key + * the actual key data + * @param random + * the random component of the key + * + * @return BlobKey subclass + */ + static BlobKey createKey(BlobType type, byte[] key, byte[] random) { + if (type == PERMANENT_BLOB) { + return new PermanentBlobKey(key, random); + } else { + return new TransientBlobKey(key, random); + } } /** @@ -140,6 +187,15 @@ byte[] getHash() { return key; } + /** + * Returns the (internal) BLOB type which is reflected by the inheriting sub-class. + * + * @return BLOB type, i.e. permanent or transient + */ + BlobType getType() { + return type; + } + /** * Adds the BLOB key to the given {@link MessageDigest}. * @@ -159,13 +215,16 @@ public boolean equals(final Object obj) { final BlobKey bk = (BlobKey) obj; - return Arrays.equals(this.key, bk.key) && this.type == bk.type; + return Arrays.equals(this.key, bk.key) && + this.type == bk.type && + this.random.equals(bk.random); } @Override public int hashCode() { int result = Arrays.hashCode(this.key); result = 37 * result + this.type.hashCode(); + result = 37 * result + this.random.hashCode(); return result; } @@ -183,7 +242,7 @@ public String toString() { // this actually never happens! throw new IllegalStateException("Invalid BLOB type"); } - return typeString + StringUtils.byteToHexString(this.key); + return typeString + StringUtils.byteToHexString(this.key) + "-" + random.toString(); } @Override @@ -203,7 +262,13 @@ public int compareTo(BlobKey o) { if (aarr.length == barr.length) { // same hash contents - compare the BLOB types - return this.type.compareTo(o.type); + int typeCompare = this.type.compareTo(o.type); + if (typeCompare == 0) { + // same type - compare random components + return this.random.compareTo(o.random); + } else { + return typeCompare; + } } else { return aarr.length - barr.length; } @@ -223,6 +288,7 @@ public int compareTo(BlobKey o) { static BlobKey readFromInputStream(InputStream inputStream) throws IOException { final byte[] key = new byte[BlobKey.SIZE]; + final byte[] random = new byte[AbstractID.SIZE]; int bytesRead = 0; // read key @@ -233,6 +299,7 @@ static BlobKey readFromInputStream(InputStream inputStream) throws IOException { } bytesRead += read; } + // read BLOB type final BlobType blobType; { @@ -248,7 +315,17 @@ static BlobKey readFromInputStream(InputStream inputStream) throws IOException { } } - return createKey(blobType, key); + // read random component + bytesRead = 0; + while (bytesRead < AbstractID.SIZE) { + final int read = inputStream.read(random, bytesRead, AbstractID.SIZE - bytesRead); + if (read < 0) { + throw new EOFException("Read an incomplete BLOB key"); + } + bytesRead += read; + } + + return createKey(blobType, key, random); } /** @@ -262,5 +339,6 @@ static BlobKey readFromInputStream(InputStream inputStream) throws IOException { void writeToOutputStream(final OutputStream outputStream) throws IOException { outputStream.write(this.key); outputStream.write(this.type.ordinal()); + outputStream.write(this.random.getBytes()); } } 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 7804dfd73e34c..bc61ef76c9f80 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 @@ -474,8 +474,13 @@ void getFileInternal(@Nullable JobID jobId, BlobKey blobKey, File localFile) thr incomingFile = createTemporaryFilename(); blobStore.get(jobId, blobKey, incomingFile); - BlobUtils.moveTempFileToStore( - incomingFile, jobId, blobKey, localFile, readWriteLock.writeLock(), LOG, null); + readWriteLock.writeLock().lock(); + try { + BlobUtils.moveTempFileToStore( + incomingFile, jobId, blobKey, localFile, LOG, null); + } finally { + readWriteLock.writeLock().unlock(); + } return; } finally { @@ -586,10 +591,8 @@ private BlobKey putBuffer(@Nullable JobID jobId, byte[] value, BlobKey.BlobType md.update(value); fos.write(value); - blobKey = BlobKey.createKey(blobType, md.digest()); - // persist file - moveTempFileToStore(incomingFile, jobId, blobKey); + blobKey = moveTempFileToStore(incomingFile, jobId, md.digest(), blobType); return blobKey; } finally { @@ -642,10 +645,8 @@ private BlobKey putInputStream( md.update(buf, 0, bytesRead); } - blobKey = BlobKey.createKey(blobType, md.digest()); - // persist file - moveTempFileToStore(incomingFile, jobId, blobKey); + blobKey = moveTempFileToStore(incomingFile, jobId, md.digest(), blobType); return blobKey; } finally { @@ -665,20 +666,53 @@ private BlobKey putInputStream( * temporary file created during transfer * @param jobId * ID of the job this blob belongs to or null if job-unrelated - * @param blobKey - * BLOB key identifying the file + * @param digest + * BLOB content digest, i.e. hash + * @param blobType + * whether this file is a permanent or transient BLOB + * + * @return unique BLOB key that identifies the BLOB on the server * * @throws IOException * thrown if an I/O error occurs while moving the file or uploading it to the HA store */ - void moveTempFileToStore( - File incomingFile, @Nullable JobID jobId, BlobKey blobKey) throws IOException { + BlobKey moveTempFileToStore( + File incomingFile, @Nullable JobID jobId, byte[] digest, BlobKey.BlobType blobType) + throws IOException { + + int retries = 10; - File storageFile = BlobUtils.getStorageLocation(storageDir, jobId, blobKey); + int attempt = 0; + while (true) { + // add unique component independent of the BLOB content + BlobKey blobKey = BlobKey.createKey(blobType, digest); + File storageFile = BlobUtils.getStorageLocation(storageDir, jobId, blobKey); - BlobUtils.moveTempFileToStore( - incomingFile, jobId, blobKey, storageFile, readWriteLock.writeLock(), LOG, - blobKey instanceof PermanentBlobKey ? blobStore : null); + // try again until the key is unique (put the existence check into the lock!) + readWriteLock.writeLock().lock(); + try { + if (!storageFile.exists()) { + BlobUtils.moveTempFileToStore( + incomingFile, jobId, blobKey, storageFile, LOG, + blobKey instanceof PermanentBlobKey ? blobStore : null); + return blobKey; + } + } finally { + readWriteLock.writeLock().unlock(); + } + + ++attempt; + if (attempt >= retries) { + String message = "Failed to find a unique key for BLOB of job " + jobId + " (last tried " + storageFile.getAbsolutePath() + "."; + LOG.error(message + " No retries left."); + throw new IOException(message); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Trying to find a unique key for BLOB of job {} (retry {}, last tried {})", + jobId, attempt, storageFile.getAbsolutePath()); + } + } + } } /** 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 be625811da547..fa8427e1fd660 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 @@ -37,6 +37,8 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; +import static org.apache.flink.runtime.blob.BlobKey.BlobType.PERMANENT_BLOB; +import static org.apache.flink.runtime.blob.BlobKey.BlobType.TRANSIENT_BLOB; import static org.apache.flink.runtime.blob.BlobServerProtocol.BUFFER_SIZE; import static org.apache.flink.runtime.blob.BlobServerProtocol.GET_OPERATION; import static org.apache.flink.runtime.blob.BlobServerProtocol.JOB_RELATED_CONTENT; @@ -44,8 +46,6 @@ import static org.apache.flink.runtime.blob.BlobServerProtocol.PUT_OPERATION; import static org.apache.flink.runtime.blob.BlobServerProtocol.RETURN_ERROR; import static org.apache.flink.runtime.blob.BlobServerProtocol.RETURN_OKAY; -import static org.apache.flink.runtime.blob.BlobKey.BlobType.PERMANENT_BLOB; -import static org.apache.flink.runtime.blob.BlobKey.BlobType.TRANSIENT_BLOB; 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; @@ -346,9 +346,9 @@ private void put(InputStream inputStream, OutputStream outputStream, byte[] buf) } incomingFile = blobServer.createTemporaryFilename(); - BlobKey blobKey = readFileFully(inputStream, incomingFile, buf, blobType); + byte[] digest = readFileFully(inputStream, incomingFile, buf); - blobServer.moveTempFileToStore(incomingFile, jobId, blobKey); + BlobKey blobKey = blobServer.moveTempFileToStore(incomingFile, jobId, digest, blobType); // Return computed key to client for validation outputStream.write(RETURN_OKAY); @@ -387,16 +387,14 @@ private void put(InputStream inputStream, OutputStream outputStream, byte[] buf) * file to write to * @param buf * An auxiliary buffer for data serialization/deserialization - * @param blobType - * whether to make the data permanent or transient * - * @return the received file's content hash as a BLOB key + * @return the received file's content hash * * @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, BlobKey.BlobType blobType) + private static byte[] readFileFully( + final InputStream inputStream, final File incomingFile, final byte[] buf) throws IOException { MessageDigest md = BlobUtils.createMessageDigest(); @@ -417,7 +415,7 @@ private static BlobKey readFileFully( md.update(buf, 0, bytesExpected); } - return BlobKey.createKey(blobType, md.digest()); + return md.digest(); } } 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 d8223c8b77680..04f2cdb7a3e7b 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 @@ -30,6 +30,7 @@ import org.slf4j.Logger; import javax.annotation.Nullable; + import java.io.Closeable; import java.io.EOFException; import java.io.File; @@ -42,7 +43,6 @@ import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; import java.util.UUID; -import java.util.concurrent.locks.Lock; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly; @@ -422,7 +422,7 @@ private BlobUtils() { /** * Moves the temporary incomingFile to its permanent location where it is available for - * use. + * use (not thread-safe!). * * @param incomingFile * temporary file created during transfer @@ -432,8 +432,6 @@ private BlobUtils() { * BLOB key identifying the file * @param storageFile * (local) file where the blob is/should be stored - * @param writeLock - * lock to acquire before doing the move * @param log * logger for debug information * @param blobStore @@ -444,9 +442,7 @@ private BlobUtils() { */ static void moveTempFileToStore( File incomingFile, @Nullable JobID jobId, BlobKey blobKey, File storageFile, - Lock writeLock, Logger log, @Nullable BlobStore blobStore) throws IOException { - - writeLock.lock(); + Logger log, @Nullable BlobStore blobStore) throws IOException { try { // first check whether the file already exists @@ -483,8 +479,6 @@ static void moveTempFileToStore( if (incomingFile != null && !incomingFile.delete() && incomingFile.exists()) { log.warn("Could not delete the staging file {} for blob key {} and job {}.", incomingFile, blobKey, jobId); } - - writeLock.unlock(); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/PermanentBlobKey.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/PermanentBlobKey.java index 2ad8f7268412a..40732fad51474 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/PermanentBlobKey.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/PermanentBlobKey.java @@ -42,4 +42,16 @@ public PermanentBlobKey() { PermanentBlobKey(byte[] key) { super(BlobType.PERMANENT_BLOB, key); } + + /** + * Constructs a new BLOB key from the given byte array. + * + * @param key + * the actual key data + * @param random + * the random component of the key + */ + PermanentBlobKey(byte[] key, byte[] random) { + super(BlobType.PERMANENT_BLOB, key, random); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/TransientBlobKey.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/TransientBlobKey.java index 43e0f5f50b29d..15a9637244153 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/TransientBlobKey.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/TransientBlobKey.java @@ -42,4 +42,16 @@ public TransientBlobKey() { TransientBlobKey(byte[] key) { super(BlobType.TRANSIENT_BLOB, key); } + + /** + * Constructs a new BLOB key from the given byte array. + * + * @param key + * the actual key data + * @param random + * the random component of the key + */ + TransientBlobKey(byte[] key, byte[] random) { + super(BlobType.TRANSIENT_BLOB, key, random); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagerLogHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagerLogHandler.java index 54725e13e6c92..cf5bfcb83b3a0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagerLogHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagerLogHandler.java @@ -190,6 +190,9 @@ protected void respondAsLeader(final ChannelHandlerContext ctx, final Routed rou //delete previous log file, if it is different than the current one HashMap lastSubmittedFile = fileMode == FileMode.LOG ? lastSubmittedLog : lastSubmittedStdout; if (lastSubmittedFile.containsKey(taskManagerID)) { + // the BlobKey will almost certainly be different but the old file + // may not exist anymore so we cannot rely on it and need to + // download the new file anyway, even if the hashes match if (!Objects.equals(blobKey, lastSubmittedFile.get(taskManagerID))) { if (!blobCache.deleteFromCache(lastSubmittedFile.get(taskManagerID))) { throw new CompletionException(new FlinkException("Could not delete file for " + taskManagerID + '.')); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheDeleteTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheDeleteTest.java index 8ea2a5ea4331a..37a5128f561e1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheDeleteTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheDeleteTest.java @@ -50,7 +50,6 @@ import static org.apache.flink.runtime.blob.BlobServerGetTest.verifyDeleted; import static org.apache.flink.runtime.blob.BlobServerPutTest.put; import static org.apache.flink.runtime.blob.BlobServerPutTest.verifyContents; -import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; @@ -86,6 +85,12 @@ public void testDeleteTransient4() throws IOException { testDelete(new JobID(), new JobID()); } + @Test + public void testDeleteTransient5() throws IOException { + JobID jobId = new JobID(); + testDelete(jobId, jobId); + } + /** * Uploads a (different) byte array for each of the given jobs and verifies that deleting one of * them (via the {@link BlobCacheService}) does not influence the other. @@ -97,7 +102,6 @@ public void testDeleteTransient4() throws IOException { */ private void testDelete(@Nullable JobID jobId1, @Nullable JobID jobId2) throws IOException { - final boolean sameJobId = (jobId1 == jobId2) || (jobId1 != null && jobId1.equals(jobId2)); final Configuration config = new Configuration(); config.setString(BlobServerOptions.STORAGE_DIRECTORY, temporaryFolder.newFolder().getAbsolutePath()); @@ -121,9 +125,10 @@ config, new VoidBlobStore())) { // put two more BLOBs (same key, other key) for another job ID TransientBlobKey key2a = (TransientBlobKey) put(server, jobId2, data, TRANSIENT_BLOB); assertNotNull(key2a); - assertEquals(key1, key2a); + BlobKeyTest.verifyKeyDifferentHashEquals(key1, key2a); TransientBlobKey key2b = (TransientBlobKey) put(server, jobId2, data2, TRANSIENT_BLOB); assertNotNull(key2b); + BlobKeyTest.verifyKeyDifferentHashDifferent(key1, key2b); // issue a DELETE request assertTrue(delete(cache, jobId1, key1)); @@ -133,16 +138,15 @@ config, new VoidBlobStore())) { // delete on server so that the cache cannot re-download assertTrue(server.deleteInternal(jobId1, key1)); verifyDeleted(cache, jobId1, key1); - // deleting a one BLOB should not affect another BLOB, even with the same key if job IDs are different - if (!sameJobId) { - verifyContents(server, jobId2, key2a, data); - } + // deleting one BLOB should not affect another BLOB with a different key + // (and keys are always different now) + verifyContents(server, jobId2, key2a, data); verifyContents(server, jobId2, key2b, data2); // delete first file of second job assertTrue(delete(cache, jobId2, key2a)); - // delete only works on local cache (unless already deleted - key1 == key2a)! - assertTrue(sameJobId || server.getStorageLocation(jobId2, key2a).exists()); + // delete only works on local cache + assertTrue(server.getStorageLocation(jobId2, key2a).exists()); // delete on server so that the cache cannot re-download assertTrue(server.deleteInternal(jobId2, key2a)); verifyDeleted(cache, jobId2, key2a); @@ -150,7 +154,7 @@ config, new VoidBlobStore())) { // delete second file of second job assertTrue(delete(cache, jobId2, key2b)); - // delete only works on local cache (unless already deleted - key1 == key2a)! + // delete only works on local cache assertTrue(server.getStorageLocation(jobId2, key2b).exists()); // delete on server so that the cache cannot re-download assertTrue(server.deleteInternal(jobId2, key2b)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheGetTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheGetTest.java index bed27d8b9f705..c760d04cbd188 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheGetTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheGetTest.java @@ -40,7 +40,6 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.nio.file.AccessDeniedException; -import java.security.MessageDigest; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -57,6 +56,7 @@ import static org.apache.flink.runtime.blob.BlobClientTest.validateGetAndClose; import static org.apache.flink.runtime.blob.BlobKey.BlobType.PERMANENT_BLOB; import static org.apache.flink.runtime.blob.BlobKey.BlobType.TRANSIENT_BLOB; +import static org.apache.flink.runtime.blob.BlobKeyTest.verifyKeyDifferentHashEquals; import static org.apache.flink.runtime.blob.BlobKeyTest.verifyType; import static org.apache.flink.runtime.blob.BlobServerDeleteTest.delete; import static org.apache.flink.runtime.blob.BlobServerGetTest.get; @@ -152,37 +152,37 @@ config, new VoidBlobStore())) { // add the same data under a second jobId BlobKey key2 = put(server, jobId2, data, blobType); - assertNotNull(key); - assertEquals(key, key2); + assertNotNull(key2); + verifyKeyDifferentHashEquals(key, key2); // request for jobId2 should succeed - get(cache, jobId2, key); + get(cache, jobId2, key2); // request for jobId1 should still fail verifyDeleted(cache, jobId1, key); if (blobType == PERMANENT_BLOB) { // still existing on server - assertTrue(server.getStorageLocation(jobId2, key).exists()); + assertTrue(server.getStorageLocation(jobId2, key2).exists()); // delete jobId2 on cache - blobFile = cache.getPermanentBlobService().getStorageLocation(jobId2, key); + blobFile = cache.getPermanentBlobService().getStorageLocation(jobId2, key2); assertTrue(blobFile.delete()); // try to retrieve again - get(cache, jobId2, key); + get(cache, jobId2, key2); // delete on cache and server, verify that it is not accessible anymore - blobFile = cache.getPermanentBlobService().getStorageLocation(jobId2, key); + blobFile = cache.getPermanentBlobService().getStorageLocation(jobId2, key2); assertTrue(blobFile.delete()); - blobFile = server.getStorageLocation(jobId2, key); + blobFile = server.getStorageLocation(jobId2, key2); assertTrue(blobFile.delete()); - verifyDeleted(cache, jobId2, key); + verifyDeleted(cache, jobId2, key2); } else { // deleted eventually on the server by the GET request above - verifyDeletedEventually(server, jobId2, key); + verifyDeletedEventually(server, jobId2, key2); // delete jobId2 on cache - blobFile = cache.getTransientBlobService().getStorageLocation(jobId2, key); + blobFile = cache.getTransientBlobService().getStorageLocation(jobId2, key2); assertTrue(blobFile.delete()); // verify that it is not accessible anymore - verifyDeleted(cache, jobId2, key); + verifyDeleted(cache, jobId2, key2); } } } @@ -548,11 +548,6 @@ private void testConcurrentGetOperations(final JobID jobId, final BlobKey.BlobTy final byte[] data = {1, 2, 3, 4, 99, 42}; - MessageDigest md = BlobUtils.createMessageDigest(); - - // create the correct blob key by hashing our input data - final BlobKey blobKey = BlobKey.createKey(blobType, md.digest(data)); - final ExecutorService executor = Executors.newFixedThreadPool(numberConcurrentGetOperations); try ( @@ -563,7 +558,7 @@ private void testConcurrentGetOperations(final JobID jobId, final BlobKey.BlobTy server.start(); // upload data first - assertEquals(blobKey, put(server, jobId, data, blobType)); + final BlobKey blobKey = put(server, jobId, data, blobType); // now try accessing it concurrently (only HA mode will be able to retrieve it from HA store!) for (int i = 0; i < numberConcurrentGetOperations; i++) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCachePutTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCachePutTest.java index aa23c8095ca48..56258c30c9cd7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCachePutTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCachePutTest.java @@ -58,6 +58,7 @@ import static org.apache.flink.runtime.blob.BlobCacheCleanupTest.checkFilesExist; import static org.apache.flink.runtime.blob.BlobKey.BlobType.PERMANENT_BLOB; import static org.apache.flink.runtime.blob.BlobKey.BlobType.TRANSIENT_BLOB; +import static org.apache.flink.runtime.blob.BlobKeyTest.verifyKeyDifferentHashEquals; import static org.apache.flink.runtime.blob.BlobKeyTest.verifyType; import static org.apache.flink.runtime.blob.BlobServerGetTest.verifyDeleted; import static org.apache.flink.runtime.blob.BlobServerPutTest.BlockingInputStream; @@ -275,6 +276,11 @@ config, new VoidBlobStore())) { BlobKey key1a = put(cache, jobId1, data, blobType); assertNotNull(key1a); verifyType(blobType, key1a); + // second upload of same data should yield a different BlobKey + BlobKey key1a2 = put(cache, jobId1, data, blobType); + assertNotNull(key1a2); + verifyType(blobType, key1a2); + verifyKeyDifferentHashEquals(key1a, key1a2); BlobKey key1b = put(cache, jobId1, data2, blobType); assertNotNull(key1b); @@ -282,19 +288,23 @@ config, new VoidBlobStore())) { // files should be available on the server verifyContents(server, jobId1, key1a, data); + verifyContents(server, jobId1, key1a2, data); verifyContents(server, jobId1, key1b, data2); // now put data for jobId2 and verify that both are ok BlobKey key2a = put(cache, jobId2, data, blobType); assertNotNull(key2a); - assertEquals(key1a, key2a); + verifyType(blobType, key2a); + verifyKeyDifferentHashEquals(key1a, key2a); BlobKey key2b = put(cache, jobId2, data2, blobType); assertNotNull(key2b); - assertEquals(key1b, key2b); + verifyType(blobType, key2b); + verifyKeyDifferentHashEquals(key1b, key2b); // verify the accessibility and the BLOB contents verifyContents(server, jobId1, key1a, data); + verifyContents(server, jobId1, key1a2, data); verifyContents(server, jobId1, key1b, data2); verifyContents(server, jobId2, key2a, data); verifyContents(server, jobId2, key2b, data2); @@ -382,26 +392,32 @@ config, new VoidBlobStore())) { TransientBlobKey key1a = (TransientBlobKey) put(cache, jobId1, new ByteArrayInputStream(data), TRANSIENT_BLOB); assertNotNull(key1a); + // second upload of same data should yield a different BlobKey + BlobKey key1a2 = put(cache, jobId1, new ByteArrayInputStream(data), TRANSIENT_BLOB); + assertNotNull(key1a2); + verifyKeyDifferentHashEquals(key1a, key1a2); TransientBlobKey key1b = (TransientBlobKey) put(cache, jobId1, new ByteArrayInputStream(data2), TRANSIENT_BLOB); assertNotNull(key1b); // files should be available on the server verifyContents(server, jobId1, key1a, data); + verifyContents(server, jobId1, key1a2, data); verifyContents(server, jobId1, key1b, data2); // now put data for jobId2 and verify that both are ok TransientBlobKey key2a = (TransientBlobKey) put(cache, jobId2, new ByteArrayInputStream(data), TRANSIENT_BLOB); assertNotNull(key2a); - assertEquals(key1a, key2a); + verifyKeyDifferentHashEquals(key1a, key2a); TransientBlobKey key2b = (TransientBlobKey) put(cache, jobId2, new ByteArrayInputStream(data2), TRANSIENT_BLOB); assertNotNull(key2b); - assertEquals(key1b, key2b); + verifyKeyDifferentHashEquals(key1b, key2b); // verify the accessibility and the BLOB contents verifyContents(server, jobId1, key1a, data); + verifyContents(server, jobId1, key1a2, data); verifyContents(server, jobId1, key1b, data2); verifyContents(server, jobId2, key2a, data); verifyContents(server, jobId2, key2b, data2); @@ -486,6 +502,10 @@ config, new VoidBlobStore())) { TransientBlobKey key1a = (TransientBlobKey) put(cache, jobId1, new ChunkedInputStream(data, 19), TRANSIENT_BLOB); assertNotNull(key1a); + // second upload of same data should yield a different BlobKey + BlobKey key1a2 = put(cache, jobId1, new ChunkedInputStream(data, 19), TRANSIENT_BLOB); + assertNotNull(key1a2); + verifyKeyDifferentHashEquals(key1a, key1a2); TransientBlobKey key1b = (TransientBlobKey) put(cache, jobId1, new ChunkedInputStream(data2, 19), TRANSIENT_BLOB); @@ -493,21 +513,23 @@ config, new VoidBlobStore())) { // files should be available on the server verifyContents(server, jobId1, key1a, data); + verifyContents(server, jobId1, key1a2, data); verifyContents(server, jobId1, key1b, data2); // now put data for jobId2 and verify that both are ok TransientBlobKey key2a = (TransientBlobKey) put(cache, jobId2, new ChunkedInputStream(data, 19), TRANSIENT_BLOB); assertNotNull(key2a); - assertEquals(key1a, key2a); + verifyKeyDifferentHashEquals(key1a, key2a); TransientBlobKey key2b = (TransientBlobKey) put(cache, jobId2, new ChunkedInputStream(data2, 19), TRANSIENT_BLOB); assertNotNull(key2b); - assertEquals(key1b, key2b); + verifyKeyDifferentHashEquals(key1b, key2b); // verify the accessibility and the BLOB contents verifyContents(server, jobId1, key1a, data); + verifyContents(server, jobId1, key1a2, data); verifyContents(server, jobId1, key1b, data2); verifyContents(server, jobId2, key2a, data); verifyContents(server, jobId2, key2b, data2); @@ -858,7 +880,8 @@ private void testConcurrentPutOperations( // make sure that all blob keys are the same while (blobKeyIterator.hasNext()) { - assertEquals(blobKey, blobKeyIterator.next()); + // check for unique BlobKey, but should have same hash + verifyKeyDifferentHashEquals(blobKey, blobKeyIterator.next()); } // check the uploaded file's contents diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheRecoveryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheRecoveryTest.java index 1a3f161725f90..e275949db1656 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheRecoveryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheRecoveryTest.java @@ -38,13 +38,11 @@ import static org.apache.flink.runtime.blob.BlobKey.BlobType.PERMANENT_BLOB; import static org.apache.flink.runtime.blob.BlobKey.BlobType.TRANSIENT_BLOB; +import static org.apache.flink.runtime.blob.BlobKeyTest.verifyKeyDifferentHashDifferent; +import static org.apache.flink.runtime.blob.BlobKeyTest.verifyKeyDifferentHashEquals; import static org.apache.flink.runtime.blob.BlobServerGetTest.verifyDeleted; import static org.apache.flink.runtime.blob.BlobServerPutTest.put; import static org.apache.flink.runtime.blob.BlobServerPutTest.verifyContents; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.not; -import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; /** @@ -131,10 +129,8 @@ public static void testBlobCacheRecovery( // put non-HA data nonHAKey = put(cache0, jobId[0], expected2, TRANSIENT_BLOB); - assertNotEquals(keys[0], nonHAKey); - assertThat(keys[0].getHash(), not(equalTo(nonHAKey.getHash()))); - assertNotEquals(keys[1], nonHAKey); - assertThat(keys[1].getHash(), equalTo(nonHAKey.getHash())); + verifyKeyDifferentHashDifferent(keys[0], nonHAKey); + verifyKeyDifferentHashEquals(keys[1], nonHAKey); // check that the storage directory exists final Path blobServerPath = new Path(storagePath, "blob"); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobClientTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobClientTest.java index 0e97604425a8d..9e4f4b71f8069 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobClientTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobClientTest.java @@ -47,6 +47,7 @@ import static org.apache.flink.runtime.blob.BlobCachePutTest.verifyDeletedEventually; import static org.apache.flink.runtime.blob.BlobKey.BlobType.PERMANENT_BLOB; import static org.apache.flink.runtime.blob.BlobKey.BlobType.TRANSIENT_BLOB; +import static org.apache.flink.runtime.blob.BlobKeyTest.verifyKeyDifferentHashEquals; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; @@ -113,15 +114,13 @@ private static byte[] createTestBuffer() { * * @param file * the file to prepare for the unit tests - * @param blobType - * whether the BLOB should become permanent or transient * * @return the BLOB key of the prepared file * * @throws IOException * thrown if an I/O error occurs while writing to the test file */ - private static BlobKey prepareTestFile(File file, BlobKey.BlobType blobType) throws IOException { + private static byte[] prepareTestFile(File file) throws IOException { MessageDigest md = BlobUtils.createMessageDigest(); @@ -145,7 +144,7 @@ private static BlobKey prepareTestFile(File file, BlobKey.BlobType blobType) thr } } - return BlobKey.createKey(blobType, md.digest()); + return md.digest(); } /** @@ -254,35 +253,38 @@ private void testContentAddressableBuffer(BlobKey.BlobType blobType) byte[] testBuffer = createTestBuffer(); MessageDigest md = BlobUtils.createMessageDigest(); md.update(testBuffer); - BlobKey origKey = BlobKey.createKey(blobType, md.digest()); + byte[] digest = md.digest(); InetSocketAddress serverAddress = new InetSocketAddress("localhost", getBlobServer().getPort()); client = new BlobClient(serverAddress, getBlobClientConfig()); JobID jobId = new JobID(); - BlobKey receivedKey; // Store the data (job-unrelated) + BlobKey receivedKey1 = null; if (blobType == TRANSIENT_BLOB) { - receivedKey = client.putBuffer(null, testBuffer, 0, testBuffer.length, blobType); - assertEquals(origKey, receivedKey); + receivedKey1 = client.putBuffer(null, testBuffer, 0, testBuffer.length, blobType); + assertArrayEquals(digest, receivedKey1.getHash()); } // try again with a job-related BLOB: - receivedKey = client.putBuffer(jobId, testBuffer, 0, testBuffer.length, blobType); - assertEquals(origKey, receivedKey); + BlobKey receivedKey2 = client.putBuffer(jobId, testBuffer, 0, testBuffer.length, blobType); + assertArrayEquals(digest, receivedKey2.getHash()); + if (blobType == TRANSIENT_BLOB) { + verifyKeyDifferentHashEquals(receivedKey1, receivedKey2); + } // Retrieve the data (job-unrelated) if (blobType == TRANSIENT_BLOB) { - validateGetAndClose(client.getInternal(null, receivedKey), testBuffer); + validateGetAndClose(client.getInternal(null, receivedKey1), testBuffer); // transient BLOBs should be deleted from the server, eventually - verifyDeletedEventually(getBlobServer(), null, receivedKey); + verifyDeletedEventually(getBlobServer(), null, receivedKey1); } // job-related - validateGetAndClose(client.getInternal(jobId, receivedKey), testBuffer); + validateGetAndClose(client.getInternal(jobId, receivedKey2), testBuffer); if (blobType == TRANSIENT_BLOB) { // transient BLOBs should be deleted from the server, eventually - verifyDeletedEventually(getBlobServer(), jobId, receivedKey); + verifyDeletedEventually(getBlobServer(), jobId, receivedKey2); } // Check reaction to invalid keys for job-unrelated blobs @@ -342,41 +344,42 @@ private void testContentAddressableStream(BlobKey.BlobType blobType) throws IOException, InterruptedException { File testFile = temporaryFolder.newFile(); - BlobKey origKey = prepareTestFile(testFile, blobType); + byte[] digest = prepareTestFile(testFile); InputStream is = null; try (BlobClient client = new BlobClient(new InetSocketAddress("localhost", getBlobServer().getPort()), getBlobClientConfig())) { JobID jobId = new JobID(); - BlobKey receivedKey; + BlobKey receivedKey1 = null; // Store the data (job-unrelated) if (blobType == TRANSIENT_BLOB) { is = new FileInputStream(testFile); - receivedKey = client.putInputStream(null, is, blobType); - assertEquals(origKey, receivedKey); + receivedKey1 = client.putInputStream(null, is, blobType); + assertArrayEquals(digest, receivedKey1.getHash()); } // try again with a job-related BLOB: is = new FileInputStream(testFile); - receivedKey = client.putInputStream(jobId, is, blobType); - assertEquals(origKey, receivedKey); + BlobKey receivedKey2 = client.putInputStream(jobId, is, blobType); is.close(); is = null; // Retrieve the data (job-unrelated) if (blobType == TRANSIENT_BLOB) { - validateGetAndClose(client.getInternal(null, receivedKey), testFile); + verifyKeyDifferentHashEquals(receivedKey1, receivedKey2); + + validateGetAndClose(client.getInternal(null, receivedKey1), testFile); // transient BLOBs should be deleted from the server, eventually - verifyDeletedEventually(getBlobServer(), null, receivedKey); + verifyDeletedEventually(getBlobServer(), null, receivedKey1); } // job-related - validateGetAndClose(client.getInternal(jobId, receivedKey), testFile); + validateGetAndClose(client.getInternal(jobId, receivedKey2), testFile); if (blobType == TRANSIENT_BLOB) { // transient BLOBs should be deleted from the server, eventually - verifyDeletedEventually(getBlobServer(), jobId, receivedKey); + verifyDeletedEventually(getBlobServer(), jobId, receivedKey2); } } finally { if (is != null) { @@ -463,7 +466,7 @@ public void testUploadJarFilesHelper() throws Exception { static void uploadJarFile(BlobServer blobServer, Configuration blobClientConfig) throws Exception { final File testFile = File.createTempFile("testfile", ".dat"); testFile.deleteOnExit(); - prepareTestFile(testFile, PERMANENT_BLOB); + prepareTestFile(testFile); InetSocketAddress serverAddress = new InetSocketAddress("localhost", blobServer.getPort()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobKeyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobKeyTest.java index 49f4fc2b457c3..ae538aa30181e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobKeyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobKeyTest.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.blob; import org.apache.flink.core.testutils.CommonTestUtils; +import org.apache.flink.util.AbstractID; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -29,14 +30,17 @@ import static org.apache.flink.runtime.blob.BlobKey.BlobType.PERMANENT_BLOB; import static org.apache.flink.runtime.blob.BlobKey.BlobType.TRANSIENT_BLOB; +import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.lessThan; +import static org.hamcrest.core.IsNot.not; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; /** @@ -46,21 +50,35 @@ public final class BlobKeyTest extends TestLogger { /** * The first key array to be used during the unit tests. */ - private static final byte[] KEY_ARRAY_1 = new byte[20]; + private static final byte[] KEY_ARRAY_1 = new byte[BlobKey.SIZE]; /** * The second key array to be used during the unit tests. */ - private static final byte[] KEY_ARRAY_2 = new byte[20]; + private static final byte[] KEY_ARRAY_2 = new byte[BlobKey.SIZE]; + + /** + * First byte array to use for the random component of a {@link BlobKey}. + */ + private static final byte[] RANDOM_ARRAY_1 = new byte[AbstractID.SIZE]; + + /** + * Second byte array to use for the random component of a {@link BlobKey}. + */ + private static final byte[] RANDOM_ARRAY_2 = new byte[AbstractID.SIZE]; /* - * Initialize the key array. + * Initialize the key and random arrays. */ static { for (int i = 0; i < KEY_ARRAY_1.length; ++i) { KEY_ARRAY_1[i] = (byte) i; KEY_ARRAY_2[i] = (byte) (i + 1); } + for (int i = 0; i < RANDOM_ARRAY_1.length; ++i) { + RANDOM_ARRAY_1[i] = (byte) i; + RANDOM_ARRAY_2[i] = (byte) (i + 1); + } } @Test @@ -89,7 +107,7 @@ public void testSerializationPermanent() throws Exception { * Tests the serialization/deserialization of BLOB keys. */ private void testSerialization(BlobKey.BlobType blobType) throws Exception { - final BlobKey k1 = BlobKey.createKey(blobType, KEY_ARRAY_1); + final BlobKey k1 = BlobKey.createKey(blobType, KEY_ARRAY_1, RANDOM_ARRAY_1); final BlobKey k2 = CommonTestUtils.createCopySerializable(k1); assertEquals(k1, k2); assertEquals(k1.hashCode(), k2.hashCode()); @@ -107,16 +125,25 @@ public void testEqualsPermanent() { } /** - * Tests the equals method. + * Tests the {@link BlobKey#equals(Object)} and {@link BlobKey#hashCode()} methods. */ private void testEquals(BlobKey.BlobType blobType) { - final BlobKey k1 = BlobKey.createKey(blobType, KEY_ARRAY_1); - final BlobKey k2 = BlobKey.createKey(blobType, KEY_ARRAY_1); - final BlobKey k3 = BlobKey.createKey(blobType, KEY_ARRAY_2); + final BlobKey k1 = BlobKey.createKey(blobType, KEY_ARRAY_1, RANDOM_ARRAY_1); + final BlobKey k2 = BlobKey.createKey(blobType, KEY_ARRAY_1, RANDOM_ARRAY_1); + final BlobKey k3 = BlobKey.createKey(blobType, KEY_ARRAY_2, RANDOM_ARRAY_1); + final BlobKey k4 = BlobKey.createKey(blobType, KEY_ARRAY_1, RANDOM_ARRAY_2); assertTrue(k1.equals(k2)); assertTrue(k2.equals(k1)); + assertEquals(k1.hashCode(), k2.hashCode()); assertFalse(k1.equals(k3)); assertFalse(k3.equals(k1)); + assertFalse(k1.equals(k4)); + assertFalse(k4.equals(k1)); + + //noinspection ObjectEqualsNull + assertFalse(k1.equals(null)); + //noinspection EqualsBetweenInconvertibleTypes + assertFalse(k1.equals(this)); } /** @@ -124,8 +151,8 @@ private void testEquals(BlobKey.BlobType blobType) { */ @Test public void testEqualsDifferentBlobType() { - final BlobKey k1 = BlobKey.createKey(TRANSIENT_BLOB, KEY_ARRAY_1); - final BlobKey k2 = BlobKey.createKey(PERMANENT_BLOB, KEY_ARRAY_1); + final BlobKey k1 = BlobKey.createKey(TRANSIENT_BLOB, KEY_ARRAY_1, RANDOM_ARRAY_1); + final BlobKey k2 = BlobKey.createKey(PERMANENT_BLOB, KEY_ARRAY_1, RANDOM_ARRAY_1); assertFalse(k1.equals(k2)); assertFalse(k2.equals(k1)); } @@ -144,19 +171,22 @@ public void testComparesPermanent() { * Tests the compares method. */ private void testCompares(BlobKey.BlobType blobType) { - final BlobKey k1 = BlobKey.createKey(blobType, KEY_ARRAY_1); - final BlobKey k2 = BlobKey.createKey(blobType, KEY_ARRAY_1); - final BlobKey k3 = BlobKey.createKey(blobType, KEY_ARRAY_2); + final BlobKey k1 = BlobKey.createKey(blobType, KEY_ARRAY_1, RANDOM_ARRAY_1); + final BlobKey k2 = BlobKey.createKey(blobType, KEY_ARRAY_1, RANDOM_ARRAY_1); + final BlobKey k3 = BlobKey.createKey(blobType, KEY_ARRAY_2, RANDOM_ARRAY_1); + final BlobKey k4 = BlobKey.createKey(blobType, KEY_ARRAY_1, RANDOM_ARRAY_2); assertThat(k1.compareTo(k2), is(0)); assertThat(k2.compareTo(k1), is(0)); assertThat(k1.compareTo(k3), lessThan(0)); + assertThat(k1.compareTo(k4), lessThan(0)); assertThat(k3.compareTo(k1), greaterThan(0)); + assertThat(k4.compareTo(k1), greaterThan(0)); } @Test public void testComparesDifferentBlobType() { - final BlobKey k1 = BlobKey.createKey(TRANSIENT_BLOB, KEY_ARRAY_1); - final BlobKey k2 = BlobKey.createKey(PERMANENT_BLOB, KEY_ARRAY_1); + final BlobKey k1 = BlobKey.createKey(TRANSIENT_BLOB, KEY_ARRAY_1, RANDOM_ARRAY_1); + final BlobKey k2 = BlobKey.createKey(PERMANENT_BLOB, KEY_ARRAY_1, RANDOM_ARRAY_1); assertThat(k1.compareTo(k2), greaterThan(0)); assertThat(k2.compareTo(k1), lessThan(0)); } @@ -175,7 +205,7 @@ public void testStreamsPermanent() throws Exception { * Test the serialization/deserialization using input/output streams. */ private void testStreams(BlobKey.BlobType blobType) throws IOException { - final BlobKey k1 = BlobKey.createKey(blobType, KEY_ARRAY_1); + final BlobKey k1 = BlobKey.createKey(blobType, KEY_ARRAY_1, RANDOM_ARRAY_1); final ByteArrayOutputStream baos = new ByteArrayOutputStream(20); k1.writeToOutputStream(baos); @@ -187,6 +217,28 @@ private void testStreams(BlobKey.BlobType blobType) throws IOException { assertEquals(k1, k2); } + /** + * Verifies that the two given key's are different in total but share the same hash. + * + * @param key1 first blob key + * @param key2 second blob key + */ + static void verifyKeyDifferentHashEquals(BlobKey key1, BlobKey key2) { + assertNotEquals(key1, key2); + assertThat(key1.getHash(), equalTo(key2.getHash())); + } + + /** + * Verifies that the two given key's are different in total and also have different hashes. + * + * @param key1 first blob key + * @param key2 second blob key + */ + static void verifyKeyDifferentHashDifferent(BlobKey key1, BlobKey key2) { + assertNotEquals(key1, key2); + assertThat(key1.getHash(), not(equalTo(key2.getHash()))); + } + /** * Verifies that the given key is of an expected type. * diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java index a110d4a80d803..fde21ba36d75a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java @@ -47,10 +47,11 @@ import static org.apache.flink.runtime.blob.BlobCacheCleanupTest.checkFileCountForJob; import static org.apache.flink.runtime.blob.BlobKey.BlobType.PERMANENT_BLOB; import static org.apache.flink.runtime.blob.BlobKey.BlobType.TRANSIENT_BLOB; +import static org.apache.flink.runtime.blob.BlobKeyTest.verifyKeyDifferentHashEquals; import static org.apache.flink.runtime.blob.BlobServerGetTest.verifyDeleted; import static org.apache.flink.runtime.blob.BlobServerPutTest.put; import static org.apache.flink.runtime.blob.BlobServerPutTest.verifyContents; -import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; @@ -86,6 +87,12 @@ public void testDeleteTransient4() throws IOException { testDeleteTransient(new JobID(), new JobID()); } + @Test + public void testDeleteTransient5() throws IOException { + JobID jobId = new JobID(); + testDeleteTransient(jobId, jobId); + } + /** * Uploads a (different) byte array for each of the given jobs and verifies that deleting one of * them (via the {@link BlobServer}) does not influence the other. @@ -97,7 +104,6 @@ public void testDeleteTransient4() throws IOException { */ private void testDeleteTransient(@Nullable JobID jobId1, @Nullable JobID jobId2) throws IOException { - final boolean sameJobId = (jobId1 == jobId2) || (jobId1 != null && jobId1.equals(jobId2)); final Configuration config = new Configuration(); config.setString(BlobServerOptions.STORAGE_DIRECTORY, temporaryFolder.newFolder().getAbsolutePath()); @@ -118,7 +124,7 @@ private void testDeleteTransient(@Nullable JobID jobId1, @Nullable JobID jobId2) // put two more BLOBs (same key, other key) for another job ID TransientBlobKey key2a = (TransientBlobKey) put(server, jobId2, data, TRANSIENT_BLOB); assertNotNull(key2a); - assertEquals(key1, key2a); + verifyKeyDifferentHashEquals(key1, key2a); TransientBlobKey key2b = (TransientBlobKey) put(server, jobId2, data2, TRANSIENT_BLOB); assertNotNull(key2b); @@ -126,10 +132,9 @@ private void testDeleteTransient(@Nullable JobID jobId1, @Nullable JobID jobId2) assertTrue(delete(server, jobId1, key1)); verifyDeleted(server, jobId1, key1); - // deleting a one BLOB should not affect another BLOB, even with the same key if job IDs are different - if (!sameJobId) { - verifyContents(server, jobId2, key2a, data); - } + // deleting a one BLOB should not affect another BLOB with a different key + // (and keys are always different now) + verifyContents(server, jobId2, key2a, data); verifyContents(server, jobId2, key2b, data2); // delete first file of second job @@ -284,7 +289,7 @@ private void testJobCleanup(BlobKey.BlobType blobType) throws IOException { BlobKey key1a = put(server, jobId1, data, blobType); BlobKey key2 = put(server, jobId2, data, blobType); - assertEquals(key1a, key2); + assertArrayEquals(key1a.getHash(), key2.getHash()); BlobKey key1b = put(server, jobId1, data2, blobType); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerGetTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerGetTest.java index 492727935624c..e3b53096e2c90 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerGetTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerGetTest.java @@ -42,7 +42,6 @@ import java.io.IOException; import java.nio.file.AccessDeniedException; import java.nio.file.NoSuchFileException; -import java.security.MessageDigest; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -58,6 +57,7 @@ import static org.apache.flink.runtime.blob.BlobClientTest.validateGetAndClose; import static org.apache.flink.runtime.blob.BlobKey.BlobType.PERMANENT_BLOB; import static org.apache.flink.runtime.blob.BlobKey.BlobType.TRANSIENT_BLOB; +import static org.apache.flink.runtime.blob.BlobKeyTest.verifyKeyDifferentHashEquals; import static org.apache.flink.runtime.blob.BlobServerPutTest.put; import static org.apache.flink.runtime.blob.BlobUtils.JOB_DIR_PREFIX; import static org.junit.Assert.assertArrayEquals; @@ -141,18 +141,18 @@ private void testGetFailsDuringLookup( // add the same data under a second jobId BlobKey key2 = put(server, jobId2, data, blobType); - assertNotNull(key); - assertEquals(key, key2); + assertNotNull(key2); + verifyKeyDifferentHashEquals(key, key2); // request for jobId2 should succeed - get(server, jobId2, key); + get(server, jobId2, key2); // request for jobId1 should still fail verifyDeleted(server, jobId1, key); // same checks as for jobId1 but for jobId2 should also work: - blobFile = server.getStorageLocation(jobId2, key); + blobFile = server.getStorageLocation(jobId2, key2); assertTrue(blobFile.delete()); - verifyDeleted(server, jobId2, key); + verifyDeleted(server, jobId2, key2); } } @@ -373,11 +373,6 @@ private void testConcurrentGetOperations( final byte[] data = {1, 2, 3, 4, 99, 42}; - MessageDigest md = BlobUtils.createMessageDigest(); - - // create the correct blob key by hashing our input data - final BlobKey blobKey = BlobKey.createKey(blobType, md.digest(data)); - doAnswer( new Answer() { @Override @@ -398,7 +393,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { server.start(); // upload data first - assertEquals(blobKey, put(server, jobId, data, blobType)); + final BlobKey blobKey = put(server, jobId, data, blobType); // now try accessing it concurrently (only HA mode will be able to retrieve it from HA store!) if (blobType == PERMANENT_BLOB) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerPutTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerPutTest.java index aefd0a302e417..dcf49a5ca697e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerPutTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerPutTest.java @@ -62,6 +62,7 @@ import static org.apache.flink.runtime.blob.BlobClientTest.validateGetAndClose; import static org.apache.flink.runtime.blob.BlobKey.BlobType.PERMANENT_BLOB; import static org.apache.flink.runtime.blob.BlobKey.BlobType.TRANSIENT_BLOB; +import static org.apache.flink.runtime.blob.BlobKeyTest.verifyKeyDifferentHashEquals; import static org.apache.flink.runtime.blob.BlobServerGetTest.get; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; @@ -226,21 +227,26 @@ private void testPutBufferSuccessfulGet( // put data for jobId1 and verify BlobKey key1a = put(server, jobId1, data, blobType); assertNotNull(key1a); + // second upload of same data should yield a different BlobKey + BlobKey key1a2 = put(server, jobId1, data, blobType); + assertNotNull(key1a2); + verifyKeyDifferentHashEquals(key1a, key1a2); BlobKey key1b = put(server, jobId1, data2, blobType); assertNotNull(key1b); verifyContents(server, jobId1, key1a, data); + verifyContents(server, jobId1, key1a2, data); verifyContents(server, jobId1, key1b, data2); // now put data for jobId2 and verify that both are ok BlobKey key2a = put(server, jobId2, data, blobType); assertNotNull(key2a); - assertEquals(key1a, key2a); + verifyKeyDifferentHashEquals(key1a, key2a); BlobKey key2b = put(server, jobId2, data2, blobType); assertNotNull(key2b); - assertEquals(key1b, key2b); + verifyKeyDifferentHashEquals(key1b, key2b); // verify the accessibility and the BLOB contents verifyContents(server, jobId2, key2a, data); @@ -249,6 +255,7 @@ private void testPutBufferSuccessfulGet( // verify the accessibility and the BLOB contents one more time (transient BLOBs should // not be deleted here) verifyContents(server, jobId1, key1a, data); + verifyContents(server, jobId1, key1a2, data); verifyContents(server, jobId1, key1b, data2); verifyContents(server, jobId2, key2a, data); verifyContents(server, jobId2, key2b, data2); @@ -311,21 +318,26 @@ private void testPutStreamSuccessfulGet( // put data for jobId1 and verify BlobKey key1a = put(server, jobId1, new ByteArrayInputStream(data), blobType); assertNotNull(key1a); + // second upload of same data should yield a different BlobKey + BlobKey key1a2 = put(server, jobId1, new ByteArrayInputStream(data), blobType); + assertNotNull(key1a2); + verifyKeyDifferentHashEquals(key1a, key1a2); BlobKey key1b = put(server, jobId1, new ByteArrayInputStream(data2), blobType); assertNotNull(key1b); verifyContents(server, jobId1, key1a, data); + verifyContents(server, jobId1, key1a2, data); verifyContents(server, jobId1, key1b, data2); // now put data for jobId2 and verify that both are ok BlobKey key2a = put(server, jobId2, new ByteArrayInputStream(data), blobType); assertNotNull(key2a); - assertEquals(key1a, key2a); + verifyKeyDifferentHashEquals(key1a, key2a); BlobKey key2b = put(server, jobId2, new ByteArrayInputStream(data2), blobType); assertNotNull(key2b); - assertEquals(key1b, key2b); + verifyKeyDifferentHashEquals(key1b, key2b); // verify the accessibility and the BLOB contents verifyContents(server, jobId2, key2a, data); @@ -334,6 +346,7 @@ private void testPutStreamSuccessfulGet( // verify the accessibility and the BLOB contents one more time (transient BLOBs should // not be deleted here) verifyContents(server, jobId1, key1a, data); + verifyContents(server, jobId1, key1a2, data); verifyContents(server, jobId1, key1b, data2); verifyContents(server, jobId2, key2a, data); verifyContents(server, jobId2, key2b, data2); @@ -396,21 +409,26 @@ private void testPutChunkedStreamSuccessfulGet( // put data for jobId1 and verify BlobKey key1a = put(server, jobId1, new ChunkedInputStream(data, 19), blobType); assertNotNull(key1a); + // second upload of same data should yield a different BlobKey + BlobKey key1a2 = put(server, jobId1, new ChunkedInputStream(data, 19), blobType); + assertNotNull(key1a2); + verifyKeyDifferentHashEquals(key1a, key1a2); BlobKey key1b = put(server, jobId1, new ChunkedInputStream(data2, 19), blobType); assertNotNull(key1b); verifyContents(server, jobId1, key1a, data); + verifyContents(server, jobId1, key1a2, data); verifyContents(server, jobId1, key1b, data2); // now put data for jobId2 and verify that both are ok BlobKey key2a = put(server, jobId2, new ChunkedInputStream(data, 19), blobType); assertNotNull(key2a); - assertEquals(key1a, key2a); + verifyKeyDifferentHashEquals(key1a, key2a); BlobKey key2b = put(server, jobId2, new ChunkedInputStream(data2, 19), blobType); assertNotNull(key2b); - assertEquals(key1b, key2b); + verifyKeyDifferentHashEquals(key1b, key2b); // verify the accessibility and the BLOB contents verifyContents(server, jobId2, key2a, data); @@ -419,6 +437,7 @@ private void testPutChunkedStreamSuccessfulGet( // verify the accessibility and the BLOB contents one more time (transient BLOBs should // not be deleted here) verifyContents(server, jobId1, key1a, data); + verifyContents(server, jobId1, key1a2, data); verifyContents(server, jobId1, key1b, data2); verifyContents(server, jobId2, key2a, data); verifyContents(server, jobId2, key2b, data2); @@ -700,7 +719,7 @@ private void testConcurrentPutOperations( // make sure that all blob keys are the same while (blobKeyIterator.hasNext()) { - assertEquals(blobKey, blobKeyIterator.next()); + verifyKeyDifferentHashEquals(blobKey, blobKeyIterator.next()); } // check the uploaded file's contents diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerRecoveryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerRecoveryTest.java index 35575b5f20155..5b8d0e9e3531f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerRecoveryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerRecoveryTest.java @@ -39,12 +39,10 @@ import static org.apache.flink.runtime.blob.BlobKey.BlobType.PERMANENT_BLOB; import static org.apache.flink.runtime.blob.BlobKey.BlobType.TRANSIENT_BLOB; +import static org.apache.flink.runtime.blob.BlobKeyTest.verifyKeyDifferentHashEquals; import static org.apache.flink.runtime.blob.BlobServerGetTest.verifyDeleted; import static org.apache.flink.runtime.blob.BlobServerPutTest.put; import static org.apache.flink.runtime.blob.BlobServerPutTest.verifyContents; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.equalTo; -import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -126,8 +124,7 @@ public static void testBlobServerRecovery(final Configuration config, final Blob // put non-HA data nonHAKey = put(server0, jobId[0], expected2, TRANSIENT_BLOB); - assertNotEquals(keys[1], nonHAKey); - assertThat(keys[1].getHash(), equalTo(nonHAKey.getHash())); + verifyKeyDifferentHashEquals(keys[1], nonHAKey); // check that the storage directory exists final Path blobServerPath = new Path(storagePath, "blob"); From 5a1ddfa5da779d0416516c235ddfbc301903c55c Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Thu, 5 Oct 2017 15:18:58 +0200 Subject: [PATCH 2/6] [FLINK-7196][blob] add a TTL to all transient BLOBs Transient BLOB files should not exist for long and are only deleted manually from the caches and after access on the server. This uses the BLOB storage's cleanup interval to set a TTL on all transient BLOB files as a backup cleanup path. The cleanup task itself runs every cleanupInterval seconds and removes all transient BLOBs for which the TTL is older than the current time. This way, a transient BLOB stays at most 2*cleanupInterval seconds before getting deleted automatically. --- docs/ops/config.md | 12 +- .../apache/flink/runtime/blob/BlobServer.java | 67 ++++- .../runtime/blob/PermanentBlobCache.java | 10 +- .../runtime/blob/TransientBlobCache.java | 63 ++++- .../blob/TransientBlobCleanupTask.java | 115 ++++++++ .../runtime/blob/BlobCacheCleanupTest.java | 220 +++++++++------ .../flink/runtime/blob/BlobCachePutTest.java | 14 +- .../runtime/blob/BlobServerCleanupTest.java | 264 ++++++++++++++++++ .../runtime/blob/BlobServerDeleteTest.java | 2 +- .../BlobLibraryCacheManagerTest.java | 4 +- 10 files changed, 670 insertions(+), 101 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/blob/TransientBlobCleanupTask.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java diff --git a/docs/ops/config.md b/docs/ops/config.md index 9d2405e525e19..ac9c813ca64c4 100644 --- a/docs/ops/config.md +++ b/docs/ops/config.md @@ -199,12 +199,14 @@ will be used under the directory specified by jobmanager.web.tmpdir. - `blob.storage.directory`: Directory for storing blobs (such as user JARs) on the TaskManagers. -- `blob.service.cleanup.interval`: Cleanup interval (in seconds) of the blob caches (DEFAULT: 1 hour). -Whenever a job is not referenced at the cache anymore, we set a TTL and let the periodic cleanup task -(executed every `blob.service.cleanup.interval` seconds) remove its blob files after this TTL has passed. +- `blob.service.cleanup.interval`: Cleanup interval (in seconds) of transient blobs at server and caches as well as permanent blobs at the caches (DEFAULT: 1 hour). +Whenever a job is not referenced at the cache anymore, we set a TTL for its permanent blob files and +let the periodic cleanup task (executed every `blob.service.cleanup.interval` seconds) remove them +after this TTL has passed. We do the same for transient blob files at both server and caches but +immediately after accessing them, i.e. an put or get operation. This means that a blob will be retained at most 2 * `blob.service.cleanup.interval` seconds after -not being referenced anymore. Therefore, a recovery still has the chance to use existing files rather -than to download them again. +not being referenced anymore (permanent blobs) or their last access (transient blobs). For permanent blobs, +this means that a recovery still has the chance to use existing files rather downloading them again. - `blob.server.port`: Port definition for the blob server (serving user JARs) on the TaskManagers. By default the port is set to 0, which means that the operating system is picking an ephemeral port. Flink also accepts a list of ports ("50100,50101"), ranges ("50100-50200") or a combination of both. It is recommended to set a range of ports to avoid collisions when multiple JobManagers are running on the same machine. 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 bc61ef76c9f80..402a73993c68f 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 @@ -20,6 +20,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.net.SSLUtils; @@ -46,14 +47,17 @@ import java.util.Iterator; import java.util.List; import java.util.Set; +import java.util.Timer; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; -import static org.apache.flink.runtime.blob.BlobServerProtocol.BUFFER_SIZE; import static org.apache.flink.runtime.blob.BlobKey.BlobType.PERMANENT_BLOB; import static org.apache.flink.runtime.blob.BlobKey.BlobType.TRANSIENT_BLOB; +import static org.apache.flink.runtime.blob.BlobServerProtocol.BUFFER_SIZE; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -102,6 +106,23 @@ public class BlobServer extends Thread implements BlobService, PermanentBlobServ */ private final Thread shutdownHook; + // -------------------------------------------------------------------------------------------- + + /** + * Map to store the TTL of each element stored in the local storage, i.e. via one of the {@link + * #getFile} methods. + **/ + private final ConcurrentHashMap, Long> blobExpiryTimes = + new ConcurrentHashMap<>(); + + /** Time interval (ms) to run the cleanup task; also used as the default TTL. */ + private final long cleanupInterval; + + /** + * Timer task to execute the cleanup at regular intervals. + */ + private final Timer cleanupTimer; + /** * Instantiates a new BLOB server and binds it to a free network port. * @@ -141,6 +162,14 @@ public BlobServer(Configuration config, BlobStore blobStore) throws IOException backlog = BlobServerOptions.FETCH_BACKLOG.defaultValue(); } + // Initializing the clean up task + this.cleanupTimer = new Timer(true); + + this.cleanupInterval = config.getLong(BlobServerOptions.CLEANUP_INTERVAL) * 1000; + this.cleanupTimer + .schedule(new TransientBlobCleanupTask(blobExpiryTimes, readWriteLock.writeLock(), + storageDir, LOG), cleanupInterval, cleanupInterval); + this.shutdownHook = BlobUtils.addShutdownHook(this, LOG); if (config.getBoolean(BlobServerOptions.SSL_ENABLED)) { @@ -273,6 +302,8 @@ public void run() { */ @Override public void close() throws IOException { + cleanupTimer.cancel(); + if (shutdownRequested.compareAndSet(false, true)) { Exception exception = null; @@ -462,6 +493,15 @@ void getFileInternal(@Nullable JobID jobId, BlobKey blobKey, File localFile) thr // assume readWriteLock.readLock() was already locked (cannot really check that) if (localFile.exists()) { + // update TTL for transient BLOBs: + if (blobKey instanceof TransientBlobKey) { + // regarding concurrent operations, it is not really important which timestamp makes + // it into the map as they are close to each other anyway, also we can simply + // overwrite old values as long as we are in the read (or write) lock + blobExpiryTimes + .put(Tuple2.of(jobId, (TransientBlobKey) blobKey), + System.currentTimeMillis() + cleanupInterval); + } return; } else if (blobKey instanceof PermanentBlobKey) { // Try the HA blob store @@ -695,6 +735,13 @@ BlobKey moveTempFileToStore( BlobUtils.moveTempFileToStore( incomingFile, jobId, blobKey, storageFile, LOG, blobKey instanceof PermanentBlobKey ? blobStore : null); + // add TTL for transient BLOBs: + if (blobKey instanceof TransientBlobKey) { + // must be inside read or write lock to add a TTL + blobExpiryTimes + .put(Tuple2.of(jobId, (TransientBlobKey) blobKey), + System.currentTimeMillis() + cleanupInterval); + } return blobKey; } } finally { @@ -769,6 +816,8 @@ boolean deleteInternal(@Nullable JobID jobId, TransientBlobKey key) { LOG.warn("Failed to locally delete BLOB " + key + " at " + localFile.getAbsolutePath()); return false; } + // this needs to happen inside the write lock in case of concurrent getFile() calls + blobExpiryTimes.remove(Tuple2.of(jobId, key)); return true; } finally { readWriteLock.writeLock().unlock(); @@ -797,6 +846,12 @@ public boolean cleanupJob(JobID jobId) { boolean deletedLocally = false; try { FileUtils.deleteDirectory(jobDir); + + // NOTE: Instead of going through blobExpiryTimes, keep lingering entries - they + // will be cleaned up by the timer task which tolerates non-existing files + // If inserted again with the same IDs (via put()), the TTL will be updated + // again. + deletedLocally = true; } catch (IOException e) { LOG.warn("Failed to locally delete BLOB storage directory at " + @@ -832,6 +887,16 @@ public int getPort() { return this.serverSocket.getLocalPort(); } + /** + * Returns the blob expiry times - for testing purposes only! + * + * @return blob expiry times (internal state!) + */ + @VisibleForTesting + ConcurrentMap, Long> getBlobExpiryTimes() { + return blobExpiryTimes; + } + /** * Tests whether the BLOB server has been requested to shut down. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/PermanentBlobCache.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/PermanentBlobCache.java index 63003a20caf47..ae59f590b8db5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/PermanentBlobCache.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/PermanentBlobCache.java @@ -40,9 +40,10 @@ /** * Provides a cache for permanent BLOB files including a per-job ref-counting and a staged cleanup. * - *

When requesting BLOBs via {@link #getPermanentFile(JobID, BlobKey)}, the cache will first attempt to - * serve the file from its local cache. Only if the local cache does not contain the desired BLOB, - * it will try to download it from a distributed HA file system (if available) or the BLOB server. + *

When requesting BLOBs via {@link #getFile(JobID, PermanentBlobKey)}, the cache will first + * attempt to serve the file from its local cache. Only if the local cache does not contain the + * desired BLOB, it will try to download it from a distributed HA file system (if available) or the + * BLOB server. * *

If files for a job are not needed any more, they will enter a staged, i.e. deferred, cleanup. * Files may thus still be be accessible upon recovery and do not need to be re-downloaded. @@ -76,6 +77,9 @@ static class RefCount { */ private final long cleanupInterval; + /** + * Timer task to execute the cleanup at regular intervals. + */ private final Timer cleanupTimer; /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/TransientBlobCache.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/TransientBlobCache.java index 941973c610a53..bbd008ae3939a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/TransientBlobCache.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/TransientBlobCache.java @@ -20,6 +20,8 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; import org.slf4j.LoggerFactory; @@ -30,6 +32,9 @@ import java.io.IOException; import java.io.InputStream; import java.net.InetSocketAddress; +import java.util.Timer; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import static org.apache.flink.runtime.blob.BlobKey.BlobType.TRANSIENT_BLOB; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -42,6 +47,23 @@ */ public class TransientBlobCache extends AbstractBlobCache implements TransientBlobService { + /** + * Map to store the TTL of each element stored in the local storage, i.e. via one of the {@link + * #getFile} methods. + **/ + private final ConcurrentHashMap, Long> blobExpiryTimes = + new ConcurrentHashMap<>(); + + /** + * Time interval (ms) to run the cleanup task; also used as the default TTL. + */ + private final long cleanupInterval; + + /** + * Timer task to execute the cleanup at regular intervals. + */ + private final Timer cleanupTimer; + /** * Instantiates a new BLOB cache. * @@ -59,6 +81,14 @@ public TransientBlobCache( super(serverAddress, blobClientConfig, new VoidBlobStore(), LoggerFactory.getLogger(TransientBlobCache.class)); + + // Initializing the clean up task + this.cleanupTimer = new Timer(true); + + this.cleanupInterval = blobClientConfig.getLong(BlobServerOptions.CLEANUP_INTERVAL) * 1000; + this.cleanupTimer + .schedule(new TransientBlobCleanupTask(blobExpiryTimes, readWriteLock.writeLock(), + storageDir, log), cleanupInterval, cleanupInterval); } @Override @@ -72,6 +102,24 @@ public File getFile(JobID jobId, TransientBlobKey key) throws IOException { return getFileInternal(jobId, key); } + @Override + protected File getFileInternal(@Nullable JobID jobId, BlobKey blobKey) throws IOException { + File file = super.getFileInternal(jobId, blobKey); + + readWriteLock.readLock().lock(); + try { + // regarding concurrent operations, it is not really important which timestamp makes + // it into the map as they are close to each other anyway, also we can simply + // overwrite old values as long as we are in the read (or write) lock + blobExpiryTimes.put(Tuple2.of(jobId, (TransientBlobKey) blobKey), + System.currentTimeMillis() + cleanupInterval); + } finally { + readWriteLock.readLock().unlock(); + } + + return file; + } + @Override public TransientBlobKey putTransient(byte[] value) throws IOException { try (BlobClient bc = new BlobClient(serverAddress, blobClientConfig)) { @@ -134,6 +182,9 @@ private boolean deleteInternal(@Nullable JobID jobId, TransientBlobKey key) { log.warn("Failed to delete locally cached BLOB {} at {}", key, localFile.getAbsolutePath()); return false; + } else { + // this needs to happen inside the write lock in case of concurrent getFile() calls + blobExpiryTimes.remove(Tuple2.of(jobId, key)); } } finally { readWriteLock.writeLock().unlock(); @@ -141,6 +192,16 @@ private boolean deleteInternal(@Nullable JobID jobId, TransientBlobKey key) { return true; } + /** + * Returns the blob expiry times - for testing purposes only! + * + * @return blob expiry times (internal state!) + */ + @VisibleForTesting + ConcurrentMap, Long> getBlobExpiryTimes() { + return blobExpiryTimes; + } + /** * Returns a file handle to the file associated with the given blob key on the blob * server. @@ -166,6 +227,6 @@ private BlobClient createClient() throws IOException { @Override protected void cancelCleanupTask() { - // nothing to do here + cleanupTimer.cancel(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/TransientBlobCleanupTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/TransientBlobCleanupTask.java new file mode 100644 index 0000000000000..5dc76787677dc --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/TransientBlobCleanupTask.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.blob; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.java.tuple.Tuple2; + +import org.slf4j.Logger; + +import java.io.File; +import java.util.Map; +import java.util.Set; +import java.util.TimerTask; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.locks.Lock; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Cleanup task for transient BLOBs. + */ +class TransientBlobCleanupTask extends TimerTask { + + /** + * The log object used for debugging. + */ + private final Logger log; + + /** + * Map to store the TTL of each element stored in the local storage. + **/ + private ConcurrentMap, Long> blobExpiryTimes; + + /** + * Lock to acquire before changing file contents. + */ + private Lock writeLock; + + /** + * Local storage directory to work on. + */ + private File storageDir; + + /** + * Creates a new cleanup timer task working with the given parameters from {@link BlobServer} + * and {@link TransientBlobCache}. + * + * @param blobExpiryTimes + * map to store the TTL of each element stored in the local storage + * @param writeLock + * lock to acquire before changing file contents + * @param storageDir + * local storage directory to work on + * @param log + * logger instance for debugging + */ + TransientBlobCleanupTask( + ConcurrentMap, Long> blobExpiryTimes, Lock writeLock, + File storageDir, Logger log) { + this.blobExpiryTimes = checkNotNull(blobExpiryTimes); + this.writeLock = checkNotNull(writeLock); + this.storageDir = checkNotNull(storageDir); + this.log = checkNotNull(log); + } + + /** + * Cleans up transient BLOBs whose TTL is up, tolerating that files do not exist (anymore). + */ + @Override + public void run() { + // let's cache the current time - we do not operate on a millisecond precision anyway + final long currentTimeMillis = System.currentTimeMillis(); + // iterate through all entries and remove those where the current time is past their expiry + Set, Long>> entries = blobExpiryTimes.entrySet(); + for (Map.Entry, Long> entry : entries) { + if (currentTimeMillis >= entry.getValue()) { + JobID jobId = entry.getKey().f0; + BlobKey blobKey = entry.getKey().f1; + + final File localFile = + new File(BlobUtils.getStorageLocationPath(storageDir.getAbsolutePath(), jobId, blobKey)); + + // deleting the file or changing blobExpiryTimes' contents needs to be protected by the lock + writeLock.lock(); + + try { + if (!localFile.delete() && localFile.exists()) { + log.warn("Failed to locally delete blob " + localFile.getAbsolutePath()); + } else { + // this needs to happen inside the write lock in case of concurrent getFile() calls + entries.remove(entry); + } + } finally { + writeLock.unlock(); + } + } + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheCleanupTest.java index d6fab509c2300..5ec9b8917a0b3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheCleanupTest.java @@ -19,8 +19,11 @@ package org.apache.flink.runtime.blob; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; import org.junit.Ignore; @@ -28,23 +31,41 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.File; +import javax.annotation.Nullable; + import java.io.IOException; import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.List; - +import java.util.Random; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import static org.apache.flink.runtime.blob.BlobCachePutTest.verifyDeletedEventually; +import static org.apache.flink.runtime.blob.BlobKey.BlobType.TRANSIENT_BLOB; +import static org.apache.flink.runtime.blob.BlobServerCleanupTest.checkFileCountForJob; +import static org.apache.flink.runtime.blob.BlobServerCleanupTest.checkFilesExist; +import static org.apache.flink.runtime.blob.BlobServerGetTest.get; +import static org.apache.flink.runtime.blob.BlobServerPutTest.put; +import static org.apache.flink.runtime.blob.BlobServerPutTest.verifyContents; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; /** - * A few tests for the deferred ref-counting based cleanup inside the {@link PermanentBlobCache}. + * A few tests for the cleanup of {@link PermanentBlobCache} and {@link TransientBlobCache}. */ public class BlobCacheCleanupTest extends TestLogger { + private final Random rnd = new Random(); + @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -52,7 +73,7 @@ public class BlobCacheCleanupTest extends TestLogger { * Tests that {@link PermanentBlobCache} cleans up after calling {@link PermanentBlobCache#releaseJob(JobID)}. */ @Test - public void testJobCleanup() throws IOException, InterruptedException { + public void testPermanentBlobCleanup() throws IOException, InterruptedException { JobID jobId = new JobID(); List keys = new ArrayList<>(); @@ -131,7 +152,7 @@ public void testJobCleanup() throws IOException, InterruptedException { * when registering, releasing, and re-registering jobs. */ @Test - public void testJobReferences() throws IOException, InterruptedException { + public void testPermanentJobReferences() throws IOException, InterruptedException { JobID jobId = new JobID(); @@ -184,12 +205,13 @@ public void testJobReferences() throws IOException, InterruptedException { } /** - * Tests that {@link PermanentBlobCache} cleans up after calling {@link PermanentBlobCache#releaseJob(JobID)} - * but only after preserving the file for a bit longer. + * Tests the deferred cleanup of {@link PermanentBlobCache}, i.e. after calling {@link + * PermanentBlobCache#releaseJob(JobID)} the file should be preserved a bit longer and then + * cleaned up. */ @Test @Ignore("manual test due to stalling: ensures a BLOB is retained first and only deleted after the (long) timeout ") - public void testJobDeferredCleanup() throws IOException, InterruptedException { + public void testPermanentBlobDeferredCleanup() throws IOException, InterruptedException { // file should be deleted between 5 and 10s after last job release long cleanupInterval = 5L; @@ -278,6 +300,114 @@ public void testJobDeferredCleanup() throws IOException, InterruptedException { } } + @Test + public void testTransientBlobNoJobCleanup() + throws IOException, InterruptedException, ExecutionException { + testTransientBlobCleanup(null); + } + + @Test + public void testTransientBlobForJobCleanup() + throws IOException, InterruptedException, ExecutionException { + testTransientBlobCleanup(new JobID()); + } + + /** + * Tests that {@link TransientBlobCache} cleans up after a default TTL and keeps files which are + * constantly accessed. + */ + private void testTransientBlobCleanup(@Nullable final JobID jobId) + throws IOException, InterruptedException, ExecutionException { + + // 1s should be a safe-enough buffer to still check for existence after a BLOB's last access + long cleanupInterval = 1L; // in seconds + final int numberConcurrentGetOperations = 3; + + final List> getOperations = new ArrayList<>(numberConcurrentGetOperations); + + byte[] data = new byte[2000000]; + rnd.nextBytes(data); + byte[] data2 = Arrays.copyOfRange(data, 10, 54); + + Configuration config = new Configuration(); + config.setString(BlobServerOptions.STORAGE_DIRECTORY, + temporaryFolder.newFolder().getAbsolutePath()); + config.setLong(BlobServerOptions.CLEANUP_INTERVAL, cleanupInterval); + + long cleanupLowerBound; + + try ( + BlobServer server = new BlobServer(config, new VoidBlobStore()); + final BlobCacheService cache = new BlobCacheService( + new InetSocketAddress("localhost", server.getPort()), config, + new VoidBlobStore())) { + ConcurrentMap, Long> transientBlobExpiryTimes = + cache.getTransientBlobService().getBlobExpiryTimes(); + + server.start(); + + final TransientBlobKey key1 = + (TransientBlobKey) put(server, jobId, data, TRANSIENT_BLOB); + final TransientBlobKey key2 = + (TransientBlobKey) put(server, jobId, data2, TRANSIENT_BLOB); + + // access key1, verify expiry times + cleanupLowerBound = System.currentTimeMillis() + cleanupInterval; + verifyContents(cache, jobId, key1, data); + final Long key1ExpiryFirstAccess = transientBlobExpiryTimes.get(Tuple2.of(jobId, key1)); + assertThat(key1ExpiryFirstAccess, greaterThanOrEqualTo(cleanupLowerBound)); + assertNull(transientBlobExpiryTimes.get(Tuple2.of(jobId, key2))); + + // access key2, verify expiry times (delay at least 1ms to also verify key1 expiry is unchanged) + Thread.sleep(1); + cleanupLowerBound = System.currentTimeMillis() + cleanupInterval; + verifyContents(cache, jobId, key2, data2); + assertEquals(key1ExpiryFirstAccess, transientBlobExpiryTimes.get(Tuple2.of(jobId, key1))); + assertThat(transientBlobExpiryTimes.get(Tuple2.of(jobId, key2)), + greaterThanOrEqualTo(cleanupLowerBound)); + + // files are cached now for the given TTL - remove from server so that they are not re-downloaded + if (jobId != null) { + server.cleanupJob(jobId); + } else { + server.deleteFromCache(key1); + server.deleteFromCache(key2); + } + checkFileCountForJob(0, jobId, server); + + // cleanup task is run every cleanupInterval seconds + // => unaccessed file should remain at most 2*cleanupInterval seconds + // (use 3*cleanupInterval to check that we can still access it) + final long finishTime = System.currentTimeMillis() + 3 * cleanupInterval; + + final ExecutorService executor = Executors.newFixedThreadPool(numberConcurrentGetOperations); + for (int i = 0; i < numberConcurrentGetOperations; i++) { + CompletableFuture getOperation = CompletableFuture + .supplyAsync( + () -> { + try { + // constantly access key1 so this should not get deleted + while (System.currentTimeMillis() < finishTime) { + get(cache, jobId, key1); + } + + return null; + } catch (IOException e) { + throw new CompletionException(new FlinkException( + "Could not retrieve blob.", e)); + } + }, executor); + + getOperations.add(getOperation); + } + + FutureUtils.ConjunctFuture> filesFuture = FutureUtils.combineAll(getOperations); + filesFuture.get(); + + verifyDeletedEventually(server, jobId, key1, key2); + } + } + /** * Checks that BLOBs for the given jobId are cleaned up eventually (after calling * {@link PermanentBlobCache#releaseJob(JobID)}, which is not done by this method!) (waits at @@ -307,78 +437,4 @@ static void verifyJobCleanup(PermanentBlobCache cache, JobID jobId, Listfalse), or thrown (true) - * - * @return number of files existing at {@link BlobServer#getStorageLocation(JobID, BlobKey)} and - * {@link PermanentBlobCache#getStorageLocation(JobID, BlobKey)}, respectively - */ - public static int checkFilesExist( - JobID jobId, Collection keys, PermanentBlobService blobService, boolean doThrow) - throws IOException { - - int numFiles = 0; - - for (BlobKey key : keys) { - final File blobFile; - if (blobService instanceof BlobServer) { - BlobServer server = (BlobServer) blobService; - blobFile = server.getStorageLocation(jobId, key); - } else { - PermanentBlobCache cache = (PermanentBlobCache) blobService; - blobFile = cache.getStorageLocation(jobId, key); - } - if (blobFile.exists()) { - ++numFiles; - } else if (doThrow) { - throw new IOException("File " + blobFile + " does not exist."); - } - } - - return numFiles; - } - - /** - * Checks how many of the files given by blob keys are accessible. - * - * @param expectedCount - * number of expected files in the blob service for the given job - * @param jobId - * ID of a job - * @param blobService - * BLOB store to use - */ - public static void checkFileCountForJob( - int expectedCount, JobID jobId, PermanentBlobService blobService) - throws IOException { - - final File jobDir; - if (blobService instanceof BlobServer) { - BlobServer server = (BlobServer) blobService; - jobDir = server.getStorageLocation(jobId, new PermanentBlobKey()).getParentFile(); - } else { - PermanentBlobCache cache = (PermanentBlobCache) blobService; - jobDir = cache.getStorageLocation(jobId, new PermanentBlobKey()).getParentFile(); - } - File[] blobsForJob = jobDir.listFiles(); - if (blobsForJob == null) { - if (expectedCount != 0) { - throw new IOException("File " + jobDir + " does not exist."); - } - } else { - assertEquals("Too many/few files in job dir: " + - Arrays.asList(blobsForJob).toString(), expectedCount, - blobsForJob.length); - } - } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCachePutTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCachePutTest.java index 56258c30c9cd7..264c7cbc4031f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCachePutTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCachePutTest.java @@ -55,11 +55,11 @@ import java.util.concurrent.Executors; import java.util.function.Supplier; -import static org.apache.flink.runtime.blob.BlobCacheCleanupTest.checkFilesExist; import static org.apache.flink.runtime.blob.BlobKey.BlobType.PERMANENT_BLOB; import static org.apache.flink.runtime.blob.BlobKey.BlobType.TRANSIENT_BLOB; import static org.apache.flink.runtime.blob.BlobKeyTest.verifyKeyDifferentHashEquals; import static org.apache.flink.runtime.blob.BlobKeyTest.verifyType; +import static org.apache.flink.runtime.blob.BlobServerCleanupTest.checkFilesExist; import static org.apache.flink.runtime.blob.BlobServerGetTest.verifyDeleted; import static org.apache.flink.runtime.blob.BlobServerPutTest.BlockingInputStream; import static org.apache.flink.runtime.blob.BlobServerPutTest.ChunkedInputStream; @@ -909,19 +909,21 @@ private void testConcurrentPutOperations( * BLOB server * @param jobId * job ID or null if job-unrelated - * @param key - * key identifying the BLOB to request + * @param keys + * key(s) identifying the BLOB to request */ - static void verifyDeletedEventually(BlobServer server, @Nullable JobID jobId, BlobKey key) + static void verifyDeletedEventually(BlobServer server, @Nullable JobID jobId, BlobKey... keys) throws IOException, InterruptedException { long deadline = System.currentTimeMillis() + 30_000L; do { Thread.sleep(10); } - while (checkFilesExist(jobId, Collections.singletonList(key), server, false) != 0 && + while (checkFilesExist(jobId, Arrays.asList(keys), server, false) != 0 && System.currentTimeMillis() < deadline); - verifyDeleted(server, jobId, key); + for (BlobKey key : keys) { + verifyDeleted(server, jobId, key); + } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java new file mode 100644 index 0000000000000..aafba306dfa0c --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java @@ -0,0 +1,264 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.blob; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.BlobServerOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.TestLogger; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import javax.annotation.Nullable; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Random; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import static org.apache.flink.runtime.blob.BlobCachePutTest.verifyDeletedEventually; +import static org.apache.flink.runtime.blob.BlobKey.BlobType.PERMANENT_BLOB; +import static org.apache.flink.runtime.blob.BlobKey.BlobType.TRANSIENT_BLOB; +import static org.apache.flink.runtime.blob.BlobServerGetTest.get; +import static org.apache.flink.runtime.blob.BlobServerPutTest.put; +import static org.apache.flink.runtime.blob.BlobServerPutTest.verifyContents; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; + +/** + * A few tests for the cleanup of transient BLOBs at the {@link BlobServer}. + */ +public class BlobServerCleanupTest extends TestLogger { + + private final Random rnd = new Random(); + + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @Test + public void testTransientBlobNoJobCleanup() + throws IOException, InterruptedException, ExecutionException { + testTransientBlobCleanup(null); + } + + @Test + public void testTransientBlobForJobCleanup() + throws IOException, InterruptedException, ExecutionException { + testTransientBlobCleanup(new JobID()); + } + + /** + * Tests that {@link TransientBlobCache} cleans up after a default TTL and keeps files which are + * constantly accessed. + */ + private void testTransientBlobCleanup(@Nullable final JobID jobId) + throws IOException, InterruptedException, ExecutionException { + + // 1s should be a safe-enough buffer to still check for existence after a BLOB's last access + long cleanupInterval = 1L; // in seconds + final int numberConcurrentGetOperations = 3; + + final List> getOperations = new ArrayList<>(numberConcurrentGetOperations); + + byte[] data = new byte[2000000]; + rnd.nextBytes(data); + byte[] data2 = Arrays.copyOfRange(data, 10, 54); + + Configuration config = new Configuration(); + config.setString(BlobServerOptions.STORAGE_DIRECTORY, + temporaryFolder.newFolder().getAbsolutePath()); + config.setLong(BlobServerOptions.CLEANUP_INTERVAL, cleanupInterval); + + long cleanupLowerBound; + + try (BlobServer server = new BlobServer(config, new VoidBlobStore())) { + + ConcurrentMap, Long> transientBlobExpiryTimes = + server.getBlobExpiryTimes(); + + server.start(); + + // after put(), files are cached for the given TTL + cleanupLowerBound = System.currentTimeMillis() + cleanupInterval; + final TransientBlobKey key1 = + (TransientBlobKey) put(server, jobId, data, TRANSIENT_BLOB); + final Long key1ExpiryAfterPut = transientBlobExpiryTimes.get(Tuple2.of(jobId, key1)); + assertThat(key1ExpiryAfterPut, greaterThanOrEqualTo(cleanupLowerBound)); + + cleanupLowerBound = System.currentTimeMillis() + cleanupInterval; + final TransientBlobKey key2 = + (TransientBlobKey) put(server, jobId, data2, TRANSIENT_BLOB); + final Long key2ExpiryAfterPut = transientBlobExpiryTimes.get(Tuple2.of(jobId, key2)); + assertThat(key2ExpiryAfterPut, greaterThanOrEqualTo(cleanupLowerBound)); + + // check that HA contents are not cleaned up + final JobID jobIdHA = (jobId == null) ? new JobID() : jobId; + final BlobKey keyHA = put(server, jobIdHA, data, PERMANENT_BLOB); + + // access key1, verify expiry times (delay at least 1ms to also verify key2 expiry is unchanged) + Thread.sleep(1); + cleanupLowerBound = System.currentTimeMillis() + cleanupInterval; + verifyContents(server, jobId, key1, data); + final Long key1ExpiryAfterGet = transientBlobExpiryTimes.get(Tuple2.of(jobId, key1)); + assertThat(key1ExpiryAfterGet, greaterThan(key1ExpiryAfterPut)); + assertThat(key1ExpiryAfterGet, greaterThanOrEqualTo(cleanupLowerBound)); + assertEquals(key2ExpiryAfterPut, transientBlobExpiryTimes.get(Tuple2.of(jobId, key2))); + + // access key2, verify expiry times (delay at least 1ms to also verify key1 expiry is unchanged) + Thread.sleep(1); + cleanupLowerBound = System.currentTimeMillis() + cleanupInterval; + verifyContents(server, jobId, key2, data2); + assertEquals(key1ExpiryAfterGet, transientBlobExpiryTimes.get(Tuple2.of(jobId, key1))); + assertThat(transientBlobExpiryTimes.get(Tuple2.of(jobId, key2)), + greaterThan(key2ExpiryAfterPut)); + assertThat(transientBlobExpiryTimes.get(Tuple2.of(jobId, key2)), + greaterThanOrEqualTo(cleanupLowerBound)); + + // cleanup task is run every cleanupInterval seconds + // => unaccessed file should remain at most 2*cleanupInterval seconds + // (use 3*cleanupInterval to check that we can still access it) + final long finishTime = System.currentTimeMillis() + 3 * cleanupInterval; + + final ExecutorService executor = Executors.newFixedThreadPool(numberConcurrentGetOperations); + for (int i = 0; i < numberConcurrentGetOperations; i++) { + CompletableFuture getOperation = CompletableFuture + .supplyAsync( + () -> { + try { + // constantly access key1 so this should not get deleted + while (System.currentTimeMillis() < finishTime) { + get(server, jobId, key1); + } + + return null; + } catch (IOException e) { + throw new CompletionException(new FlinkException( + "Could not retrieve blob.", e)); + } + }, executor); + + getOperations.add(getOperation); + } + + FutureUtils.ConjunctFuture> filesFuture = FutureUtils.combineAll(getOperations); + filesFuture.get(); + + verifyDeletedEventually(server, jobId, key1, key2); + + // HA content should be unaffected + verifyContents(server, jobIdHA, keyHA, data); + } + } + + /** + * Checks how many of the files given by blob keys are accessible. + * + * @param jobId + * ID of a job + * @param keys + * blob keys to check + * @param blobService + * BLOB store to use + * @param doThrow + * whether exceptions should be ignored (false), or thrown (true) + * + * @return number of files existing at {@link BlobServer#getStorageLocation(JobID, BlobKey)} and + * {@link PermanentBlobCache#getStorageLocation(JobID, BlobKey)}, respectively + */ + public static int checkFilesExist( + JobID jobId, Collection keys, T blobService, boolean doThrow) + throws IOException { + + int numFiles = 0; + + for (BlobKey key : keys) { + final File blobFile; + if (blobService instanceof BlobServer) { + BlobServer server = (BlobServer) blobService; + blobFile = server.getStorageLocation(jobId, key); + } else if (blobService instanceof PermanentBlobCache) { + PermanentBlobCache cache = (PermanentBlobCache) blobService; + blobFile = cache.getStorageLocation(jobId, key); + } else if (blobService instanceof TransientBlobCache) { + TransientBlobCache cache = (TransientBlobCache) blobService; + blobFile = cache.getStorageLocation(jobId, key); + } else { + throw new UnsupportedOperationException( + "unsupported BLOB service class: " + blobService.getClass().getCanonicalName()); + } + if (blobFile.exists()) { + ++numFiles; + } else if (doThrow) { + throw new IOException("File " + blobFile + " does not exist."); + } + } + + return numFiles; + } + + /** + * Checks how many of the files given by blob keys are accessible. + * + * @param expectedCount + * number of expected files in the blob service for the given job + * @param jobId + * ID of a job + * @param blobService + * BLOB store to use + */ + public static void checkFileCountForJob( + int expectedCount, JobID jobId, PermanentBlobService blobService) + throws IOException { + + final File jobDir; + if (blobService instanceof BlobServer) { + BlobServer server = (BlobServer) blobService; + jobDir = server.getStorageLocation(jobId, new PermanentBlobKey()).getParentFile(); + } else { + PermanentBlobCache cache = (PermanentBlobCache) blobService; + jobDir = cache.getStorageLocation(jobId, new PermanentBlobKey()).getParentFile(); + } + File[] blobsForJob = jobDir.listFiles(); + if (blobsForJob == null) { + if (expectedCount != 0) { + throw new IOException("File " + jobDir + " does not exist."); + } + } else { + assertEquals("Too many/few files in job dir: " + + Arrays.asList(blobsForJob).toString(), expectedCount, + blobsForJob.length); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java index fde21ba36d75a..21680349570d4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java @@ -44,10 +44,10 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import static org.apache.flink.runtime.blob.BlobCacheCleanupTest.checkFileCountForJob; import static org.apache.flink.runtime.blob.BlobKey.BlobType.PERMANENT_BLOB; import static org.apache.flink.runtime.blob.BlobKey.BlobType.TRANSIENT_BLOB; import static org.apache.flink.runtime.blob.BlobKeyTest.verifyKeyDifferentHashEquals; +import static org.apache.flink.runtime.blob.BlobServerCleanupTest.checkFileCountForJob; import static org.apache.flink.runtime.blob.BlobServerGetTest.verifyDeleted; import static org.apache.flink.runtime.blob.BlobServerPutTest.put; import static org.apache.flink.runtime.blob.BlobServerPutTest.verifyContents; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java index cb5d6087a8d45..080f7432cdcd4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java @@ -42,8 +42,8 @@ import java.util.Collections; import java.util.List; -import static org.apache.flink.runtime.blob.BlobCacheCleanupTest.checkFileCountForJob; -import static org.apache.flink.runtime.blob.BlobCacheCleanupTest.checkFilesExist; +import static org.apache.flink.runtime.blob.BlobServerCleanupTest.checkFileCountForJob; +import static org.apache.flink.runtime.blob.BlobServerCleanupTest.checkFilesExist; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; From 29bea6f319472a09e8cac31dc47c38d8b1ddbc00 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Tue, 25 Jul 2017 14:49:58 +0200 Subject: [PATCH 3/6] [FLINK-7262][blob] remove the unused FallbackLibraryCacheManager This class was basically only used in unit tests and not really needed there either. The code path inside TaskManager was also dead. --- .../FallbackLibraryCacheManager.java | 62 ------------------- .../runtime/taskmanager/TaskManager.scala | 39 +++++------- .../runtime/util/JvmExitOnFatalErrorTest.java | 9 ++- .../tasks/BlockingCheckpointsTest.java | 9 ++- .../tasks/InterruptSensitiveRestoreTest.java | 7 ++- .../tasks/StreamTaskTerminationTest.java | 7 ++- 6 files changed, 39 insertions(+), 94 deletions(-) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FallbackLibraryCacheManager.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FallbackLibraryCacheManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FallbackLibraryCacheManager.java deleted file mode 100644 index 6564e9e78d169..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FallbackLibraryCacheManager.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.execution.librarycache; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.runtime.blob.PermanentBlobKey; -import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.net.URL; -import java.util.Collection; - -public class FallbackLibraryCacheManager implements LibraryCacheManager { - - private static Logger LOG = LoggerFactory.getLogger(FallbackLibraryCacheManager.class); - - @Override - public ClassLoader getClassLoader(JobID id) { - return getClass().getClassLoader(); - } - - @Override - public void registerJob(JobID id, Collection requiredJarFiles, Collection requiredClasspaths) { - LOG.warn("FallbackLibraryCacheManager cannot download files associated with blob keys."); - } - - @Override - public void registerTask(JobID id, ExecutionAttemptID execution, Collection requiredJarFiles, - Collection requiredClasspaths) { - LOG.warn("FallbackLibraryCacheManager cannot download files associated with blob keys."); - } - - @Override - public void unregisterJob(JobID id) { - LOG.warn("FallbackLibraryCacheManager does not book keeping of job IDs."); - } - - @Override - public void unregisterTask(JobID id, ExecutionAttemptID execution) { - LOG.warn("FallbackLibraryCacheManager does not book keeping of job IDs."); - } - - @Override - public void shutdown() {} -} diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index 8d1f6f760f999..f11c837690aa0 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -42,7 +42,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID import org.apache.flink.runtime.concurrent.Executors import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor import org.apache.flink.runtime.execution.ExecutionState -import org.apache.flink.runtime.execution.librarycache.{BlobLibraryCacheManager, FallbackLibraryCacheManager, LibraryCacheManager} +import org.apache.flink.runtime.execution.librarycache.{BlobLibraryCacheManager, LibraryCacheManager} import org.apache.flink.runtime.executiongraph.{ExecutionAttemptID, PartitionInfo} import org.apache.flink.runtime.filecache.FileCache import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils.AddressResolution @@ -955,31 +955,26 @@ class TaskManager( } // start a blob service, if a blob server is specified - if (blobPort > 0) { - val jmHost = jobManager.path.address.host.getOrElse("localhost") - val address = new InetSocketAddress(jmHost, blobPort) + val jmHost = jobManager.path.address.host.getOrElse("localhost") + val address = new InetSocketAddress(jmHost, blobPort) - log.info(s"Determined BLOB server address to be $address. Starting BLOB cache.") + log.info(s"Determined BLOB server address to be $address. Starting BLOB cache.") - try { - val blobcache = new BlobCacheService( - address, - config.getConfiguration(), - highAvailabilityServices.createBlobStore()) - blobCache = Option(blobcache) - libraryCacheManager = Some( - new BlobLibraryCacheManager( + try { + val blobcache = new BlobCacheService( + address, + config.getConfiguration(), + highAvailabilityServices.createBlobStore()) + blobCache = Option(blobcache) + libraryCacheManager = Some( + new BlobLibraryCacheManager( blobcache.getPermanentBlobService, config.getClassLoaderResolveOrder())) - } - catch { - case e: Exception => - val message = "Could not create BLOB cache or library cache." - log.error(message, e) - throw new RuntimeException(message, e) - } } - else { - libraryCacheManager = Some(new FallbackLibraryCacheManager) + catch { + case e: Exception => + val message = "Could not create BLOB cache or library cache." + log.error(message, e) + throw new RuntimeException(message, e) } taskManagerMetricGroup = diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java index 57a383172482d..82b6b298361f8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java @@ -34,7 +34,8 @@ import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.execution.ExecutionState; -import org.apache.flink.runtime.execution.librarycache.FallbackLibraryCacheManager; +import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager; +import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.JobInformation; import org.apache.flink.runtime.executiongraph.TaskInformation; @@ -182,8 +183,10 @@ public static void main(String[] args) throws Exception { new NoOpTaskManagerActions(), new NoOpInputSplitProvider(), new NoOpCheckpointResponder(), - blobService, - new FallbackLibraryCacheManager(), + blobService, + new BlobLibraryCacheManager( + blobService.getPermanentBlobService(), + FlinkUserCodeClassLoaders.ResolveOrder.CHILD_FIRST), new FileCache(tmInfo.getTmpDirectories()), tmInfo, new UnregisteredTaskMetricsGroup(), diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/BlockingCheckpointsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/BlockingCheckpointsTest.java index b43f30a44b87f..5a38615d26c96 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/BlockingCheckpointsTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/BlockingCheckpointsTest.java @@ -37,7 +37,8 @@ import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.execution.ExecutionState; -import org.apache.flink.runtime.execution.librarycache.FallbackLibraryCacheManager; +import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager; +import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.JobInformation; import org.apache.flink.runtime.executiongraph.TaskInformation; @@ -160,8 +161,10 @@ private static Task createTask(Configuration taskConfig) throws IOException { mock(TaskManagerActions.class), mock(InputSplitProvider.class), mock(CheckpointResponder.class), - blobService, - new FallbackLibraryCacheManager(), + blobService, + new BlobLibraryCacheManager( + blobService.getPermanentBlobService(), + FlinkUserCodeClassLoaders.ResolveOrder.CHILD_FIRST), new FileCache(new String[] { EnvironmentInformation.getTemporaryFileDirectory() }), new TestingTaskManagerRuntimeInfo(), new UnregisteredTaskMetricsGroup(), diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java index 19823ecd21d50..d049e316260e9 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java @@ -34,7 +34,8 @@ import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.execution.ExecutionState; -import org.apache.flink.runtime.execution.librarycache.FallbackLibraryCacheManager; +import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager; +import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.JobInformation; import org.apache.flink.runtime.executiongraph.TaskInformation; @@ -262,7 +263,9 @@ private static Task createTask( mock(InputSplitProvider.class), mock(CheckpointResponder.class), blobService, - new FallbackLibraryCacheManager(), + new BlobLibraryCacheManager( + blobService.getPermanentBlobService(), + FlinkUserCodeClassLoaders.ResolveOrder.CHILD_FIRST), new FileCache(new String[] { EnvironmentInformation.getTemporaryFileDirectory() }), new TestingTaskManagerRuntimeInfo(), new UnregisteredTaskMetricsGroup(), diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java index a989fa4d2e43d..7eb2655083ef5 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java @@ -34,7 +34,8 @@ import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.execution.ExecutionState; -import org.apache.flink.runtime.execution.librarycache.FallbackLibraryCacheManager; +import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager; +import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.JobInformation; import org.apache.flink.runtime.executiongraph.TaskInformation; @@ -158,7 +159,9 @@ public void testConcurrentAsyncCheckpointCannotFailFinishedStreamTask() throws E mock(InputSplitProvider.class), mock(CheckpointResponder.class), blobService, - new FallbackLibraryCacheManager(), + new BlobLibraryCacheManager( + blobService.getPermanentBlobService(), + FlinkUserCodeClassLoaders.ResolveOrder.CHILD_FIRST), mock(FileCache.class), taskManagerRuntimeInfo, new UnregisteredTaskMetricsGroup(), From 28c092d3027088db534344f5a3c940a849a9a264 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Thu, 27 Jul 2017 13:56:13 +0200 Subject: [PATCH 4/6] [hotfix] make BlobServer#serverSSLContext final --- .../main/java/org/apache/flink/runtime/blob/BlobServer.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) 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 402a73993c68f..ec6044af8bc24 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 @@ -78,7 +78,7 @@ public class BlobServer extends Thread implements BlobService, PermanentBlobServ private final ServerSocket serverSocket; /** The SSL server context if ssl is enabled for the connections */ - private SSLContext serverSSLContext = null; + private final SSLContext serverSSLContext; /** Blob Server configuration */ private final Configuration blobServiceConfiguration; @@ -178,6 +178,8 @@ public BlobServer(Configuration config, BlobStore blobStore) throws IOException } catch (Exception e) { throw new IOException("Failed to initialize SSLContext for the blob server", e); } + } else { + serverSSLContext = null; } // ----------------------- start the server ------------------- From 8133deb8cde946d3c53c4e1c949ff7248e67b3c8 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Fri, 6 Oct 2017 12:02:54 +0200 Subject: [PATCH 5/6] [FLINK-6046] offload large data from the TaskDeploymentDescriptor to the BlobServer This only includes potentially big parts, i.e. serializedJobInformation and serializedTaskInformation, which are both offloaded only once for all parallel instances. - adds a configurable akka.rpc.offload.minsize threshold for large data - serialized task information is uploaded only once per task, irrespective of the parallelism --- docs/ops/config.md | 4 + .../flink/configuration/AkkaOptions.java | 1 + .../configuration/JobManagerOptions.java | 7 + .../apache/flink/runtime/blob/BlobKey.java | 3 +- .../apache/flink/runtime/blob/BlobServer.java | 9 + .../deployment/TaskDeploymentDescriptor.java | 181 +++++++++++++++++- .../executiongraph/ExecutionGraph.java | 129 ++++++++++--- .../executiongraph/ExecutionGraphBuilder.java | 7 +- .../executiongraph/ExecutionJobVertex.java | 110 ++++++++--- .../executiongraph/ExecutionVertex.java | 38 +++- .../flink/runtime/jobmaster/JobMaster.java | 1 + .../runtime/taskexecutor/TaskExecutor.java | 37 ++-- .../flink/runtime/jobmanager/JobManager.scala | 1 + .../runtime/taskmanager/TaskManager.scala | 15 +- .../CheckpointSettingsSerializableTest.java | 1 + ...ecutionGraphCheckpointCoordinatorTest.java | 3 +- .../TaskDeploymentDescriptorTest.java | 5 +- .../ExecutionGraphDeploymentTest.java | 124 ++++++++---- ...utionGraphDeploymentWithBlobCacheTest.java | 59 ++++++ ...tionGraphDeploymentWithBlobServerTest.java | 101 ++++++++++ .../ExecutionGraphSchedulingTest.java | 1 + .../ExecutionGraphTestUtils.java | 47 ++++- .../ExecutionVertexLocalityTest.java | 1 + .../executiongraph/FailoverRegionTest.java | 12 +- .../executiongraph/GlobalModVersionTest.java | 3 +- .../IndividualRestartsConcurrencyTest.java | 3 +- ...ipelinedRegionFailoverConcurrencyTest.java | 3 +- .../RestartPipelinedRegionStrategyTest.java | 12 +- .../PipelinedFailoverRegionBuildingTest.java | 1 + .../runtime/jobmaster/JobMasterTest.java | 23 ++- .../taskexecutor/TaskExecutorTest.java | 10 +- .../runtime/taskmanager/TaskManagerTest.java | 5 +- .../testingUtils/TestingTaskManagerLike.scala | 24 +-- .../partitioner/RescalePartitionerTest.java | 3 +- 34 files changed, 816 insertions(+), 168 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentWithBlobCacheTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentWithBlobServerTest.java diff --git a/docs/ops/config.md b/docs/ops/config.md index ac9c813ca64c4..5e652c953b427 100644 --- a/docs/ops/config.md +++ b/docs/ops/config.md @@ -309,6 +309,10 @@ The following parameters configure Flink's JobManager and TaskManagers. - `taskmanager.exit-on-fatal-akka-error`: Whether the TaskManager shall be terminated in case of a fatal Akka error (quarantining event). (DEFAULT: **false**) +- `jobmanager.tdd.offload.minsize`: Maximum size of of the `TaskDeploymentDescriptor`'s serialized +task and job information to still transmit them via RPC. Larger blobs may be offloaded to the BLOB +server. (DEFAULT: **1 KiB**). + ### Distributed Coordination (via Akka) - `akka.ask.timeout`: Timeout used for all futures and blocking Akka calls. If Flink fails due to timeouts then you should try to increase this value. Timeouts can be caused by slow machines or a congested network. The timeout value requires a time-unit specifier (ms/s/min/h/d) (DEFAULT: **10 s**). diff --git a/flink-core/src/main/java/org/apache/flink/configuration/AkkaOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/AkkaOptions.java index 9bfc237546473..55dcd01b35ca1 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/AkkaOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/AkkaOptions.java @@ -139,4 +139,5 @@ public class AkkaOptions { public static final ConfigOption JVM_EXIT_ON_FATAL_ERROR = ConfigOptions .key("akka.jvm-exit-on-fatal-error") .defaultValue(true); + } diff --git a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java index ef3306e4c82fa..a39927dde2331 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java @@ -96,6 +96,13 @@ public class JobManagerOptions { key("jobmanager.archive.fs.dir") .noDefaultValue(); + /** + * The maximum size of the TaskDeploymentDescriptor's serialized task and job + * information to still transmit them via RPC. Larger blobs may be offloaded to the BLOB server. + */ + public static final ConfigOption TDD_OFFLOAD_MINSIZE = key("jobmanager.tdd.offload.minsize") + .defaultValue(1_024); // 1KiB by default + // --------------------------------------------------------------------------------------------- private JobManagerOptions() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobKey.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobKey.java index ef2d64db3c0f1..4b1d498586971 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobKey.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobKey.java @@ -183,7 +183,8 @@ static BlobKey createKey(BlobType type, byte[] key, byte[] random) { * * @return a 20 bit hash of the contents the key refers to */ - byte[] getHash() { + @VisibleForTesting + public byte[] getHash() { return key; } 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 ec6044af8bc24..7249c8b54cd21 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 @@ -879,6 +879,15 @@ public TransientBlobService getTransientBlobService() { return this; } + /** + * Returns the configuration used by the BLOB server. + * + * @return configuration + */ + public final Configuration getConfiguration() { + return blobServiceConfiguration; + } + /** * Returns the port on which the server is listening. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java index 1fa5eb5b51484..0c7e308bfd7bd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java @@ -18,6 +18,9 @@ package org.apache.flink.runtime.deployment; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.blob.PermanentBlobKey; +import org.apache.flink.runtime.blob.PermanentBlobService; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; @@ -26,7 +29,12 @@ import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; +import javax.annotation.Nullable; + +import java.io.File; +import java.io.IOException; import java.io.Serializable; +import java.nio.file.Files; import java.util.Collection; /** @@ -36,11 +44,82 @@ public final class TaskDeploymentDescriptor implements Serializable { private static final long serialVersionUID = -3233562176034358530L; - /** Serialized job information. */ - private final SerializedValue serializedJobInformation; + /** + * Wrapper class for serialized values which may be offloaded to the {@link + * org.apache.flink.runtime.blob.BlobServer} or not. + * + * @param + * type of the serialized value + */ + @SuppressWarnings("unused") + public static class MaybeOffloaded implements Serializable { + private static final long serialVersionUID = 5977104446396536907L; + } + + /** + * A serialized value that is not offloaded to the {@link org.apache.flink.runtime.blob.BlobServer}. + * + * @param + * type of the serialized value + */ + public static class NonOffloaded extends MaybeOffloaded { + private static final long serialVersionUID = 4246628617754862463L; + + /** + * The serialized value. + */ + public SerializedValue serializedValue; + + @SuppressWarnings("unused") + public NonOffloaded() { + } + + public NonOffloaded(SerializedValue serializedValue) { + this.serializedValue = Preconditions.checkNotNull(serializedValue); + } + } + + /** + * Reference to a serialized value that was offloaded to the {@link + * org.apache.flink.runtime.blob.BlobServer}. + * + * @param + * type of the serialized value + */ + public static class Offloaded extends MaybeOffloaded { + private static final long serialVersionUID = 4544135485379071679L; + + /** + * The key of the offloaded value BLOB. + */ + public PermanentBlobKey serializedValueKey; + + @SuppressWarnings("unused") + public Offloaded() { + } - /** Serialized task information. */ - private final SerializedValue serializedTaskInformation; + public Offloaded(PermanentBlobKey serializedValueKey) { + this.serializedValueKey = Preconditions.checkNotNull(serializedValueKey); + } + } + + /** + * Serialized job information or null if offloaded. + */ + private MaybeOffloaded serializedJobInformation; + + /** + * Serialized task information or null if offloaded. + */ + private MaybeOffloaded serializedTaskInformation; + + /** + * The ID referencing the job this task belongs to. + * + *

NOTE: this is redundant to the information stored in {@link #serializedJobInformation} but + * needed in order to restore offloaded data.

+ */ + private final JobID jobId; /** The ID referencing the attempt to execute the task. */ private final ExecutionAttemptID executionId; @@ -67,8 +146,9 @@ public final class TaskDeploymentDescriptor implements Serializable { private final TaskStateSnapshot taskStateHandles; public TaskDeploymentDescriptor( - SerializedValue serializedJobInformation, - SerializedValue serializedTaskInformation, + JobID jobId, + MaybeOffloaded serializedJobInformation, + MaybeOffloaded serializedTaskInformation, ExecutionAttemptID executionAttemptId, AllocationID allocationId, int subtaskIndex, @@ -78,8 +158,11 @@ public TaskDeploymentDescriptor( Collection resultPartitionDeploymentDescriptors, Collection inputGateDeploymentDescriptors) { + this.jobId = Preconditions.checkNotNull(jobId); + this.serializedJobInformation = Preconditions.checkNotNull(serializedJobInformation); this.serializedTaskInformation = Preconditions.checkNotNull(serializedTaskInformation); + this.executionId = Preconditions.checkNotNull(executionAttemptId); this.allocationId = Preconditions.checkNotNull(allocationId); @@ -101,19 +184,46 @@ public TaskDeploymentDescriptor( /** * Return the sub task's serialized job information. * - * @return serialized job information + * @return serialized job information (may be null before a call to {@link + * #loadBigData(PermanentBlobService)}). */ + @Nullable public SerializedValue getSerializedJobInformation() { - return serializedJobInformation; + if (serializedJobInformation instanceof NonOffloaded) { + NonOffloaded jobInformation = + (NonOffloaded) serializedJobInformation; + return jobInformation.serializedValue; + } else { + throw new IllegalStateException( + "Trying to work with offloaded serialized job information."); + } } /** * Return the sub task's serialized task information. * - * @return serialized task information + * @return serialized task information (may be null before a call to {@link + * #loadBigData(PermanentBlobService)}). */ + @Nullable public SerializedValue getSerializedTaskInformation() { - return serializedTaskInformation; + if (serializedJobInformation instanceof NonOffloaded) { + NonOffloaded jobInformation = + (NonOffloaded) serializedTaskInformation; + return jobInformation.serializedValue; + } else { + throw new IllegalStateException( + "Trying to work with offloaded serialized job information."); + } + } + + /** + * Returns the task's job ID. + * + * @return the job ID this task belongs to + */ + public JobID getJobId() { + return jobId; } public ExecutionAttemptID getExecutionAttemptId() { @@ -161,6 +271,57 @@ public AllocationID getAllocationId() { return allocationId; } + /** + * Loads externalized data from the BLOB store back to the object. + * + * @param blobService + * the blob store to use (may be null if {@link #serializedJobInformation} and {@link + * #serializedTaskInformation} are non-null) + * + * @throws IOException + * during errors retrieving or reading the BLOBs + * @throws ClassNotFoundException + * Class of a serialized object cannot be found. + */ + public void loadBigData(@Nullable PermanentBlobService blobService) + throws IOException, ClassNotFoundException { + + // re-integrate offloaded job info from blob + // here, if this fails, we need to throw the exception as there is no backup path anymore + if (serializedJobInformation instanceof Offloaded) { + PermanentBlobKey jobInfoKey = ((Offloaded) serializedJobInformation).serializedValueKey; + + Preconditions.checkNotNull(blobService); + + final File dataFile = blobService.getFile(jobId, jobInfoKey); + // NOTE: Do not delete the job info BLOB since it may be needed again during recovery. + // (it is deleted automatically on the BLOB server and cache when the job + // enters a terminal state) + SerializedValue serializedValue = + SerializedValue.fromBytes(Files.readAllBytes(dataFile.toPath())); + serializedJobInformation = new NonOffloaded<>(serializedValue); + } + + // re-integrate offloaded task info from blob + if (serializedTaskInformation instanceof Offloaded) { + PermanentBlobKey taskInfoKey = ((Offloaded) serializedTaskInformation).serializedValueKey; + + Preconditions.checkNotNull(blobService); + + final File dataFile = blobService.getFile(jobId, taskInfoKey); + // NOTE: Do not delete the task info BLOB since it may be needed again during recovery. + // (it is deleted automatically on the BLOB server and cache when the job + // enters a terminal state) + SerializedValue serializedValue = + SerializedValue.fromBytes(Files.readAllBytes(dataFile.toPath())); + serializedTaskInformation = new NonOffloaded<>(serializedValue); + } + + // make sure that the serialized job and task information fields are filled + Preconditions.checkNotNull(serializedJobInformation); + Preconditions.checkNotNull(serializedTaskInformation); + } + @Override public String toString() { return String.format("TaskDeploymentDescriptor [execution id: %s, attempt: %d, " + diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 42cefc62fac55..121b788041b9f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -27,10 +27,12 @@ import org.apache.flink.api.common.accumulators.AccumulatorHelper; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.runtime.JobException; import org.apache.flink.runtime.StoppingException; import org.apache.flink.runtime.accumulators.AccumulatorSnapshot; import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult; +import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.blob.PermanentBlobKey; import org.apache.flink.runtime.checkpoint.CheckpointCoordinator; import org.apache.flink.runtime.checkpoint.CheckpointIDCounter; @@ -75,6 +77,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; import java.io.IOException; import java.net.URL; import java.util.ArrayList; @@ -124,20 +127,20 @@ * about deployment of tasks and updates in the task status always use the ExecutionAttemptID to * address the message receiver. * - * + * *

Global and local failover

- * + * * The Execution Graph has two failover modes: global failover and local failover. - * + * *

A global failover aborts the task executions for all vertices and restarts whole * data flow graph from the last completed checkpoint. Global failover is considered the * "fallback strategy" that is used when a local failover is unsuccessful, or when a issue is * found in the state of the ExecutionGraph that could mark it as inconsistent (caused by a bug). - * + * *

A local failover is triggered when an individual vertex execution (a task) fails. * The local failover is coordinated by the {@link FailoverStrategy}. A local failover typically * attempts to restart as little as possible, but as much as necessary. - * + * *

Between local- and global failover, the global failover always takes precedence, because it * is the core mechanism that the ExecutionGraph relies on to bring back consistency. The * guard that, the ExecutionGraph maintains a global modification version, which is incremented @@ -145,7 +148,7 @@ * failure). Local failover is always scoped by the modification version that the execution graph * had when the failover was triggered. If a new global modification version is reached during * local failover (meaning there is a concurrent global failover), the failover strategy has to - * yield before the global failover. + * yield before the global failover. */ public class ExecutionGraph implements AccessExecutionGraph, Archiveable { @@ -176,6 +179,13 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable serializedJobInformation; + /** + * The key of the offloaded job information BLOB containing {@link #serializedJobInformation} or + * null if not offloaded. + */ + @Nullable + private final PermanentBlobKey jobInformationBlobKey; + /** The executor which is used to execute futures. */ private final ScheduledExecutorService futureExecutor; @@ -235,6 +245,9 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable serializedConfig, + Time timeout, + RestartStrategy restartStrategy, + SlotProvider slotProvider, + BlobServer blobServer) { this( futureExecutor, ioExecutor, @@ -307,7 +339,9 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable requiredJarFiles, List requiredClasspaths, SlotProvider slotProvider, - ClassLoader userClassLoader) { + ClassLoader userClassLoader, + BlobServer blobServer) { checkNotNull(futureExecutor); checkNotNull(jobId); @@ -379,6 +414,52 @@ public ExecutionGraph( // is ready by the time the failover strategy sees it this.failoverStrategy = checkNotNull(failoverStrategyFactory.create(this), "null failover strategy"); LOG.info("Job recovers via failover strategy: {}", failoverStrategy.getStrategyName()); + + this.blobServer = blobServer; + this.jobInformationBlobKey = tryOffLoadJobInformation(); + } + + /** + * Tries to store {@link #serializedJobInformation} and in the graph's {@link + * ExecutionGraph#blobServer} (if not null) so that RPC messages do not need to include + * it. + * + * @return the BLOB key of the uploaded job information or null if the upload failed + */ + @Nullable + private PermanentBlobKey tryOffLoadJobInformation() { + if (blobServer == null) { + return null; + } + + // If the serialized job information inside serializedJobInformation is larger than this, + // we try to offload it to the BLOB server. + final int rpcOffloadMinSize = + blobServer.getConfiguration().getInteger(JobManagerOptions.TDD_OFFLOAD_MINSIZE); + + if (serializedJobInformation.getByteArray().length > rpcOffloadMinSize) { + LOG.info("Storing job {} information at the BLOB server", getJobID()); + + // TODO: do not overwrite existing job info and thus speed up recovery? + try { + return blobServer.putPermanent(getJobID(), serializedJobInformation.getByteArray()); + } catch (IOException e) { + LOG.warn("Failed to offload job " + getJobID() + " information data to BLOB store", e); + } + } + + return null; + } + + /** + * Returns the key of the offloaded job information BLOB containing {@link + * #serializedJobInformation}. + * + * @return the BLOB key or null if not offloaded + */ + @Nullable + public PermanentBlobKey getJobInformationBlobKey() { + return jobInformationBlobKey; } // -------------------------------------------------------------------------------------------- @@ -603,7 +684,7 @@ public ErrorInfo getFailureCause() { * Gets the number of full restarts that the execution graph went through. * If a full restart recovery is currently pending, this recovery is included in the * count. - * + * * @return The number of full restarts so far */ public long getNumberOfFullRestarts() { @@ -679,6 +760,10 @@ public long getStatusTimestamp(JobStatus status) { return this.stateTimestamps[status.ordinal()]; } + public final BlobServer getBlobServer() { + return blobServer; + } + /** * Returns the ExecutionContext associated with this ExecutionGraph. * @@ -829,8 +914,8 @@ private void scheduleLazy(SlotProvider slotProvider) throws NoResourceAvailableE } /** - * - * + * + * * @param slotProvider The resource provider from which the slots are allocated * @param timeout The maximum time that the deployment may take, before a * TimeoutException is thrown. @@ -923,7 +1008,7 @@ public void run() { // we catch everything here to make sure cleanup happens and the // ExecutionGraph notices the error - // we need to to release all slots before going into recovery! + // we need to to release all slots before going into recovery! try { ExecutionGraphUtils.releaseAllSlotsSilently(resources); } @@ -934,7 +1019,7 @@ public void run() { // Wouldn't it be nice if we could return an actual Void object? // return (Void) Unsafe.getUnsafe().allocateInstance(Void.class); - return null; + return null; }, futureExecutor); @@ -1084,12 +1169,12 @@ public void failGlobal(Throwable error) { /** * Fails the execution graph globally. This failure will not be recovered by a specific * failover strategy, but results in a full restart of all tasks. - * + * *

This global failure is meant to be triggered in cases where the consistency of the * execution graph' state cannot be guaranteed any more (for example when catching unexpected * exceptions that indicate a bug or an unexpected call race), and where a full restart is the * safe way to get consistency back. - * + * * @param errorInfo ErrorInfo containing the exception that caused the failure. */ public void failGlobal(ErrorInfo errorInfo) { @@ -1427,7 +1512,7 @@ private boolean tryRestartOrFail(long globalModVersionForRestart) { return true; } else if (!isRestartable && transitionState(currentState, JobStatus.FAILED, failureCause)) { - final String cause1 = isFailureCauseAllowingRestart ? null : + final String cause1 = isFailureCauseAllowingRestart ? null : "a type of SuppressRestartsException was thrown"; final String cause2 = isRestartStrategyAllowingRestart ? null : "the restart strategy prevented it"; @@ -1485,25 +1570,25 @@ public boolean updateState(TaskExecutionState state) { switch (state.getExecutionState()) { case RUNNING: return attempt.switchToRunning(); - + case FINISHED: // this deserialization is exception-free accumulators = deserializeAccumulators(state); attempt.markFinished(accumulators, state.getIOMetrics()); return true; - + case CANCELED: // this deserialization is exception-free accumulators = deserializeAccumulators(state); attempt.cancelingComplete(accumulators, state.getIOMetrics()); return true; - + case FAILED: // this deserialization is exception-free accumulators = deserializeAccumulators(state); attempt.markFailed(state.getError(userClassLoader), accumulators, state.getIOMetrics()); return true; - + default: // we mark as failed and return false, which triggers the TaskManager // to remove the task @@ -1526,9 +1611,9 @@ public boolean updateState(TaskExecutionState state) { /** * Deserializes accumulators from a task state update. - * + * *

This method never throws an exception! - * + * * @param state The task execution state from which to deserialize the accumulators. * @return The deserialized accumulators, of null, if there are no accumulators or an error occurred. */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java index ba66a2b6162f0..3b7250547bdad 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java @@ -27,6 +27,7 @@ import org.apache.flink.configuration.WebOptions; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.checkpoint.CheckpointIDCounter; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.checkpoint.CheckpointStatsTracker; @@ -90,6 +91,7 @@ public static ExecutionGraph buildGraph( RestartStrategy restartStrategy, MetricGroup metrics, int parallelismForAutoMax, + BlobServer blobServer, Logger log) throws JobExecutionException, JobException { @@ -98,7 +100,7 @@ public static ExecutionGraph buildGraph( final String jobName = jobGraph.getName(); final JobID jobId = jobGraph.getJobID(); - final FailoverStrategy.Factory failoverStrategy = + final FailoverStrategy.Factory failoverStrategy = FailoverStrategyLoader.loadFailoverStrategy(jobManagerConfig, log); // create a new execution graph, if none exists so far @@ -116,7 +118,8 @@ public static ExecutionGraph buildGraph( jobGraph.getUserJarBlobKeys(), jobGraph.getClasspaths(), slotProvider, - classLoader); + classLoader, + blobServer); // set the basic properties diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java index e6d49d261def3..33a43592776b2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java @@ -32,6 +32,8 @@ import org.apache.flink.core.io.InputSplitSource; import org.apache.flink.runtime.JobException; import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult; +import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.blob.PermanentBlobKey; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.instance.SlotProvider; @@ -46,9 +48,10 @@ import org.apache.flink.runtime.state.KeyGroupRangeAssignment; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; - import org.slf4j.Logger; +import javax.annotation.Nullable; + import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -59,9 +62,9 @@ import java.util.concurrent.CompletableFuture; /** - * An {@code ExecutionJobVertex} is part of the {@link ExecutionGraph}, and the peer + * An {@code ExecutionJobVertex} is part of the {@link ExecutionGraph}, and the peer * to the {@link JobVertex}. - * + * *

The {@code ExecutionJobVertex} corresponds to a parallelized operation. It * contains an {@link ExecutionVertex} for each parallel instance of that operation. */ @@ -95,7 +98,7 @@ public class ExecutionJobVertex implements AccessExecutionJobVertex, Archiveable * The ID's are in the same order as {@link ExecutionJobVertex#operatorIDs}. */ private final List userDefinedOperatorIds; - + private final ExecutionVertex[] taskVertices; private final IntermediateResult[] producedDataSets; @@ -121,6 +124,13 @@ public class ExecutionJobVertex implements AccessExecutionJobVertex, Archiveable */ private SerializedValue serializedTaskInformation; + /** + * The key of the offloaded task information BLOB containing {@link #serializedTaskInformation} + * or null if not offloaded. + */ + @Nullable + private PermanentBlobKey taskInformationBlobKey = null; + private InputSplitAssigner splitAssigner; /** @@ -169,7 +179,7 @@ public ExecutionJobVertex( this.taskVertices = new ExecutionVertex[numTaskVertices]; this.operatorIDs = Collections.unmodifiableList(jobVertex.getOperatorIDs()); this.userDefinedOperatorIds = Collections.unmodifiableList(jobVertex.getUserDefinedOperatorIDs()); - + this.inputs = new ArrayList<>(jobVertex.getInputs().size()); // take the sharing group @@ -351,28 +361,80 @@ public List getInputs() { public SerializedValue getSerializedTaskInformation() throws IOException { - if (null == serializedTaskInformation) { + // only one thread should offload the task information, so let's also let only one thread + // serialize the task information! + synchronized (stateMonitor) { + if (null == serializedTaskInformation) { - int parallelism = getParallelism(); - int maxParallelism = getMaxParallelism(); + int parallelism = getParallelism(); + int maxParallelism = getMaxParallelism(); - if (LOG.isDebugEnabled()) { - LOG.debug("Creating task information for " + generateDebugString()); - } + if (LOG.isDebugEnabled()) { + LOG.debug("Creating task information for " + generateDebugString()); + } - serializedTaskInformation = new SerializedValue<>( + serializedTaskInformation = new SerializedValue<>( new TaskInformation( - jobVertex.getID(), - jobVertex.getName(), - parallelism, - maxParallelism, - jobVertex.getInvokableClassName(), - jobVertex.getConfiguration())); + jobVertex.getID(), + jobVertex.getName(), + parallelism, + maxParallelism, + jobVertex.getInvokableClassName(), + jobVertex.getConfiguration())); + + taskInformationBlobKey = tryOffLoadTaskInformation(); + } } return serializedTaskInformation; } + /** + * Returns the key of the offloaded task information BLOB containing {@link + * #serializedTaskInformation}. + *

+ * This may be true after the first call to {@link #getSerializedTaskInformation()}. + * + * @return the BLOB key or null if not offloaded + */ + @Nullable + public PermanentBlobKey getTaskInformationBlobKey() { + return taskInformationBlobKey; + } + + /** + * Tries to store {@link #serializedTaskInformation} and in the graph's {@link + * ExecutionGraph#blobServer} (if not null) so that RPC messages do not need to include + * it. + * + * @return the BLOB key of the uploaded task information or null if the upload failed + */ + @Nullable + private PermanentBlobKey tryOffLoadTaskInformation() { + BlobServer blobServer = graph.getBlobServer(); + if (blobServer == null) { + return null; + } + + // If the serialized task information inside #serializedTaskInformation is larger than this, + // we try to offload it to the BLOB server. + final int rpcOffloadMinSize = + blobServer.getConfiguration().getInteger(JobManagerOptions.TDD_OFFLOAD_MINSIZE); + + if (serializedTaskInformation.getByteArray().length > rpcOffloadMinSize) { + LOG.info("Storing task {} information at the BLOB server", getJobVertexId()); + + // TODO: do not overwrite existing task info and thus speed up recovery? + try { + return blobServer.putPermanent(getJobId(), serializedTaskInformation.getByteArray()); + } catch (IOException e) { + LOG.warn("Failed to offload task " + getJobVertexId() + " information data to BLOB store", e); + } + } + + return null; + } + @Override public ExecutionState getAggregateState() { int[] num = new int[ExecutionState.values().length]; @@ -504,7 +566,7 @@ public void cancel() { /** * Cancels all currently running vertex executions. - * + * * @return A future that is complete once all tasks have canceled. */ public CompletableFuture cancelWithFuture() { @@ -588,21 +650,21 @@ public ArchivedExecutionJobVertex archive() { /** * A utility function that computes an "aggregated" state for the vertex. - * + * *

This state is not used anywhere in the coordination, but can be used for display * in dashboards to as a summary for how the particular parallel operation represented by * this ExecutionJobVertex is currently behaving. - * + * *

For example, if at least one parallel task is failed, the aggregate state is failed. * If not, and at least one parallel task is cancelling (or cancelled), the aggregate state * is cancelling (or cancelled). If all tasks are finished, the aggregate state is finished, * and so on. - * + * * @param verticesPerState The number of vertices in each state (indexed by the ordinal of * the ExecutionState values). * @param parallelism The parallelism of the ExecutionJobVertex - * - * @return The aggregate state of this ExecutionJobVertex. + * + * @return The aggregate state of this ExecutionJobVertex. */ public static ExecutionState getAggregateJobVertexState(int[] verticesPerState, int parallelism) { if (verticesPerState == null || verticesPerState.length != ExecutionState.values().length) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java index 9aac133060bab..17ad3c8d74a7c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java @@ -24,6 +24,7 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.blob.PermanentBlobKey; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; @@ -47,7 +48,6 @@ import org.apache.flink.runtime.util.EvictingBoundedList; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; -import org.apache.flink.util.SerializedValue; import org.slf4j.Logger; @@ -770,19 +770,39 @@ TaskDeploymentDescriptor createDeploymentDescriptor( consumedPartitions.add(new InputGateDeploymentDescriptor(resultId, partitionType, queueToRequest, partitions)); } - SerializedValue serializedJobInformation = getExecutionGraph().getSerializedJobInformation(); - SerializedValue serializedJobVertexInformation = null; + TaskDeploymentDescriptor.MaybeOffloaded serializedJobInformation; + { + PermanentBlobKey jobInfoBlobKey = getExecutionGraph().getJobInformationBlobKey(); + if (jobInfoBlobKey != null) { + serializedJobInformation = + new TaskDeploymentDescriptor.Offloaded<>(jobInfoBlobKey); + } else { + serializedJobInformation = new TaskDeploymentDescriptor.NonOffloaded<>( + getExecutionGraph().getSerializedJobInformation()); + } + } - try { - serializedJobVertexInformation = jobVertex.getSerializedTaskInformation(); - } catch (IOException e) { - throw new ExecutionGraphException( - "Could not create a serialized JobVertexInformation for " + jobVertex.getJobVertexId(), e); + TaskDeploymentDescriptor.MaybeOffloaded serializedTaskInformation; + { + PermanentBlobKey taskInfoBlobKey = jobVertex.getTaskInformationBlobKey(); + if (taskInfoBlobKey != null) { + serializedTaskInformation = new TaskDeploymentDescriptor.Offloaded<>(taskInfoBlobKey); + } else { + try { + serializedTaskInformation = new TaskDeploymentDescriptor.NonOffloaded<>( + jobVertex.getSerializedTaskInformation()); + } catch (IOException e) { + throw new ExecutionGraphException( + "Could not create a serialized JobVertexInformation for " + + jobVertex.getJobVertexId(), e); + } + } } return new TaskDeploymentDescriptor( + getJobId(), serializedJobInformation, - serializedJobVertexInformation, + serializedTaskInformation, executionId, targetSlot.getAllocatedSlot().getSlotAllocationId(), subTaskIndex, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java index 7efcc0b30b2ec..0484d53b392c2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java @@ -288,6 +288,7 @@ public JobMaster( restartStrategy, jobMetricGroup, -1, + blobServer, log); // register self as job status change listener diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index 76cfb50e2e241..d6295b98815fb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -307,17 +307,7 @@ public CompletableFuture submitTask( Time timeout) { try { - // first, deserialize the pre-serialized information - final JobInformation jobInformation; - final TaskInformation taskInformation; - try { - jobInformation = tdd.getSerializedJobInformation().deserializeValue(getClass().getClassLoader()); - taskInformation = tdd.getSerializedTaskInformation().deserializeValue(getClass().getClassLoader()); - } catch (IOException | ClassNotFoundException e) { - throw new TaskSubmissionException("Could not deserialize the job or task information.", e); - } - - final JobID jobId = jobInformation.getJobId(); + final JobID jobId = tdd.getJobId(); final JobManagerConnection jobManagerConnection = jobManagerTable.get(jobId); if (jobManagerConnection == null) { @@ -344,6 +334,30 @@ public CompletableFuture submitTask( throw new TaskSubmissionException(message); } + // re-integrate offloaded data: + BlobCacheService blobCache = jobManagerConnection.getBlobService(); + try { + tdd.loadBigData(blobCache.getPermanentBlobService()); + } catch (IOException | ClassNotFoundException e) { + throw new TaskSubmissionException("Could not re-integrate offloaded TaskDeploymentDescriptor data.", e); + } + + // deserialize the pre-serialized information + final JobInformation jobInformation; + final TaskInformation taskInformation; + try { + jobInformation = tdd.getSerializedJobInformation().deserializeValue(getClass().getClassLoader()); + taskInformation = tdd.getSerializedTaskInformation().deserializeValue(getClass().getClassLoader()); + } catch (IOException | ClassNotFoundException e) { + throw new TaskSubmissionException("Could not deserialize the job or task information.", e); + } + + if (!jobId.equals(jobInformation.getJobId())) { + throw new TaskSubmissionException( + "Inconsistent job ID information inside TaskDeploymentDescriptor (" + + tdd.getJobId() + " vs. " + jobInformation.getJobId() + ")"); + } + TaskMetricGroup taskMetricGroup = taskManagerMetricGroup.addTaskForJob( jobInformation.getJobId(), jobInformation.getJobName(), @@ -361,6 +375,7 @@ public CompletableFuture submitTask( TaskManagerActions taskManagerActions = jobManagerConnection.getTaskManagerActions(); CheckpointResponder checkpointResponder = jobManagerConnection.getCheckpointResponder(); + BlobCacheService blobService = jobManagerConnection.getBlobService(); LibraryCacheManager libraryCache = jobManagerConnection.getLibraryCacheManager(); ResultPartitionConsumableNotifier resultPartitionConsumableNotifier = jobManagerConnection.getResultPartitionConsumableNotifier(); diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index 56c79bbbe5593..8e12bfd091f04 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -1321,6 +1321,7 @@ class JobManager( restartStrategy, jobMetrics, numSlots, + blobServer, log.logger) if (registerNewGraph) { diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index f11c837690aa0..c49b3bb9d1b96 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -1137,16 +1137,29 @@ class TaskManager( } // create the task. this does not grab any TaskManager resources or download - // and libraries - the operation does not block + // any libraries except for offloaded TaskDeploymentDescriptor data which + // was too big for the RPC - the operation may only block for the latter val jobManagerGateway = new AkkaActorGateway(jobManagerActor, leaderSessionID.orNull) + try { + tdd.loadBigData(blobCache.getPermanentBlobService); + } catch { + case e @ (_: IOException | _: ClassNotFoundException) => + throw new IOException("Could not deserialize the job information.", e) + } + val jobInformation = try { tdd.getSerializedJobInformation.deserializeValue(getClass.getClassLoader) } catch { case e @ (_: IOException | _: ClassNotFoundException) => throw new IOException("Could not deserialize the job information.", e) } + if (tdd.getJobId != jobInformation.getJobId) { + throw new IOException( + "Inconsistent job ID information inside TaskDeploymentDescriptor (" + + tdd.getJobId + " vs. " + jobInformation.getJobId + ")") + } val taskInformation = try { tdd.getSerializedTaskInformation.deserializeValue(getClass.getClassLoader) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointSettingsSerializableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointSettingsSerializableTest.java index 84b57747fc99e..e500036d30cb6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointSettingsSerializableTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointSettingsSerializableTest.java @@ -107,6 +107,7 @@ public void testDeserializationOfUserCodeWithUserClassLoader() throws Exception new NoRestartStrategy(), new UnregisteredMetricsGroup(), 10, + null, log); assertEquals(1, eg.getCheckpointCoordinator().getNumberOfRegisteredMasterHooks()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java index 744fd602e3d28..b89ed5d10473b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java @@ -93,7 +93,8 @@ private ExecutionGraph createExecutionGraphAndEnableCheckpointing( Collections.emptyList(), Collections.emptyList(), new Scheduler(TestingUtils.defaultExecutionContext()), - ClassLoader.getSystemClassLoader()); + ClassLoader.getSystemClassLoader(), + null); executionGraph.enableCheckpointing( 100, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java index 92613eda7723e..324702485d8f9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java @@ -76,8 +76,9 @@ public void testSerialization() { final TaskStateSnapshot taskStateHandles = new TaskStateSnapshot(); final TaskDeploymentDescriptor orig = new TaskDeploymentDescriptor( - serializedJobInformation, - serializedJobVertexInformation, + jobID, + new TaskDeploymentDescriptor.NonOffloaded<>(serializedJobInformation), + new TaskDeploymentDescriptor.NonOffloaded<>(serializedJobVertexInformation), execId, allocationId, indexInSubtaskGroup, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java index bbc232d764f49..3af417109d14c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java @@ -18,22 +18,8 @@ package org.apache.flink.runtime.executiongraph; -import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getInstance; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.fail; -import static org.junit.Assert.assertNotEquals; - -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ScheduledExecutorService; - import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.accumulators.IntCounter; import org.apache.flink.api.common.time.Time; @@ -43,6 +29,8 @@ import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; import org.apache.flink.runtime.accumulators.AccumulatorSnapshot; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.blob.PermanentBlobService; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; @@ -52,14 +40,13 @@ import org.apache.flink.runtime.instance.Instance; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; -import org.apache.flink.runtime.jobgraph.JobStatus; -import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.JobStatus; +import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings; import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; +import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings; import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings; import org.apache.flink.runtime.jobmanager.scheduler.Scheduler; import org.apache.flink.runtime.jobmanager.slots.ActorTaskManagerGateway; @@ -72,8 +59,59 @@ import org.junit.Test; import org.slf4j.LoggerFactory; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ScheduledExecutorService; + +import static junit.framework.TestCase.assertNull; +import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getInstance; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.fail; + +/** + * Tests for {@link ExecutionGraph} deployment. + */ public class ExecutionGraphDeploymentTest { + /** + * BLOB server instance to use for the job graph (may be null). + */ + protected BlobServer blobServer = null; + + /** + * Permanent BLOB cache instance to use for the actor gateway that handles the {@link + * TaskDeploymentDescriptor} loading (may be null). + */ + protected PermanentBlobService blobCache = null; + + /** + * Checks that the job information for the given ID has been offloaded successfully (if + * offloading is used). + * + * @param eg the execution graph that was created + */ + protected void checkJobOffloaded(ExecutionGraph eg) throws Exception { + assertNull(eg.getJobInformationBlobKey()); + } + + /** + * Checks that the task information for the job vertex has been offloaded successfully (if + * offloading is used). + * + * @param eg the execution graph that was created + * @param jobVertexId job vertex ID + */ + protected void checkTaskOffloaded(ExecutionGraph eg, JobVertexID jobVertexId) throws Exception { + assertNull(eg.getJobVertex(jobVertexId).getTaskInformationBlobKey()); + } + @Test public void testBuildDeploymentDescriptor() { try { @@ -106,13 +144,15 @@ public void testBuildDeploymentDescriptor() { ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutor(), TestingUtils.defaultExecutor(), - jobId, - "some job", + jobId, + "some job", new Configuration(), new SerializedValue<>(new ExecutionConfig()), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy(), - new Scheduler(TestingUtils.defaultExecutionContext())); + new Scheduler(TestingUtils.defaultExecutionContext()), + blobServer); + checkJobOffloaded(eg); List ordered = Arrays.asList(v1, v2, v3, v4); @@ -121,7 +161,10 @@ public void testBuildDeploymentDescriptor() { ExecutionJobVertex ejv = eg.getAllVertices().get(jid2); ExecutionVertex vertex = ejv.getTaskVertices()[3]; - ExecutionGraphTestUtils.SimpleActorGateway instanceGateway = new ExecutionGraphTestUtils.SimpleActorGateway(TestingUtils.directExecutionContext()); + ExecutionGraphTestUtils.SimpleActorGatewayWithTDD instanceGateway = + new ExecutionGraphTestUtils.SimpleActorGatewayWithTDD( + TestingUtils.directExecutionContext(), + blobCache == null ? blobServer : blobCache); final Instance instance = getInstance(new ActorTaskManagerGateway(instanceGateway)); @@ -132,13 +175,17 @@ public void testBuildDeploymentDescriptor() { vertex.deployToSlot(slot); assertEquals(ExecutionState.DEPLOYING, vertex.getExecutionState()); + checkTaskOffloaded(eg, vertex.getJobvertexId()); TaskDeploymentDescriptor descr = instanceGateway.lastTDD; assertNotNull(descr); - JobInformation jobInformation = descr.getSerializedJobInformation().deserializeValue(getClass().getClassLoader()); - TaskInformation taskInformation = descr.getSerializedTaskInformation().deserializeValue(getClass().getClassLoader()); + JobInformation jobInformation = + descr.getSerializedJobInformation().deserializeValue(getClass().getClassLoader()); + TaskInformation taskInformation = + descr.getSerializedTaskInformation().deserializeValue(getClass().getClassLoader()); + assertEquals(jobId, descr.getJobId()); assertEquals(jobId, jobInformation.getJobId()); assertEquals(jid2, taskInformation.getJobVertexId()); assertEquals(3, descr.getSubtaskIndex()); @@ -381,7 +428,9 @@ public void testNoResourceAvailableFailure() throws Exception { new SerializedValue<>(new ExecutionConfig()), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy(), - scheduler); + scheduler, + blobServer); + checkJobOffloaded(eg); eg.setQueuedSchedulingAllowed(false); @@ -448,16 +497,18 @@ private Tuple2> setupExecutio // execution graph that executes actions synchronously ExecutionGraph eg = new ExecutionGraph( - new DirectScheduledExecutorService(), - TestingUtils.defaultExecutor(), - jobId, - "some job", - new Configuration(), - new SerializedValue<>(new ExecutionConfig()), - AkkaUtils.getDefaultTimeout(), - new NoRestartStrategy(), - scheduler); - + new DirectScheduledExecutorService(), + TestingUtils.defaultExecutor(), + jobId, + "some job", + new Configuration(), + new SerializedValue<>(new ExecutionConfig()), + AkkaUtils.getDefaultTimeout(), + new NoRestartStrategy(), + scheduler, + blobServer); + checkJobOffloaded(eg); + eg.setQueuedSchedulingAllowed(false); List ordered = Arrays.asList(v1, v2); @@ -537,6 +588,7 @@ private ExecutionGraph createExecutionGraph(Configuration configuration) throws new NoRestartStrategy(), new UnregisteredMetricsGroup(), 1, + blobServer, LoggerFactory.getLogger(getClass())); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentWithBlobCacheTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentWithBlobCacheTest.java new file mode 100644 index 0000000000000..f1c071e4c40c5 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentWithBlobCacheTest.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.executiongraph; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.blob.PermanentBlobCache; +import org.apache.flink.runtime.blob.VoidBlobStore; + +import org.junit.After; +import org.junit.Before; + +import java.io.IOException; +import java.net.InetSocketAddress; + +/** + * Tests {@link ExecutionGraph} deployment when offloading job and task information into the BLOB + * server. + */ +public class ExecutionGraphDeploymentWithBlobCacheTest extends ExecutionGraphDeploymentWithBlobServerTest { + + @Before + @Override + public void setupBlobServer() throws IOException { + Configuration config = new Configuration(); + // always offload the serialized job and task information + config.setInteger(JobManagerOptions.TDD_OFFLOAD_MINSIZE, 0); + blobServer = new BlobServer(config, new VoidBlobStore()); + blobServer.start(); + + InetSocketAddress serverAddress = new InetSocketAddress("localhost", blobServer.getPort()); + blobCache = new PermanentBlobCache(serverAddress, config, new VoidBlobStore()); + } + + @After + @Override + public void shutdownBlobServer() throws IOException { + if (blobServer != null) { + blobServer.close(); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentWithBlobServerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentWithBlobServerTest.java new file mode 100644 index 0000000000000..2fab41350b142 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentWithBlobServerTest.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.executiongraph; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.blob.PermanentBlobKey; +import org.apache.flink.runtime.blob.VoidBlobStore; +import org.apache.flink.runtime.jobgraph.JobVertexID; + +import org.junit.After; +import org.junit.Before; +import org.mockito.Matchers; +import org.mockito.Mockito; + +import java.io.IOException; +import java.util.Collections; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.doAnswer; + +/** + * Tests {@link ExecutionGraph} deployment when offloading job and task information into the BLOB + * server. + */ +public class ExecutionGraphDeploymentWithBlobServerTest extends ExecutionGraphDeploymentTest { + + private Set seenHashes = Collections.newSetFromMap(new ConcurrentHashMap()); + + @Before + public void setupBlobServer() throws IOException { + Configuration config = new Configuration(); + // always offload the serialized job and task information + config.setInteger(JobManagerOptions.TDD_OFFLOAD_MINSIZE, 0); + blobServer = Mockito.spy(new BlobServer(config, new VoidBlobStore())); + + seenHashes.clear(); + + // verify that we do not upload the same content more than once + doAnswer( + invocation -> { + PermanentBlobKey key = (PermanentBlobKey) invocation.callRealMethod(); + + assertTrue(seenHashes.add(key.getHash())); + + return key; + } + ).when(blobServer).putPermanent(any(JobID.class), Matchers.any()); + + blobServer.start(); + } + + @After + public void shutdownBlobServer() throws IOException { + if (blobServer != null) { + blobServer.close(); + } + } + + @Override + protected void checkJobOffloaded(ExecutionGraph eg) throws Exception { + PermanentBlobKey jobInformationBlobKey = eg.getJobInformationBlobKey(); + assertNotNull(jobInformationBlobKey); + + // must not throw: + blobServer.getFile(eg.getJobID(), jobInformationBlobKey); + } + + @Override + protected void checkTaskOffloaded(ExecutionGraph eg, JobVertexID jobVertexId) throws Exception { + PermanentBlobKey taskInformationBlobKey = eg.getJobVertex(jobVertexId).getTaskInformationBlobKey(); + assertNotNull(taskInformationBlobKey); + + // must not throw: + blobServer.getFile(eg.getJobID(), taskInformationBlobKey); + } + + // TODO: test cleanup with failures? +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java index b88a9280cee14..c8cab9f471311 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java @@ -555,6 +555,7 @@ private ExecutionGraph createExecutionGraph(JobGraph jobGraph, SlotProvider slot new NoRestartStrategy(), new UnregisteredMetricsGroup(), 1, + null, log); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java index 2daf28fc25ce5..1f9fa8232da46 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java @@ -18,12 +18,15 @@ package org.apache.flink.runtime.executiongraph; +import akka.actor.Status; + import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.blob.PermanentBlobService; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -77,7 +80,7 @@ import static org.mockito.Mockito.spy; /** - * A collection of utility methods for testing the ExecutionGraph and its related classes. + * A collection of utility methods for testing the ExecutionGraph and its related classes. */ public class ExecutionGraphTestUtils { @@ -89,10 +92,10 @@ public class ExecutionGraphTestUtils { /** * Waits until the job has reached a certain state. - * + * *

This method is based on polling and might miss very fast state transitions! */ - public static void waitUntilJobStatus(ExecutionGraph eg, JobStatus status, long maxWaitMillis) + public static void waitUntilJobStatus(ExecutionGraph eg, JobStatus status, long maxWaitMillis) throws TimeoutException { checkNotNull(eg); @@ -103,7 +106,7 @@ public static void waitUntilJobStatus(ExecutionGraph eg, JobStatus status, long final long deadline = maxWaitMillis == 0 ? Long.MAX_VALUE : System.nanoTime() + (maxWaitMillis * 1_000_000); while (eg.getState() != status && System.nanoTime() < deadline) { - try { + try { Thread.sleep(2); } catch (InterruptedException ignored) {} } @@ -280,7 +283,7 @@ public static ExecutionGraph createSimpleTestGraph(RestartStrategy restartStrate /** * Creates an execution graph containing the given vertices. - * + * *

The execution graph uses {@link NoRestartStrategy} as the restart strategy. */ public static ExecutionGraph createSimpleTestGraph(JobID jid, JobVertex... vertices) throws Exception { @@ -339,6 +342,7 @@ public static ExecutionGraph createExecutionGraph( restartStrategy, new UnregisteredMetricsGroup(), 1, + null, TEST_LOGGER); } @@ -368,8 +372,7 @@ public static Instance getInstance(final TaskManagerGateway gateway, final int n @SuppressWarnings("serial") public static class SimpleActorGateway extends BaseTestingActorGateway { - - public TaskDeploymentDescriptor lastTDD; + public SimpleActorGateway(ExecutionContext executionContext){ super(executionContext); @@ -379,7 +382,6 @@ public SimpleActorGateway(ExecutionContext executionContext){ public Object handleMessage(Object message) { if (message instanceof SubmitTask) { SubmitTask submitTask = (SubmitTask) message; - lastTDD = submitTask.tasks(); return Acknowledge.get(); } else if(message instanceof CancelTask) { return Acknowledge.get(); @@ -391,6 +393,35 @@ public Object handleMessage(Object message) { } } + @SuppressWarnings("serial") + public static class SimpleActorGatewayWithTDD extends SimpleActorGateway { + + public TaskDeploymentDescriptor lastTDD; + private final PermanentBlobService blobCache; + + public SimpleActorGatewayWithTDD(ExecutionContext executionContext, PermanentBlobService blobCache) { + super(executionContext); + this.blobCache = blobCache; + } + + @Override + public Object handleMessage(Object message) { + if(message instanceof SubmitTask) { + SubmitTask submitTask = (SubmitTask) message; + lastTDD = submitTask.tasks(); + try { + lastTDD.loadBigData(blobCache); + return Acknowledge.get(); + } catch (Exception e) { + e.printStackTrace(); + return new Status.Failure(e); + } + } else { + return super.handleMessage(message); + } + } + } + @SuppressWarnings("serial") public static class SimpleFailingActorGateway extends BaseTestingActorGateway { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexLocalityTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexLocalityTest.java index c9b7a40a78b0a..2ba41940224aa 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexLocalityTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexLocalityTest.java @@ -220,6 +220,7 @@ private ExecutionGraph createTestGraph(int parallelism, boolean allToAll) throws new FixedDelayRestartStrategy(10, 0L), new UnregisteredMetricsGroup(), 1, + null, log); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/FailoverRegionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/FailoverRegionTest.java index 6b872effc22cf..d631de9b4bde2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/FailoverRegionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/FailoverRegionTest.java @@ -141,7 +141,8 @@ public void testMultiRegionsFailover() throws Exception { Collections.emptyList(), Collections.emptyList(), slotProvider, - ExecutionGraph.class.getClassLoader()); + ExecutionGraph.class.getClassLoader(), + null); eg.attachJobGraph(ordered); @@ -268,7 +269,8 @@ public void testMutilRegionFailoverAtSameTime() throws Exception { Collections.emptyList(), Collections.emptyList(), scheduler, - ExecutionGraph.class.getClassLoader()); + ExecutionGraph.class.getClassLoader(), + null); try { eg.attachJobGraph(ordered); } @@ -345,7 +347,8 @@ public void testSucceedingNoticePreceding() throws Exception { Collections.emptyList(), Collections.emptyList(), scheduler, - ExecutionGraph.class.getClassLoader()); + ExecutionGraph.class.getClassLoader(), + null); try { eg.attachJobGraph(ordered); } @@ -458,7 +461,8 @@ private static ExecutionGraph createSingleRegionExecutionGraph(RestartStrategy r Collections.emptyList(), Collections.emptyList(), scheduler, - ExecutionGraph.class.getClassLoader()); + ExecutionGraph.class.getClassLoader(), + null); try { eg.attachJobGraph(ordered); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/GlobalModVersionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/GlobalModVersionTest.java index f4419704ef50c..986fb39fd713f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/GlobalModVersionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/GlobalModVersionTest.java @@ -175,7 +175,8 @@ private ExecutionGraph createSampleGraph(FailoverStrategy failoverStrategy) thro Collections.emptyList(), Collections.emptyList(), slotProvider, - getClass().getClassLoader()); + getClass().getClassLoader(), + null); JobVertex jv = new JobVertex("test vertex"); jv.setInvokableClass(NoOpInvokable.class); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/IndividualRestartsConcurrencyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/IndividualRestartsConcurrencyTest.java index c52b59c3f37f7..9d7cd9066ff74 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/IndividualRestartsConcurrencyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/IndividualRestartsConcurrencyTest.java @@ -299,7 +299,8 @@ private ExecutionGraph createSampleGraph( Collections.emptyList(), Collections.emptyList(), slotProvider, - getClass().getClassLoader()); + getClass().getClassLoader(), + null); JobVertex jv = new JobVertex("test vertex"); jv.setInvokableClass(NoOpInvokable.class); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PipelinedRegionFailoverConcurrencyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PipelinedRegionFailoverConcurrencyTest.java index 3c58616aab417..a0351fa7879e8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PipelinedRegionFailoverConcurrencyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PipelinedRegionFailoverConcurrencyTest.java @@ -321,7 +321,8 @@ private ExecutionGraph createSampleGraph( Collections.emptyList(), Collections.emptyList(), slotProvider, - getClass().getClassLoader()); + getClass().getClassLoader(), + null); JobVertex jv = new JobVertex("test vertex"); jv.setInvokableClass(NoOpInvokable.class); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/RestartPipelinedRegionStrategyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/RestartPipelinedRegionStrategyTest.java index 7c4151bedde7e..ecbdc46af399f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/RestartPipelinedRegionStrategyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/RestartPipelinedRegionStrategyTest.java @@ -106,7 +106,8 @@ public void testSimpleFailoverRegion() throws Exception { Collections.emptyList(), Collections.emptyList(), scheduler, - ExecutionGraph.class.getClassLoader()); + ExecutionGraph.class.getClassLoader(), + null); try { eg.attachJobGraph(ordered); } @@ -191,7 +192,8 @@ public void testMultipleFailoverRegions() throws Exception { Collections.emptyList(), Collections.emptyList(), scheduler, - ExecutionGraph.class.getClassLoader()); + ExecutionGraph.class.getClassLoader(), + null); try { eg.attachJobGraph(ordered); } @@ -281,7 +283,8 @@ public void testSingleRegionWithMixedInput() throws Exception { Collections.emptyList(), Collections.emptyList(), scheduler, - ExecutionGraph.class.getClassLoader()); + ExecutionGraph.class.getClassLoader(), + null); try { eg.attachJobGraph(ordered); } @@ -362,7 +365,8 @@ public void testMultiRegionNotAllToAll() throws Exception { Collections.emptyList(), Collections.emptyList(), scheduler, - ExecutionGraph.class.getClassLoader()); + ExecutionGraph.class.getClassLoader(), + null); try { eg.attachJobGraph(ordered); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/PipelinedFailoverRegionBuildingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/PipelinedFailoverRegionBuildingTest.java index 55bf7114ea34e..5e96dfd05099a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/PipelinedFailoverRegionBuildingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/PipelinedFailoverRegionBuildingTest.java @@ -639,6 +639,7 @@ private ExecutionGraph createExecutionGraph(JobGraph jobGraph) throws JobExcepti new NoRestartStrategy(), new UnregisteredMetricsGroup(), 1000, + null, log); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java index 0f21b55b97cfe..2946d5e6b1d75 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.blob.VoidBlobStore; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.concurrent.ScheduledExecutor; @@ -91,22 +92,23 @@ public void testHeartbeatTimeoutWithTaskManager() throws Exception { final ScheduledExecutor scheduledExecutor = mock(ScheduledExecutor.class); final HeartbeatServices heartbeatServices = new TestingHeartbeatServices(heartbeatInterval, heartbeatTimeout, scheduledExecutor); - BlobServer blobServer = mock(BlobServer.class); - when(blobServer.getPort()).thenReturn(1337); final JobGraph jobGraph = new JobGraph(); - try { + Configuration configuration = new Configuration(); + try (BlobServer blobServer = new BlobServer(configuration, new VoidBlobStore())) { + blobServer.start(); + final JobMaster jobMaster = new JobMaster( rpc, jmResourceId, jobGraph, - new Configuration(), + configuration, haServices, heartbeatServices, Executors.newScheduledThreadPool(1), blobServer, - mock(BlobLibraryCacheManager.class), + new BlobLibraryCacheManager(blobServer, FlinkUserCodeClassLoaders.ResolveOrder.CHILD_FIRST), mock(RestartStrategyFactory.class), testingTimeout, null, @@ -197,17 +199,20 @@ public void testHeartbeatTimeoutWithResourceManager() throws Exception { final TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); - try { + Configuration configuration = new Configuration(); + try (BlobServer blobServer = new BlobServer(configuration, new VoidBlobStore())) { + blobServer.start(); + final JobMaster jobMaster = new JobMaster( rpc, jmResourceId, jobGraph, - new Configuration(), + configuration, haServices, heartbeatServices, Executors.newScheduledThreadPool(1), - mock(BlobServer.class), - mock(BlobLibraryCacheManager.class), + blobServer, + new BlobLibraryCacheManager(blobServer, FlinkUserCodeClassLoaders.ResolveOrder.CHILD_FIRST), mock(RestartStrategyFactory.class), testingTimeout, null, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index 04134968f5d7b..d19f94bd5b054 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -680,8 +680,9 @@ public void testTaskSubmission() throws Exception { SerializedValue serializedJobVertexInformation = new SerializedValue<>(taskInformation); final TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor( - serializedJobInformation, - serializedJobVertexInformation, + jobId, + new TaskDeploymentDescriptor.NonOffloaded<>(serializedJobInformation), + new TaskDeploymentDescriptor.NonOffloaded<>(serializedJobVertexInformation), new ExecutionAttemptID(), allocationId, 0, @@ -1279,8 +1280,9 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { SerializedValue serializedJobVertexInformation = new SerializedValue<>(taskInformation); final TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor( - serializedJobInformation, - serializedJobVertexInformation, + jobId, + new TaskDeploymentDescriptor.NonOffloaded<>(serializedJobInformation), + new TaskDeploymentDescriptor.NonOffloaded<>(serializedJobVertexInformation), new ExecutionAttemptID(), allocationId1, 0, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java index 45771850195ed..31e9e223b6900 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java @@ -2149,8 +2149,9 @@ private static TaskDeploymentDescriptor createTaskDeploymentDescriptor( SerializedValue serializedJobVertexInformation = new SerializedValue<>(taskInformation); return new TaskDeploymentDescriptor( - serializedJobInformation, - serializedJobVertexInformation, + jobId, + new TaskDeploymentDescriptor.NonOffloaded<>(serializedJobInformation), + new TaskDeploymentDescriptor.NonOffloaded<>(serializedJobVertexInformation), executionAttemptId, new AllocationID(), subtaskIndex, diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerLike.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerLike.scala index 9db11d79ac030..6d5f3c31d8b78 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerLike.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerLike.scala @@ -138,22 +138,18 @@ trait TestingTaskManagerLike extends FlinkActor { registeredSubmitTaskListeners.put(jobId, sender()) case msg@SubmitTask(tdd) => - try { - val jobId = tdd.getSerializedJobInformation.deserializeValue(getClass.getClassLoader) - .getJobId - - registeredSubmitTaskListeners.get(jobId) match { - case Some(listenerRef) => - listenerRef ! ResponseSubmitTaskListener(tdd) - case None => - // Nothing to do - } - } catch { - case e: Exception => - log.error("Could not deserialize the job information.", e) + // this loads offloaded data back into the tdd and needs to be run first! + super.handleMessage(msg) + + val jobId = tdd.getJobId + + registeredSubmitTaskListeners.get(jobId) match { + case Some(listenerRef) => + listenerRef ! ResponseSubmitTaskListener(tdd) + case None => + // Nothing to do } - super.handleMessage(msg) /** * Message from task manager that accumulator values changed and need to be reported immediately diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitionerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitionerTest.java index edea3f3d4d944..70b030c28090d 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitionerTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitionerTest.java @@ -155,7 +155,8 @@ public void flatMap(String value, new ArrayList<>(), new ArrayList<>(), new Scheduler(TestingUtils.defaultExecutionContext()), - ExecutionGraph.class.getClassLoader()); + ExecutionGraph.class.getClassLoader(), + null); try { eg.attachJobGraph(jobVertices); } From 94b4598b88a241a97c23e8b73879746084718f38 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Thu, 27 Jul 2017 14:11:44 +0200 Subject: [PATCH 6/6] [FLINK-6046][tests] added an integration test with a job with big payload This verifies that uploading a job with a payload of 100MB is successful. NOTE: This only works after also implementing JobGraph offloading during submission or akka.framesize will already limit it when sending it form the client to the jobmanager. --- .../BigUserProgramJobSubmitITCase.java | 120 ++++++++++++++++++ 1 file changed, 120 insertions(+) create mode 100644 flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/BigUserProgramJobSubmitITCase.java diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/BigUserProgramJobSubmitITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/BigUserProgramJobSubmitITCase.java new file mode 100644 index 0000000000000..a4d595877f95d --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/BigUserProgramJobSubmitITCase.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.test.streaming.runtime; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.util.TestStreamEnvironment; +import org.apache.flink.test.streaming.runtime.util.TestListResultSink; +import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.util.TestLogger; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Ignore; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Random; + +import static org.junit.Assert.assertEquals; + +/** + * Integration test that verifies that a user program with a big(ger) payload is successfully + * submitted and run. + */ +@Ignore("Fails on job submission payload being too large - [FLINK-7285]") +public class BigUserProgramJobSubmitITCase extends TestLogger { + + // ------------------------------------------------------------------------ + // The mini cluster that is shared across tests + // ------------------------------------------------------------------------ + + private static final int DEFAULT_PARALLELISM = 1; + + private static LocalFlinkMiniCluster cluster; + + private static final Logger LOG = LoggerFactory.getLogger(BigUserProgramJobSubmitITCase.class); + + // ------------------------------------------------------------------------ + // Cluster setup & teardown + // ------------------------------------------------------------------------ + + @BeforeClass + public static void setup() throws Exception { + // make sure we do not use a singleActorSystem for the tests + // (therefore, we cannot simply inherit from StreamingMultipleProgramsTestBase) + LOG.info("Starting FlinkMiniCluster"); + cluster = TestBaseUtils.startCluster(1, DEFAULT_PARALLELISM, false, false, false); + TestStreamEnvironment.setAsContext(cluster, DEFAULT_PARALLELISM); + } + + @AfterClass + public static void teardown() throws Exception { + LOG.info("Closing FlinkMiniCluster"); + TestStreamEnvironment.unsetAsContext(); + TestBaseUtils.stopCluster(cluster, TestBaseUtils.DEFAULT_TIMEOUT); + } + + private final Random rnd = new Random(); + + /** + * Use a map function that references a 100MB byte array. + */ + @Test + public void bigDataInMap() throws Exception { + + final byte[] data = new byte[100 * 1024 * 1024]; // 100 MB + rnd.nextBytes(data); // use random data so that Java does not optimise it away + data[1] = 0; + data[3] = 0; + data[5] = 0; + + TestListResultSink resultSink = new TestListResultSink<>(); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream src = env.fromElements(1, 3, 5); + + src.map(new MapFunction() { + private static final long serialVersionUID = 1L; + + @Override + public String map(Integer value) throws Exception { + return "x " + value + " " + data[value]; + } + }).addSink(resultSink); + + env.execute(); + + List expected = Arrays.asList("x 1 0", "x 3 0", "x 5 0"); + + List result = resultSink.getResult(); + + Collections.sort(expected); + Collections.sort(result); + + assertEquals(expected, result); + } +}