zhuzhurk commented on a change in pull request #16498: URL: https://github.com/apache/flink/pull/16498#discussion_r677225022
########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheSizeTrackerTest.java ########## @@ -0,0 +1,286 @@ +/* + * 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.util.TestLogger; +import org.apache.flink.util.concurrent.FutureUtils; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import javax.annotation.Nullable; + +import java.io.File; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Random; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import static org.apache.flink.runtime.blob.BlobServerPutTest.put; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** Tests for {@link BlobCacheSizeTracker}. */ +public class BlobCacheSizeTrackerTest extends TestLogger { + + private static final int blobSize = 10_000; + + private static final Random random = new Random(); + + private static final TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @BeforeClass + public static void setUp() throws IOException { + temporaryFolder.create(); + } + + @AfterClass + public static void cleanUp() throws IOException { + temporaryFolder.delete(); + } + + @Test + public void testSizeLimitTrackerStatusUpdate() { Review comment: Can be simplified to be `testUpdate ` ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheSizeTrackerTest.java ########## @@ -0,0 +1,286 @@ +/* + * 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.util.TestLogger; +import org.apache.flink.util.concurrent.FutureUtils; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import javax.annotation.Nullable; + +import java.io.File; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Random; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import static org.apache.flink.runtime.blob.BlobServerPutTest.put; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** Tests for {@link BlobCacheSizeTracker}. */ +public class BlobCacheSizeTrackerTest extends TestLogger { + + private static final int blobSize = 10_000; + + private static final Random random = new Random(); + + private static final TemporaryFolder temporaryFolder = new TemporaryFolder(); Review comment: Can be simplified to be: ``` @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder(); ``` And the `setUp()` and `cleanUp ()` will not be needed. ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheSizeTrackerTest.java ########## @@ -0,0 +1,286 @@ +/* + * 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.util.TestLogger; +import org.apache.flink.util.concurrent.FutureUtils; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import javax.annotation.Nullable; + +import java.io.File; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Random; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import static org.apache.flink.runtime.blob.BlobServerPutTest.put; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** Tests for {@link BlobCacheSizeTracker}. */ +public class BlobCacheSizeTrackerTest extends TestLogger { + + private static final int blobSize = 10_000; + + private static final Random random = new Random(); + + private static final TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @BeforeClass + public static void setUp() throws IOException { + temporaryFolder.create(); + } + + @AfterClass + public static void cleanUp() throws IOException { + temporaryFolder.delete(); + } + + @Test + public void testSizeLimitTrackerStatusUpdate() { + BlobCacheSizeTracker tracker = new BlobCacheSizeTracker(5L); + List<JobID> jobIds = new ArrayList<>(); + List<BlobKey> blobKeys = new ArrayList<>(); + for (int i = 0; i < 5; i++) { + jobIds.add(new JobID()); + blobKeys.add(BlobKey.createKey(BlobKey.BlobType.PERMANENT_BLOB)); + } + for (int i = 0; i < 5; i++) { + tracker.track(jobIds.get(i), blobKeys.get(i), 1); + } + tracker.update(jobIds.get(1), blobKeys.get(1)); + tracker.update(jobIds.get(2), blobKeys.get(2)); + + List<Tuple2<JobID, BlobKey>> blobsToDelete = tracker.checkLimit(2); + + assertThat( + blobsToDelete, + containsInAnyOrder( + Tuple2.of(jobIds.get(0), blobKeys.get(0)), + Tuple2.of(jobIds.get(3), blobKeys.get(3)))); + } + + @Test + public void testTrackPermanentBlobCache() throws Exception { + + final BlobKey.BlobType blobType = BlobKey.BlobType.PERMANENT_BLOB; + + final Configuration config = new Configuration(); + config.setString( + BlobServerOptions.STORAGE_DIRECTORY, temporaryFolder.newFolder().getAbsolutePath()); + + // The size limit is the size of 2 blobs + int sizeLimit = 2; + + try (BlobServer server = new BlobServer(config, new VoidBlobStore()); + BlobCacheService cache = + initBlobCacheServiceWithSizeLimit( + config, + new VoidBlobStore(), + new InetSocketAddress("localhost", server.getPort()), + sizeLimit * blobSize)) { + + server.start(); + + // Initialize the data structures + JobID[] jobId = new JobID[4]; + byte[][] data = new byte[4][blobSize]; + BlobKey[] key = new BlobKey[4]; + File[] file = new File[4]; + + // Prepare the blob data + byte[] blobData = new byte[blobSize]; + random.nextBytes(blobData); + + for (int i = 0; i < 4; i++) { + jobId[i] = new JobID(); + data[i] = Arrays.copyOf(blobData, blobData.length); + data[i][i] ^= 1; + } + + // Put the first, the second and the the third BLOB into the blob server one by one + // and let the cache retrieve the data from the blob + for (int i = 0; i < 3; i++) { + key[i] = put(server, jobId[i], data[i], blobType); + assertNotNull(key[i]); + + readFileAndVerifyContent(cache, jobId[i], key[i], data[i]); + file[i] = getFile(cache, jobId[i], key[i]); + assertTrue(file[i].exists()); + } + + // Since the size limit of the cache is the size of 2 blobs, + // the first blob is removed and the second one remains + assertFalse(file[0].exists()); + assertTrue(file[1].exists()); + + // Get the second blob once again, make the third blob to be the last recently used + readFileAndVerifyContent(cache, jobId[1], key[1], data[1]); + + // Then get the fourth blob, make sure the third blob is replaced. + // Here we cannot get the first blob again, because for transient blob, + // BlobServer will delete the blob once it's transmitted. + key[3] = put(server, jobId[3], data[3], blobType); + readFileAndVerifyContent(cache, jobId[3], key[3], data[3]); + file[3] = getFile(cache, jobId[3], key[3]); + + assertTrue(file[1].exists()); + assertTrue(file[3].exists()); + assertFalse(file[2].exists()); + } + } + + @Test + public void testGetPermanentCacheWithSizeLimitConcurrently() throws Exception { Review comment: It is test of `PermanentBlobCache` and should not be in `BlobCacheSizeTrackerTest`. ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheSizeTrackerTest.java ########## @@ -0,0 +1,286 @@ +/* + * 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.util.TestLogger; +import org.apache.flink.util.concurrent.FutureUtils; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import javax.annotation.Nullable; + +import java.io.File; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Random; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import static org.apache.flink.runtime.blob.BlobServerPutTest.put; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** Tests for {@link BlobCacheSizeTracker}. */ +public class BlobCacheSizeTrackerTest extends TestLogger { + + private static final int blobSize = 10_000; + + private static final Random random = new Random(); + + private static final TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @BeforeClass + public static void setUp() throws IOException { + temporaryFolder.create(); + } + + @AfterClass + public static void cleanUp() throws IOException { + temporaryFolder.delete(); + } + + @Test + public void testSizeLimitTrackerStatusUpdate() { + BlobCacheSizeTracker tracker = new BlobCacheSizeTracker(5L); + List<JobID> jobIds = new ArrayList<>(); + List<BlobKey> blobKeys = new ArrayList<>(); + for (int i = 0; i < 5; i++) { + jobIds.add(new JobID()); + blobKeys.add(BlobKey.createKey(BlobKey.BlobType.PERMANENT_BLOB)); + } + for (int i = 0; i < 5; i++) { + tracker.track(jobIds.get(i), blobKeys.get(i), 1); + } + tracker.update(jobIds.get(1), blobKeys.get(1)); + tracker.update(jobIds.get(2), blobKeys.get(2)); + + List<Tuple2<JobID, BlobKey>> blobsToDelete = tracker.checkLimit(2); + + assertThat( + blobsToDelete, + containsInAnyOrder( + Tuple2.of(jobIds.get(0), blobKeys.get(0)), + Tuple2.of(jobIds.get(3), blobKeys.get(3)))); + } + + @Test + public void testTrackPermanentBlobCache() throws Exception { Review comment: It is test of `PermanentBlobCache` and should not be in `BlobCacheSizeTrackerTest`. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCacheSizeTracker.java ########## @@ -0,0 +1,177 @@ +/* + * 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.annotation.VisibleForTesting; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.java.tuple.Tuple2; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import javax.annotation.concurrent.GuardedBy; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * BlobCacheSizeTracker uses {@link LinkedHashMap} to maintain the LRU order for the files in the + * cache. When new files are intended to be put into cache, {@code checkLimit} is called to query + * the files should be removed. This tracker maintains a lock to avoid concurrent modification. To + * avoid the deadlock, make sure that hold the READ/WRITE lock in {@link PermanentBlobCache} first + * and then hold the lock here. + */ +public class BlobCacheSizeTracker { + + private static final Logger LOG = LoggerFactory.getLogger(BlobCacheSizeTracker.class); + + private static final int INITIAL_SIZE = 10_000; + + private final Object lock = new Object(); + + protected final long sizeLimit; + + @GuardedBy("lock") + private long total; + + @GuardedBy("lock") + private final LinkedHashMap<Tuple2<JobID, BlobKey>, Long> caches; + + @GuardedBy("lock") + protected final HashMap<JobID, Set<BlobKey>> blobKeyByJob; + + public BlobCacheSizeTracker(long sizeLimit) { + this.sizeLimit = sizeLimit; + this.total = 0L; + this.caches = new LinkedHashMap<>(INITIAL_SIZE, 0.75F, true); + this.blobKeyByJob = new HashMap<>(); + } + + /** + * Check the size limit and return the files to delete. + * + * @param size size of the target file intended to put into cache + * @return list of files to delete before saving the target file + */ + public List<Tuple2<JobID, BlobKey>> checkLimit(long size) { + checkArgument(size >= 0); + + synchronized (lock) { + List<Tuple2<JobID, BlobKey>> blobsToDelete = new ArrayList<>(); + + long current = total; + + for (Map.Entry<Tuple2<JobID, BlobKey>, Long> entry : caches.entrySet()) { + if (current + size > sizeLimit) { + blobsToDelete.add(entry.getKey()); + current -= entry.getValue(); + } + } + + return blobsToDelete; + } + } + + /** Register the target file to the tracker. */ + public void track(@Nullable JobID jobId, BlobKey blobKey, long size) { + checkNotNull(blobKey); + checkArgument(size >= 0); + + synchronized (lock) { + caches.put(Tuple2.of(jobId, blobKey), size); + if (jobId != null) { + blobKeyByJob.computeIfAbsent(jobId, ignore -> new HashSet<>()).add(blobKey); + } + total += size; + if (total > sizeLimit) { + LOG.warn( + "The overall size of ShuffleDescriptors in PermanentBlobCache exceeds the limit. " + + "Limit: {}, current: {}. The size of next ShuffleDescriptors: {}.", + sizeLimit, + total, + size); + } + } + } + + /** Remove the BLOB from the tracker. */ + public void untrack(Tuple2<JobID, BlobKey> key) { + checkNotNull(key); + + synchronized (lock) { + if (key.f0 != null) { + blobKeyByJob.computeIfAbsent(key.f0, ignore -> new HashSet<>()).remove(key.f1); + } + Long size = caches.remove(key); + if (size != null && size >= 0) { + total -= size; + } + } + } + + /** Remove the BLOB from the tracker. */ + public void untrack(@Nullable JobID jobId, BlobKey blobKey) { + checkNotNull(blobKey); + + untrack(Tuple2.of(jobId, blobKey)); + } + + /** + * Update the last used index for the BLOBs so that the tracker can easily find out the last + * recently used BLOBs. + */ + public void update(@Nullable JobID jobId, BlobKey blobKey) { + checkNotNull(blobKey); + + synchronized (lock) { + caches.get(Tuple2.of(jobId, blobKey)); + } + } + + /** Unregister all the tracked BLOBs related to given job. */ + public void unregisterJob(JobID jobId) { + checkNotNull(jobId); + + synchronized (lock) { + for (BlobKey blobKey : blobKeyByJob.computeIfAbsent(jobId, ignore -> new HashSet<>())) { + untrack(jobId, blobKey); + } + blobKeyByJob.remove(jobId); + } + } + + @VisibleForTesting + Set<BlobKey> getBlobKeysByJobId(JobID jobId) { Review comment: Seems this method is never used? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
