tillrohrmann commented on a change in pull request #6898: [FLINK-10431] Extraction of scheduling-related code from SlotPool into preliminary Scheduler URL: https://github.com/apache/flink/pull/6898#discussion_r227456207
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/Scheduler.java ########## @@ -0,0 +1,516 @@ +/* + * 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.api.common.time.Time; +import org.apache.flink.runtime.clusterframework.types.SlotProfile; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.instance.SlotSharingGroupId; +import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint; +import org.apache.flink.runtime.jobmanager.scheduler.Locality; +import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException; +import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit; +import org.apache.flink.runtime.jobmaster.LogicalSlot; +import org.apache.flink.runtime.jobmaster.SlotContext; +import org.apache.flink.runtime.jobmaster.SlotInfo; +import org.apache.flink.runtime.jobmaster.SlotOwner; +import org.apache.flink.runtime.jobmaster.SlotRequestId; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.util.AbstractID; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.Executor; +import java.util.function.BooleanSupplier; + +/** + * Scheduler that assigns tasks to slots. This class is currently work in progress, comments will be updated as we + * move forward. + */ +public class Scheduler implements SlotProvider, SlotOwner { + + /** Logger */ + private final Logger log = LoggerFactory.getLogger(getClass()); + + /** Strategy that selects the best slot for a given slot allocation request. */ + @Nonnull + private final SlotSelectionStrategy slotSelectionStrategy; + + /** Managers for the different slot sharing groups. */ + @Nonnull + private final Map<SlotSharingGroupId, SlotSharingManager> slotSharingManagersMap; + + /** The slot pool from which slots are allocated. */ + @Nonnull + private final SlotPoolGateway slotPoolGateway; + + /** Executor for running tasks in the job master's main thread. */ + @Nonnull + private Executor componentMainThreadExecutor; + + /** Predicate to check if the current thread is the job master's main thread. */ + @Nonnull + private BooleanSupplier componentMainThreadCheck; + + + public Scheduler( + @Nonnull Map<SlotSharingGroupId, SlotSharingManager> slotSharingManagersMap, + @Nonnull SlotSelectionStrategy slotSelectionStrategy, + @Nonnull SlotPoolGateway slotPoolGateway) { + + this.slotSelectionStrategy = slotSelectionStrategy; + this.slotSharingManagersMap = slotSharingManagersMap; + this.slotPoolGateway = slotPoolGateway; + this.componentMainThreadExecutor = (runnable) -> { + throw new IllegalStateException("Main thread executor not initialized."); + }; + this.componentMainThreadCheck = () -> { + throw new IllegalStateException("Main thread checker not initialized"); + }; + } + + public void start(@Nonnull Executor mainThreadExecutor, @Nonnull BooleanSupplier mainThreadCheck) { + this.componentMainThreadExecutor = mainThreadExecutor; + this.componentMainThreadCheck = mainThreadCheck; + } + + //--------------------------- + + @Override + public CompletableFuture<LogicalSlot> allocateSlot( + SlotRequestId slotRequestId, + ScheduledUnit scheduledUnit, + boolean allowQueuedScheduling, + SlotProfile slotProfile, + Time allocationTimeout) { Review comment: nit: To better separate broken parameter lists from the body of the method, the runtime code usually uses double indentation. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services