xintongsong commented on code in PR #21233:
URL: https://github.com/apache/flink/pull/21233#discussion_r1040542541
##########
flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/NonSupportedResourceAllocatorImpl.java:
##########
@@ -18,11 +18,14 @@
package org.apache.flink.runtime.resourcemanager.slotmanager;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
import org.apache.flink.runtime.instance.InstanceID;
import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
/** ResourceAllocator that not support to allocate/release resources. */
public class NonSupportedResourceAllocatorImpl implements ResourceAllocator {
+ public static final NonSupportedResourceAllocatorImpl INSTANCE =
+ new NonSupportedResourceAllocatorImpl();
Review Comment:
For singleton classes, we should add a private constructor to prevent it
from being instantiated.
##########
flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManager.java:
##########
@@ -754,15 +754,16 @@ private boolean allocateResource(PendingTaskManager
pendingTaskManager) {
return false;
}
- if (!resourceAllocator.isSupported()
- || !resourceAllocator.allocateResource(
- WorkerResourceSpec.fromTotalResourceProfile(
- pendingTaskManager.getTotalResourceProfile(),
- pendingTaskManager.getNumSlots()))) {
+ if (!resourceAllocator.isSupported()) {
Review Comment:
It would be better to move this check to before the max total resource
check, because this one is cheaper.
##########
flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskExecutorManager.java:
##########
@@ -154,13 +154,15 @@ public boolean registerTaskManager(
ResourceProfile totalResourceProfile,
ResourceProfile defaultSlotResourceProfile) {
if (isMaxSlotNumExceededAfterRegistration(initialSlotReport)) {
- LOG.info(
- "The total number of slots exceeds the max limitation {},
releasing the excess task executor.",
- maxSlotNum);
- resourceActions.releaseResource(
- taskExecutorConnection.getInstanceID(),
- new FlinkExpectedException(
- "The total number of slots exceeds the max
limitation."));
+ if (resourceAllocator.isSupported()) {
Review Comment:
Same here. The if-statements can be combined.
##########
flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManager.java:
##########
@@ -341,15 +348,17 @@ public boolean registerTaskManager(
if (!matchedPendingTaskManagerOptional.isPresent()
&&
isMaxTotalResourceExceededAfterAdding(totalResourceProfile)) {
- LOG.info(
- "Releasing task manager {}. The max total resource
limitation <{}, {}> is reached.",
- taskExecutorConnection.getResourceID(),
- maxTotalCpu,
- maxTotalMem.toHumanReadableString());
- resourceActions.releaseResource(
- taskExecutorConnection.getInstanceID(),
- new FlinkExpectedException(
- "The max total resource limitation is
reached."));
+ if (resourceAllocator.isSupported()) {
Review Comment:
If- statements can be combined.
--
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]