wanglijie95 commented on code in PR #22861:
URL: https://github.com/apache/flink/pull/22861#discussion_r1262696361


##########
flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputGateDeploymentDescriptor.java:
##########
@@ -139,53 +139,67 @@ public IndexRange getConsumedSubpartitionIndexRange() {
         return consumedSubpartitionIndexRange;
     }
 
-    public void loadBigData(@Nullable PermanentBlobService blobService, JobID 
jobId)
+    public ShuffleDescriptor[] getShuffleDescriptors() {
+        if (inputChannels == null) {
+            throw new IllegalStateException("InputChannel should not be 
null.");
+        }
+        return inputChannels;
+    }
+
+    public void loadBigDataAndDeserializeShuffleDescriptors(
+            @Nullable PermanentBlobService blobService,
+            JobID jobId,
+            ShuffleDescriptorsCache shuffleDescriptorsCache)
             throws IOException {
-        for (int i = 0; i < serializedInputChannels.size(); i++) {
-            MaybeOffloaded<ShuffleDescriptorAndIndex[]> shuffleDescriptors =
-                    serializedInputChannels.get(i);
-            if (shuffleDescriptors instanceof Offloaded) {
-                PermanentBlobKey blobKey =
-                        ((Offloaded<ShuffleDescriptorAndIndex[]>) 
shuffleDescriptors)
-                                .serializedValueKey;
+        try {
+            if (inputChannels == null) {
+                inputChannels = new ShuffleDescriptor[numberOfInputChannels];
+            }
 
-                Preconditions.checkNotNull(blobService);
+            for (MaybeOffloaded<ShuffleDescriptorAndIndex[]> 
serializedShuffleDescriptors :

Review Comment:
   Maybe there is some misunderstanding in our offline discussion, I think it 
is ok to introduce `ShuffleDescriptorGroup`(or `ShuffleDescriptorList`), which 
helps to understand the serialization process (The shuffle descriptors in the 
same`ShuffleDescriptorGroup` are serialized together)



##########
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/ShuffleDescriptorsCache.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.blob.PermanentBlobKey;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.deployment.TaskDeploymentDescriptorFactory;
+
+/** Cache of shuffle descriptors in TaskExecutor. */
+public interface ShuffleDescriptorsCache {
+    /**
+     * Start cache manager.
+     *
+     * @param mainThreadExecutor of main thread executor.
+     */
+    void start(ComponentMainThreadExecutor mainThreadExecutor);
+
+    /** Stop cache manager. */
+    void stop();
+
+    /**
+     * Get shuffle descriptors in cache.
+     *
+     * @param blobKey identify the shuffle descriptors
+     * @return shuffle descriptors in cache if exists, otherwise null
+     */
+    TaskDeploymentDescriptorFactory.ShuffleDescriptorAndIndex[] 
get(PermanentBlobKey blobKey);
+
+    /**
+     * Put shuffle descriptors to cache.
+     *
+     * @param jobId of job
+     * @param blobKey identify the shuffle descriptors
+     * @param shuffleDescriptorAndIndices shuffle descriptors to cache
+     */
+    void put(
+            JobID jobId,
+            PermanentBlobKey blobKey,
+            TaskDeploymentDescriptorFactory.ShuffleDescriptorAndIndex[]
+                    shuffleDescriptorAndIndices);
+
+    /**
+     * Clear all cache of the Job.
+     *
+     * @param jobId of job
+     */
+    void clearCacheOfJob(JobID jobId);

Review Comment:
   Maybe `clearCacheForJob`



##########
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java:
##########
@@ -68,6 +69,9 @@
 public class TaskManagerServices {
     private static final Logger LOG = 
LoggerFactory.getLogger(TaskManagerServices.class);
 
+    private static final Duration SHUFFLE_DESCRIPTORS_CACHE_TIMEOUT = 
Duration.ofSeconds(300);
+    private static final int SHUFFLE_DESCRIPTORS_CACHE_SIZE_LIMIT = 100;

Review Comment:
   Is this the num of `shuffle descriptors`? or  the num of 
`Blob/ShuffleDescriptorGroup`?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java:
##########
@@ -68,6 +69,9 @@
 public class TaskManagerServices {
     private static final Logger LOG = 
LoggerFactory.getLogger(TaskManagerServices.class);
 
+    private static final Duration SHUFFLE_DESCRIPTORS_CACHE_TIMEOUT = 
Duration.ofSeconds(300);
+    private static final int SHUFFLE_DESCRIPTORS_CACHE_SIZE_LIMIT = 100;

Review Comment:
   Is this the num of `shuffle descriptors`? or  the num of 
`Blob/ShuffleDescriptorGroup`?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/DefaultShuffleDescriptorsCache.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.blob.PermanentBlobKey;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.deployment.TaskDeploymentDescriptorFactory;
+import org.apache.flink.util.clock.Clock;
+import org.apache.flink.util.clock.SystemClock;
+
+import java.time.Duration;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+/** Default implement of {@link ShuffleDescriptorsCache}. Entries will be 
expired after timeout. */
+public class DefaultShuffleDescriptorsCache implements ShuffleDescriptorsCache 
{

Review Comment:
   Can we directly use the 
`org.apache.flink.shaded.guava31.com.google.common.cache.Cache` as the 
`LRUCache` ? In this way, we can greatly simplify the logic in this class.
   
   We can hold a `Map<JobID, List<PermanentBlobKey>>` to invalidate related 
cache entry when job terminated.



-- 
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]

Reply via email to