reswqa commented on code in PR #21560: URL: https://github.com/apache/flink/pull/21560#discussion_r1062396153
########## flink-runtime/src/main/java/org/apache/flink/runtime/deployment/CachedShuffleDescriptors.java: ########## @@ -0,0 +1,111 @@ +/* + * 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.deployment; + +import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor.MaybeOffloaded; +import org.apache.flink.runtime.deployment.TaskDeploymentDescriptorFactory.ShuffleDescriptorAndIndex; +import org.apache.flink.runtime.executiongraph.IntermediateResultPartition; +import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.scheduler.strategy.ConsumedPartitionGroup; +import org.apache.flink.runtime.shuffle.ShuffleDescriptor; +import org.apache.flink.util.function.FunctionWithException; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Queue; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** {@link ShuffleDescriptor}s cache for a {@link ConsumedPartitionGroup}. */ +public class CachedShuffleDescriptors { + /** + * Stores all serialized shuffle descriptors with indexes. For a partition, it may be added a + * serialized unknown shuffle descriptor to this list first, and then added the real descriptor + * later. + */ + private final List<MaybeOffloaded<ShuffleDescriptorAndIndex[]>> serializedShuffleDescriptors; + + /** + * Stores all to be serialized shuffle descriptors, They will be serialized and added to + * serializedShuffleDescriptors during the next time TaskDeploymentDescriptor is generated. + */ + private final Queue<ShuffleDescriptorAndIndex> toBeSerialized; + + /** Stores the mapping of resultPartitionId to index subscripts in consumed partition group. */ + private final Map<IntermediateResultPartitionID, Integer> resultPartitionIdToIndex; + + public CachedShuffleDescriptors( + ConsumedPartitionGroup consumedPartitionGroup, + MaybeOffloaded<ShuffleDescriptorAndIndex[]> initialSerializedShuffleDescriptors) { + this.resultPartitionIdToIndex = new HashMap<>(); + int index = 0; + for (IntermediateResultPartitionID resultPartitionID : consumedPartitionGroup) { + resultPartitionIdToIndex.put(resultPartitionID, index++); + } + this.toBeSerialized = new ArrayDeque<>(consumedPartitionGroup.size()); + this.serializedShuffleDescriptors = new ArrayList<>(); + this.serializedShuffleDescriptors.add(initialSerializedShuffleDescriptors); + } + + public MaybeOffloaded<ShuffleDescriptorAndIndex[]> getShuffleDescriptors(int index) { Review Comment: Using `getAllSerializedShuffleDescriptors` to replace these two method's invoke and remove it. -- 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]
