wonook commented on a change in pull request #288: [NEMO-438] Create a Simulator for Simulating an Execution of an Execution Plan URL: https://github.com/apache/incubator-nemo/pull/288#discussion_r386952122
########## File path: runtime/master/src/main/java/org/apache/nemo/runtime/master/scheduler/SimulationScheduler.java ########## @@ -0,0 +1,579 @@ +/* + * 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.nemo.runtime.master.scheduler; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.commons.lang3.SerializationUtils; +import org.apache.nemo.common.Pair; +import org.apache.nemo.common.Util; +import org.apache.nemo.common.exception.IllegalMessageException; +import org.apache.nemo.common.exception.SimulationException; +import org.apache.nemo.common.exception.UnknownExecutionStateException; +import org.apache.nemo.common.exception.UnrecoverableFailureException; +import org.apache.nemo.common.ir.executionproperty.ResourceSpecification; +import org.apache.nemo.conf.JobConf; +import org.apache.nemo.runtime.common.RuntimeIdManager; +import org.apache.nemo.runtime.common.comm.ControlMessage; +import org.apache.nemo.runtime.common.message.MessageSender; +import org.apache.nemo.runtime.common.message.MessageUtils; +import org.apache.nemo.runtime.common.metric.JobMetric; +import org.apache.nemo.runtime.common.metric.Metric; +import org.apache.nemo.runtime.common.plan.PhysicalPlan; +import org.apache.nemo.runtime.common.plan.PlanRewriter; +import org.apache.nemo.runtime.common.plan.Stage; +import org.apache.nemo.runtime.common.plan.Task; +import org.apache.nemo.runtime.common.state.StageState; +import org.apache.nemo.runtime.common.state.TaskState; +import org.apache.nemo.runtime.master.BlockManagerMaster; +import org.apache.nemo.runtime.master.PlanAppender; +import org.apache.nemo.runtime.master.PlanStateManager; +import org.apache.nemo.runtime.master.metric.MetricStore; +import org.apache.nemo.runtime.master.resource.DefaultExecutorRepresenter; +import org.apache.nemo.runtime.master.resource.ExecutorRepresenter; +import org.apache.reef.annotations.audience.DriverSide; +import org.apache.reef.driver.context.ActiveContext; +import org.apache.reef.driver.evaluator.EvaluatorDescriptor; +import org.apache.reef.tang.Configuration; +import org.apache.reef.tang.annotations.Parameter; +import org.apache.reef.util.Optional; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import javax.annotation.concurrent.NotThreadSafe; +import javax.inject.Inject; +import java.util.*; +import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + +/** + * Scheduler for simulating an execution, not controlled by the runtime master. + */ +@DriverSide +@NotThreadSafe +public final class SimulationScheduler implements Scheduler { + private static final Logger LOG = LoggerFactory.getLogger(SimulationScheduler.class.getName()); + + /** + * Run-time optimizations. + */ + private final PlanRewriter planRewriter; + + /** + * Components related to scheduling the given plan. + */ + private TaskDispatcher taskDispatcher; + private final PendingTaskCollectionPointer pendingTaskCollectionPointer; + private ExecutorRegistry executorRegistry; + private PlanStateManager planStateManager; + private final ExecutorService serializationExecutorService; // Executor service for scheduling message serialization. + private final BlockManagerMaster blockManagerMaster; + private final MetricStore actualMetricStore; + private MetricStore metricStore; + private CountDownLatch metricCountDownLatch; + + private final SchedulingConstraintRegistry schedulingConstraintRegistry; + private final SchedulingPolicy schedulingPolicy; + private final String resourceSpecificationString; + private final String dagDirectory; + + private final Map<String, SimulatedTaskExecutor> simulatedTaskExecutorMap; + + /** + * The below variables depend on the submitted plan to execute. + */ + private List<List<Stage>> sortedScheduleGroups; + + @Inject + private SimulationScheduler(final PlanRewriter planRewriter, + final SchedulingConstraintRegistry schedulingConstraintRegistry, + final BlockManagerMaster blockManagerMaster, + @Parameter(JobConf.ExecutorJSONContents.class) final String resourceSpecificationString, + @Parameter(JobConf.ScheduleSerThread.class) final int scheduleSerThread, + @Parameter(JobConf.DAGDirectory.class) final String dagDirectory) { + this.planRewriter = planRewriter; + this.blockManagerMaster = blockManagerMaster; + this.pendingTaskCollectionPointer = PendingTaskCollectionPointer.newInstance(); + this.executorRegistry = ExecutorRegistry.newInstance(); + this.schedulingConstraintRegistry = schedulingConstraintRegistry; + this.schedulingPolicy = new SimulationSchedulingPolicy(); + this.resourceSpecificationString = resourceSpecificationString; + this.dagDirectory = dagDirectory; + this.planStateManager = PlanStateManager.newInstance(dagDirectory); + this.taskDispatcher = TaskDispatcher.newInstance(schedulingConstraintRegistry, schedulingPolicy, + pendingTaskCollectionPointer, executorRegistry, planStateManager); + this.serializationExecutorService = Executors.newFixedThreadPool(scheduleSerThread); + this.actualMetricStore = MetricStore.getStore(); + this.metricStore = MetricStore.newInstance(); + this.planStateManager.setMetricStore(this.metricStore); + this.simulatedTaskExecutorMap = new HashMap<>(); + setUpExecutors(); + } + + /** + * Simulate the launch of executors. + */ + private void setUpExecutors() { + final List<Pair<Integer, ResourceSpecification>> resourceSpecs = + Util.parseResourceSpecificationString(resourceSpecificationString); + // Role of ActiveContextHandler + RuntimeMaster.onExecuterLaunched. + final AtomicInteger executorIdGenerator = new AtomicInteger(0); + final AtomicInteger resourceRequestCount = new AtomicInteger(0); + resourceSpecs.forEach(p -> { + for (int i = 0; i < p.left(); i++) { + resourceRequestCount.getAndIncrement(); + final ActiveContext ac = new SimulationEvaluatorActiveContext(executorIdGenerator.getAndIncrement()); + this.onExecutorAdded(new DefaultExecutorRepresenter(ac.getId(), p.right(), + new SimulationMessageSender(ac.getId(), this), ac, serializationExecutorService, ac.getId())); + } + }); + this.metricCountDownLatch = new CountDownLatch(resourceRequestCount.get()); + } + + /** + * Reset the instance to its initial state. + */ + public void reset() { + this.terminate(); + this.executorRegistry = ExecutorRegistry.newInstance(); + this.planStateManager = PlanStateManager.newInstance(dagDirectory); + this.pendingTaskCollectionPointer.getAndSetNull(); + this.taskDispatcher = TaskDispatcher.newInstance(schedulingConstraintRegistry, schedulingPolicy, + pendingTaskCollectionPointer, executorRegistry, planStateManager); + this.metricStore = MetricStore.newInstance(); + this.planStateManager.setMetricStore(metricStore); + this.simulatedTaskExecutorMap.clear(); + setUpExecutors(); + } + + @VisibleForTesting + public PlanStateManager getPlanStateManager() { + return planStateManager; + } + + /** + * The entrance point of the simulator. Simulate a plan by submitting a plan through this method. + * @param submittedPhysicalPlan the plan to simulate. + * @param maxScheduleAttempt the max number of times this plan/sub-part of the plan should be attempted. + */ + @Override + public void schedulePlan(final PhysicalPlan submittedPhysicalPlan, final int maxScheduleAttempt) { + // Execute the given plan. + LOG.info("Plan to schedule: {}", submittedPhysicalPlan.getPlanId()); + + if (!planStateManager.isInitialized()) { + // First scheduling. + taskDispatcher.run(); + updatePlan(submittedPhysicalPlan, maxScheduleAttempt); + planStateManager.storeJSON("submitted"); + } else { + // Append the submitted plan to the original plan. + final PhysicalPlan appendedPlan = + PlanAppender.appendPlan(planStateManager.getPhysicalPlan(), submittedPhysicalPlan); + updatePlan(appendedPlan, maxScheduleAttempt); + planStateManager.storeJSON("appended"); + } + + doSchedule(); + + try { + planStateManager.waitUntilFinish(); + } finally { + planStateManager.storeJSON("final"); + } + + final Long jobDuration = this.simulatedTaskExecutorMap.values().stream() + .mapToLong(SimulatedTaskExecutor::getElapsedTime) + .max().orElse(0); + LOG.info("Simulation of {} is complete with job duration of {}!", submittedPhysicalPlan.getPlanId(), jobDuration); + this.metricStore.getOrCreateMetric(JobMetric.class, submittedPhysicalPlan.getPlanId()).setJobDuration(jobDuration); + executorRegistry.viewExecutors(executors -> executors.forEach(executor -> metricCountDownLatch.countDown())); + } + + /** + * The main entry point for task scheduling. + * This operation can be invoked at any point during job execution, as it is designed to be free of side-effects. + * <p> + * These are the reasons why. + * - We 'reset' {@link PendingTaskCollectionPointer}, and not 'add' new tasks to it + * - We make {@link TaskDispatcher} dispatch only the tasks that are READY. + */ + private void doSchedule() { + final java.util.Optional<List<Stage>> earliest = + SchedulerUtils.selectEarliestSchedulableGroup(sortedScheduleGroups, planStateManager); + + if (earliest.isPresent()) { + final List<Task> tasksToSchedule = earliest.get().stream() + .flatMap(stage -> SchedulerUtils.selectSchedulableTasks(planStateManager, blockManagerMaster, stage).stream()) + .collect(Collectors.toList()); + if (!tasksToSchedule.isEmpty()) { + LOG.info("Scheduling some tasks in {}, which are in the same ScheduleGroup", tasksToSchedule.stream() + .map(Task::getTaskId) + .map(RuntimeIdManager::getStageIdFromTaskId) + .collect(Collectors.toSet())); + + // Set the pointer to the schedulable tasks. + pendingTaskCollectionPointer.setToOverwrite(tasksToSchedule); + + // Notify the dispatcher that a new collection is available. + taskDispatcher.onNewPendingTaskCollectionAvailable(); + } + } else { + LOG.info("Skipping this round as no ScheduleGroup is schedulable."); + } + } + + @Override + public void updatePlan(final PhysicalPlan newPhysicalPlan) { + // update the physical plan in the scheduler. + // NOTE: what's already been executed is not modified in the new physical plan. + // TODO #182: Consider reshaping in run-time optimization. At now, we only consider plan appending. + updatePlan(newPhysicalPlan, planStateManager.getMaxScheduleAttempt()); + } + + /** + * Update the physical plan in the scheduler. + * + * @param newPhysicalPlan the new physical plan to update. + * @param maxScheduleAttempt the maximum number of task scheduling attempt. + */ + private void updatePlan(final PhysicalPlan newPhysicalPlan, + final int maxScheduleAttempt) { + planStateManager.updatePlan(newPhysicalPlan, maxScheduleAttempt); + this.sortedScheduleGroups = newPhysicalPlan.getStageDAG().getVertices().stream() + .collect(Collectors.groupingBy(Stage::getScheduleGroup)) + .entrySet().stream() + .sorted(Map.Entry.comparingByKey()) + .map(Map.Entry::getValue) + .collect(Collectors.toList()); + } + + @Override + public void onExecutorAdded(final ExecutorRepresenter executorRepresenter) { + LOG.info("{} added (node: {})", executorRepresenter.getExecutorId(), executorRepresenter.getNodeName()); + executorRegistry.registerExecutor(executorRepresenter); + this.simulatedTaskExecutorMap.put(executorRepresenter.getExecutorId(), + new SimulatedTaskExecutor(this, executorRepresenter, actualMetricStore)); + taskDispatcher.onExecutorSlotAvailable(); + } + + @Override + public void onExecutorRemoved(final String executorId) { + // Role of FailedEvaluatorHandler + onExecutorFailed. + metricCountDownLatch.countDown(); + LOG.info("{} removed", executorId); + + // These are tasks that were running at the time of executor removal. + final Set<String> interruptedTasks = new HashSet<>(); + executorRegistry.updateExecutor(executorId, (executor, state) -> { + interruptedTasks.addAll(executor.onExecutorFailed()); + return Pair.of(executor, ExecutorRegistry.ExecutorState.FAILED); + }); + + // Retry the interrupted tasks (and required parents) + SchedulerUtils.retryTasksAndRequiredParents(planStateManager, blockManagerMaster, interruptedTasks); + Review comment: I tried to refactor the common parts to the BatchSchedulerUtil, so it seems like this would be alright. ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: [email protected] With regards, Apache Git Services
