/**
   * Plan for some additional workers to be launched.
   *
   * @param numWorkers The number of workers to allocate.
   */
  @Override
  protected void requestNewWorkers(int numWorkers) {

    try {
      List<TaskMonitor.TaskGoalStateUpdated> toMonitor = new ArrayList<>(numWorkers);
      List<LaunchableTask> toLaunch = new ArrayList<>(numWorkers);

      // generate new workers into persistent state and launch associated actors
      for (int i = 0; i < numWorkers; i++) {
        MesosWorkerStore.Worker worker = MesosWorkerStore.Worker.newWorker(workerStore.newTaskID());
        workerStore.putWorker(worker);
        workersInNew.put(extractResourceID(worker.taskID()), worker);

        LaunchableMesosWorker launchable = createLaunchableMesosWorker(worker.taskID());

        LOG.info(
            "Scheduling Mesos task {} with ({} MB, {} cpus).",
            launchable.taskID().getValue(),
            launchable.taskRequest().getMemory(),
            launchable.taskRequest().getCPUs());

        toMonitor.add(new TaskMonitor.TaskGoalStateUpdated(extractGoalState(worker)));
        toLaunch.add(launchable);
      }

      // tell the task router about the new plans
      for (TaskMonitor.TaskGoalStateUpdated update : toMonitor) {
        taskRouter.tell(update, self());
      }

      // tell the launch coordinator to launch the new tasks
      if (toLaunch.size() >= 1) {
        launchCoordinator.tell(new LaunchCoordinator.Launch(toLaunch), self());
      }
    } catch (Exception ex) {
      fatalError("unable to request new workers", ex);
    }
  }
  /** Called when connected to Mesos as a new framework. */
  private void registered(Registered message) {
    connectionMonitor.tell(message, self());

    try {
      workerStore.setFrameworkID(Option.apply(message.frameworkId()));
    } catch (Exception ex) {
      fatalError("unable to store the assigned framework ID", ex);
      return;
    }

    launchCoordinator.tell(message, self());
    reconciliationCoordinator.tell(message, self());
    taskRouter.tell(message, self());
  }
  @Override
  protected void initialize() throws Exception {
    LOG.info("Initializing Mesos resource master");

    workerStore.start();

    // create the scheduler driver to communicate with Mesos
    schedulerCallbackHandler = new SchedulerProxy(self());

    // register with Mesos
    FrameworkInfo.Builder frameworkInfo = mesosConfig.frameworkInfo().clone().setCheckpoint(true);

    Option<Protos.FrameworkID> frameworkID = workerStore.getFrameworkID();
    if (frameworkID.isEmpty()) {
      LOG.info("Registering as new framework.");
    } else {
      LOG.info(
          "Recovery scenario: re-registering using framework ID {}.", frameworkID.get().getValue());
      frameworkInfo.setId(frameworkID.get());
    }

    MesosConfiguration initializedMesosConfig = mesosConfig.withFrameworkInfo(frameworkInfo);
    MesosConfiguration.logMesosConfig(LOG, initializedMesosConfig);
    schedulerDriver = initializedMesosConfig.createDriver(schedulerCallbackHandler, false);

    // create supporting actors
    connectionMonitor = createConnectionMonitor();
    launchCoordinator = createLaunchCoordinator();
    reconciliationCoordinator = createReconciliationCoordinator();
    taskRouter = createTaskRouter();

    recoverWorkers();

    connectionMonitor.tell(new ConnectionMonitor.Start(), self());
    schedulerDriver.start();
  }
  /**
   * Called to shut down the cluster (not a failover situation).
   *
   * @param finalStatus The application status to report.
   * @param optionalDiagnostics An optional diagnostics message.
   */
  @Override
  protected void shutdownApplication(ApplicationStatus finalStatus, String optionalDiagnostics) {

    LOG.info("Shutting down and unregistering as a Mesos framework.");
    try {
      // unregister the framework, which implicitly removes all tasks.
      schedulerDriver.stop(false);
    } catch (Exception ex) {
      LOG.warn("unable to unregister the framework", ex);
    }

    try {
      workerStore.stop(true);
    } catch (Exception ex) {
      LOG.warn("unable to stop the worker state store", ex);
    }

    context().stop(self());
  }
  /** Plan for the removal of the given worker. */
  private void releaseWorker(MesosWorkerStore.Worker worker) {
    try {
      LOG.info("Releasing worker {}", worker.taskID());

      // update persistent state of worker to Released
      worker = worker.releaseWorker();
      workerStore.putWorker(worker);
      workersBeingReturned.put(extractResourceID(worker.taskID()), worker);
      taskRouter.tell(new TaskMonitor.TaskGoalStateUpdated(extractGoalState(worker)), self());

      if (worker.hostname().isDefined()) {
        // tell the launch coordinator that the task is being unassigned from the host, for planning
        // purposes
        launchCoordinator.tell(
            new LaunchCoordinator.Unassign(worker.taskID(), worker.hostname().get()), self());
      }
    } catch (Exception ex) {
      fatalError("unable to release worker", ex);
    }
  }
  /** Recover framework/worker information persisted by a prior incarnation of the RM. */
  private void recoverWorkers() throws Exception {
    // if this application master starts as part of an ApplicationMaster/JobManager recovery,
    // then some worker tasks are most likely still alive and we can re-obtain them
    final List<MesosWorkerStore.Worker> tasksFromPreviousAttempts = workerStore.recoverWorkers();

    if (!tasksFromPreviousAttempts.isEmpty()) {
      LOG.info("Retrieved {} TaskManagers from previous attempt", tasksFromPreviousAttempts.size());

      List<Tuple2<TaskRequest, String>> toAssign =
          new ArrayList<>(tasksFromPreviousAttempts.size());
      List<LaunchableTask> toLaunch = new ArrayList<>(tasksFromPreviousAttempts.size());

      for (final MesosWorkerStore.Worker worker : tasksFromPreviousAttempts) {
        LaunchableMesosWorker launchable = createLaunchableMesosWorker(worker.taskID());

        switch (worker.state()) {
          case New:
            workersInNew.put(extractResourceID(worker.taskID()), worker);
            toLaunch.add(launchable);
            break;
          case Launched:
            workersInLaunch.put(extractResourceID(worker.taskID()), worker);
            toAssign.add(new Tuple2<>(launchable.taskRequest(), worker.hostname().get()));
            break;
          case Released:
            workersBeingReturned.put(extractResourceID(worker.taskID()), worker);
            break;
        }
        taskRouter.tell(new TaskMonitor.TaskGoalStateUpdated(extractGoalState(worker)), self());
      }

      // tell the launch coordinator about prior assignments
      if (toAssign.size() >= 1) {
        launchCoordinator.tell(new LaunchCoordinator.Assign(toAssign), self());
      }
      // tell the launch coordinator to launch any new tasks
      if (toLaunch.size() >= 1) {
        launchCoordinator.tell(new LaunchCoordinator.Launch(toLaunch), self());
      }
    }
  }
  /**
   * Accept offers as advised by the launch coordinator.
   *
   * <p>Acceptance is routed through the RM to update the persistent state before forwarding the
   * message to Mesos.
   */
  private void acceptOffers(AcceptOffers msg) {

    try {
      List<TaskMonitor.TaskGoalStateUpdated> toMonitor = new ArrayList<>(msg.operations().size());

      // transition the persistent state of some tasks to Launched
      for (Protos.Offer.Operation op : msg.operations()) {
        if (op.getType() != Protos.Offer.Operation.Type.LAUNCH) {
          continue;
        }
        for (Protos.TaskInfo info : op.getLaunch().getTaskInfosList()) {
          MesosWorkerStore.Worker worker = workersInNew.remove(extractResourceID(info.getTaskId()));
          assert (worker != null);

          worker = worker.launchWorker(info.getSlaveId(), msg.hostname());
          workerStore.putWorker(worker);
          workersInLaunch.put(extractResourceID(worker.taskID()), worker);

          LOG.info(
              "Launching Mesos task {} on host {}.",
              worker.taskID().getValue(),
              worker.hostname().get());

          toMonitor.add(new TaskMonitor.TaskGoalStateUpdated(extractGoalState(worker)));
        }
      }

      // tell the task router about the new plans
      for (TaskMonitor.TaskGoalStateUpdated update : toMonitor) {
        taskRouter.tell(update, self());
      }

      // send the acceptance message to Mesos
      schedulerDriver.acceptOffers(msg.offerIds(), msg.operations(), msg.filters());
    } catch (Exception ex) {
      fatalError("unable to accept offers", ex);
    }
  }
  /** Invoked when a Mesos task reaches a terminal status. */
  private void taskTerminated(Protos.TaskID taskID, Protos.TaskStatus status) {
    // this callback occurs for failed containers and for released containers alike

    final ResourceID id = extractResourceID(taskID);

    boolean existed;
    try {
      existed = workerStore.removeWorker(taskID);
    } catch (Exception ex) {
      fatalError("unable to remove worker", ex);
      return;
    }

    if (!existed) {
      LOG.info("Received a termination notice for an unrecognized worker: {}", id);
      return;
    }

    // check if this is a failed task or a released task
    if (workersBeingReturned.remove(id) != null) {
      // regular finished worker that we released
      LOG.info("Worker {} finished successfully with diagnostics: {}", id, status.getMessage());
    } else {
      // failed worker, either at startup, or running
      final MesosWorkerStore.Worker launched = workersInLaunch.remove(id);
      if (launched != null) {
        LOG.info(
            "Mesos task {} failed, with a TaskManager in launch or registration. "
                + "State: {} Reason: {} ({})",
            id,
            status.getState(),
            status.getReason(),
            status.getMessage());
        // we will trigger re-acquiring new workers at the end
      } else {
        // failed registered worker
        LOG.info(
            "Mesos task {} failed, with a registered TaskManager. " + "State: {} Reason: {} ({})",
            id,
            status.getState(),
            status.getReason(),
            status.getMessage());

        // notify the generic logic, which notifies the JobManager, etc.
        notifyWorkerFailed(id, "Mesos task " + id + " failed.  State: " + status.getState());
      }

      // general failure logging
      failedTasksSoFar++;

      String diagMessage =
          String.format(
              "Diagnostics for task %s in state %s : " + "reason=%s message=%s",
              id, status.getState(), status.getReason(), status.getMessage());
      sendInfoMessage(diagMessage);

      LOG.info(diagMessage);
      LOG.info("Total number of failed tasks so far: {}", failedTasksSoFar);

      // maxFailedTasks == -1 is infinite number of retries.
      if (maxFailedTasks >= 0 && failedTasksSoFar > maxFailedTasks) {
        String msg =
            "Stopping Mesos session because the number of failed tasks ("
                + failedTasksSoFar
                + ") exceeded the maximum failed tasks ("
                + maxFailedTasks
                + "). This number is controlled by the '"
                + ConfigConstants.MESOS_MAX_FAILED_TASKS
                + "' configuration setting. "
                + "By default its the number of requested tasks.";

        LOG.error(msg);
        self()
            .tell(
                decorateMessage(new StopCluster(ApplicationStatus.FAILED, msg)),
                ActorRef.noSender());

        // no need to do anything else
        return;
      }
    }

    // in case failed containers were among the finished containers, make
    // sure we re-examine and request new ones
    triggerCheckWorkers();
  }