RocMarshal commented on code in PR #23599:
URL: https://github.com/apache/flink/pull/23599#discussion_r1375961136


##########
flink-runtime/src/main/java/org/apache/flink/runtime/util/DefaultGroupCache.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.util;
+
+import org.apache.flink.annotation.VisibleForTesting;
+
+import org.apache.flink.shaded.guava31.com.google.common.base.Ticker;
+import org.apache.flink.shaded.guava31.com.google.common.cache.Cache;
+import org.apache.flink.shaded.guava31.com.google.common.cache.CacheBuilder;
+import 
org.apache.flink.shaded.guava31.com.google.common.cache.RemovalNotification;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.time.Duration;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+/** Default implement of {@link GroupCache}. Entries will be expired after 
timeout. */
+@NotThreadSafe
+public class DefaultGroupCache<G, K, V> implements GroupCache<G, K, V> {
+    private final Cache<CacheKey<G, K>, V> cache;
+    private final Map<G, Set<CacheKey<G, K>>> cachedBlobKeysPerJob;
+
+    private DefaultGroupCache(Duration expireTimeout, int cacheSizeLimit, 
Ticker ticker) {
+        this.cachedBlobKeysPerJob = new HashMap<>();
+        this.cache =
+                CacheBuilder.newBuilder()
+                        .concurrencyLevel(1)
+                        .maximumSize(cacheSizeLimit)
+                        .expireAfterAccess(expireTimeout)
+                        .ticker(ticker)
+                        .removalListener(this::onCacheRemoval)
+                        .build();
+    }
+
+    @Override
+    public void clear() {
+        cachedBlobKeysPerJob.clear();
+        cache.cleanUp();
+    }
+
+    @Override
+    public V get(G group, K key) {
+        return cache.getIfPresent(new CacheKey<>(group, key));
+    }
+
+    @Override
+    public void put(G group, K key, V value) {
+        CacheKey<G, K> cacheKey = new CacheKey<>(group, key);
+        cache.put(cacheKey, value);
+        cachedBlobKeysPerJob.computeIfAbsent(group, ignore -> new 
HashSet<>()).add(cacheKey);
+    }
+
+    @Override
+    public void clearCacheForGroup(G group) {
+        Set<CacheKey<G, K>> removed = cachedBlobKeysPerJob.remove(group);
+        if (removed != null) {
+            cache.invalidateAll(removed);
+        }
+    }
+
+    /**
+     * Removal listener that remove the cache key of this group .
+     *
+     * @param removalNotification of removed element.
+     */
+    private void onCacheRemoval(RemovalNotification<CacheKey<G, K>, V> 
removalNotification) {
+        CacheKey<G, K> cacheKey = removalNotification.getKey();
+        V value = removalNotification.getValue();
+        if (cacheKey != null && value != null) {
+            cachedBlobKeysPerJob.computeIfPresent(
+                    cacheKey.getGroup(),
+                    (group, keys) -> {
+                        keys.remove(cacheKey);
+                        if (keys.isEmpty()) {
+                            return null;
+                        } else {
+                            return keys;
+                        }
+                    });

Review Comment:
   Would be there a risk of memory leakage here?
   For example, let's talk about the situation:
   
   - There are too many groups
   - Perform the following operations on each of these groups one by one
     - Add a set for one group and then remove set for the same one group, but 
the key has not been removed. Would there be many Entries in the form of 
`Entry-i<Group-i, set-i>`(set-i is empty or null) ?
     
   In short, would `cachedBlobKeysPerJob` degenerate into a collection with too 
many elements?
   
   Please correct me if needed for my limited read.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java:
##########
@@ -151,39 +166,43 @@ public TaskDeploymentDescriptor(
     }
 
     /**
-     * Return the sub task's serialized job information.
+     * Return the sub task's job information.
      *
-     * @return serialized job information (may throw {@link 
IllegalStateException} if {@link
-     *     #loadBigData} is not called beforehand).
+     * @return job information (may throw {@link IllegalStateException} if 
{@link #loadBigData} is
+     *     not called beforehand).
      * @throws IllegalStateException If job information is offloaded to BLOB 
store.
      */
-    public SerializedValue<JobInformation> getSerializedJobInformation() {
+    public JobInformation getJobInformation() throws IOException, 
ClassNotFoundException {
+        if (jobInformation != null) {
+            return jobInformation;
+        }
         if (serializedJobInformation instanceof NonOffloaded) {
             NonOffloaded<JobInformation> jobInformation =
                     (NonOffloaded<JobInformation>) serializedJobInformation;
-            return jobInformation.serializedValue;
-        } else {
-            throw new IllegalStateException(
-                    "Trying to work with offloaded serialized job 
information.");
+            return 
jobInformation.serializedValue.deserializeValue(getClass().getClassLoader());
         }
+        throw new IllegalStateException(
+                "Trying to work with offloaded serialized job information.");
     }
 
     /**
-     * Return the sub task's serialized task information.
+     * Return the sub task's task information.
      *
-     * @return serialized task information (may throw {@link 
IllegalStateException} if {@link
-     *     #loadBigData} is not called beforehand)).
+     * @return task information (may throw {@link IllegalStateException} if 
{@link #loadBigData} is
+     *     not called beforehand)).
      * @throws IllegalStateException If job information is offloaded to BLOB 
store.
      */
-    public SerializedValue<TaskInformation> getSerializedTaskInformation() {
+    public TaskInformation getTaskInformation() throws IOException, 
ClassNotFoundException {
+        if (taskInformation != null) {
+            return taskInformation;
+        }
         if (serializedTaskInformation instanceof NonOffloaded) {
             NonOffloaded<TaskInformation> taskInformation =
                     (NonOffloaded<TaskInformation>) serializedTaskInformation;
-            return taskInformation.serializedValue;
-        } else {
-            throw new IllegalStateException(
-                    "Trying to work with offloaded serialized job 
information.");
+            return 
taskInformation.serializedValue.deserializeValue(getClass().getClassLoader());
         }
+        throw new IllegalStateException(
+                "Trying to work with offloaded serialized task information.");

Review Comment:
   How about 
   ```
           Preconditions.checkState(
                   serializedJobInformation instanceof NonOffloaded,
                    "Trying to work with offloaded serialized job 
information.");
           NonOffloaded<JobInformation> jobInformation =
                   (NonOffloaded<JobInformation>) serializedJobInformation;
           return 
jobInformation.serializedValue.deserializeValue(getClass().getClassLoader());
   
   
   ```
   
   ?



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to