zhuzhurk commented on a change in pull request #16307: URL: https://github.com/apache/flink/pull/16307#discussion_r662721409
########## File path: flink-core/src/main/java/org/apache/flink/api/common/operators/util/SlotSharingGroupUtils.java ########## @@ -0,0 +1,53 @@ +/* + * 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.api.common.operators.util; + +import org.apache.flink.api.common.operators.ResourceSpec; +import org.apache.flink.api.common.operators.SlotSharingGroup; +import org.apache.flink.api.common.resources.ExternalResource; +import org.apache.flink.util.Preconditions; + +import java.util.stream.Collectors; + +/** Utils for {@link SlotSharingGroup}. */ +public class SlotSharingGroupUtils { + public static ResourceSpec extractResourceSpec(SlotSharingGroup slotSharingGroup) { + if (!slotSharingGroup.getCpuCores().isPresent()) { + return ResourceSpec.UNKNOWN; + } + + Preconditions.checkState(slotSharingGroup.getCpuCores().isPresent()); + Preconditions.checkState(slotSharingGroup.getTaskHeapMemory().isPresent()); + Preconditions.checkState(slotSharingGroup.getTaskOffHeapMemory().isPresent()); + Preconditions.checkState(slotSharingGroup.getManagedMemory().isPresent()); + + return ResourceSpec.newBuilder(slotSharingGroup.getCpuCores().get(), 0) Review comment: `0` -> `slotSharingGroup.getTaskHeapMemory().get()` And the next line to `setTaskHeapMemory()` can be removed. ########## File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java ########## @@ -749,6 +750,60 @@ public void testSettingSavepointRestoreSettingsSetterOverrides() { equalTo(SavepointRestoreSettings.forPath("/tmp/savepoint1"))); } + @Test + public void testConfigureSlotSharingGroupResource() { + final SlotSharingGroup ssg1 = Review comment: Given that `UNKNOWN` will not be included, let's also check cases that 1. operator without SSG set 2. operator with `UNKNOWN` resource SSG set 3. env with `UNKNOWN` resource SSG set -- 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]
