This is an automated email from the ASF dual-hosted git repository. guoyangze pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git
commit a69f96514f3da893f6a048b107ea287411f58714 Author: Weihua Hu <[email protected]> AuthorDate: Fri Jul 21 15:12:35 2023 +0800 [FLINK-31843][runtime] delete SlotInfoWithUtilization since it's unused. --- .../jobmaster/slotpool/AllocatedSlotPool.java | 2 +- .../jobmaster/slotpool/DeclarativeSlotPool.java | 2 +- .../slotpool/DeclarativeSlotPoolBridge.java | 4 +- .../slotpool/DeclarativeSlotPoolService.java | 2 +- .../slotpool/DefaultAllocatedSlotPool.java | 27 +++----- .../slotpool/DefaultDeclarativeSlotPool.java | 2 +- .../slotpool/SlotInfoWithUtilization.java | 77 ---------------------- .../flink/runtime/jobmaster/slotpool/SlotPool.java | 10 +-- .../flink/runtime/jobmaster/JobMasterTest.java | 6 +- .../slotpool/DefaultAllocatedSlotPoolTest.java | 26 +++++--- .../slotpool/DefaultDeclarativeSlotPoolTest.java | 4 +- .../slotpool/TestingDeclarativeSlotPool.java | 6 +- .../TestingDeclarativeSlotPoolBuilder.java | 5 +- .../slotpool/TestingFreeSlotInfoTracker.java | 4 +- 14 files changed, 49 insertions(+), 128 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/AllocatedSlotPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/AllocatedSlotPool.java index 03152e364cf..4339e0b8f49 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/AllocatedSlotPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/AllocatedSlotPool.java @@ -133,7 +133,7 @@ public interface AllocatedSlotPool { /** Information about a free slot. */ interface FreeSlotInfo { - SlotInfoWithUtilization asSlotInfo(); + SlotInfo asSlotInfo(); /** * Returns since when this slot is free. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DeclarativeSlotPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DeclarativeSlotPool.java index b4a01c9d18b..981d6682777 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DeclarativeSlotPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DeclarativeSlotPool.java @@ -113,7 +113,7 @@ public interface DeclarativeSlotPool { * * @return collection of free slot information */ - Collection<SlotInfoWithUtilization> getFreeSlotsInformation(); + Collection<SlotInfo> getFreeSlotsInformation(); /** * Returns the free slot tracker. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DeclarativeSlotPoolBridge.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DeclarativeSlotPoolBridge.java index 36fb9692726..1d4af3db7ac 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DeclarativeSlotPoolBridge.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DeclarativeSlotPoolBridge.java @@ -423,7 +423,7 @@ public class DeclarativeSlotPoolBridge extends DeclarativeSlotPoolService implem getDeclarativeSlotPool().getAllSlotsInformation(); final Set<AllocationID> freeSlots = getDeclarativeSlotPool().getFreeSlotsInformation().stream() - .map(SlotInfoWithUtilization::getAllocationId) + .map(SlotInfo::getAllocationId) .collect(Collectors.toSet()); return allSlotsInformation.stream() @@ -433,7 +433,7 @@ public class DeclarativeSlotPoolBridge extends DeclarativeSlotPoolService implem @Override @Nonnull - public Collection<SlotInfoWithUtilization> getAvailableSlotsInformation() { + public Collection<SlotInfo> getAvailableSlotsInformation() { assertRunningInMainThread(); return getDeclarativeSlotPool().getFreeSlotsInformation(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DeclarativeSlotPoolService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DeclarativeSlotPoolService.java index 5379c96cfea..ee5fadbc788 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DeclarativeSlotPoolService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DeclarativeSlotPoolService.java @@ -247,7 +247,7 @@ public class DeclarativeSlotPoolService implements SlotPoolService { slotInfo.getTaskManagerLocation() .getResourceID() .equals(taskManagerId)) - .map(SlotInfoWithUtilization::getAllocationId) + .map(SlotInfo::getAllocationId) .collect(Collectors.toSet()); for (AllocationID allocationId : freeSlots) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultAllocatedSlotPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultAllocatedSlotPool.java index c768054e3a3..a8020152b66 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultAllocatedSlotPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultAllocatedSlotPool.java @@ -195,11 +195,7 @@ public class DefaultAllocatedSlotPool implements AllocatedSlotPool { final AllocatedSlot allocatedSlot = Preconditions.checkNotNull(registeredSlots.get(freeSlot.getKey())); - final SlotInfoWithUtilization slotInfoWithUtilization = - SlotInfoWithUtilization.from(allocatedSlot, this::getTaskExecutorUtilization); - - freeSlotInfos.add( - DefaultFreeSlotInfo.create(slotInfoWithUtilization, freeSlot.getValue())); + freeSlotInfos.add(DefaultFreeSlotInfo.create(allocatedSlot, freeSlot.getValue())); } return freeSlotInfos; @@ -210,9 +206,7 @@ public class DefaultAllocatedSlotPool implements AllocatedSlotPool { Preconditions.checkNotNull(registeredSlots.get(allocationId)); final Long idleSince = Preconditions.checkNotNull(freeSlots.getFreeSlotsSince().get(allocationId)); - final SlotInfoWithUtilization slotInfoWithUtilization = - SlotInfoWithUtilization.from(allocatedSlot, this::getTaskExecutorUtilization); - return DefaultFreeSlotInfo.create(slotInfoWithUtilization, idleSince); + return DefaultFreeSlotInfo.create(allocatedSlot, idleSince); } @Override @@ -280,19 +274,18 @@ public class DefaultAllocatedSlotPool implements AllocatedSlotPool { private static final class DefaultFreeSlotInfo implements AllocatedSlotPool.FreeSlotInfo { - private final SlotInfoWithUtilization slotInfoWithUtilization; + private final SlotInfo slotInfo; private final long freeSince; - private DefaultFreeSlotInfo( - SlotInfoWithUtilization slotInfoWithUtilization, long freeSince) { - this.slotInfoWithUtilization = slotInfoWithUtilization; + private DefaultFreeSlotInfo(SlotInfo slotInfo, long freeSince) { + this.slotInfo = slotInfo; this.freeSince = freeSince; } @Override - public SlotInfoWithUtilization asSlotInfo() { - return slotInfoWithUtilization; + public SlotInfo asSlotInfo() { + return slotInfo; } @Override @@ -300,10 +293,8 @@ public class DefaultAllocatedSlotPool implements AllocatedSlotPool { return freeSince; } - private static DefaultFreeSlotInfo create( - SlotInfoWithUtilization slotInfoWithUtilization, long idleSince) { - return new DefaultFreeSlotInfo( - Preconditions.checkNotNull(slotInfoWithUtilization), idleSince); + private static DefaultFreeSlotInfo create(SlotInfo slotInfo, long idleSince) { + return new DefaultFreeSlotInfo(Preconditions.checkNotNull(slotInfo), idleSince); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultDeclarativeSlotPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultDeclarativeSlotPool.java index 82147530355..9b2e188c074 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultDeclarativeSlotPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultDeclarativeSlotPool.java @@ -564,7 +564,7 @@ public class DefaultDeclarativeSlotPool implements DeclarativeSlotPool { } @Override - public Collection<SlotInfoWithUtilization> getFreeSlotsInformation() { + public Collection<SlotInfo> getFreeSlotsInformation() { return slotPool.getFreeSlotsInformation().stream() .map(AllocatedSlotPool.FreeSlotInfo::asSlotInfo) .collect(Collectors.toList()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotInfoWithUtilization.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotInfoWithUtilization.java deleted file mode 100644 index 6e2b34deeac..00000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotInfoWithUtilization.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * 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.jobmaster.slotpool; - -import org.apache.flink.runtime.clusterframework.types.AllocationID; -import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.clusterframework.types.ResourceProfile; -import org.apache.flink.runtime.jobmaster.SlotInfo; -import org.apache.flink.runtime.taskmanager.TaskManagerLocation; - -import java.util.function.Function; - -/** - * Container for {@link SlotInfo} and the task executors utilization (freeSlots / - * totalOfferedSlots). - */ -public final class SlotInfoWithUtilization implements SlotInfo { - private final SlotInfo slotInfoDelegate; - private final Function<ResourceID, Double> taskExecutorUtilizationLookup; - - private SlotInfoWithUtilization( - SlotInfo slotInfo, Function<ResourceID, Double> taskExecutorUtilizationLookup) { - this.slotInfoDelegate = slotInfo; - this.taskExecutorUtilizationLookup = taskExecutorUtilizationLookup; - } - - public double getTaskExecutorUtilization() { - return taskExecutorUtilizationLookup.apply( - slotInfoDelegate.getTaskManagerLocation().getResourceID()); - } - - @Override - public AllocationID getAllocationId() { - return slotInfoDelegate.getAllocationId(); - } - - @Override - public TaskManagerLocation getTaskManagerLocation() { - return slotInfoDelegate.getTaskManagerLocation(); - } - - @Override - public int getPhysicalSlotNumber() { - return slotInfoDelegate.getPhysicalSlotNumber(); - } - - @Override - public ResourceProfile getResourceProfile() { - return slotInfoDelegate.getResourceProfile(); - } - - @Override - public boolean willBeOccupiedIndefinitely() { - return slotInfoDelegate.willBeOccupiedIndefinitely(); - } - - public static SlotInfoWithUtilization from( - SlotInfo slotInfo, Function<ResourceID, Double> taskExecutorUtilizationLookup) { - return new SlotInfoWithUtilization(slotInfo, taskExecutorUtilizationLookup); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java index c9b01df298f..8bd2383891d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java @@ -116,13 +116,13 @@ public interface SlotPool extends AllocatedSlotActions, AutoCloseable { // ------------------------------------------------------------------------ /** - * Returns a list of {@link SlotInfoWithUtilization} objects about all slots that are currently - * available in the slot pool. + * Returns a list of {@link SlotInfo} objects about all slots that are currently available in + * the slot pool. * - * @return a list of {@link SlotInfoWithUtilization} objects about all slots that are currently - * available in the slot pool. + * @return a list of {@link SlotInfo} objects about all slots that are currently available in + * the slot pool. */ - Collection<SlotInfoWithUtilization> getAvailableSlotsInformation(); + Collection<SlotInfo> getAvailableSlotsInformation(); /** * Returns all free slot tracker. diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java index 04dabcbb86e..da7154c2ba4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java @@ -84,7 +84,6 @@ import org.apache.flink.runtime.jobmaster.slotpool.DeclarativeSlotPoolFactory; import org.apache.flink.runtime.jobmaster.slotpool.FreeSlotInfoTracker; import org.apache.flink.runtime.jobmaster.slotpool.FreeSlotInfoTrackerTestUtils; import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot; -import org.apache.flink.runtime.jobmaster.slotpool.SlotInfoWithUtilization; import org.apache.flink.runtime.jobmaster.slotpool.SlotPool; import org.apache.flink.runtime.jobmaster.slotpool.SlotPoolService; import org.apache.flink.runtime.jobmaster.slotpool.SlotPoolServiceFactory; @@ -575,11 +574,10 @@ class JobMasterTest { @Nonnull @Override - public Collection<SlotInfoWithUtilization> getAvailableSlotsInformation() { - final Collection<SlotInfoWithUtilization> allSlotInfos = + public Collection<SlotInfo> getAvailableSlotsInformation() { + final Collection<SlotInfo> allSlotInfos = registeredSlots.values().stream() .flatMap(Collection::stream) - .map(slot -> SlotInfoWithUtilization.from(slot, ignored -> 0.0d)) .collect(Collectors.toList()); return Collections.unmodifiableCollection(allSlotInfos); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultAllocatedSlotPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultAllocatedSlotPoolTest.java index f1b5b276268..9a539176ea6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultAllocatedSlotPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultAllocatedSlotPoolTest.java @@ -23,6 +23,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.jobmaster.RpcTaskManagerGateway; +import org.apache.flink.runtime.jobmaster.SlotInfo; import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; @@ -223,7 +224,12 @@ class DefaultAllocatedSlotPoolTest { assertThat(slotPool.getFreeSlotsInformation()) .allSatisfy( freeSlotInfo -> - assertThat(freeSlotInfo.asSlotInfo().getTaskExecutorUtilization()) + assertThat( + slotPool.getTaskExecutorUtilization( + freeSlotInfo + .asSlotInfo() + .getTaskManagerLocation() + .getResourceID())) .isCloseTo(0, offset(0.1))); int numAllocatedSlots = 0; @@ -233,7 +239,12 @@ class DefaultAllocatedSlotPoolTest { for (AllocatedSlotPool.FreeSlotInfo freeSlotInfo : slotPool.getFreeSlotsInformation()) { final double utilization = (double) numAllocatedSlots / slots.size(); - assertThat(freeSlotInfo.asSlotInfo().getTaskExecutorUtilization()) + assertThat( + slotPool.getTaskExecutorUtilization( + freeSlotInfo + .asSlotInfo() + .getTaskManagerLocation() + .getResourceID())) .isCloseTo(utilization, offset(0.1)); } } @@ -310,16 +321,15 @@ class DefaultAllocatedSlotPoolTest { AllocatedSlot allocatedSlot = allocatedSlotMap.get(freeSlotInfo.getAllocationId()); assertThat(allocatedSlot).isNotNull(); - SlotInfoWithUtilization slotInfoWithUtilization = - freeSlotInfo.asSlotInfo(); + SlotInfo slotInfo = freeSlotInfo.asSlotInfo(); assertThat(allocatedSlot.getAllocationId()) - .isEqualTo(slotInfoWithUtilization.getAllocationId()); + .isEqualTo(slotInfo.getAllocationId()); assertThat(allocatedSlot.getPhysicalSlotNumber()) - .isEqualTo(slotInfoWithUtilization.getPhysicalSlotNumber()); + .isEqualTo(slotInfo.getPhysicalSlotNumber()); assertThat(allocatedSlot.getResourceProfile()) - .isEqualTo(slotInfoWithUtilization.getResourceProfile()); + .isEqualTo(slotInfo.getResourceProfile()); assertThat(allocatedSlot.getTaskManagerLocation()) - .isEqualTo(slotInfoWithUtilization.getTaskManagerLocation()); + .isEqualTo(slotInfo.getTaskManagerLocation()); }); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultDeclarativeSlotPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultDeclarativeSlotPoolTest.java index 089de0c0b60..5f0a9cb6df4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultDeclarativeSlotPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultDeclarativeSlotPoolTest.java @@ -569,7 +569,7 @@ class DefaultDeclarativeSlotPoolTest { createSlotOffersForResourceRequirements( ResourceCounter.withResource(ResourceProfile.ANY, 1))); - final SlotInfoWithUtilization slot = slotPool.getFreeSlotsInformation().iterator().next(); + final SlotInfo slot = slotPool.getFreeSlotsInformation().iterator().next(); slotPool.reserveFreeSlot(slot.getAllocationId(), largeResourceProfile); assertThat( @@ -614,7 +614,7 @@ class DefaultDeclarativeSlotPoolTest { slotPool.increaseResourceRequirementsBy( ResourceCounter.withResource(smallResourceProfile, 1)); - final SlotInfoWithUtilization largeSlot = + final SlotInfo largeSlot = slotPool.getFreeSlotsInformation().stream() .filter(slot -> slot.getResourceProfile().equals(largeResourceProfile)) .findFirst() diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/TestingDeclarativeSlotPool.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/TestingDeclarativeSlotPool.java index 4f12a42956b..6fc7ef32e3e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/TestingDeclarativeSlotPool.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/TestingDeclarativeSlotPool.java @@ -64,7 +64,7 @@ final class TestingDeclarativeSlotPool implements DeclarativeSlotPool { Collection<SlotOffer>> registerSlotsFunction; - private final Supplier<Collection<SlotInfoWithUtilization>> getFreeSlotsInformationSupplier; + private final Supplier<Collection<SlotInfo>> getFreeSlotsInformationSupplier; private final Supplier<FreeSlotInfoTracker> getFreeSlotInfoTrackerSupplier; @@ -105,7 +105,7 @@ final class TestingDeclarativeSlotPool implements DeclarativeSlotPool { Long, Collection<SlotOffer>> registerSlotsFunction, - Supplier<Collection<SlotInfoWithUtilization>> getFreeSlotsInformationSupplier, + Supplier<Collection<SlotInfo>> getFreeSlotsInformationSupplier, Supplier<FreeSlotInfoTracker> getFreeSlotInfoTrackerSupplier, Supplier<Collection<? extends SlotInfo>> getAllSlotsInformationSupplier, BiFunction<ResourceID, Exception, ResourceCounter> releaseSlotsFunction, @@ -175,7 +175,7 @@ final class TestingDeclarativeSlotPool implements DeclarativeSlotPool { } @Override - public Collection<SlotInfoWithUtilization> getFreeSlotsInformation() { + public Collection<SlotInfo> getFreeSlotsInformation() { return getFreeSlotsInformationSupplier.get(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/TestingDeclarativeSlotPoolBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/TestingDeclarativeSlotPoolBuilder.java index fb27a5797e1..a2c345e1852 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/TestingDeclarativeSlotPoolBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/TestingDeclarativeSlotPoolBuilder.java @@ -54,8 +54,7 @@ public class TestingDeclarativeSlotPoolBuilder { Collection<SlotOffer>> offerSlotsFunction = (ignoredA, ignoredB, ignoredC, ignoredD) -> Collections.emptyList(); - private Supplier<Collection<SlotInfoWithUtilization>> getFreeSlotsInformationSupplier = - Collections::emptyList; + private Supplier<Collection<SlotInfo>> getFreeSlotsInformationSupplier = Collections::emptyList; private Supplier<Collection<? extends SlotInfo>> getAllSlotsInformationSupplier = Collections::emptyList; private Supplier<FreeSlotInfoTracker> getFreeSlotInfoTrackerSupplier = @@ -130,7 +129,7 @@ public class TestingDeclarativeSlotPoolBuilder { } public TestingDeclarativeSlotPoolBuilder setGetFreeSlotsInformationSupplier( - Supplier<Collection<SlotInfoWithUtilization>> getFreeSlotsInformationSupplier) { + Supplier<Collection<SlotInfo>> getFreeSlotsInformationSupplier) { this.getFreeSlotsInformationSupplier = getFreeSlotsInformationSupplier; return this; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/TestingFreeSlotInfoTracker.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/TestingFreeSlotInfoTracker.java index af55233bb48..d59ab1608b2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/TestingFreeSlotInfoTracker.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/TestingFreeSlotInfoTracker.java @@ -180,8 +180,8 @@ public class TestingFreeSlotInfoTracker implements FreeSlotInfoTracker { } @Override - public SlotInfoWithUtilization asSlotInfo() { - return SlotInfoWithUtilization.from(slotInfo, ignore -> 0d); + public SlotInfo asSlotInfo() { + return slotInfo; } @Override
