From 587074d6c686f03622c98ad8635db022270bca47 Mon Sep 17 00:00:00 2001 From: Andy Zhang Date: Fri, 5 Dec 2025 17:29:16 -0800 Subject: [PATCH 1/2] reservation scheduling --- .../reservation/context-scheduling-logic.md | 107 ++ .../plan-reservation-registry-v3.md | 164 +++ .../plan-reservation-scheduler-integration.md | 983 ++++++++++++++ .../plan-scaler-reservation-integration.md | 982 ++++++++++++++ .../reservation/plan-scheduling-registry.md | 135 ++ context/reservation-scheduling-context.md | 108 ++ .../resourcecluster/ResourceCluster.java | 48 + .../resourcecluster/ResourceClusters.java | 10 + .../TaskExecutorAllocationRequest.java | 14 +- ...MantisResourceClusterReservationProto.java | 219 ++++ .../master/JobClustersManagerActor.java | 17 + .../master/jobcluster/job/JobActor.java | 360 +++++- .../jobcluster/job/worker/JobWorker.java | 5 +- .../master/jobcluster/proto/JobProto.java | 113 ++ .../AssignmentHandlerActor.java | 361 ++++++ .../resourcecluster/ExecutorStateManager.java | 18 + .../ExecutorStateManagerActor.java | 1124 +++++++++++++++++ .../ExecutorStateManagerImpl.java | 247 +++- .../ReservationRegistryActor.java | 684 ++++++++++ .../resourcecluster/ResourceClusterActor.java | 1083 ++++++---------- .../ResourceClusterActorMetrics.java | 10 + .../ResourceClusterAkkaImpl.java | 42 +- .../ResourceClusterScalerActor.java | 105 +- .../ResourceClustersAkkaImpl.java | 12 + .../ResourceClustersManagerActor.java | 58 +- .../TaskAssignmentManagerActor.java | 20 + .../proto/GetClusterUsageResponse.java | 11 + .../master/ExecuteStageRequestFactory.java | 31 + .../master/config/MasterConfiguration.java | 4 + .../master/scheduler/LaunchTaskRequest.java | 67 - .../master/scheduler/MantisScheduler.java | 22 + .../scheduler/MantisSchedulerFactory.java | 13 + .../scheduler/MantisSchedulerFactoryImpl.java | 67 +- ...ourceClusterReservationAwareScheduler.java | 120 ++ .../master/events/WorkerRegistryV2Test.java | 6 +- .../master/jobcluster/JobClusterAkkaTest.java | 100 +- .../job/JobActorReservationTests.java | 793 ++++++++++++ .../job/JobActorSmartRefreshTest.java | 40 +- .../job/JobClusterManagerAkkaTest.java | 2 +- .../job/JobHeartbeatTimeoutTest.java | 297 ++++- .../jobcluster/job/JobScaleUpDownTests.java | 246 +++- .../master/jobcluster/job/JobTestHelper.java | 21 + .../jobcluster/job/JobTestLifecycle.java | 625 ++++++++- .../jobcluster/job/JobTestMigrationTests.java | 14 + .../jobcluster/job/JobTestScalerRule.java | 2 +- .../AssignmentHandlerActorTest.java | 284 +++++ ...utorStateManagerReservationUsageTests.java | 509 ++++++++ .../ExecutorStateManagerTests.java | 60 +- ...servationRegistryActorIntegrationTest.java | 486 +++++++ .../ReservationRegistryActorTest.java | 1055 ++++++++++++++++ .../ReservationScalerIntegrationTests.java | 675 ++++++++++ .../resourcecluster/ReservationTest.java | 442 +++++++ ...ourceClusterActorClusterUsageAkkaTest.java | 7 +- .../ResourceClusterActorTest.java | 70 +- ...rReservationSchedulerIntegrationTests.java | 433 +++++++ .../ResourceClusterScalerActorTests.java | 367 +++++- ...skExecutorReconnectionIntegrationTest.java | 3 +- .../master/scheduler/FakeMantisScheduler.java | 16 + 58 files changed, 12844 insertions(+), 1073 deletions(-) create mode 100644 context/archive/reservation/context-scheduling-logic.md create mode 100644 context/archive/reservation/plan-reservation-registry-v3.md create mode 100644 context/archive/reservation/plan-reservation-scheduler-integration.md create mode 100644 context/archive/reservation/plan-scaler-reservation-integration.md create mode 100644 context/archive/reservation/plan-scheduling-registry.md create mode 100644 context/reservation-scheduling-context.md create mode 100644 mantis-control-plane/mantis-control-plane-core/src/main/java/io/mantisrx/server/master/resourcecluster/proto/MantisResourceClusterReservationProto.java create mode 100644 mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/AssignmentHandlerActor.java create mode 100644 mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ExecutorStateManagerActor.java create mode 100644 mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ReservationRegistryActor.java create mode 100644 mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/TaskAssignmentManagerActor.java delete mode 100644 mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/scheduler/LaunchTaskRequest.java create mode 100644 mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/scheduler/ResourceClusterReservationAwareScheduler.java create mode 100644 mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/jobcluster/job/JobActorReservationTests.java create mode 100644 mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/resourcecluster/AssignmentHandlerActorTest.java create mode 100644 mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/resourcecluster/ExecutorStateManagerReservationUsageTests.java create mode 100644 mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/resourcecluster/ReservationRegistryActorIntegrationTest.java create mode 100644 mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/resourcecluster/ReservationRegistryActorTest.java create mode 100644 mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/resourcecluster/ReservationScalerIntegrationTests.java create mode 100644 mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/resourcecluster/ReservationTest.java create mode 100644 mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/resourcecluster/ResourceClusterReservationSchedulerIntegrationTests.java diff --git a/context/archive/reservation/context-scheduling-logic.md b/context/archive/reservation/context-scheduling-logic.md new file mode 100644 index 000000000..1d262cfe9 --- /dev/null +++ b/context/archive/reservation/context-scheduling-logic.md @@ -0,0 +1,107 @@ +# Scheduling Logic and Resource Cluster Interactions + +This document explains how job creation and job scaling translate into scheduling and resource changes, and how those changes interact with the Resource Cluster and the Resource Cluster Scaler. + +## A) Job Creation → Scheduling → Task Execution + +```mermaid +sequenceDiagram + autonumber + participant Client + participant JCM as JobClustersManagerActor + participant JC as JobClusterActor + participant JA as JobActor + participant MS as MantisScheduler + participant RCSA as ResourceClusterAwareSchedulerActor + participant RC as ResourceClusterActor + participant TE as Task Executor + + Client->>JCM: SubmitJobRequest + JCM->>JC: forward(request) + JC->>JA: InitJob (initialize workers) + JA->>MS: scheduleWorkers(BatchScheduleRequest) + MS->>RCSA: BatchScheduleRequestEvent + RCSA->>RC: getTaskExecutorsFor(allocationRequests) + RC-->>RCSA: allocations (TaskExecutorID -> AllocationRequest) + RCSA->>TE: submitTask(executeStageRequestFactory.of(...)) + TE-->>RCSA: Ack / Failure + RCSA->>JA: WorkerLaunched / WorkerLaunchFailed + Note over RC: Inventory & matching by ExecutorStateManagerImpl +``` + +Key notes: +- Job submission enters at `JobClustersManagerActor.onJobSubmit`, forwarded to `JobClusterActor`, then `JobActor.initialize` constructs workers. +- `JobActor` enqueues workers via batch scheduling to the resource-cluster-aware scheduler. +- `ResourceClusterAwareSchedulerActor` queries `ResourceClusterActor` for best-fit executors and pushes tasks to TEs via `TaskExecutorGateway`. + +## B) Job Stage Scale-Up/Down and Resource Cluster Scaler Loop + +```mermaid +sequenceDiagram + autonumber + participant JA as JobActor + participant WM as WorkerManager + participant MS as MantisScheduler + participant RCSA as ResourceClusterAwareSchedulerActor + participant RC as ResourceClusterActor + participant RCS as ResourceClusterScalerActor + participant Host as ResourceClusterHostActor + participant TE as Task Executor + + JA->>JA: onScaleStage(StageScalingPolicy, ScalerRule min/max) + JA->>WM: scaleStage(newNumWorkers) + alt Scale Up + WM->>MS: scheduleWorkers(BatchScheduleRequest) + MS->>RCSA: BatchScheduleRequestEvent + RCSA->>RC: getTaskExecutorsFor(...) + RC-->>RCSA: allocations or NoResourceAvailable + RCSA->>TE: submitTask(...) + note over RC: pendingJobRequests updated when not enough TEs + else Scale Down + WM->>RC: unscheduleAndTerminateWorker(...) + end + + loop Periodic Capacity Loop (every scalerPullThreshold) + RCS->>RC: GetClusterUsageRequest + RC-->>RCS: GetClusterUsageResponse (idle/total by SKU) + RCS->>RCS: apply rules (minIdleToKeep, maxIdleToKeep, cooldown) + alt ScaleUp decision + RCS->>Host: ScaleResourceRequest(desired size increase) + Host-->>RC: New TEs provisioned + TE->>RC: TaskExecutorRegistration + Heartbeats + RC->>RC: mark available (unless disabled) + else ScaleDown decision + RCS->>RC: GetClusterIdleInstancesRequest + RC-->>RCS: idle TE list + RCS->>Host: ScaleResourceRequest(desired size decrease, idleInstances) + RCS->>RC: DisableTaskExecutorsRequest(idle targets during drain) + end + end +``` + +Key notes: +- Job scale-up adds workers; if TEs are insufficient, the scheduler records demand (pending) so the scaler sees fewer effective idle slots and scales up capacity. +- Scale-down removes workers and may free idle TEs; the scaler can shrink capacity by selecting idle instances to drain and disabling them during deprovisioning. + +## Code Reference Map + +- Submission & job initialization: + - `JobClustersManagerActor.onJobSubmit(...)` + - `JobActor.onJobInitialize(...)`, `initialize(...)`, `submitInitialWorkers(...)`, `queueTasks(...)` + +- Scheduling (push model): + - `ResourceClusterAwareSchedulerActor.onBatchScheduleRequestEvent(...)`, `onAssignedScheduleRequestEvent(...)` + - `ResourceClusterActor` + `ExecutorStateManagerImpl.findBestFit(...)`, `findBestGroupBySizeNameMatch(...)`, `findBestGroupByFitnessCalculator(...)` + +- Job scaling: + - `JobActor.onScaleStage(...)` → `WorkerManager.scaleStage(...)` + +- Resource-cluster autoscaling: + - `ResourceClusterScalerActor.onTriggerClusterUsageRequest(...)`, `onGetClusterUsageResponse(...)`, `onGetClusterIdleInstancesResponse(...)` + - `ExecutorStateManagerImpl.getClusterUsage(...)` (idle = available − pending), `pendingJobRequests` + - `ResourceClusterActor.onTaskExecutorRegistration(...)` (TE joins, becomes available) + +## Coupling Between Scheduling and Scaling + +- When allocations fail due to lack of capacity, the resource cluster records pending demand per job/group; the scaler subtracts this pending from available to compute effective idle, triggering scale-up. +- New TEs register with the resource cluster and are immediately considered for subsequent scheduling waves. diff --git a/context/archive/reservation/plan-reservation-registry-v3.md b/context/archive/reservation/plan-reservation-registry-v3.md new file mode 100644 index 000000000..4e11a2751 --- /dev/null +++ b/context/archive/reservation/plan-reservation-registry-v3.md @@ -0,0 +1,164 @@ +# Plan: Reservation Registry v3 + +## Goal +- Deliver reliable, strict-batch worker scheduling by moving reservation processing and task assignment into dedicated child actors under `ResourceClusterActor`. +- Preserve legacy scheduling path while enabling a pass-through mode where the resource cluster drives assignment, using the Excalidraw concept diagram as the execution blueprint. + +## References +- `context/context-scheduling-logic.md` +- `context/Mantis Resource Registry.excalidraw` + +## Summary of Gaps in Current Scheduling Flow +- Batch allocation from the scheduler is best-effort; no single actor owns the full reservation lifecycle, so retries become per-worker and inconsistent. +- `ExecutorStateManager` (ESM) pending state is not synchronized with the scaler, leading to inaccurate idle capacity reporting. +- Job creation honors strict batching while stage scaling mixes batch and non-batch behaviors, causing fairness and starvation issues. +- Scheduler retrying each worker independently creates no shared state for partial failures; duplicates and missing workers are possible. +- `ResourceClusterActor` already handles TE registration and heartbeats; adding reservation retry loops there would overload its dispatcher, so the new design must split responsibilities. + +## Terminology +- Task Executor (TE): agent host registered to control plane from one of the resource clusters with a pre-defined container spec. Can be assigned job worker. +- Worker: a worker in one of a job's stage. A job can have multiple stages and each stage will have a list of workers with worker index and worker number to id each one. JobActor is responsible to keep track of the workers and inform scheduler to add/remove workers from scaling/creation/deletion operations. +- ESM (executor state manager): class in ResourceClusterActor tracking TE status (registration and heartbeats). + +## Architectural Changes +- Introduce a child-actor hierarchy under `ResourceClusterActor`: `ReservationRegistryActor`, `AssignmentTrackerActor`, and `ExecutorStateManagerActor` (wrapping the existing ESM logic). The child actors should talk to each other via parent actor instead of using direct actor ref. +- Add a scheduler factory toggle that selects either the legacy `ResourceClusterAwareSchedulerActor` or a new pass-through scheduler that forwards batch requests directly to the resource cluster. +- Shift the batch scheduling loop out of the scheduler: the scheduler enqueues a reservation, the resource cluster allocates TEs, and a dedicated tracker assigns workers and publishes results back to the job. +- Provide a feedback channel so successful assignments notify `JobActor`, allowing it to transition workers to `STARTING` (re-purposing the status to mean "assigned and waiting for heartbeat"). +- Strict batching (immutable reservation). Use detailed metrics to see the pending status. Support cancellation on reservations fully wired into job action (kill, scale down). Simpler mechanism, less error prone but won't handle extreme case where large batch starving everyone else. + +## Actor Catalog + +### ResourceClusterActor (parent) +- **State** + - References to child actors (`ReservationRegistryActor`, `AssignmentTrackerActor`, `ExecutorStateManagerActor`). + - Existing TE registration tables (disabled/available/leasing), mapped by cluster SKU. + - Pending command queue for escalations (scale requests, disablement). + - Configuration for dispatcher choices and polling intervals. +- **Behavior** + - Routes incoming `BatchScheduleRequest` (from scheduler or pass-through factory) to the reservation registry. + - Continues to process TE lifecycle events (registration, heartbeats, disable, drain) and forwards inventory changes to the registry/assignment tracker when relevant. + - Bridges scaler requests by composing data from ESM and reservations (pending demand) before replying. + - Supervises child actors, restarting them and rebuilding in-memory state on failure. + +### ReservationRegistryActor +- **State** + - Strict priority queue of reservations keyed by stage/job priority; each entry includes canonicalized scheduling constraints and desired worker count. + - Reservation index by `JobId` and stage for dedupe/cancellation. + - Rate limiter / minimum wait timestamp to avoid thrashing the dispatcher. + - Metrics counters (queue length, successful allocations, blocked durations). +- **Behavior** + - Enqueue reservations when jobs submit new workers or scale up; merge requests when total target size matches existing reservation. + - Process loop (timer-driven and on-demand): peek top reservation, invoke `tryAllocateAndLease` on ESM, and react to the result. + - On successful lease of the full batch, emit an `AssignmentTaskCreated` message to the assignment tracker and remove the reservation from the queue. + - On failed allocation, leave the reservation in place, optionally downgrade priority based on policy. + - Handle cancellations triggered by job scale-down, worker kill, or reservation expiry. + - Publish queue metrics to the scaler so pending demand is visible immediately. + - on startup this registry actor should wait for all pending reservations to be resent by the jobactor init process. Do not return partial results to scaler until this init is marked as finished via the signal from jobClustersManagerActor -> schedulerFactory + + +### ExecutorStateManagerActor +- **State** + - Wraps the existing ESM tables for TE availability, leasing contracts, disabled/draining sets, and pending reservations. + - Data snapshots used for scaler responses (ready, idle, pending by SKU). +- **Behavior** + - Responds to `tryAllocateAndLease(reservation)` messages from the registry, performing fitness calculations and returning `LeaseAccepted` or reasons for failure. + - Updates pending counts when reservations are leasing, ensuring the scaler subtracts pending from available. + - Releases leases on cancellation or on assignment tracker completion/failure. + - Emits notifications to the registry whenever inventory becomes available (TE registered, released, or re-enabled). + +### AssignmentTrackerActor +- **State** + - Map of `AssignmentTaskId -> AssignmentTask` (see below), keyed by job/stage and reservation epoch. + - Retry budgets (configurable) and exponential backoff timers per worker. + - Channels to route IO-intensive work (TaskExecutor RPCs) onto a dedicated IO dispatcher while control logic stays on the actor dispatcher. + - Metrics for in-flight assignments, retry counts, and time-to-assign. +- **Behavior** + - On `AssignmentTaskCreated`, spawn tracking for the task, moving each worker to the IO dispatcher to call `TaskExecutorGateway.submitTask`. + - Transition assigned workers into the success set; notify `JobActor` via existing message router so it can monitor heartbeats. + - For RPC failures or timeouts, increment retry count; if the limit is reached, emit a `ReservationRequeueRequested` event (high priority) instead of silently replacing the worker. + - Cancel tasks when job termination or worker kill/replace commands arrive. + - Close tasks when all workers acknowledge, releasing leases back to ESM. + +### AssignmentTask (entity, tracked by AssignmentTrackerActor) +- **State Fields** + - Original reservation payload: job/stage, list of worker descriptors, scheduling constraints. + - `assignedWorkers`: worker ids already acknowledged by TEs. + - `pendingWorkers`: worker ids pending assignment with per-worker retry counters and last attempt timestamps. + - Associated leases (TE ids) from ESM to ensure release on completion/failure. +- **Behavior** + - Moves workers from pending to assigned on success. + - Exposes next retry timestamps to the tracker so the dispatcher timer can wake the task at the right time. + +### ResourceClusterAwarePassThroughSchedulerActor (new optional scheduler) +- **State** + - Reference to resource cluster routing endpoint. + - Mode flag (legacy vs pass-through) configured via scheduler factory. +- **Behavior** + - In pass-through mode, accepts `BatchScheduleRequest` and forwards a lightweight reservation envelope to `ResourceClusterActor`, skipping TE selection. + - Handles legacy compatibility by falling back to existing behavior when feature flag is disabled. + - Reports reservation enqueue success/failure back to the caller (`JobActor`). + +## Message Flow (Strict Batch Scenario) +1. `(0)` Scheduler (legacy or pass-through) receives `BatchScheduleRequest` from `JobActor` and forwards `EnqueueReservation(job, stage, workers)` to `ResourceClusterActor`. +2. `ResourceClusterActor` pushes the reservation into `ReservationRegistryActor`, which records metadata and schedules processing (timer or immediate wake). +3. `(1)` Registry peeks the highest priority reservation and calls `ExecutorStateManagerActor.tryAllocateAndLease`. If successful for the entire batch, it proceeds; otherwise the reservation stays queued. +4. `(2)` Registry emits `AssignmentTaskCreated(reservation, leases)` to `AssignmentTrackerActor`, breaking the operation into a separate actor message. +5. `AssignmentTrackerActor` fans out per-worker assignment on the IO dispatcher, calling into `TaskExecutorGateway`. +6. `(3)` On success, the tracker sends `WorkerAssigned(job, stage, worker)` to `JobActor`, which updates worker state and monitors heartbeats. +7. `(3')` If a worker exceeds retry budget, the tracker raises `ReservationRequeueRequested` (high priority) to the registry, and the job actor is asked to orchestrate worker replacement rather than blindly reassigning the same TE. +8. On completion (all workers assigned), the tracker releases leases through ESM and marks the task finished. + +## Failure and Retry Handling +- TE assignment retries are capped; once exceeded, the system re-queues a fresh reservation and lets `JobActor` manage replacement to avoid duplicate workers on partially initialized executors. +- Reservation cancellations propagate to both registry and tracker, ensuring leased TEs are released promptly. +- Registry processing has a minimum delay between attempts to prevent tight loops if inventory is unavailable. +- On actor restart, state recovery sequences reload reservations and assignments from persisted snapshots (implementation detail TBD) before resuming processing. + +## Concurrency and Dispatchers +- Dispatcher threads handle registry logic and timers; IO threads handle TE RPCs (`TaskExecutorGateway.submitTask`). +- `AssignmentTrackerActor` bridges both via message passing, keeping actor state mutations single-threaded. +- Registry to ESM calls remain asynchronous (`ask`/`pipeTo`) to avoid blocking the dispatcher. + +## Scheduler and API Updates +- Introduce a scheduler factory flag (`passThroughReservationScheduling`) to select between legacy scheduling and the new registry-driven flow. +- Maintain compatibility with existing APIs for job submission and scaling; the difference is purely in how the scheduler fulfills the batch request. +- Update job lifecycle so that success notifications from the tracker move workers into the `STARTING` state, aligning with the TODOs called out in the diagram. + +## Observability +- New metrics: reservation queue depth, time-in-queue, assignment retry counts, assignment latency per job, registry wake-ups, and ESM lease success rate. +- Add debug tracing for `(0)` through `(3')` steps to correlate reservations, assignments, and job notifications. + +## ResourceClusterActor.java Changes +- In `mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClusterActor.java`, declare child `ActorRef`s for the reservation registry, assignment tracker, and executor state manager actors. Construct them during `preStart` (or constructor) using dedicated dispatchers and register them under predictable names for restart. +- Replace direct `ExecutorStateManager` invocations with asynchronous `tell/ask` calls to the new ESM actor. Update all scheduling handlers to work with future responses, releasing leases through the actor instead of mutating local state. +- Extend the receive pipeline to handle reservation lifecycle messages: enqueue, cancel, flush, and assignment completion. Legacy inline scheduling remains behind a feature flag; when pass-through mode is enabled, immediately forward requests to the registry actor and return an enqueue acknowledgement. +- When TE lifecycle events arrive (registration, heartbeats, disable/enable, disconnection), continue updating existing tables but also notify both ESM and registry actors so queued reservations can retry promptly. +- Update scaler request handlers (`GetClusterUsage*`, `GetClusterIdleInstances*`) to compose responses from ESM inventory plus registry pending demand. Ensure responses include pending batch sizes so autoscaler decisions remain accurate. +- Given the state restoration mode, ensure these actors do not terminate on error and should try to resume to avoid losing state. +- Emit new metrics from `ResourceClusterActorMetrics` for registry queue depth, assignment latency, and lease durations by subscribing to child actor metric events or probing their state via ask-pattern calls. + +- ReservationRegistryActor: add `ReservationRegistry` owned by `ResourceClusterActor` + - Types: `ReservationKey(jobId, stageNum)`, `Reservation{ReservationKey, canonicalSchedulingConstraint, workerRequests, stageTargetSize, lastUpdatedAt, priorityEpoch}` + - Indexes: `reservationsByKey`, `perSchedulingConstraintPriorityQueue` ordered by `(priorityEpoch asc)`. (make sure there is proper GC on these index) + - API (actor message behavior): + -- `upsertReservations`: recevied from JobActor/Scheduler. insert if new. If there is existing key, replace the old reservation. + -- `cancelReservation`: cancel an existing reservation. Remove from index tables. + -- `getPendingReservationsView`: return a map of summary of pending reservations grouped by reservation's schedulingConstraint. + -- `markReady`: signal from external actors to indicate the registry has received all the state (init messages) from job actors. This is important since a partial pending result to resource cluster actor and scaler can interrupt ongoing provisoin tasks during a leader switch. + - Behavior: + - the general reservations from job actors should be processed FIFO + - same reservation request doesn't change existing reservation priority. + - init: return not ready from getPendingReservationsBySchedulingConstraints until ready signal from schduler factory. + - reservation priority: default on request epoch; override to (0/1/2 etc. on manual overide e.g. replace worker request). Leave interface to add custom logic in future too. + - same key (jobId + stageNum) with different stage target size: dedupe logic (ignore same target size && same worker request size. Replace old request with new one if target size changed). + - reservation class should be immutable. Replace exisitng if needed. + - internal loop to process reservation queue: timer + ESM signal (need to be mini-batch style to avoid too many trigger from ESM). + - abstract and config to support different priority behavior. Default strategy setup to use epoch time on creation + override signal (e.g. priorityEpoch as 0) + - main processing logic: when reservation registry try to process the active reservations, it should go through the priority (so it honors the priority). Start with top of the queue of each "size" or "sku", check if the reservation can be fulfilled (matched) from executor state manager. Once matched, invoke resource cluster actor to do the actual assignement to TEs and finish (remove) the reservation in registry. If the target reservation cannot be matched, stop the process on this queue. Proceed to next reservation if the top one is successfully filled, otherwise stop (only top of the queue shall be processed). Error in assigning TEs tasks should be properly handled such that it moves the reservation to the tail of the queue (re-assigned priority). + - (optional) new reservation trigger resource cluster scaler process. + +- AssignmentTrackerActor: tracks and manages actual worker assignments + - Host similar funcationality as onAssignedScheduleRequestEvent(AssignedScheduleRequestEvent event) in ResourceClusterAwareSchedulerActor; Since this is now a child actor in resourceClusterActor it doesn't need to go throgh resource cluster interface to get the connections etc. but can directly ask the parent actor for the needed objects. + - track ongoing IO tasks to each TE and handle retry similar to logic in scheduler actor. + - assignment tasks can be cancelled due to job kill and worker replacement. diff --git a/context/archive/reservation/plan-reservation-scheduler-integration.md b/context/archive/reservation/plan-reservation-scheduler-integration.md new file mode 100644 index 000000000..cb25e3cb7 --- /dev/null +++ b/context/archive/reservation/plan-reservation-scheduler-integration.md @@ -0,0 +1,983 @@ +# Reservation-Based Scheduler Integration Plan + +## Overview + +This document outlines the integration plan for the new reservation-based scheduling system with the existing `JobActor` worker scheduling logic. The reservation system provides prioritized, batched allocation of Task Executors through the `ReservationRegistryActor`. + +## Architecture + +``` +┌─────────────────────────────────────────────────────────────────────────┐ +│ JobClustersManagerActor │ +│ initialize() ──────> mantisSchedulerFactory.markAllRegistriesReady() │ +└─────────────────────────────────────────────────────────────────────────┘ + │ + ▼ +┌─────────────────────────────────────────────────────────────────────────┐ +│ JobActor │ +│ ┌─────────────────────────────────────────────────────────────────┐ │ +│ │ WorkerManager │ │ +│ │ submitInitialWorkers() ──> scheduler.upsertReservation(NEW_JOB)│ │ +│ │ resubmitWorker() ──> scheduler.upsertReservation(REPLACE)│ │ +│ │ scaleStage() ──> scheduler.upsertReservation(SCALE) │ │ +│ │ shutdown() ──> scheduler.cancelReservation() │ │ +│ └─────────────────────────────────────────────────────────────────┘ │ +└─────────────────────────────────────────────────────────────────────────┘ + │ + ▼ +┌─────────────────────────────────────────────────────────────────────────┐ +│ MantisScheduler (interface) │ +│ ┌──────────────────────────┐ ┌────────────────────────────────┐ │ +│ │ResourceClusterAware │ │ResourceClusterReservationAware │ │ +│ │Scheduler (existing) │ │Scheduler (NEW) │ │ +│ │- scheduleWorkers() │ │- upsertReservation() │ │ +│ │- unscheduleJob() │ │- cancelReservation() │ │ +│ └────────────┬─────────────┘ └───────────────┬────────────────┘ │ +└─────────────────┼──────────────────────────────────┼────────────────────┘ + │ │ + ▼ ▼ +┌─────────────────────────────────────────────────────────────────────────┐ +│ ResourceCluster (interface) │ +│ ┌───────────────────────────────────────────────────────────┐ │ +│ │ ResourceClusterAkkaImpl │ │ +│ │ - getTaskExecutorsFor() (existing) │ │ +│ │ - upsertReservation() (NEW) │ │ +│ │ - cancelReservation() (NEW) │ │ +│ └───────────────────────────────────────────────────────────┘ │ +└─────────────────────────────────────────────────────────────────────────┘ + │ + ▼ +┌─────────────────────────────────────────────────────────────────────────┐ +│ ResourceClusters (interface) │ +│ - getClusterFor(ClusterID) │ +│ - markAllRegistriesReady() (NEW) ─────────────────────────────────┐ │ +└──────────────────────────────────────────────────────────────────────│──┘ + │ │ + ▼ │ +┌─────────────────────────────────────────────────────────────────────────┐ +│ ResourceClusterActor │ +│ ┌──────────────────────────────────────────────────────────────┐ │ +│ │ ReservationRegistryActor ExecutorStateManagerActor │ │ +│ │ - UpsertReservation - TaskExecutorBatchAssignment │◄──┘ +│ │ - CancelReservation - WorkerLaunched events │ +│ │ - MarkReady │ +│ └──────────────────────────────────────────────────────────────┘ │ +└─────────────────────────────────────────────────────────────────────────┘ +``` + +## Priority Types + +The reservation system supports three priority types for different scheduling scenarios: + +| Priority Type | Use Case | Priority Order | +|--------------|----------|----------------| +| `REPLACE` | Worker resubmission due to failure/heartbeat timeout | Highest (processed first) | +| `SCALE` | Scale-up requests for existing jobs | Medium | +| `NEW_JOB` | Initial workers for new job submissions | Lowest | + +Within the same priority type, jobs are ordered by: +1. **Tier** (ascending) - Lower tier = higher priority +2. **Timestamp** (ascending) - Earlier requests processed first (FIFO) + +--- + +## Implementation Plan + +### Phase 1: Refactor Existing Classes for Reuse + +#### 1.1 Move Reservation Classes to Shared Location (No Duplication) + +The existing reservation-related classes in `ResourceClusterActor.java` should be **moved** to a new shared file rather than duplicating them. This avoids code duplication and ensures consistency. + +**Move FROM:** `ResourceClusterActor.java` (static inner classes) +**Move TO:** New file `mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/proto/MantisResourceClusterReservationProto.java` + +```java +package io.mantisrx.master.resourcecluster.proto; + +import io.mantisrx.server.core.domain.WorkerId; +import io.mantisrx.server.core.scheduler.SchedulingConstraints; +import io.mantisrx.server.master.resourcecluster.TaskExecutorAllocationRequest; +import lombok.Builder; +import lombok.Value; +import java.util.Collections; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Protocol classes for the reservation-based scheduling system. + * These classes are shared between ResourceClusterActor, ReservationRegistryActor, + * MantisScheduler, and JobActor. + */ +public final class MantisResourceClusterReservationProto { + + private MantisResourceClusterReservationProto() {} + + /** + * Unique identifier for a reservation (job + stage). + */ + @Value + @Builder + public static class ReservationKey { + String jobId; + int stageNumber; + } + + /** + * Priority for ordering reservations within constraint groups. + * Ordering: REPLACE < SCALE < NEW_JOB (REPLACE processed first). + */ + @Value + @Builder + public static class ReservationPriority implements Comparable { + public enum PriorityType { + REPLACE, // Worker replacement due to failure (highest priority) + SCALE, // Scale-up request for existing job + NEW_JOB // New job submission (lowest priority) + } + + PriorityType type; + int tier; // Job tier (lower = higher priority within same type) + long timestamp; // FIFO ordering within same priority and tier + + @Override + public int compareTo(ReservationPriority other) { + int typeComparison = this.type.compareTo(other.type); + if (typeComparison != 0) return typeComparison; + int tierComparison = Integer.compare(this.tier, other.tier); + if (tierComparison != 0) return tierComparison; + return Long.compare(this.timestamp, other.timestamp); + } + } + + /** + * Full reservation data including scheduling constraints and allocation requests. + */ + @Value + @Builder(toBuilder = true) + public static class Reservation { + ReservationKey key; + SchedulingConstraints schedulingConstraints; + String canonicalConstraintKey; + Set requestedWorkers; + Set allocationRequests; + int stageTargetSize; + ReservationPriority priority; + + public boolean hasSameShape(Reservation other) { + return other != null + && Objects.equals(key, other.key) + && Objects.equals(canonicalConstraintKey, other.canonicalConstraintKey) + && Objects.equals(requestedWorkers, other.requestedWorkers) + && stageTargetSize == other.stageTargetSize; + } + + public int getRequestedWorkersCount() { + return requestedWorkers != null ? requestedWorkers.size() : 0; + } + + public static Reservation fromUpsertReservation(UpsertReservation upsert, String canonicalConstraintKey) { + return Reservation.builder() + .key(upsert.getReservationKey()) + .schedulingConstraints(upsert.getSchedulingConstraints()) + .canonicalConstraintKey(canonicalConstraintKey) + .requestedWorkers(upsert.getAllocationRequests() != null ? + upsert.getAllocationRequests().stream() + .map(TaskExecutorAllocationRequest::getWorkerId) + .collect(Collectors.toSet()) + : Collections.emptySet()) + .allocationRequests(upsert.getAllocationRequests() != null ? + upsert.getAllocationRequests() : Collections.emptySet()) + .stageTargetSize(upsert.getStageTargetSize()) + .priority(upsert.getPriority()) + .build(); + } + } + + /** + * Message to insert/update a reservation. + */ + @Value + @Builder + public static class UpsertReservation { + ReservationKey reservationKey; + SchedulingConstraints schedulingConstraints; + Set allocationRequests; + int stageTargetSize; + ReservationPriority priority; + } + + /** + * Message to cancel a reservation. + */ + @Value + @Builder + public static class CancelReservation { + ReservationKey reservationKey; + } + + /** + * Response to reservation cancellation. + */ + @Value + public static class CancelReservationAck { + ReservationKey reservationKey; + boolean cancelled; + } + + /** + * Marker message to indicate registry is ready to process reservations. + */ + public enum MarkReady { + INSTANCE + } +} +``` + +#### 1.2 Update `ResourceClusterActor.java` and `ReservationRegistryActor.java` + +Replace the static inner classes with imports from the new shared location: + +**In `ResourceClusterActor.java`:** +```java +// Add imports +import io.mantisrx.master.resourcecluster.proto.MantisResourceClusterReservationProto.*; +// Or use static imports for cleaner code: +import static io.mantisrx.master.resourcecluster.proto.MantisResourceClusterReservationProto.*; + +// Remove the following static inner classes (moved to proto file): +// - ReservationKey +// - ReservationPriority +// - Reservation +// - UpsertReservation +// - CancelReservation +// - CancelReservationAck +// - MarkReady + +// Keep these as they are specific to the actor's internal operation: +// - ProcessReservationsTick +// - ForceProcessReservationsTick +// - GetPendingReservationsView +``` + +**In `ReservationRegistryActor.java`:** +```java +// Replace references like ResourceClusterActor.ReservationKey with direct import: +import io.mantisrx.master.resourcecluster.proto.MantisResourceClusterReservationProto.*; +// Or static import for cleaner code +``` + +#### 1.3 Extend `MantisScheduler` Interface (Using Shared Classes) + +**File:** `mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/scheduler/MantisScheduler.java` + +```java +import io.mantisrx.master.resourcecluster.proto.MantisResourceClusterReservationProto.UpsertReservation; +import io.mantisrx.master.resourcecluster.proto.MantisResourceClusterReservationProto.CancelReservation; +import io.mantisrx.master.resourcecluster.proto.MantisResourceClusterReservationProto.CancelReservationAck; + +// Add new methods: + +/** + * Insert or update a reservation for workers. + * @param request The reservation request (reuses existing UpsertReservation class) + * @return Future that completes when the reservation is accepted + */ +default CompletableFuture upsertReservation(UpsertReservation request) { + throw new UnsupportedOperationException("Reservation-based scheduling not supported"); +} + +/** + * Cancel all pending reservations for a given job stage. + * @param request The cancel request (reuses existing CancelReservation class) + * @return Future containing cancellation acknowledgement + */ +default CompletableFuture cancelReservation(CancelReservation request) { + throw new UnsupportedOperationException("Reservation-based scheduling not supported"); +} + +/** + * Check if this scheduler supports reservation-based scheduling. + */ +default boolean supportsReservationScheduling() { + return false; +} +``` + +**Note:** No new DTO classes needed! We reuse `UpsertReservation`, `CancelReservation`, and `CancelReservationAck` directly. + +--- + +### Phase 2: ResourceCluster Interface Extension + +#### 2.1 Extend `ResourceCluster` Interface (Using Shared Classes) + +**File:** `mantis-control-plane/mantis-control-plane-core/src/main/java/io/mantisrx/server/master/resourcecluster/ResourceCluster.java` + +Add imports and new methods: + +```java +import io.mantisrx.master.resourcecluster.proto.MantisResourceClusterReservationProto.UpsertReservation; +import io.mantisrx.master.resourcecluster.proto.MantisResourceClusterReservationProto.CancelReservation; +import io.mantisrx.master.resourcecluster.proto.MantisResourceClusterReservationProto.CancelReservationAck; + +/** + * Insert or update a reservation for workers in this cluster. + * Reuses the shared UpsertReservation class from MantisResourceClusterReservationProto. + * + * @param request The reservation request + * @return Future that completes when reservation is accepted + */ +default CompletableFuture upsertReservation(UpsertReservation request) { + throw new UnsupportedOperationException("Reservation not supported"); +} + +/** + * Cancel reservations for a job stage. + * Reuses the shared CancelReservation class from MantisResourceClusterReservationProto. + * + * @param request The cancellation request + * @return Future containing cancellation response + */ +default CompletableFuture cancelReservation(CancelReservation request) { + throw new UnsupportedOperationException("Reservation not supported"); +} +``` + +#### 2.2 Extend `ResourceClusters` Interface + +**File:** `mantis-control-plane/mantis-control-plane-core/src/main/java/io/mantisrx/server/master/resourcecluster/ResourceClusters.java` + +Add new method: + +```java +/** + * Mark all reservation registries as ready to process reservations. + * This should be called after master initialization is complete and all + * existing jobs have been recovered. + * + * @return Future that completes when all clusters are marked ready + */ +CompletableFuture markAllRegistriesReady(); +``` + +#### 2.3 Implement in `ResourceClusterAkkaImpl` (Simplified - No Conversion Needed) + +**File:** `mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClusterAkkaImpl.java` + +Since we're using the shared classes directly, the implementation is much simpler - **no type conversion needed**: + +```java +import io.mantisrx.master.resourcecluster.proto.MantisResourceClusterReservationProto.UpsertReservation; +import io.mantisrx.master.resourcecluster.proto.MantisResourceClusterReservationProto.CancelReservation; +import io.mantisrx.master.resourcecluster.proto.MantisResourceClusterReservationProto.CancelReservationAck; + +@Override +public CompletableFuture upsertReservation(UpsertReservation request) { + // Direct passthrough - no conversion needed since we use shared classes! + return Patterns.ask(resourceClusterManagerActor, request, askTimeout) + .thenApply(Ack.class::cast) + .toCompletableFuture(); +} + +@Override +public CompletableFuture cancelReservation(CancelReservation request) { + // Direct passthrough - no conversion needed! + return Patterns.ask(resourceClusterManagerActor, request, askTimeout) + .thenApply(CancelReservationAck.class::cast) + .toCompletableFuture(); +} +``` + +**Note:** By reusing the same classes across all layers, we eliminate all type conversion code! + +#### 2.4 Implement in `ResourceClustersAkkaImpl` (or equivalent) + +Implement `markAllRegistriesReady()` using the shared `MarkReady` enum: + +```java +import io.mantisrx.master.resourcecluster.proto.MantisResourceClusterReservationProto.MarkReady; + +@Override +public CompletableFuture markAllRegistriesReady() { + return listActiveClusters() + .thenCompose(clusterIds -> { + List> futures = clusterIds.stream() + .map(this::getClusterFor) + .map(cluster -> { + // Send MarkReady to each cluster's reservation registry + // Uses shared MarkReady enum from proto file + return Patterns.ask( + resourceClusterManagerActor, + MarkReady.INSTANCE, // From shared proto + askTimeout) + .thenApply(Ack.class::cast) + .toCompletableFuture(); + }) + .collect(Collectors.toList()); + + return CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])) + .thenApply(v -> Ack.getInstance()); + }); +} +``` + +--- + +### Phase 3: New Scheduler Implementation (Using Shared Classes) + +#### 3.1 Create `ResourceClusterReservationAwareScheduler` + +**File:** `mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/scheduler/ResourceClusterReservationAwareScheduler.java` + +```java +package io.mantisrx.server.master.scheduler; + +import io.mantisrx.common.Ack; +import io.mantisrx.master.resourcecluster.proto.MantisResourceClusterReservationProto.UpsertReservation; +import io.mantisrx.master.resourcecluster.proto.MantisResourceClusterReservationProto.CancelReservation; +import io.mantisrx.master.resourcecluster.proto.MantisResourceClusterReservationProto.CancelReservationAck; +import io.mantisrx.master.resourcecluster.proto.MantisResourceClusterReservationProto.ReservationKey; +import io.mantisrx.server.core.domain.WorkerId; +import io.mantisrx.server.master.resourcecluster.ResourceCluster; +import io.mantisrx.server.master.resourcecluster.TaskExecutorID; +import lombok.RequiredArgsConstructor; +import lombok.extern.slf4j.Slf4j; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; + +/** + * Scheduler implementation that uses the reservation-based scheduling system. + * + * This scheduler delegates scheduling decisions to the ReservationRegistryActor + * which processes reservations in priority order per constraint group: + * - REPLACE (worker failures) - highest priority + * - SCALE (scale-up requests) - medium priority + * - NEW_JOB (new submissions) - lowest priority + * + * Uses shared classes from MantisResourceClusterReservationProto - no type conversion needed. + * WorkerLaunched events are still routed via jobMessageRouter from ExecutorStateManagerActor. + */ +@Slf4j +@RequiredArgsConstructor +public class ResourceClusterReservationAwareScheduler implements MantisScheduler { + + private final ResourceCluster resourceCluster; + + // ==================== Reservation APIs (Using Shared Classes Directly) ==================== + + @Override + public CompletableFuture upsertReservation(UpsertReservation request) { + ReservationKey key = request.getReservationKey(); + log.info("Upserting reservation for job {} stage {} with {} workers (priority={})", + key.getJobId(), + key.getStageNumber(), + request.getAllocationRequests().size(), + request.getPriority().getType()); + // Direct passthrough - same type used everywhere! + return resourceCluster.upsertReservation(request); + } + + @Override + public CompletableFuture cancelReservation(CancelReservation request) { + ReservationKey key = request.getReservationKey(); + log.info("Cancelling reservation for job {} stage {}", key.getJobId(), key.getStageNumber()); + // Direct passthrough - same type used everywhere! + return resourceCluster.cancelReservation(request); + } + + @Override + public boolean supportsReservationScheduling() { + return true; + } + + @Override + public boolean schedulerHandlesAllocationRetries() { + // Reservation registry handles retries via timeout mechanism + return true; + } + + // ==================== Legacy APIs ==================== + + @Override + public void scheduleWorkers(BatchScheduleRequest scheduleRequest) { + throw new UnsupportedOperationException( + "Use upsertReservation() instead of scheduleWorkers() with reservation-based scheduling"); + } + + @Override + public void unscheduleJob(String jobId) { + // For reservation-based scheduling, caller should use cancelReservation per stage + log.warn("unscheduleJob({}) called - use cancelReservation() per stage instead", jobId); + } + + @Override + public void unscheduleWorker(WorkerId workerId, Optional hostname) { + throw new UnsupportedOperationException( + "unscheduleWorker not supported - use cancelReservation() for pending workers"); + } + + @Override + public void unscheduleAndTerminateWorker(WorkerId workerId, Optional hostname) { + // Still needed for terminating running workers + log.info("Terminating worker {}", workerId); + resourceCluster.markTaskExecutorWorkerCancelled(workerId) + .whenComplete((ack, ex) -> { + if (ex != null) { + log.warn("Failed to mark worker {} as cancelled", workerId, ex); + } + }); + } + + @Override + public void updateWorkerSchedulingReadyTime(WorkerId workerId, long when) { + throw new UnsupportedOperationException( + "Reservation registry handles retry timing internally"); + } + + @Override + public void initializeRunningWorker(ScheduleRequest scheduleRequest, String hostname, String hostID) { + // Still needed for master failover recovery + log.info("Initializing running worker {} on host {}", scheduleRequest.getWorkerId(), hostname); + resourceCluster.initializeTaskExecutor(TaskExecutorID.of(hostID), scheduleRequest.getWorkerId()) + .whenComplete((ack, ex) -> { + if (ex != null) { + log.warn("Failed to initialize running worker {}", scheduleRequest.getWorkerId(), ex); + } + }); + } +} +``` + +--- + +### Phase 4: Factory Updates + +#### 4.1 Add Configuration Flag + +**File:** `mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/config/MasterConfiguration.java` + +```java +@Config("mantis.scheduling.reservation.enabled") +@Default("false") +boolean isReservationSchedulingEnabled(); +``` + +#### 4.2 Update `MantisSchedulerFactoryImpl` + +**File:** `mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/scheduler/MantisSchedulerFactoryImpl.java` + +```java +@RequiredArgsConstructor +@Slf4j +public class MantisSchedulerFactoryImpl implements MantisSchedulerFactory { + + private final ActorSystem actorSystem; + private final ResourceClusters resourceClusters; + private final ExecuteStageRequestFactory executeStageRequestFactory; + private final JobMessageRouter jobMessageRouter; + private final MasterConfiguration masterConfiguration; + private final MetricsRegistry metricsRegistry; + private final Map actorRefMap = new ConcurrentHashMap<>(); + + @Override + public MantisScheduler forClusterID(ClusterID clusterID) { + if (clusterID == null || Strings.isNullOrEmpty(clusterID.getResourceID())) { + throw new RuntimeException("Invalid clusterID for MantisScheduler"); + } + return actorRefMap.computeIfAbsent(clusterID, this::createScheduler); + } + + private MantisScheduler createScheduler(ClusterID clusterID) { + log.info("Creating scheduler for cluster: {} (reservationEnabled={})", + clusterID.getResourceID(), + masterConfiguration.isReservationSchedulingEnabled()); + + if (masterConfiguration.isReservationSchedulingEnabled()) { + return new ResourceClusterReservationAwareScheduler( + resourceClusters.getClusterFor(clusterID)); + } else { + return new ResourceClusterAwareScheduler( + actorSystem.actorOf( + ResourceClusterAwareSchedulerActor.props( + masterConfiguration.getSchedulerMaxRetries(), + masterConfiguration.getSchedulerMaxRetries(), + masterConfiguration.getSchedulerIntervalBetweenRetries(), + resourceClusters.getClusterFor(clusterID), + executeStageRequestFactory, + jobMessageRouter, + metricsRegistry), + "scheduler-for-" + clusterID.getResourceID()), + masterConfiguration.getSchedulerHandlesAllocationRetries()); + } + } + + /** + * Mark all reservation registries as ready after master initialization. + * Should be called after all jobs have been recovered. + * + * @return Future that completes when all registries are ready + */ + public CompletableFuture markAllRegistriesReady() { + if (masterConfiguration.isReservationSchedulingEnabled()) { + log.info("Marking all reservation registries as ready"); + return resourceClusters.markAllRegistriesReady(); + } else { + log.debug("Reservation scheduling disabled, skipping markAllRegistriesReady"); + return CompletableFuture.completedFuture(Ack.getInstance()); + } + } +} +``` + +--- + +### Phase 5: JobClustersManagerActor Integration + +#### 5.1 Call `markAllRegistriesReady` After Initialization + +**File:** `mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/JobClustersManagerActor.java` + +In the `initialize()` method, after all job clusters are initialized: + +```java +// In initialize() method, after Observable.from(jobClusterMap.values())...toBlocking().subscribe() +// Around line 405-411 + +.toBlocking() +.subscribe((clusterInit) -> { + logger.info("JobCluster {} inited with code {}", clusterInit.jobClusterName, clusterInit.responseCode); + numJobClusterInitSuccesses.increment(); +}, (error) -> { + logger.warn("Exception initializing clusters {}", error.getMessage(), error); + logger.error("JobClusterManagerActor had errors during initialization"); + sender.tell(new JobClustersManagerInitializeResponse(initMsg.requestId, SERVER_ERROR, + "JobClustersManager inited with errors"), getSelf()); +}, () -> { + logger.info("JobClusterManagerActor transitioning to initialized behavior"); + getContext().become(initializedBehavior); + + // NEW: Mark all reservation registries as ready + if (mantisSchedulerFactory != null) { + mantisSchedulerFactory.markAllRegistriesReady() + .whenComplete((ack, ex) -> { + if (ex != null) { + logger.error("Failed to mark reservation registries as ready", ex); + } else { + logger.info("All reservation registries marked ready"); + } + }); + } + + sender.tell(new JobClustersManagerInitializeResponse(initMsg.requestId, SUCCESS, + "JobClustersManager successfully inited"), getSelf()); +}); +``` + +--- + +### Phase 6: JobActor Integration (Using Shared Classes Directly) + +#### 6.1 Update `WorkerManager` in `JobActor.java` + +**File:** `mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/job/JobActor.java` + +Add imports for the shared classes at the top of the file: + +```java +import io.mantisrx.master.resourcecluster.proto.MantisResourceClusterReservationProto.UpsertReservation; +import io.mantisrx.master.resourcecluster.proto.MantisResourceClusterReservationProto.CancelReservation; +import io.mantisrx.master.resourcecluster.proto.MantisResourceClusterReservationProto.ReservationKey; +import io.mantisrx.master.resourcecluster.proto.MantisResourceClusterReservationProto.ReservationPriority; +import io.mantisrx.master.resourcecluster.proto.MantisResourceClusterReservationProto.ReservationPriority.PriorityType; +``` + +##### 6.1.1 Add Field to Track Reservation Support + +```java +class WorkerManager implements IWorkerManager { + // Existing fields... + + private final boolean useReservationScheduling; + + WorkerManager(...) { + // Existing initialization... + this.useReservationScheduling = // from master config value + } +} +``` + +##### 6.1.2 Modify `queueTasks()` Method + +```java +private void queueTasks(List workerRequests, Optional readyAt) { + if (useReservationScheduling) { + queueTasksViaReservation(workerRequests, PriorityType.NEW_JOB); + } else { + queueTasksViaLegacyScheduler(workerRequests, readyAt); + } +} + +private void queueTasksViaLegacyScheduler(List workerRequests, Optional readyAt) { + // Existing implementation - move current queueTasks logic here + final List scheduleRequests = workerRequests + .stream() + .map(wR -> createSchedulingRequest(wR, readyAt)) + .collect(Collectors.toList()); + LOGGER.info("Queueing up batch schedule request for {} workers", workerRequests.size()); + try { + scheduler.scheduleWorkers(new BatchScheduleRequest(scheduleRequests)); + } catch (Exception e) { + LOGGER.error("Exception queueing tasks", e); + } +} + +/** + * Queue tasks via reservation system using shared proto classes directly. + * No type conversion needed - JobActor uses the same classes as ResourceClusterActor. + */ +private void queueTasksViaReservation( + List workerRequests, + PriorityType priorityType) { + + // Group by stage + Map> byStage = workerRequests.stream() + .collect(Collectors.groupingBy(IMantisWorkerMetadata::getStageNum)); + + for (Map.Entry> entry : byStage.entrySet()) { + int stageNum = entry.getKey(); + List stageWorkers = entry.getValue(); + + if (stageWorkers.isEmpty()) continue; + + // Create allocation requests + Set allocationRequests = stageWorkers.stream() + .map(wm -> { + ScheduleRequest sr = createSchedulingRequest(wm, Optional.empty()); + return TaskExecutorAllocationRequest.of( + sr.getWorkerId(), + sr.getSchedulingConstraints(), + sr.getJobMetadata(), + sr.getStageNum()); + }) + .collect(Collectors.toSet()); + + // Get stage target size + int stageTargetSize = mantisJobMetaData.getStageMetadata(stageNum) + .map(IMantisStageMetadata::getNumWorkers) + .orElse(stageWorkers.size()); + + // Build reservation request using shared UpsertReservation class directly! + UpsertReservation request = UpsertReservation.builder() + .reservationKey(ReservationKey.builder() + .jobId(jobId.getId()) + .stageNumber(stageNum) + .build()) + .schedulingConstraints(stageWorkers.get(0).getSchedulingConstraints()) + .allocationRequests(allocationRequests) + .stageTargetSize(stageTargetSize) + .priority(buildPriority(priorityType)) + .build(); + + LOGGER.info("Upserting reservation for job {} stage {} with {} workers (priority={})", + jobId, stageNum, allocationRequests.size(), priorityType); + + scheduler.upsertReservation(request) + .whenComplete((ack, ex) -> { + if (ex != null) { + LOGGER.warn("Failed to upsert reservation for job {} stage {}", jobId, stageNum, ex); + } + }); + } +} + +/** + * Build priority using shared ReservationPriority class directly. + */ +private ReservationPriority buildPriority(PriorityType type) { + // Get tier from job SLA (durationType ordinal gives tier) + int tier = mantisJobMetaData.getJobDefinition().getJobSla().getDurationType().ordinal(); + + return ReservationPriority.builder() + .type(type) + .tier(tier) + .timestamp(System.currentTimeMillis()) + .build(); +} +``` + +##### 6.1.3 Modify `resubmitWorker()` Method + +```java +private void resubmitWorker(JobWorker oldWorker) throws Exception { + LOGGER.info("Resubmitting worker {}", oldWorker.getMetadata()); + // ... existing validation and worker creation logic ... + + JobWorker newWorker = new JobWorker.Builder() + // ... existing builder code ... + .build(); + + mantisJobMetaData.replaceWorkerMetaData(...); + + // Kill the old task + scheduler.unscheduleAndTerminateWorker( + oldWorkerMetadata.getWorkerId(), + Optional.ofNullable(oldWorkerMetadata.getSlave())); + + // Queue the new worker + if (useReservationScheduling) { + // Use REPLACE priority (highest) for worker resubmission + queueTasksViaReservation( + Collections.singletonList(newWorker.getMetadata()), + PriorityType.REPLACE); // Direct enum reference + } else { + long workerResubmitTime = resubmitRateLimiter.getWorkerResubmitTime(...); + queueTasks(Collections.singletonList(newWorker.getMetadata()), Optional.of(workerResubmitTime)); + } + + LOGGER.info("Worker {} successfully queued for scheduling", newWorker); + numWorkerResubmissions.increment(); +} +``` + +##### 6.1.4 Modify `scaleStage()` Method + +```java +@Override +public int scaleStage(MantisStageMetadataImpl stageMetaData, int ruleMax, int ruleMin, + int numWorkers, String reason) { + // ... existing validation and worker count calculation ... + + if (newNumWorkerCount > oldNumWorkers) { + List workerRequests = new ArrayList<>(); + for (int i = 0; i < newNumWorkerCount - oldNumWorkers; i++) { + try { + int newWorkerIndex = oldNumWorkers + i; + IMantisWorkerMetadata workerRequest = addWorker(schedInfo, stageMetaData.getStageNum(), newWorkerIndex); + jobStore.storeNewWorker(workerRequest); + markStageAssignmentsChanged(true); + workerRequests.add(workerRequest); + } catch (Exception e) { + LOGGER.warn("Exception adding new worker for {}", stageMetaData.getJobId().getId(), e); + } + } + + // Queue all new workers + if (useReservationScheduling) { + // Use SCALE priority (medium) for scaling operations + queueTasksViaReservation(workerRequests, PriorityType.SCALE); // Direct enum reference + } else { + queueTasks(workerRequests, Optional.empty()); + } + } else { + // Scale down logic - unchanged + // ... + } + + return newNumWorkerCount; +} +``` + +##### 6.1.5 Modify `shutdown()` Method + +```java +@Override +public void shutdown() { + if (useReservationScheduling) { + // Cancel all pending reservations for each stage using shared CancelReservation class + for (IMantisStageMetadata stage : mantisJobMetaData.getStageMetadata().values()) { + CancelReservation cancelRequest = CancelReservation.builder() + .reservationKey(ReservationKey.builder() + .jobId(jobId.getId()) + .stageNumber(stage.getStageNum()) + .build()) + .build(); + + scheduler.cancelReservation(cancelRequest) + .whenComplete((resp, ex) -> { + if (ex != null) { + LOGGER.warn("Failed to cancel reservation for job {} stage {}", + jobId, stage.getStageNum(), ex); + } else { + LOGGER.debug("Cancelled reservation for job {} stage {} (cancelled={})", + jobId, stage.getStageNum(), resp.isCancelled()); + } + }); + } + } else { + scheduler.unscheduleJob(jobId.getId()); + } + + // Rest of existing shutdown logic... + if (!allWorkerCompleted()) { + terminateAllWorkersAsync(); + } + jobSchedulingInfoBehaviorSubject.onNext(new JobSchedulingInfo(this.jobMgr.getJobId().getId(), new HashMap<>())); + jobSchedulingInfoBehaviorSubject.onCompleted(); +} +``` + +--- + +## Summary of Files to Modify/Create + +### Key Design Decision: Reuse Existing Classes (No Duplication!) + +By moving the existing reservation classes from `ResourceClusterActor.java` to a shared proto file, we: +- **Eliminate type conversion code** throughout the stack +- **Avoid duplication** of `ReservationKey`, `ReservationPriority`, `Reservation`, etc. +- **Ensure consistency** - same class used by JobActor, Scheduler, ResourceCluster, and Actor layers +- **Simplify maintenance** - single source of truth for reservation types + +### New Files + +| File | Description | +|------|-------------| +| `MantisResourceClusterReservationProto.java` | **Shared proto file** containing `ReservationKey`, `ReservationPriority`, `Reservation`, `UpsertReservation`, `CancelReservation`, `CancelReservationAck`, `MarkReady` - moved from `ResourceClusterActor.java` | +| `ResourceClusterReservationAwareScheduler.java` | New scheduler implementation using shared classes | + +### Modified Files + +| File | Changes | +|------|---------| +| `ResourceClusterActor.java` | Remove inner classes (moved to proto), import from `MantisResourceClusterReservationProto` | +| `ReservationRegistryActor.java` | Import from `MantisResourceClusterReservationProto` instead of `ResourceClusterActor` inner classes | +| `MantisScheduler.java` | Add `upsertReservation(UpsertReservation)`, `cancelReservation(CancelReservation)`, `supportsReservationScheduling()` | +| `ResourceCluster.java` | Add `upsertReservation(UpsertReservation)`, `cancelReservation(CancelReservation)` | +| `ResourceClusters.java` | Add `markAllRegistriesReady()` | +| `ResourceClusterAkkaImpl.java` | Implement reservation methods (direct passthrough - no conversion!) | +| `ResourceClustersAkkaImpl.java` | Implement `markAllRegistriesReady()` | +| `MasterConfiguration.java` | Add `isReservationSchedulingEnabled()` config | +| `MantisSchedulerFactoryImpl.java` | Add config toggle, create appropriate scheduler, add `markAllRegistriesReady()` | +| `JobClustersManagerActor.java` | Call `markAllRegistriesReady()` after init success | +| `JobActor.java` | Update `WorkerManager` to use reservations with priority types using shared classes | + +--- + +## Rollout Plan + +1. **Phase 1**: Deploy with `mantis.scheduling.reservation.enabled=false` (no behavior change) +2. **Phase 2**: Enable on test clusters, validate: + - New job submission uses `NEW_JOB` priority + - Worker resubmission uses `REPLACE` priority + - Scale-up uses `SCALE` priority + - Job kill cancels reservations properly +3. **Phase 3**: Gradual production rollout with feature flag +4. **Phase 4**: Enable by default, deprecate legacy path + +--- + +## Testing Checklist + +- [ ] New job submission creates reservation with `NEW_JOB` priority +- [ ] Worker failure triggers resubmission with `REPLACE` priority +- [ ] Scale-up creates reservation with `SCALE` priority +- [ ] Scale-down does not create reservations (direct termination) +- [ ] Job kill cancels all pending reservations +- [ ] Master failover correctly initializes running workers +- [ ] Priority ordering: `REPLACE` processed before `SCALE` before `NEW_JOB` +- [ ] Same priority ordered by tier then timestamp (FIFO) +- [ ] `WorkerLaunched` events correctly routed to JobActor +- [ ] Legacy scheduler path still works when config disabled + diff --git a/context/archive/reservation/plan-scaler-reservation-integration.md b/context/archive/reservation/plan-scaler-reservation-integration.md new file mode 100644 index 000000000..886501764 --- /dev/null +++ b/context/archive/reservation/plan-scaler-reservation-integration.md @@ -0,0 +1,982 @@ +# Scaler-Reservation Integration Plan + +## Overview + +This document outlines the integration plan for the reservation-based scheduling system with the `ResourceClusterScalerActor`. The goal is to make scaling decisions aware of pending reservations so the cluster can proactively scale up to meet demand and avoid premature scale-down. + +## Problem Statement + +The `ResourceClusterScalerActor` currently makes scaling decisions based solely on **current cluster usage** (`idleCount`, `totalCount`). With the new reservation-based scheduling system, there's a critical gap: + +1. **Pending reservations are not factored into scaling decisions** - If 10 workers are in the reservation queue, scaling down would be premature +2. **Scale-up should be more aggressive** when reservations are pending - The system should proactively scale up to fulfill pending demand +3. **Key mismatch** - Scaler uses `ContainerSkuID` (SKU), while reservations are grouped by `canonicalConstraintKey` (SchedulingConstraints) + +## Architecture + +``` +┌─────────────────────────────────────────────────────────────────────────────────┐ +│ ResourceClusterScalerActor │ +│ (+ reservationSchedulingEnabled flag from MasterConfiguration) │ +│ │ +│ GetReservationAwareClusterUsageRequest ─────────────────────────────────┐ │ +└───────────────────────────────────────────────────────────────────────────│─────┘ + │ + ▼ +┌─────────────────────────────────────────────────────────────────────────────────┐ +│ ResourceClusterActor │ +│ onGetReservationAwareClusterUsage(): │ +│ 1. Ask ReservationRegistryActor for GetPendingReservationsForScaler │ +│ 2. Forward to ExecutorStateManagerActor with pending reservations │ +└─────────────────────────────────────────────────────────────────────────────────┘ + │ │ + ▼ ▼ +┌─────────────────────────┐ ┌────────────────────────────────────────┐ +│ ReservationRegistryActor│ │ ExecutorStateManagerActor │ +│ │ │ (+ reservationSchedulingEnabled flag) │ +│ GetPendingReservations- │ │ │ +│ ForScaler │ │ GetClusterUsageWithReservationsRequest│ +│ → List │ │ to SKUs via findBestGroupForUsage() │ +│ (with actual │ │ │ +│ SchedulingConstraints) │ │ +└─────────────────────────┘ └────────────────────────────────────────┘ + │ + ▼ + ┌────────────────────────────────────────┐ + │ ExecutorStateManagerImpl │ + │ (+ reservationSchedulingEnabled flag) │ + │ │ + │ - findBestGroupForUsage(): Like │ + │ findBestGroup but relaxed (no need │ + │ for available TEs) │ + │ - mapGroupKeyToSkuViaGroupKeyFunc(): │ + │ Sample TE from executorsByGroup │ + │ → SKU via groupKeyFunc │ + │ - getClusterUsageWithReservations(): │ + │ Merge current usage + pending │ + │ reservation counts │ + └────────────────────────────────────────┘ +``` + +--- + +## Implementation Plan + +### Phase 1: Configuration Flag Propagation + +#### 1.1 MasterConfiguration + +**File:** `mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/config/MasterConfiguration.java` + +```java +@Config("mantis.scheduling.reservation.enabled") +@Default("false") +boolean isReservationSchedulingEnabled(); +``` + +#### 1.2 Update ResourceClusterScalerActor Constructor + +**File:** `mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClusterScalerActor.java` + +```java +public class ResourceClusterScalerActor extends AbstractActorWithTimers { + // ... existing fields ... + + // NEW: Flag to enable reservation-aware scaling + private final boolean reservationSchedulingEnabled; + + public static Props props( + ClusterID clusterId, + Clock clock, + Duration scalerPullThreshold, + Duration ruleRefreshThreshold, + IMantisPersistenceProvider storageProvider, + ActorRef resourceClusterHostActor, + ActorRef resourceClusterActor, + boolean reservationSchedulingEnabled) { // NEW parameter + return Props.create( + ResourceClusterScalerActor.class, + clusterId, + clock, + scalerPullThreshold, + ruleRefreshThreshold, + storageProvider, + resourceClusterHostActor, + resourceClusterActor, + reservationSchedulingEnabled); + } + + public ResourceClusterScalerActor( + ClusterID clusterId, + Clock clock, + Duration scalerPullThreshold, + Duration ruleRefreshThreshold, + IMantisPersistenceProvider storageProvider, + ActorRef resourceClusterHostActor, + ActorRef resourceClusterActor, + boolean reservationSchedulingEnabled) { // NEW parameter + // ... existing initialization ... + this.reservationSchedulingEnabled = reservationSchedulingEnabled; + } +} +``` + +#### 1.3 Update ExecutorStateManagerActor Constructor + +**File:** `mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ExecutorStateManagerActor.java` + +```java +public class ExecutorStateManagerActor extends AbstractActorWithTimers { + // ... existing fields ... + + // NEW: Flag to enable reservation-aware usage computation + private final boolean reservationSchedulingEnabled; + + public static Props props( + ExecutorStateManagerImpl delegate, + Clock clock, + RpcService rpcService, + JobMessageRouter jobMessageRouter, + MantisJobStore mantisJobStore, + Duration heartbeatTimeout, + Duration assignmentTimeout, + Duration disabledTaskExecutorsCheckInterval, + ClusterID clusterID, + boolean isJobArtifactCachingEnabled, + String jobClustersWithArtifactCachingEnabled, + ResourceClusterActorMetrics metrics, + ExecuteStageRequestFactory executeStageRequestFactory, + boolean reservationSchedulingEnabled) { // NEW parameter + // ... update Props.create call ... + } + + ExecutorStateManagerActor( + ExecutorStateManagerImpl delegate, + // ... existing params ... + boolean reservationSchedulingEnabled) { // NEW parameter + // ... existing initialization ... + this.reservationSchedulingEnabled = reservationSchedulingEnabled; + } +} +``` + +#### 1.4 Update ExecutorStateManagerImpl Constructor + +**File:** `mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ExecutorStateManagerImpl.java` + +```java +public class ExecutorStateManagerImpl implements ExecutorStateManager { + // ... existing fields ... + + // NEW: Flag to enable reservation-aware usage computation + private final boolean reservationSchedulingEnabled; + + ExecutorStateManagerImpl( + Map schedulingAttributes, + FitnessCalculator fitnessCalculator, + Duration schedulerLeaseExpirationDuration, + AvailableTaskExecutorMutatorHook availableTaskExecutorMutatorHook, + boolean reservationSchedulingEnabled) { // NEW parameter + this.schedulingAttributes = schedulingAttributes; + this.fitnessCalculator = fitnessCalculator; + this.schedulerLeaseExpirationDuration = schedulerLeaseExpirationDuration; + this.availableTaskExecutorMutatorHook = availableTaskExecutorMutatorHook; + this.reservationSchedulingEnabled = reservationSchedulingEnabled; + } +} +``` + +--- + +### Phase 2: New Message Types + +#### 2.1 Add New Messages in ResourceClusterActor + +**File:** `mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClusterActor.java` + +```java +/** + * Request from ScalerActor to get cluster usage with pending reservation counts. + * This triggers a two-phase query: first to ReservationRegistryActor, then to ExecutorStateManagerActor. + */ +@Value +static class GetReservationAwareClusterUsageRequest { + ClusterID clusterID; + Function> groupKeyFunc; +} + +/** + * Pending reservation info with actual SchedulingConstraints (not just constraint key string). + * This avoids the need to parse constraint keys back into machine definitions. + */ +@Value +@Builder +static class PendingReservationInfo { + String canonicalConstraintKey; + SchedulingConstraints schedulingConstraints; // Actual constraints - no parsing needed! + int totalRequestedWorkers; + int reservationCount; +} + +/** + * Internal message to pass pending reservations from ReservationRegistryActor + * to ExecutorStateManagerActor for final usage computation. + * Contains actual SchedulingConstraints for direct matching. + */ +@Value +static class GetClusterUsageWithReservationsRequest { + ClusterID clusterID; + Function> groupKeyFunc; + List pendingReservations; // Contains actual constraints +} +``` + +#### 2.2 Extend GetClusterUsageResponse with Pending Reservation Count + +**File:** `mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/proto/GetClusterUsageResponse.java` + +```java +@Value +@Builder +public class GetClusterUsageResponse { + ClusterID clusterID; + + @Singular + List usages; + + @Value + @Builder + public static class UsageByGroupKey { + String usageGroupKey; + int idleCount; + int totalCount; + + // NEW: Count of workers pending in reservation queue for this SKU + @Builder.Default + int pendingReservationCount = 0; + + /** + * Effective idle count accounting for pending reservations. + * If pending reservations exist, those "idle" TEs will soon be consumed. + */ + public int getEffectiveIdleCount() { + return Math.max(0, idleCount - pendingReservationCount); + } + } +} +``` + +--- + +### Phase 3: ReservationRegistryActor Changes + +#### 3.1 Add New Message Types and Handler + +**File:** `mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ReservationRegistryActor.java` + +```java +/** + * Request for pending reservations with full scheduling constraints (for scaler integration). + */ +enum GetPendingReservationsForScaler { + INSTANCE +} + +/** + * Response containing pending reservations with actual SchedulingConstraints. + */ +@Value +@Builder +public static class PendingReservationsForScalerResponse { + boolean ready; + @Builder.Default + List reservations = Collections.emptyList(); +} + +/** + * Snapshot of pending reservation info including actual SchedulingConstraints. + */ +@Value +@Builder +public static class PendingReservationInfoSnapshot { + String canonicalConstraintKey; + SchedulingConstraints schedulingConstraints; + int totalRequestedWorkers; + int reservationCount; +} +``` + +#### 3.2 Add Handler in createReceive() + +```java +@Override +public Receive createReceive() { + return receiveBuilder() + .match(UpsertReservation.class, this::onUpsertReservation) + .match(CancelReservation.class, this::onCancelReservation) + .match(GetPendingReservationsView.class, this::onGetPendingReservationsView) + .match(GetPendingReservationsForScaler.class, this::onGetPendingReservationsForScaler) // NEW + .match(MarkReady.class, message -> onMarkReady()) + .match(ProcessReservationsTick.class, message -> onProcessReservationsTick(false)) + .match(ResourceClusterActor.ForceProcessReservationsTick.class, message -> onProcessReservationsTick(true)) + .match(AutoMarkReadyTick.class, message -> onAutoMarkReadyTick()) + .match(ResourceClusterActor.TaskExecutorsAllocation.class, this::onTaskExecutorBatchAssignmentResult) + .match(Status.Failure.class, this::onStatusFailure) + .build(); +} + +/** + * Handler for scaler integration - returns pending reservations with actual SchedulingConstraints. + */ +private void onGetPendingReservationsForScaler(GetPendingReservationsForScaler request) { + if (!ready) { + sender().tell( + PendingReservationsForScalerResponse.builder() + .ready(false) + .build(), + self()); + return; + } + + List reservationInfos = new ArrayList<>(); + + for (ConstraintGroup group : reservationsByConstraint.values()) { + // Get a sample reservation from this group to extract SchedulingConstraints + Optional sampleReservation = group.peekTop(); + if (sampleReservation.isEmpty()) { + continue; + } + + reservationInfos.add(PendingReservationInfoSnapshot.builder() + .canonicalConstraintKey(group.getCanonicalConstraintKey()) + .schedulingConstraints(sampleReservation.get().getSchedulingConstraints()) // Actual constraints! + .totalRequestedWorkers(group.getTotalRequestedWorkers()) + .reservationCount(group.size()) + .build()); + } + + sender().tell( + PendingReservationsForScalerResponse.builder() + .ready(true) + .reservations(reservationInfos) + .build(), + self()); +} +``` + +#### 3.3 Expose ConstraintGroup Methods + +Add getter methods to the `ConstraintGroup` inner class: + +```java +private static final class ConstraintGroup { + // ... existing fields and methods ... + + int getTotalRequestedWorkers() { + return totalRequestedWorkers; + } + + int size() { + return queue.size(); + } +} +``` + +--- + +### Phase 4: ResourceClusterActor Handler Implementation + +#### 4.1 Add Handler for GetReservationAwareClusterUsageRequest + +**File:** `mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClusterActor.java` + +```java +@Override +public Receive createReceive() { + return + ReceiveBuilder + .create() + // ... existing handlers ... + .match(GetReservationAwareClusterUsageRequest.class, + metrics.withTracking(this::onGetReservationAwareClusterUsage)) + .build(); +} + +/** + * Handler for reservation-aware cluster usage request. + * Two-phase approach: + * 1. Ask ReservationRegistryActor for pending reservations with actual SchedulingConstraints + * 2. Forward to ExecutorStateManagerActor with the enriched reservation info + */ +private void onGetReservationAwareClusterUsage(GetReservationAwareClusterUsageRequest request) { + final ActorRef originalSender = sender(); + final ActorRef self = self(); + + // Phase 1: Get pending reservations with actual constraints from ReservationRegistryActor + CompletionStage reservationsFuture = + FutureConverters.toJava(Patterns.ask( + reservationRegistryActor, + ReservationRegistryActor.GetPendingReservationsForScaler.INSTANCE, + Duration.ofSeconds(5).toMillis())) + .thenApply(ReservationRegistryActor.PendingReservationsForScalerResponse.class::cast); + + reservationsFuture.whenComplete((reservationsResponse, error) -> { + if (error != null) { + log.warn("Failed to get pending reservations for usage request", error); + // Fall back to regular usage request without reservations + executorStateManagerActor.tell( + new GetClusterUsageRequest(request.getClusterID(), request.getGroupKeyFunc()), + originalSender); + } else { + // Phase 2: Convert to PendingReservationInfo and forward to ExecutorStateManagerActor + List pendingReservations; + if (reservationsResponse.isReady()) { + pendingReservations = reservationsResponse.getReservations().stream() + .map(snapshot -> PendingReservationInfo.builder() + .canonicalConstraintKey(snapshot.getCanonicalConstraintKey()) + .schedulingConstraints(snapshot.getSchedulingConstraints()) + .totalRequestedWorkers(snapshot.getTotalRequestedWorkers()) + .reservationCount(snapshot.getReservationCount()) + .build()) + .collect(Collectors.toList()); + } else { + pendingReservations = Collections.emptyList(); + } + + log.debug("Forwarding usage request with {} pending reservation groups", pendingReservations.size()); + + executorStateManagerActor.tell( + new GetClusterUsageWithReservationsRequest( + request.getClusterID(), + request.getGroupKeyFunc(), + pendingReservations), + originalSender); + } + }); +} +``` + +--- + +### Phase 5: ExecutorStateManagerActor Handler + +#### 5.1 Add Handler for GetClusterUsageWithReservationsRequest + +**File:** `mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ExecutorStateManagerActor.java` + +```java +@Override +public Receive createReceive() { + return receiveBuilder() + // ... existing handlers ... + .match(GetClusterUsageRequest.class, this::onGetClusterUsage) + .match(GetClusterUsageWithReservationsRequest.class, this::onGetClusterUsageWithReservations) // NEW + .build(); +} + +private void onGetClusterUsage(GetClusterUsageRequest req) { + sender().tell(this.delegate.getClusterUsage(req), self()); +} + +// NEW: Handler for reservation-aware usage request +private void onGetClusterUsageWithReservations(GetClusterUsageWithReservationsRequest req) { + sender().tell( + this.delegate.getClusterUsageWithReservations( + req.getClusterID(), + req.getGroupKeyFunc(), + req.getPendingReservations()), + self()); +} +``` + +--- + +### Phase 6: ExecutorStateManagerImpl Implementation + +#### 6.1 Add Reservation-Aware Usage Computation + +**File:** `mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ExecutorStateManagerImpl.java` + +```java +/** + * Compute cluster usage with pending reservation counts. + * Uses actual SchedulingConstraints from reservations - no parsing needed! + * + * @param clusterID The cluster ID + * @param groupKeyFunc Function to extract SKU key from TaskExecutorRegistration + * @param pendingReservations Pending reservations with actual SchedulingConstraints + * @return Usage response with pending reservation counts per SKU + */ +public GetClusterUsageResponse getClusterUsageWithReservations( + ClusterID clusterID, + Function> groupKeyFunc, + List pendingReservations) { + + // Step 1: Get base usage (same as existing getClusterUsage logic) + Map> usageByGroupKey = computeBaseUsage(groupKeyFunc); + Map pendingCountByGroupKey = computePendingCountByGroupKey(groupKeyFunc); + + // Step 2: Map pending reservations to SKU keys using actual SchedulingConstraints + Map reservationCountBySku = + mapReservationsToSku(pendingReservations, groupKeyFunc); + + // Step 3: Build response with merged counts + GetClusterUsageResponseBuilder resBuilder = GetClusterUsageResponse.builder().clusterID(clusterID); + + // Collect all SKU keys (from both current usage and reservations) + Set allSkuKeys = new HashSet<>(); + allSkuKeys.addAll(usageByGroupKey.keySet()); + allSkuKeys.addAll(reservationCountBySku.keySet()); + + for (String skuKey : allSkuKeys) { + Pair usage = usageByGroupKey.getOrDefault(skuKey, Pair.of(0, 0)); + int pendingFromScheduler = pendingCountByGroupKey.getOrDefault(skuKey, 0); + int pendingFromReservations = reservationCountBySku.getOrDefault(skuKey, 0); + + resBuilder.usage(UsageByGroupKey.builder() + .usageGroupKey(skuKey) + .idleCount(usage.getLeft() - pendingFromScheduler) + .totalCount(usage.getRight()) + .pendingReservationCount(pendingFromReservations) + .build()); + } + + GetClusterUsageResponse res = resBuilder.build(); + log.info("Usage result with reservations: {}", res); + return res; +} + +/** + * Compute base usage by SKU. + */ +private Map> computeBaseUsage( + Function> groupKeyFunc) { + Map> usageByGroupKey = new HashMap<>(); + + taskExecutorStateMap.forEach((key, value) -> { + if (value == null || value.getRegistration() == null) { + return; + } + + Optional groupKeyO = groupKeyFunc.apply(value.getRegistration()); + if (!groupKeyO.isPresent()) { + return; + } + + String groupKey = groupKeyO.get(); + Pair kvState = Pair.of( + value.isAvailable() && !value.isDisabled() ? 1 : 0, + value.isRegistered() ? 1 : 0); + + usageByGroupKey.merge(groupKey, kvState, + (prev, curr) -> Pair.of(prev.getLeft() + curr.getLeft(), prev.getRight() + curr.getRight())); + }); + + return usageByGroupKey; +} + +/** + * Compute pending scheduler request counts by SKU. + */ +private Map computePendingCountByGroupKey( + Function> groupKeyFunc) { + Map result = new HashMap<>(); + + // Get all unique TaskExecutorGroupKeys from registered executors + taskExecutorStateMap.values().stream() + .filter(state -> state.getRegistration() != null) + .map(state -> state.getRegistration().getGroup()) + .distinct() + .forEach(group -> { + int pendingCount = getPendingCountByTaskExecutorGroup(group); + if (pendingCount > 0) { + // Map TaskExecutorGroupKey to SKU using sample TE + Optional skuKey = mapGroupKeyToSkuViaGroupKeyFunc(group, groupKeyFunc); + skuKey.ifPresent(sku -> result.merge(sku, pendingCount, Integer::sum)); + } + }); + + return result; +} + +/** + * Map pending reservations to SKU keys using actual SchedulingConstraints. + * Uses findBestGroupForUsage with real constraints - NO PARSING NEEDED! + */ +private Map mapReservationsToSku( + List pendingReservations, + Function> groupKeyFunc) { + + Map reservationCountBySku = new HashMap<>(); + + for (PendingReservationInfo reservation : pendingReservations) { + int totalRequestedWorkers = reservation.getTotalRequestedWorkers(); + if (totalRequestedWorkers <= 0) { + continue; + } + + // Use actual SchedulingConstraints directly - no parsing needed! + SchedulingConstraints constraints = reservation.getSchedulingConstraints(); + Optional bestGroup = findBestGroupForUsage(constraints); + + if (bestGroup.isPresent()) { + // Map TaskExecutorGroupKey to SKU using sample TE + Optional skuKey = mapGroupKeyToSkuViaGroupKeyFunc(bestGroup.get(), groupKeyFunc); + skuKey.ifPresent(sku -> + reservationCountBySku.merge(sku, totalRequestedWorkers, Integer::sum)); + + log.debug("Mapped reservation {} ({} workers) to SKU {} via group {}", + reservation.getCanonicalConstraintKey(), + totalRequestedWorkers, + skuKey.orElse("unknown"), + bestGroup.get()); + } else { + log.warn("Cannot map reservation constraints {} to any TaskExecutorGroup; " + + "{} pending workers will not be counted in scaling", + constraints, totalRequestedWorkers); + } + } + + return reservationCountBySku; +} + +/** + * Find the best matching TaskExecutorGroupKey for given SchedulingConstraints. + * This is a relaxed version of findBestGroup - it doesn't require available TEs, + * just finds which group would be used for scheduling. + * + * Reuses existing matching logic from findBestGroupBySizeNameMatch and + * findBestGroupByFitnessCalculator but without availability requirements. + * + * @param constraints The actual SchedulingConstraints from the reservation + * @return Optional TaskExecutorGroupKey that would match these constraints + */ +private Optional findBestGroupForUsage(SchedulingConstraints constraints) { + if (constraints == null) { + return executorsByGroup.keySet().stream().findFirst(); + } + + // Try size name match first (same logic as findBestGroupBySizeNameMatch) + Optional bySizeName = executorsByGroup.keySet().stream() + .filter(group -> group.getSizeName().isPresent()) + .filter(group -> constraints.getSizeName().isPresent()) + .filter(group -> group.getSizeName().get().equalsIgnoreCase(constraints.getSizeName().get())) + .filter(group -> areSchedulingAttributeConstraintsSatisfied(constraints, group.getSchedulingAttributes())) + .findFirst(); + + if (bySizeName.isPresent()) { + return bySizeName; + } + + // Fall back to fitness calculator (same logic as findBestGroupByFitnessCalculator) + if (constraints.getMachineDefinition() != null) { + return executorsByGroup.keySet().stream() + // Filter out if both sizeName exist and are different + .filter(group -> { + Optional teGroupSizeName = group.getSizeName(); + Optional requestSizeName = constraints.getSizeName(); + return !(teGroupSizeName.isPresent() && requestSizeName.isPresent() + && !teGroupSizeName.get().equalsIgnoreCase(requestSizeName.get())); + }) + // Verify scheduling attribute constraints + .filter(group -> areSchedulingAttributeConstraintsSatisfied(constraints, group.getSchedulingAttributes())) + // Calculate fitness and filter positive scores + .map(group -> Pair.of(group, + fitnessCalculator.calculate(constraints.getMachineDefinition(), group.getMachineDefinition()))) + .filter(pair -> pair.getRight() > 0) + // Get highest fitness score + .max(Comparator.comparingDouble(Pair::getRight)) + .map(Pair::getLeft); + } + + // Last resort: return any group that satisfies attribute constraints + return executorsByGroup.keySet().stream() + .filter(group -> areSchedulingAttributeConstraintsSatisfied(constraints, group.getSchedulingAttributes())) + .findFirst(); +} + +/** + * Map a TaskExecutorGroupKey to SKU key using the provided groupKeyFunc. + * Uses a sample TE from the group to get the registration for mapping. + */ +private Optional mapGroupKeyToSkuViaGroupKeyFunc( + TaskExecutorGroupKey groupKey, + Function> groupKeyFunc) { + + NavigableSet holders = executorsByGroup.get(groupKey); + if (holders == null || holders.isEmpty()) { + log.debug("No TaskExecutors found for group {}", groupKey); + return Optional.empty(); + } + + // Get sample TE registration + TaskExecutorHolder sampleHolder = holders.first(); + TaskExecutorState state = taskExecutorStateMap.get(sampleHolder.getId()); + if (state == null || state.getRegistration() == null) { + log.debug("Sample TE {} has no registration", sampleHolder.getId()); + return Optional.empty(); + } + + // Apply the same groupKeyFunc that scaler uses + return groupKeyFunc.apply(state.getRegistration()); +} +``` + +--- + +### Phase 7: Update ResourceClusterScalerActor Usage Logic + +#### 7.1 Update Usage Request to Use Reservation-Aware Query + +**File:** `mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClusterScalerActor.java` + +```java +private void onTriggerClusterUsageRequest(TriggerClusterUsageRequest req) { + log.trace("Requesting cluster usage: {}", this.clusterId); + if (this.skuToRuleMap.isEmpty()) { + log.info("{} scaler is disabled due to no rules", this.clusterId); + return; + } + + if (reservationSchedulingEnabled) { + // Use reservation-aware usage request + this.resourceClusterActor.tell( + new GetReservationAwareClusterUsageRequest( + this.clusterId, + ResourceClusterScalerActor.groupKeyFromTaskExecutorDefinitionIdFunc), + self()); + } else { + // Legacy behavior + this.resourceClusterActor.tell( + new GetClusterUsageRequest( + this.clusterId, + ResourceClusterScalerActor.groupKeyFromTaskExecutorDefinitionIdFunc), + self()); + } +} +``` + +#### 7.2 Update ClusterAvailabilityRule to Use Effective Idle Count + +```java +public Optional apply(UsageByGroupKey usage) { + Optional decision = Optional.empty(); + + // Use effective idle count that accounts for pending reservations + int effectiveIdleCount = usage.getEffectiveIdleCount(); + int pendingReservations = usage.getPendingReservationCount(); + + log.debug("Evaluating scale rule for SKU {}: idle={}, effectiveIdle={}, pending={}, total={}", + usage.getUsageGroupKey(), + usage.getIdleCount(), + effectiveIdleCount, + pendingReservations, + usage.getTotalCount()); + + // SCALE DOWN: Only if effective idle (after reservations) exceeds max + if (effectiveIdleCount > scaleSpec.getMaxIdleToKeep()) { + if (isLastActionOlderThan(scaleSpec.getCoolDownSecs() * 5)) { + log.debug("Scale Down CoolDown skip: {}, {}", + this.scaleSpec.getClusterId(), this.scaleSpec.getSkuId()); + return Optional.empty(); + } + + int step = effectiveIdleCount - scaleSpec.getMaxIdleToKeep(); + int newSize = Math.max(usage.getTotalCount() - step, this.scaleSpec.getMinSize()); + + decision = Optional.of( + ScaleDecision.builder() + .clusterId(this.scaleSpec.getClusterId()) + .skuId(this.scaleSpec.getSkuId()) + .desireSize(newSize) + .maxSize(newSize) + .minSize(newSize) + .type(newSize == usage.getTotalCount() ? ScaleType.NoOpReachMin : ScaleType.ScaleDown) + .build()); + } + // SCALE UP: If effective idle is below min OR if there are pending reservations + else if (effectiveIdleCount < scaleSpec.getMinIdleToKeep() || pendingReservations > 0) { + if (isLastActionOlderThan(scaleSpec.getCoolDownSecs())) { + log.debug("Scale Up CoolDown skip: {}, {}", + this.scaleSpec.getClusterId(), this.scaleSpec.getSkuId()); + return Optional.empty(); + } + + // Scale up to cover both idle deficit and pending reservations + int idleDeficit = Math.max(0, scaleSpec.getMinIdleToKeep() - effectiveIdleCount); + int step = idleDeficit + pendingReservations; + + int newSize = Math.min(usage.getTotalCount() + step, this.scaleSpec.getMaxSize()); + + decision = Optional.of( + ScaleDecision.builder() + .clusterId(this.scaleSpec.getClusterId()) + .skuId(this.scaleSpec.getSkuId()) + .desireSize(newSize) + .maxSize(newSize) + .minSize(newSize) + .type(newSize == usage.getTotalCount() ? ScaleType.NoOpReachMax : ScaleType.ScaleUp) + .build()); + } + + log.info("Scale Decision for {}-{}: {} (effectiveIdle={}, pending={})", + this.scaleSpec.getClusterId(), + this.scaleSpec.getSkuId(), + decision, + effectiveIdleCount, + pendingReservations); + + if (decision.isPresent() && + (decision.get().type.equals(ScaleType.ScaleDown) || + decision.get().type.equals(ScaleType.ScaleUp))) { + resetLastActionInstant(); + } + return decision; +} +``` + +--- + +## Summary of Changes + +| File | Changes | +|------|---------| +| `MasterConfiguration.java` | Add `isReservationSchedulingEnabled()` config flag | +| `ResourceClusterActor.java` | Add `GetReservationAwareClusterUsageRequest`, `PendingReservationInfo`, `GetClusterUsageWithReservationsRequest` messages and handler | +| `ReservationRegistryActor.java` | Add `GetPendingReservationsForScaler`, `PendingReservationsForScalerResponse`, `PendingReservationInfoSnapshot`, expose `ConstraintGroup.getTotalRequestedWorkers()` and `size()` | +| `ExecutorStateManagerActor.java` | Add `reservationSchedulingEnabled` flag, handler for `GetClusterUsageWithReservationsRequest` | +| `ExecutorStateManagerImpl.java` | Add `reservationSchedulingEnabled` flag, `getClusterUsageWithReservations()`, `findBestGroupForUsage(SchedulingConstraints)`, `mapGroupKeyToSkuViaGroupKeyFunc()`, helper methods | +| `GetClusterUsageResponse.java` | Add `pendingReservationCount` field, `getEffectiveIdleCount()` method to `UsageByGroupKey` | +| `ResourceClusterScalerActor.java` | Add `reservationSchedulingEnabled` flag, conditional usage request, update `ClusterAvailabilityRule.apply()` | + +--- + +## Data Flow Summary + +``` +1. ScalerActor.onTriggerClusterUsageRequest() + │ + ├─ (reservationEnabled=false) ──> GetClusterUsageRequest ──> ESMActor ──> existing logic + │ + └─ (reservationEnabled=true) ──> GetReservationAwareClusterUsageRequest + │ + ▼ +2. ResourceClusterActor.onGetReservationAwareClusterUsage() + │ + ├─ Phase 1: Ask ReservationRegistryActor for GetPendingReservationsForScaler + │ Response: PendingReservationsForScalerResponse + │ - ready: boolean + │ - reservations: List + │ - canonicalConstraintKey: String + │ - schedulingConstraints: SchedulingConstraints ← Actual constraints! + │ - totalRequestedWorkers: int + │ - reservationCount: int + │ + └─ Phase 2: Convert to PendingReservationInfo, forward GetClusterUsageWithReservationsRequest to ESMActor + (includes List with actual SchedulingConstraints) + │ + ▼ +3. ExecutorStateManagerActor.onGetClusterUsageWithReservations() + │ + └─ Delegate to ExecutorStateManagerImpl.getClusterUsageWithReservations() + │ + ▼ +4. ExecutorStateManagerImpl.getClusterUsageWithReservations() + │ + ├─ computeBaseUsage(groupKeyFunc): idleCount, totalCount per SKU + │ + ├─ computePendingCountByGroupKey(groupKeyFunc): pending scheduler requests per SKU + │ + ├─ mapReservationsToSku(pendingReservations, groupKeyFunc): + │ │ + │ ├─ For each PendingReservationInfo: + │ │ ├─ findBestGroupForUsage(schedulingConstraints) ──> TaskExecutorGroupKey + │ │ │ (uses actual SchedulingConstraints - no parsing!) + │ │ └─ mapGroupKeyToSkuViaGroupKeyFunc(groupKey, groupKeyFunc) ──> SKU + │ │ (uses sample TE from executorsByGroup) + │ │ + │ └─ Aggregate: Map + │ + └─ Build Response: UsageByGroupKey with idleCount, totalCount, pendingReservationCount + │ + ▼ +5. ScalerActor.ClusterAvailabilityRule.apply(UsageByGroupKey) + │ + ├─ effectiveIdleCount = idleCount - pendingReservationCount + │ + ├─ Scale Down: Only if effectiveIdleCount > maxIdleToKeep + │ + └─ Scale Up: If effectiveIdleCount < minIdleToKeep OR pendingReservationCount > 0 +``` + +--- + +## Key Behavioral Changes + +### Scale-Up Logic (Enhanced) + +**Before:** +``` +IF idleCount < minIdleToKeep: + scaleUp by (minIdleToKeep - idleCount) +``` + +**After:** +``` +effectiveIdle = idleCount - pendingReservations +IF effectiveIdle < minIdleToKeep OR pendingReservations > 0: + scaleUp by (minIdleToKeep - effectiveIdle) + pendingReservations +``` + +### Scale-Down Logic (Protected) + +**Before:** +``` +IF idleCount > maxIdleToKeep: + scaleDown by (idleCount - maxIdleToKeep) +``` + +**After:** +``` +effectiveIdle = idleCount - pendingReservations +IF effectiveIdle > maxIdleToKeep: + scaleDown by (effectiveIdle - maxIdleToKeep) +// Pending reservations prevent premature scale-down +``` + +--- + +## Key Design Decisions + +1. **Actual SchedulingConstraints passed (no parsing)** + - `PendingReservationInfoSnapshot` includes the actual `SchedulingConstraints` object + - `ExecutorStateManagerImpl` can directly use `findBestGroupForUsage(constraints)` without parsing constraint keys + +2. **Reuses existing matching logic** + - `findBestGroupForUsage()` uses the same `areSchedulingAttributeConstraintsSatisfied()` and fitness calculator as real scheduling + - Only difference: doesn't require available TEs, just finds the matching group + +3. **Sample TE for SKU mapping** + - `mapGroupKeyToSkuViaGroupKeyFunc()` uses a sample TE from `executorsByGroup` to get registration + - Applies the same `groupKeyFunc` that scaler uses for consistency + +4. **Feature flag controlled** + - `reservationSchedulingEnabled` flag propagated to all relevant components + - Falls back to legacy behavior when disabled + +--- + +## Testing Checklist + +- [ ] Scaler receives pending reservation count per SKU +- [ ] Scale-up triggered when pending reservations exist +- [ ] Scale-down blocked when pending reservations would consume idle TEs +- [ ] Effective idle count correctly computed +- [ ] Constraint-to-SKU mapping works for size-based matching +- [ ] Constraint-to-SKU mapping works for machine-definition-based matching +- [ ] Feature flag controls new vs. legacy behavior +- [ ] New metrics emitted for reservation-aware decisions +- [ ] Backward compatible when flag is disabled +- [ ] Graceful fallback when ReservationRegistryActor query fails + diff --git a/context/archive/reservation/plan-scheduling-registry.md b/context/archive/reservation/plan-scheduling-registry.md new file mode 100644 index 000000000..67d7074a5 --- /dev/null +++ b/context/archive/reservation/plan-scheduling-registry.md @@ -0,0 +1,135 @@ +# Plan: Support Reservation-based Task Executors Scheduling +The objective here is to introduce a reservation registry in resource cluster and scheduler to be able to handle reliable batched scheduling for job workers to be allocated to task executors (TE). +Current Problems to be solved: +- Only support batch mode in new job creations but not job scaling. +- Unreliable cache tracking on scheduling requests. +- Resource scaling is not efficient and cannot handle large scale concurrent job scaling operations. + +## Design: Reservation-based Scheduling +### Scenarios to handle +- Starvation/blocking from single large requests. On new job submission this should remain as strict batch. On scaling scenario there are tradeoffs to be made. We can stick to strict batching (so it's same as new job) or allow partial batch to be fulfilled (in multi-job scaling situations this can make the allocation process more fair and less unexpected blocking). +- Cascading scaling requests from same job needs to be deduped. +- Error recovery: if something went wrong in a job/reservation and block the whole pipeline, we need some ways to break the gridlock. options include (not ideal: restart leader to re-construct registry), (failed reservation is re-prioritized and move to the end of the queue), (api to force a reset). + +### Options +- Strict batching (immutable reservation). Use detailed metrics to see the pending status. Support cancellation on reservations fully wired into job action (kill, scale down). Simpler mechanism, less error prone but won't handle extreme case where large batch starving everyone else. +- Mutable batching: allow partial batch update to be made to reduce locking from a large batch to give more faireness but added more complexity and error prone to the system. + +Thoughts: strict batching is preferred to reduce risk and complexity for now while we can try keep most interfaces open to both modes. + +## Terminology +- Task Executor (TE): agent host registered to control plane from one of the resource clusters with a pre-defined container spec. Can be assigned job worker. +- Worker: a worker in one of a job's stage. A job can have multiple stages and each stage will have a list of workers with worker index and worker number to id each one. JobActor is responsible to keep track of the workers and inform scheduler to add/remove workers from scaling/creation/deletion operations. +- ESM (executor state manager): class in ResourceClusterActor tracking TE status (registration and heartbeats). + +### Key Logic Changes +- A job's scheduling requests is no longer tracked in loop inside scheduler. Scheduler is only responsible to track the insertion of the reservation. Resource cluster actor and reservation registry is responsible (looping) the pending requests and push scheduled assignments into TEs. +- ESM needs to be able to interact with registry to do operations: (try to allocate TEs) and (Notify registry to process). +- Registry's main process loop shall trigger every x seconds (configurable) and also triggered by ESM message. Introduce minimum wait period to avoid live locking the dispatcher. +- Operations' within actor e.g. reservation updates runs in actor dispather thread so no locking is needed. +- Batch Scheduling Logic: when a reservation secures the full batch e.g 100 TEs, we also need to assign the TE with actual worker requests but this step can fail. To handle partial failure here e.g 3 out of 100 TE's worker assignment failed, we cannot requeue a reservation with the 3 failed TE as it breaks the batch contract. Thus we need to track/retry the failed assignment requests (same as what is happening today in scheduler). however now we have a messy state to manage: 97 TEs running the assigned workers while 3 keep failing (e.g. bad container/disk full etc.). + * Option 1: limit retry attempts and reserve again in reservation separately with high priority (e.g. similar to replace worker request) for each failure assignment. If worker kill or job kill happened in the middle of retry or reservation, they shall be cancelled. +- + +### Action List v2 +[Direction] use strict batching mode only. keep current code path as fallback with config options. + +- Reservation: one reservation instance represents one job's given stage's request to allocate/assign TEs. The assumption here is in one stage the scheduling constraint should be the same for all workers/TEs. For a given request from the JobActor, the reservation can be constructed to include the list of worker's TaskExecutorAllocationRequest and also the total target size for the given stage. + - The reservation should use total target stage size + request worker list to help determine: dedupe requests, determining whether the request is for a new job vs a scaling request. + - On new job (total target stage size == worker request list size), the reservation fullfillment should not allow partial allocation. + - On scaling request, partial allocation is also not allowed. + - Scheduling constrain should be converted to canonical format to reduce potential error and make logging cleaner. + +- ReservationRegistry: add `ReservationRegistry` owned by `ResourceClusterActor` + - Types: `ReservationKey(jobId, stageNum)`, `Reservation{ReservationKey, canonicalSchedulingConstraint, workerRequests, stageTargetSize, lastUpdatedAt, priorityEpoch}` + - Indexes: `reservationsByKey`, `perSchedulingConstraintPriorityQueue` ordered by `(priorityEpoch desc)`. (make sure there is proper GC on these index) + - API (actor message behavior): + -- `upsertReservations`: recevied from JobActor/Scheduler. insert if new. If there is existing key, replace the old reservation. + -- `completeReservation`: mark a reservation to be completed. Remove index entries. + -- `cancelReservation`: cancel an existing reservation. Remove from index tables. + -- `getPendingReservationsBySchedulingConstraints`: return a map of summary of pending reservations grouped by reservation's schedulingConstraint. + -- `markReady`: signal from external actors to indicate the registry has received all the state (init messages) from job actors. This is important since a partial pending result to resource cluster actor and scaler can interrupt ongoing provisoin tasks during a leader switch. + - Behavior: + - the general reservations from job actors should be processed FIFO + - same reservation request doesn't change existing reservation priority. + - init: return not ready from getPendingReservationsBySchedulingConstraints until ready signal from schduler factory. + - reservation priority: default on request epoch; override to (0/1/2 etc. on manual overide e.g. replace worker request). Leave interface to add custom logic in future too. + - same key (jobId + stageNum) with different stage target size: dedupe logic (ignore same target size && same worker request size. Replace old request with new one if target size changed). + - reservation class should be immutable. Replace exisitng if needed. + - internal loop to process reservation queue: timer + ESM signal (need to be mini-batch style to avoid too many trigger from ESM). + - abstract and config to support different priority behavior. Default strategy setup to use epoch time on creation + override signal (e.g. priorityEpoch as 0) + - main processing logic: when reservation registry try to process the active reservations, it should go through the priority (so it honors the priority). Start with top of the queue of each "size" or "sku", check if the reservation can be fulfilled (matched) from executor state manager. Once matched, invoke resource cluster actor to do the actual assignement to TEs and finish (remove) the reservation in registry. If the target reservation cannot be matched, stop the process on this queue. Proceed to next reservation if the top one is successfully filled, otherwise stop (only top of the queue shall be processed). Error in assigning TEs tasks should be properly handled such that it moves the reservation to the tail of the queue (re-assigned priority). + - (optional) new reservation trigger resource cluster scaler process. + +- ResourceClusterActor: integrate registry with matching and TE lifecycle + - Integrate registry calls: getTaskExecutorFor from scheduler; getUsage from RC scaler (combine state from ESM and registry). + - Delegate calls to registry and calls between registry and ExecutorStateManager + - Support Assignement function (currently in scheduler actor) to schedule assignments to TEs and failure handling (update reservation). + +- ExecutorStateManagerImpl: keep the original findBestFit part unchanged + and build a new method named "tryAllocateTaskExecutors(ReservationAllocationRequest request, boolean + allowsPartialMatch), which does similar operation as findbestFit except that ReservationAllocationRequest directly represents a reservation so the scheduling constraints in this request is the same for all of the workers inside. + +- Scheduler: simplify and become stateless (no longer tracks pending requests) (keep the current code path as fallback enabled by config flags) + - Extend `BatchScheduleRequest` to each job stage's target size. + - On schedule: keep the same `getTaskExecutorsFor` call to RC but no longer expect the allocation result. Only retry if RC returns a failure message indicating reservation insert failed. + - On assignment: no longer responsible to call TE for assignment (now handled directly inside RC). + - On leader init: in scheduler factory wait for `JobActor`s init, then call `markRegistryReady()` to RC. + +- Autoscaler coupling and readiness + - RC usage response: `{ready:boolean, bySku: {available, pending, idle=available-pending}}`; still exclude disabled from `available` + - Scaler ignores scale-down while registry not ready; optionally prompt scaler when `pending` reservations increases materially + +- Observability & ops + - Metrics: track state in reservation registry (size, pending, actions). Metrics on usage result to scaler. + - Structured logs for upserts, assignment decisions, submit outcomes, readiness transitions + +- Recovery, flags, and rollout + - Feature-flag the registry path; retain fallback to current behavior; Add depreaction annotation and notes to old code path that can be removed later. + - Ensure idempotent upserts and submit result handling; timeouts recover assigned-but-unlaunched + +- Tests + - Unit: registry and state transition. ESM/ResourceClsuterActor actions. + - Unit: ESM lease/assign/unassign/timeout invariants + - Integration: multi-reservation handling, failed TE assignements, init behavior, RC restart with scheduler replay, scaler polling with readiness gating + +### Reservation-based Scheduling v2.1 (Mermaid) + +```mermaid +sequenceDiagram + autonumber + participant JA as JobActor + participant SCH as ResourceClusterAwareSchedulerActor + participant RC as ResourceClusterActor + participant REG as ReservationRegistry + participant ESM as ExecutorStateManagerImpl + participant RCS as ResourceClusterScalerActor + participant TE as TaskExecutor + + SCH->>RC: MarkRegistryReady() after all job actor init on leader switch + JA->>SCH: BatchScheduleRequest(targetCount) + SCH->>RC: getTaskExecutorsFor(key: jobId+stage, targetCount) + RC->>REG: upsertReservations(key, targetCount) + REG->>REG: enqueue key in perSkuPriorityQueue + loop match reservations (periodic or triggered by TE updates) + REG->>RC: tryMatchTaskExecutors(reservation) + + rect rgb(245, 245, 245) + RC->>ESM: findBestFit(reservations) + ESM-->>RC: matched (TE ids) + RC->>TE: submitTask(executeStageRequest) with TE lease + TE-->>RC: Ack / Fail + RC->>REG: completeReservation(key) if success, re-enqueue reservation if failed + end + end + + + + + loop Scaler poll (periodic or prompted) + RCS->>RC: GetClusterUsage() + RC-->>RCS: {ready, bySku: {merge state from ESM and REG}} + end + + +``` diff --git a/context/reservation-scheduling-context.md b/context/reservation-scheduling-context.md new file mode 100644 index 000000000..6f3a21936 --- /dev/null +++ b/context/reservation-scheduling-context.md @@ -0,0 +1,108 @@ +# Reservation Scheduling Context + +This change promotes the reservation-based scheduling plan into production-ready behavior across the master, scheduler, resource cluster, and scaler stacks. The goal is to give each job stage a strict, priority-ordered reservation that the control plane owns end‑to‑end so that batching, failures, autoscaling, and retries can be reasoned about holistically. + +## Key behavioral shifts +- **Job → Scheduler → Resource Cluster handoff** now happens through `UpsertReservation`/`CancelReservation` envelopes (see `context/plan-reservation-scheduler-integration.md`). The `JobActor.WorkerManager` groups new workers by stage, builds `TaskExecutorAllocationRequest`s, derives a `ReservationPriority` (REPLACE > SCALE > NEW_JOB), and sends the reservation to whatever scheduler was created by `MantisSchedulerFactoryImpl`. +- `MantisSchedulerFactoryImpl` now branches on `mantis.scheduling.reservation.enabled`. When the flag is `true` (default in `MasterConfiguration`), it wires `ResourceClusterReservationAwareScheduler` instead of `ResourceClusterAwareSchedulerActor`. The new scheduler simply proxies the shared protobuf (`MantisResourceClusterReservationProto`) request straight down to `ResourceCluster.upsertReservation()`/`cancelReservation()` so there is zero type conversion and the scheduler no longer tracks pending allocations itself. +- The scheduler still exposes legacy hooks (`scheduleWorkers`, `unscheduleJob`, `unscheduleAndTerminateWorker`) for the paths that bypass reservations (i.e., when the flag is `false`), but the reservation-aware path now advertises `schedulerHandlesAllocationRetries() == true` because retries live in the registry/assignment actors rather than in the scheduler. + +## ResourceCluster + Reservation Registry responsibilities +- `ResourceClusterActor` became a thin router: it now creates child actors for `ReservationRegistryActor`, `ExecutorStateManagerActor`, and `AssignmentHandlerActor` and forwards reservation messages and lifecycle updates to them (`TaskExecutorRegistration`, heartbeats, disable/enable, etc.). Everything else (TE management, usage queries) still goes through the executor state manager. +- The registry (`mantis-control-plane/.../ReservationRegistryActor.java`) keeps one queue per canonical constraint key, enforces FIFO per priority, tracks a single in-flight reservation, and sends `TaskExecutorBatchAssignmentRequest` messages to the parent when a reservation is ready. It auto‑marks itself ready via `MarkReady`/timer, exposes `GetPendingReservationsView` for introspection, and answers `GetPendingReservationsForScaler` with real `SchedulingConstraints` so the scaler can map reservations to SKUs. +- `AssignmentHandlerActor` now handles the RPC-heavy `TaskExecutorAssignmentRequest` lifecycle, reusing `TaskExecutorGateway` futures, enforcing retries, and reporting failures back to `ResourceClusterActor` so the queue can decide whether to requeue a reservation or propagate the failure downstream. +- `ExecutorStateManagerActor`/`ExecutorStateManagerImpl` gained reservation-aware APIs (`GetClusterUsageWithReservationsRequest` → `getClusterUsageWithReservations(...)`) that accept the streamed `PendingReservationInfo` snapshots. They reuse the existing fitness/matching logic to map each reservation back to a `containerSku` and expose `pendingReservationCount` in the `GetClusterUsageResponse.UsageByGroupKey`. + +## Autoscaler integration +- `ResourceClusterScalerActor` now accepts `reservationSchedulingEnabled` and, when true, requests `GetReservationAwareClusterUsageRequest`. `ResourceClusterActor` handles this by querying the registry for pending reservations, translating them into `PendingReservationInfo`, and forwarding them to the executor state manager before replying to the scaler. +- `ClusterAvailabilityRule.apply()` operates on the new fields `pendingReservationCount`/`effectiveIdleCount`, so scale-up becomes: `if effectiveIdle < minIdle OR pending > 0 → grow by missing idle + pending`, and scale-down is blocked unless `effectiveIdle > maxIdle`. This keeps the cluster from killing capacity that already has hard reservations behind it (`context/plan-scaler-reservation-integration.md` covers the rationale). + +## Configuration + readiness +- `ResourceClusters` now exposes `markAllRegistriesReady()` so `JobClustersManagerActor.initialize()` can signal the registries once every current job cluster has recovered; the default scheduler factory implementation forwards that call to `ResourceClustersAkkaImpl`, which relays `MarkReady.INSTANCE` to every `ResourceClusterActor`. +- The shared proto `MantisResourceClusterReservationProto` holds `ReservationKey`, `ReservationPriority`, `Reservation`, `UpsertReservation`, `CancelReservation`, and `CancelReservationAck`. Every layer (job actor, scheduler, resource cluster, registry, tests) now imports these types, avoiding duplicated classes. +- `ResourceCluster` interface gained `upsertReservation`, `cancelReservation`, and `markRegistryReady` defaults so clients can interact with reservations without leaking actor internals. + +## Key classes & actors +- **`JobActor.WorkerManager` (mantis-control-plane-server/src/main/java/.../JobActor.java)** groups worker requests by stage, derives `ReservationPriority`, and serializes the work into `UpsertReservation`/`CancelReservation` calls. It also exposes the delayed reservation timer, handles scaling/resubmit priorities, and cancels pending reservations during shutdown. +- **`ResourceClusterReservationAwareScheduler` & `MantisSchedulerFactoryImpl`** replace the legacy scheduler when `mantis.scheduling.reservation.enabled` is `true`, forwarding the shared proto objects directly to `ResourceClusterAkkaImpl` without duplicating DTOs or tracking pending states. +- **`ResourceClusterActor`** now supervises `ReservationRegistryActor`, `ExecutorStateManagerActor`, and `AssignmentHandlerActor`, forwarding reservation/lifecycle messages, exposing `GetReservationAwareClusterUsageRequest`, and bridging TE lifecycle updates to the child actors while still serving executor queries. +- **`ReservationRegistryActor`** stores per-constraint queues, enforces REPLACE/SCALE/NEW_JOB priority, tracks a single in-flight reservation per SKU, emits `TaskExecutorBatchAssignmentRequest`, and answers scaler snapshots (`GetPendingReservationsForScaler`) with actual `SchedulingConstraints`. +- **`AssignmentHandlerActor`** concentrates the RPC-heavy gateway retries, keeps retry budgets, and reports assignment success/failure back to the parent so the registry can requeue or finish reservations without re-implementing gateway logic. +- **`ExecutorStateManagerActor` / `ExecutorStateManagerImpl`** gained reservation-aware handlers and `getClusterUsageWithReservations()`, mapping pending reservations through `findBestGroupForUsage()`/`mapGroupKeyToSkuViaGroupKeyFunc()` before enriching the `GetClusterUsageResponse`. +- **`ResourceClusterScalerActor`** queries `GetReservationAwareClusterUsageRequest` when enabled, works with `ClusterAvailabilityRule.apply()` to respect `pendingReservationCount`, and gate scale-down decisions behind `effectiveIdleCount`. +- **`ResourceClustersManagerActor` / `ResourceClustersAkkaImpl`** now orchestrate `markAllRegistriesReady()` for every active cluster, which in turn triggers `ResourceClusterActor` → `ReservationRegistryActor` readiness once leader initialization finishes. + +## Actor flow comparison + +### Legacy scheduling interactions + +```mermaid +sequenceDiagram + autonumber + participant JobActor + participant Scheduler + participant ResourceClusterActor + participant ExecutorStateManager + participant TaskExecutor + + JobActor->>Scheduler: scheduleWorkers(batchRequest) + Scheduler->>ResourceClusterActor: getTaskExecutorsFor(allocationRequests) + ResourceClusterActor->>ExecutorStateManager: findBestFit(...) + ExecutorStateManager-->>ResourceClusterActor: matched executors + ResourceClusterActor->>Scheduler: BatchScheduleResult(TE ids) + Scheduler->>TaskExecutor: submitTask(scheduleRequest) + TaskExecutor-->>Scheduler: ack/failure + Scheduler->>JobActor: WorkerLaunched/failed + loop scaler polling + ResourceClusterActor->>ExecutorStateManager: getClusterUsage() + ExecutorStateManager-->>ResourceClusterActor: usage + ResourceClusterActor->>Scaler: usage + end +``` + +### Reservation-based interactions + +```mermaid +sequenceDiagram + autonumber + participant JobActor + participant ReservationScheduler + participant ResourceClusterActor + participant Registry + participant ExecutorStateManager + participant AssignmentHandler + participant TaskExecutor + + JobActor->>ReservationScheduler: upsertReservation(stage, priority) + ReservationScheduler->>ResourceClusterActor: upsertReservation(request) + ResourceClusterActor->>Registry: enqueue reservation + Registry->>ExecutorStateManager: tryAllocateAndLease(reservation) + ExecutorStateManager-->>Registry: leases / failure + Registry->>AssignmentHandler: TaskExecutorBatchAssignmentRequest + AssignmentHandler->>TaskExecutor: submitTask(...) + TaskExecutor-->>AssignmentHandler: ack/failure + AssignmentHandler->>ResourceClusterActor: WorkerAssigned / failure + loop scaler polling + Scaler->>ResourceClusterActor: GetReservationAwareClusterUsageRequest + ResourceClusterActor->>Registry: GetPendingReservationsForScaler + Registry-->>ResourceClusterActor: pending snapshots + ResourceClusterActor->>ExecutorStateManager: GetClusterUsageWithReservations + ExecutorStateManager-->>ResourceClusterActor: usage with pending counts + ResourceClusterActor->>Scaler: augmented usage + end +``` + +## Testing & verification +- Added focused tests for the new pieces: + - Reservation path: `ReservationRegistryActorTest`, `ReservationRegistryActorIntegrationTest`, `ReservationRegistryActor` metrics coverage, and `ReservationTest`. + - Reservation-aware scheduler: `JobActorReservationTests`, `AssignmentHandlerActorTest`. + - Executor state: `ExecutorStateManagerReservationUsageTests`, scaled behavior: `ReservationScalerIntegrationTests`. + - End-to-end: `ResourceClusterReservationSchedulerIntegrationTests`. +- Existing scheduler tests (`JobActorSmartRefreshTest`, `JobScaleUpDownTests`, etc.) were updated to exercise the reservation branch so that job scaling, resubmissions, and migration still behave correctly under the new flow. + +## References +- `context/archive/reservation/context-scheduling-logic.md` – baseline diagram for job submission/scale loops. +- `context/archive/reservation/plan-reservation-registry-v3.md` – design notes for the registry/assignment duo shown in the Excalidraw. +- `context/archive/reservation/plan-reservation-scheduler-integration.md` – explains how JobActor/WorkerManager talk to the scheduler and registry. +- `context/archive/reservation/plan-scaler-reservation-integration.md` – outlines how the scaler becomes reservation-aware. + diff --git a/mantis-control-plane/mantis-control-plane-core/src/main/java/io/mantisrx/server/master/resourcecluster/ResourceCluster.java b/mantis-control-plane/mantis-control-plane-core/src/main/java/io/mantisrx/server/master/resourcecluster/ResourceCluster.java index a98b35735..2dff69006 100644 --- a/mantis-control-plane/mantis-control-plane-core/src/main/java/io/mantisrx/server/master/resourcecluster/ResourceCluster.java +++ b/mantis-control-plane/mantis-control-plane-core/src/main/java/io/mantisrx/server/master/resourcecluster/ResourceCluster.java @@ -17,6 +17,8 @@ package io.mantisrx.server.master.resourcecluster; import io.mantisrx.common.Ack; +import io.mantisrx.server.master.resourcecluster.proto.MantisResourceClusterReservationProto.CancelReservation; +import io.mantisrx.server.master.resourcecluster.proto.MantisResourceClusterReservationProto.UpsertReservation; import io.mantisrx.server.core.domain.ArtifactID; import io.mantisrx.server.core.domain.WorkerId; import io.mantisrx.server.worker.TaskExecutorGateway; @@ -192,10 +194,56 @@ CompletableFuture getActiveJobOverview( CompletableFuture> getTaskExecutorWorkerMapping( Map attributes); + /** + * Insert or update a reservation for workers in this cluster. + * Reuses the shared UpsertReservation class from MantisResourceClusterReservationProto. + * + * @param request The reservation request + * @return Future that completes when reservation is accepted + */ + default CompletableFuture upsertReservation(UpsertReservation request) { + throw new UnsupportedOperationException("Reservation not supported"); + } + + /** + * Cancel reservations for a job stage. + * Reuses the shared CancelReservation class from MantisResourceClusterReservationProto. + * + * @param request The cancellation request + * @return Future containing acknowledgement + */ + default CompletableFuture cancelReservation(CancelReservation request) { + throw new UnsupportedOperationException("Reservation not supported"); + } + + /** + * Mark the reservation registry as ready to process reservations. + * This should be called after master initialization is complete and all + * existing jobs have been recovered. + * + * @return Future that completes when the registry is marked ready + */ + default CompletableFuture markRegistryReady() { + throw new UnsupportedOperationException("Reservation not supported"); + } + class NoResourceAvailableException extends Exception { + @Nullable + private final String constraintKey; public NoResourceAvailableException(String message) { super(message); + this.constraintKey = null; + } + + public NoResourceAvailableException(String message, @Nullable String constraintKey) { + super(message); + this.constraintKey = constraintKey; + } + + @Nullable + public String getConstraintKey() { + return constraintKey; } } diff --git a/mantis-control-plane/mantis-control-plane-core/src/main/java/io/mantisrx/server/master/resourcecluster/ResourceClusters.java b/mantis-control-plane/mantis-control-plane-core/src/main/java/io/mantisrx/server/master/resourcecluster/ResourceClusters.java index 91cc9c644..ef397ece2 100644 --- a/mantis-control-plane/mantis-control-plane-core/src/main/java/io/mantisrx/server/master/resourcecluster/ResourceClusters.java +++ b/mantis-control-plane/mantis-control-plane-core/src/main/java/io/mantisrx/server/master/resourcecluster/ResourceClusters.java @@ -16,6 +16,7 @@ package io.mantisrx.server.master.resourcecluster; +import io.mantisrx.common.Ack; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -26,4 +27,13 @@ public interface ResourceClusters { ResourceCluster getClusterFor(ClusterID clusterID); CompletableFuture> listActiveClusters(); + + /** + * Mark all reservation registries as ready to process reservations. + * This should be called after master initialization is complete and all + * existing jobs have been recovered. + * + * @return Future that completes when all clusters are marked ready + */ + CompletableFuture markAllRegistriesReady(); } diff --git a/mantis-control-plane/mantis-control-plane-core/src/main/java/io/mantisrx/server/master/resourcecluster/TaskExecutorAllocationRequest.java b/mantis-control-plane/mantis-control-plane-core/src/main/java/io/mantisrx/server/master/resourcecluster/TaskExecutorAllocationRequest.java index a969eb42d..4a6b6bf43 100644 --- a/mantis-control-plane/mantis-control-plane-core/src/main/java/io/mantisrx/server/master/resourcecluster/TaskExecutorAllocationRequest.java +++ b/mantis-control-plane/mantis-control-plane-core/src/main/java/io/mantisrx/server/master/resourcecluster/TaskExecutorAllocationRequest.java @@ -16,6 +16,7 @@ package io.mantisrx.server.master.resourcecluster; +import io.mantisrx.runtime.MantisJobDurationType; import io.mantisrx.server.core.domain.JobMetadata; import io.mantisrx.server.core.domain.WorkerId; import io.mantisrx.server.core.scheduler.SchedulingConstraints; @@ -23,10 +24,21 @@ import lombok.Value; @Value -@AllArgsConstructor(staticName = "of") +@AllArgsConstructor public class TaskExecutorAllocationRequest { WorkerId workerId; SchedulingConstraints constraints; JobMetadata jobMetadata; int stageNum; + long readyAt; + MantisJobDurationType durationType; //TODO: update job actor to fill these fields. + + // Static factory method with defaults for backward compatibility + public static TaskExecutorAllocationRequest of( + WorkerId workerId, + SchedulingConstraints constraints, + JobMetadata jobMetadata, + int stageNum) { + return new TaskExecutorAllocationRequest(workerId, constraints, jobMetadata, stageNum, -1L, MantisJobDurationType.Perpetual); + } } diff --git a/mantis-control-plane/mantis-control-plane-core/src/main/java/io/mantisrx/server/master/resourcecluster/proto/MantisResourceClusterReservationProto.java b/mantis-control-plane/mantis-control-plane-core/src/main/java/io/mantisrx/server/master/resourcecluster/proto/MantisResourceClusterReservationProto.java new file mode 100644 index 000000000..5cb111941 --- /dev/null +++ b/mantis-control-plane/mantis-control-plane-core/src/main/java/io/mantisrx/server/master/resourcecluster/proto/MantisResourceClusterReservationProto.java @@ -0,0 +1,219 @@ +/* + * Copyright 2022 Netflix, Inc. + * + * Licensed 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 io.mantisrx.server.master.resourcecluster.proto; + +import io.mantisrx.server.core.domain.WorkerId; +import io.mantisrx.server.core.scheduler.SchedulingConstraints; +import io.mantisrx.server.master.resourcecluster.TaskExecutorAllocationRequest; +import java.util.Collections; +import java.util.HashSet; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; +import lombok.Builder; +import lombok.Value; + +/** + * Protocol classes for the reservation-based scheduling system. + * These classes are shared between ResourceClusterActor, ReservationRegistryActor, + * MantisScheduler, and JobActor. + */ +public final class MantisResourceClusterReservationProto { + + private MantisResourceClusterReservationProto() {} + + /** + * Unique identifier for a reservation (job + stage). + */ + @Value + @Builder + public static class ReservationKey { + String jobId; + int stageNumber; + } + + /** + * Priority for ordering reservations within constraint groups. + * Ordering: REPLACE < SCALE < NEW_JOB (REPLACE processed first). + */ + @Value + @Builder + public static class ReservationPriority implements Comparable { + public enum PriorityType { + REPLACE, // Worker replacement due to failure (highest priority) + SCALE, // Scale-up request for existing job + NEW_JOB // New job submission (lowest priority) + } + + PriorityType type; + int tier; // Job tier (lower = higher priority within same type) + long timestamp; // FIFO ordering within same priority and tier + + @Override + public int compareTo(ReservationPriority other) { + // 1. First, compare by PriorityType (REPLACE < SCALE < NEW_JOB) + // Enums compareTo method uses their ordinal (declaration order) + int typeComparison = this.type.compareTo(other.type); + if (typeComparison != 0) { + return typeComparison; + } + + // 2. If types are equal, compare by tier (ascending) + int tierComparison = Integer.compare(this.tier, other.tier); + if (tierComparison != 0) { + return tierComparison; + } + + // 3. If types and tiers are equal, compare by timestamp (ascending) + return Long.compare(this.timestamp, other.timestamp); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ReservationPriority priority = (ReservationPriority) o; + return tier == priority.tier && + timestamp == priority.timestamp && + type == priority.type; + } + + /** + * Generates a hash code based on type, tier, and timestamp. + */ + @Override + public int hashCode() { + return Objects.hash(type, tier, timestamp); + } + } + + /** + * Full reservation data including scheduling constraints and allocation requests. + */ + @Value + @Builder(toBuilder = true) + public static class Reservation { + ReservationKey key; + SchedulingConstraints schedulingConstraints; + String canonicalConstraintKey; + Set requestedWorkers; + Set allocationRequests; + int stageTargetSize; + ReservationPriority priority; + long createdAt; + + public boolean hasSameShape(Reservation other) { + return other != null + && Objects.equals(key, other.key) + && Objects.equals(canonicalConstraintKey, other.canonicalConstraintKey) + && Objects.equals(requestedWorkers, other.requestedWorkers) + && stageTargetSize == other.stageTargetSize; + } + + public int getRequestedWorkersCount() { + return requestedWorkers != null ? requestedWorkers.size() : 0; + } + + public static Reservation fromUpsertReservation(UpsertReservation upsert, String canonicalConstraintKey) { + return Reservation.builder() + .key(upsert.getReservationKey()) + .schedulingConstraints(upsert.getSchedulingConstraints()) + .canonicalConstraintKey(canonicalConstraintKey) + .requestedWorkers(upsert.getAllocationRequests() != null ? + Collections.unmodifiableSet( + upsert.getAllocationRequests().stream() + .map(TaskExecutorAllocationRequest::getWorkerId) + .collect(Collectors.toSet())) + : Collections.emptySet()) + .allocationRequests(upsert.getAllocationRequests() != null ? + Collections.unmodifiableSet(new HashSet<>(upsert.getAllocationRequests())) + : Collections.emptySet()) + .stageTargetSize(upsert.getStageTargetSize()) + .priority(upsert.getPriority()) + .createdAt(System.currentTimeMillis()) + .build(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Reservation that = (Reservation) o; + return stageTargetSize == that.stageTargetSize + && Objects.equals(key, that.key) + && Objects.equals(canonicalConstraintKey, that.canonicalConstraintKey) + && Objects.equals(priority, that.priority) + && Objects.equals(requestedWorkers, that.requestedWorkers); + } + + @Override + public int hashCode() { + return Objects.hash(key, canonicalConstraintKey, priority, stageTargetSize, requestedWorkers); + } + + @Override + public String toString() { + return "Reservation(" + + "key=" + key + + ", schedulingConstraints=" + schedulingConstraints + + ", canonicalConstraintKey=" + canonicalConstraintKey + + ", stageTargetSize=" + stageTargetSize + + ", priority=" + priority + + ", createdAt=" + createdAt + + ")"; + } + } + + /** + * Message to insert/update a reservation. + */ + @Value + @Builder + public static class UpsertReservation { + ReservationKey reservationKey; + SchedulingConstraints schedulingConstraints; + Set allocationRequests; + int stageTargetSize; + ReservationPriority priority; + } + + /** + * Message to cancel a reservation. + */ + @Value + @Builder + public static class CancelReservation { + ReservationKey reservationKey; + } + + /** + * Response to reservation cancellation. + */ + @Value + public static class CancelReservationAck { + ReservationKey reservationKey; + boolean cancelled; + } + + /** + * Marker message to indicate registry is ready to process reservations. + */ + public enum MarkReady { + INSTANCE + } +} + diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/JobClustersManagerActor.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/JobClustersManagerActor.java index 0091b8029..00b2ebbd6 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/JobClustersManagerActor.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/JobClustersManagerActor.java @@ -410,6 +410,23 @@ private void initialize(JobClustersManagerInitialize initMsg) { }, () -> { logger.info("JobClusterManagerActor transitioning to initialized behavior"); getContext().become(initializedBehavior); + + // Mark all reservation registries as ready. + // This step is redundant in current setup as resource cluster actors won't be created till + // the job cluster level init finish and there is nothing at this point of startup to send + // markReady signal to. Consider whether to keep this logic as future-proof in case the + // lifecycle between job clusters and resource clusters changed. + if (mantisSchedulerFactory != null) { + mantisSchedulerFactory.markAllRegistriesReady() + .whenComplete((ack, ex) -> { + if (ex != null) { + logger.error("Failed to mark reservation registries as ready", ex); + } else { + logger.info("All existing reservation registries marked ready"); + } + }); + } + sender.tell(new JobClustersManagerInitializeResponse(initMsg.requestId, SUCCESS, "JobClustersManager successfully inited"), getSelf()); }); diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/job/JobActor.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/job/JobActor.java index 294458839..239583763 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/job/JobActor.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/job/JobActor.java @@ -31,6 +31,8 @@ import static java.util.Optional.of; import static java.util.Optional.ofNullable; +import static akka.pattern.Patterns.pipe; + import akka.actor.AbstractActorWithTimers; import akka.actor.ActorRef; import akka.actor.PoisonPill; @@ -100,7 +102,19 @@ import io.mantisrx.server.master.persistence.exceptions.InvalidJobException; import io.mantisrx.server.master.persistence.exceptions.InvalidWorkerStateChangeException; import io.mantisrx.server.master.resourcecluster.ClusterID; -import io.mantisrx.server.master.scheduler.*; +import io.mantisrx.server.master.resourcecluster.TaskExecutorAllocationRequest; +import io.mantisrx.server.master.resourcecluster.proto.MantisResourceClusterReservationProto; +import io.mantisrx.server.master.resourcecluster.proto.MantisResourceClusterReservationProto.CancelReservation; +import io.mantisrx.server.master.resourcecluster.proto.MantisResourceClusterReservationProto.ReservationKey; +import io.mantisrx.server.master.resourcecluster.proto.MantisResourceClusterReservationProto.ReservationPriority; +import io.mantisrx.server.master.resourcecluster.proto.MantisResourceClusterReservationProto.ReservationPriority.PriorityType; +import io.mantisrx.server.master.resourcecluster.proto.MantisResourceClusterReservationProto.UpsertReservation; +import io.mantisrx.server.master.scheduler.BatchScheduleRequest; +import io.mantisrx.server.master.scheduler.MantisScheduler; +import io.mantisrx.server.master.scheduler.ScheduleRequest; +import io.mantisrx.server.master.scheduler.WorkerEvent; +import io.mantisrx.server.master.scheduler.WorkerOnDisabledVM; +import io.mantisrx.server.master.scheduler.WorkerUnscheduleable; import io.mantisrx.shaded.com.fasterxml.jackson.databind.ObjectMapper; import io.mantisrx.shaded.com.google.common.base.Preconditions; import io.mantisrx.shaded.com.google.common.cache.Cache; @@ -114,6 +128,8 @@ import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; + +import lombok.extern.slf4j.Slf4j; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import rx.Observable; @@ -131,6 +147,10 @@ public class JobActor extends AbstractActorWithTimers implements IMantisJobManag private static final String CHECK_HB_TIMER_KEY = "CHECK_HB"; private static final String REFRESH_SEND_STAGE_ASSIGNEMNTS_KEY = "REFRESH_SEND_STAGE_ASSIGNMENTS"; + private static final String DELAYED_RESERVATION_TIMER_PREFIX = "DELAYED_RESERVATION_"; + private static final String RESERVATION_RETRY_TIMER_PREFIX = "RESERVATION_RETRY_"; + private static final int MAX_RESERVATION_UPSERT_RETRIES = 3; + private static final long RESERVATION_RETRY_BASE_DELAY_MS = 1000; // 1 second base delay with exponential backoff private static final Logger LOGGER = LoggerFactory.getLogger(JobActor.class); private static final double DEFAULT_JOB_MASTER_CORES = 1; private static final double DEFAULT_JOB_MASTER_MEM = 1024; @@ -146,6 +166,8 @@ public class JobActor extends AbstractActorWithTimers implements IMantisJobManag private final Counter numSchedulingChangesRefreshed; private final Counter numMissingWorkerPorts; private final Counter numPeriodicRefreshSkipped; + private final Counter numReservationUpsertRetries; + private final Counter numReservationUpsertFailures; /** * Behavior after being initialized. @@ -275,6 +297,8 @@ public JobActor( .addCounter("numMissingWorkerPorts") .addCounter("numWorkerMissingHeartbeat") .addCounter("numPeriodicRefreshSkipped") + .addCounter("numReservationUpsertRetries") + .addCounter("numReservationUpsertFailures") .build(); this.metrics = MetricsRegistry.getInstance().registerAndGet(m); this.numWorkerResubmissions = metrics.getCounter("numWorkerResubmissions"); @@ -284,6 +308,8 @@ public JobActor( this.numSchedulingChangesRefreshed = metrics.getCounter("numSchedulingChangesRefreshed"); this.numMissingWorkerPorts = metrics.getCounter("numMissingWorkerPorts"); this.numPeriodicRefreshSkipped = metrics.getCounter("numPeriodicRefreshSkipped"); + this.numReservationUpsertRetries = metrics.getCounter("numReservationUpsertRetries"); + this.numReservationUpsertFailures = metrics.getCounter("numReservationUpsertFailures"); } /** @@ -326,6 +352,7 @@ void initialize(boolean isSubmit) throws Exception { this.mantisScheduler, isSubmit, ConfigurationProvider.getConfig().isBatchSchedulingEnabled(), + ConfigurationProvider.getConfig().isReservationSchedulingEnabled(), this.metrics); long checkAgainInSeconds = getWorkerTimeoutSecs(); @@ -629,6 +656,9 @@ private Receive getActiveBehavior() { .match(JobProto.SendWorkerAssignementsIfChanged.class, this::onSendWorkerAssignments) .match(IJobClusterScalerRuleData.class, this::onScalerRuleDataUpdate) .match(JobClusterScalerRuleProto.GetJobScalerRuleStreamRequest.class, this::onGetJobScalerRuleStreamRequest) + .match(JobProto.DelayedReservationRequest.class, this::onDelayedReservationRequest) + .match(JobProto.ReservationUpsertResult.class, this::onReservationUpsertResult) + .match(JobProto.ReservationRetryRequest.class, this::onReservationRetryRequest) // EXPECTED MESSAGES END// // UNEXPECTED MESSAGES BEGIN // @@ -672,6 +702,9 @@ private Receive getInitializedBehavior() { .match(JobProto.SendWorkerAssignementsIfChanged.class, this::onSendWorkerAssignments) .match(IJobClusterScalerRuleData.class, this::onScalerRuleDataUpdate) .match(JobClusterScalerRuleProto.GetJobScalerRuleStreamRequest.class, this::onGetJobScalerRuleStreamRequest) + .match(JobProto.DelayedReservationRequest.class, this::onDelayedReservationRequest) + .match(JobProto.ReservationUpsertResult.class, this::onReservationUpsertResult) + .match(JobProto.ReservationRetryRequest.class, this::onReservationRetryRequest) // EXPECTED MESSAGES END// @@ -945,6 +978,73 @@ public void onSendWorkerAssignments(final JobProto.SendWorkerAssignementsIfChang this.workerManager.refreshAndSendWorkerAssignments(); } + /** + * Handle delayed reservation request that was scheduled via timer. + * This processes the reservation request when the readyAt time is reached. + */ + public void onDelayedReservationRequest(final JobProto.DelayedReservationRequest r) { + LOGGER.info("Processing delayed reservation request for {} workers (priority={})", + r.workerRequests.size(), r.priorityType); + // Cast to WorkerManager since we know it's always that implementation + ((WorkerManager) workerManager).queueWorkersViaReservation(r.workerRequests, r.priorityType); + } + + /** + * Handle the result of a reservation upsert operation piped back from the scheduler. + * On success: logs completion. On failure: schedules retry with exponential backoff. + * Uses actor timers for retry scheduling so they are automatically cancelled when the actor stops. + */ + public void onReservationUpsertResult(final JobProto.ReservationUpsertResult result) { + if (result.success) { + LOGGER.debug("Reservation upsert succeeded for job {} stage {} (attempt {})", + jobId, result.stageNum, result.attemptNumber); + return; + } + + // Handle failure + LOGGER.warn("Reservation upsert failed for job {} stage {} (attempt {})", + jobId, result.stageNum, result.attemptNumber, result.error); + + if (result.attemptNumber < MAX_RESERVATION_UPSERT_RETRIES) { + // Schedule retry with exponential backoff using actor timers + // Actor timers are automatically cancelled when the actor stops + long delayMs = RESERVATION_RETRY_BASE_DELAY_MS * (1L << (result.attemptNumber - 1)); + + LOGGER.info("Scheduling retry for job {} stage {} in {}ms (attempt {}/{})", + jobId, result.stageNum, delayMs, result.attemptNumber + 1, MAX_RESERVATION_UPSERT_RETRIES); + + String timerKey = RESERVATION_RETRY_TIMER_PREFIX + result.stageNum + "_" + result.attemptNumber; + getTimers().startSingleTimer( + timerKey, + new JobProto.ReservationRetryRequest( + result.workerRequests, + result.priorityType, + result.stageNum, + result.attemptNumber + 1), + Duration.ofMillis(delayMs)); + } else { + LOGGER.error("Reservation upsert failed after {} attempts for job {} stage {}", + MAX_RESERVATION_UPSERT_RETRIES, jobId, result.stageNum, result.error); + numReservationUpsertFailures.increment(); + eventPublisher.publishStatusEvent(new LifecycleEventsProto.JobStatusEvent(ERROR, + String.format("Failed to create reservation for stage %d after %d attempts: %s", + result.stageNum, MAX_RESERVATION_UPSERT_RETRIES, + result.error != null ? result.error.getMessage() : "unknown error"), + getJobId(), getJobState())); + } + } + + /** + * Handle retry request for reservation upsert after delay. + */ + public void onReservationRetryRequest(final JobProto.ReservationRetryRequest request) { + LOGGER.info("Processing reservation retry for job {} stage {} (attempt {}/{})", + jobId, request.stageNum, request.attemptNumber, MAX_RESERVATION_UPSERT_RETRIES); + numReservationUpsertRetries.increment(); + ((WorkerManager) workerManager).upsertReservationForStage( + request.workerRequests, request.priorityType, request.stageNum, request.attemptNumber); + } + public void onScalerRuleDataUpdate(final IJobClusterScalerRuleData ruleData) { LOGGER.info("Job actor {} received new Scaler Rule Data: {}", this.jobId, ruleData); this.scalerRuleInfoBehaviorSubject.onNext(JobScalerRuleInfo.builder() @@ -1281,6 +1381,37 @@ private boolean isAutoscaled(SchedulingInfo schedulingInfo) { return false; } + /** + * Schedule a delayed reservation request using Akka timer. + * Called by WorkerManager when workers need to be queued with a delay. + */ + private void scheduleDelayedReservationTimer( + final List workerRequests, + final PriorityType priorityType, + final long readyAt) { + long delayMs = readyAt - System.currentTimeMillis(); + if (delayMs <= 0) { + // Already past readyAt, queue immediately + // Cast to WorkerManager since we know it's always that implementation + ((WorkerManager) workerManager).queueWorkersViaReservation(workerRequests, priorityType); + return; + } + + // Cast to WorkerManager to access delayedReservationTimerCounter + WorkerManager wm = (WorkerManager) workerManager; + String timerKey = DELAYED_RESERVATION_TIMER_PREFIX + (++wm.delayedReservationTimerCounter); + JobProto.DelayedReservationRequest delayedRequest = new JobProto.DelayedReservationRequest( + workerRequests, priorityType); + + LOGGER.info("Scheduling delayed reservation for {} workers (priority={}) in {}ms", + workerRequests.size(), priorityType, delayMs); + + getTimers().startSingleTimer( + timerKey, + delayedRequest, + Duration.ofMillis(delayMs)); + } + /*package protected*/ /** @@ -1425,8 +1556,10 @@ class WorkerManager implements IWorkerManager { private volatile long lastWorkerTransitionTime = 0; private final boolean batchSchedulingEnabled; + private final boolean useReservationScheduling; private final Counter numWorkerStuckInAccepted; private final Counter numWorkerMissingHeartbeat; + private long delayedReservationTimerCounter = 0; /** * Creates an instance of this class. @@ -1435,11 +1568,14 @@ class WorkerManager implements IWorkerManager { * @param migrationConfig * @param scheduler * @param isSubmit + * @param batchSchedulingEnabled + * @param useReservationScheduling + * @param metrics * @throws Exception */ WorkerManager( IMantisJobManager jobMgr, WorkerMigrationConfig migrationConfig, MantisScheduler scheduler, - boolean isSubmit, boolean batchSchedulingEnabled, Metrics metrics) throws Exception { + boolean isSubmit, boolean batchSchedulingEnabled, boolean useReservationScheduling, Metrics metrics) throws Exception { this.numWorkerStuckInAccepted = metrics.getCounter("numWorkerStuckInAccepted"); this.numWorkerMissingHeartbeat = metrics.getCounter("numWorkerMissingHeartbeat"); @@ -1448,6 +1584,7 @@ class WorkerManager implements IWorkerManager { this.scheduler = scheduler; this.jobMgr = jobMgr; this.batchSchedulingEnabled = batchSchedulingEnabled; + this.useReservationScheduling = useReservationScheduling; migrationStrategy = MigrationStrategyFactory.getStrategy(jobId.getId(), migrationConfig); int noOfStages = mantisJobMetaData.getStageMetadata().size(); if (noOfStages == 1) { @@ -1541,7 +1678,7 @@ private void initializeRunningWorkers() { } if (JobState.isAcceptedState(mantisJobMetaData.getState()) && !workersToSubmit.isEmpty()) { - queueTasks(workersToSubmit, empty()); + queueWorkers(workersToSubmit, PriorityType.NEW_JOB, empty()); } // queue a refresh check after re-init workers. @@ -1679,12 +1816,8 @@ private void submitInitialWorkers() throws Exception { markStageAssignmentsChanged(true); if (!workers.isEmpty()) { - // queue to scheduler - if (batchSchedulingEnabled) { - queueTasks(workers, empty()); - } else { - workers.forEach(this::queueTask); - } + // Always batch queue workers (batchSchedulingEnabled only affects legacy scheduler behavior) + queueWorkers(workers, PriorityType.NEW_JOB, empty()); } } catch (Exception e) { LOGGER.error("Error {} storing workers of job {}", e.getMessage(), jobId.getId(), e); @@ -1692,7 +1825,51 @@ private void submitInitialWorkers() throws Exception { } } - private void queueTasks(final List workerRequests, final Optional readyAt) { + /** + * Unified method to queue workers for scheduling. + * Routes to reservation-based or legacy scheduler based on configuration. + * For reservation-based scheduling with readyAt, schedules a timer to delay the request. + * + * @param workerRequests List of workers to queue + * @param priorityType Priority for reservation-based scheduling (ignored for legacy) + * @param readyAt Optional ready time - for legacy: sets ready time; for reservation: delays request until readyAt + */ + private void queueWorkers( + final List workerRequests, + final PriorityType priorityType, + final Optional readyAt) { + if (useReservationScheduling) { + if (readyAt.isPresent() && readyAt.get() > System.currentTimeMillis()) { + // Schedule delayed reservation request via timer + scheduleDelayedReservation(workerRequests, priorityType, readyAt.get()); + } else { + // Queue immediately + queueWorkersViaReservation(workerRequests, priorityType); + } + } else { + queueWorkersViaLegacyScheduler(workerRequests, readyAt); + } + } + + /** + * Schedule a delayed reservation request using Akka timer. + * The reservation will be submitted when readyAt time is reached. + */ + private void scheduleDelayedReservation( + final List workerRequests, + final PriorityType priorityType, + final long readyAt) { + // Delegate to JobActor to schedule the timer + JobActor.this.scheduleDelayedReservationTimer(workerRequests, priorityType, readyAt); + } + + /** + * Queue workers using legacy batch scheduler. + * readyAt is used to set when workers are ready for scheduling (for rate limiting). + */ + private void queueWorkersViaLegacyScheduler( + final List workerRequests, + final Optional readyAt) { final List scheduleRequests = workerRequests .stream() .map(wR -> createSchedulingRequest(wR, readyAt)) @@ -1705,8 +1882,119 @@ private void queueTasks(final List workerRequests, final } } + /** + * Queue workers via reservation system using shared proto classes directly. + * Groups workers by stage and creates reservations per stage. + * Priority determines scheduling order: REPLACE > SCALE > NEW_JOB + */ + void queueWorkersViaReservation( + final List workerRequests, + final PriorityType priorityType) { + // Group by stage - reservation system operates at stage level + Map> byStage = workerRequests.stream() + .collect(Collectors.groupingBy(IMantisWorkerMetadata::getStageNum)); + + for (Map.Entry> entry : byStage.entrySet()) { + int stageNum = entry.getKey(); + List stageWorkers = entry.getValue(); + + if (stageWorkers.isEmpty()) { + continue; + } + + upsertReservationForStage(stageWorkers, priorityType, stageNum, 1); + } + } + + /** + * Upsert reservation for a single stage with retry support. + * Uses Akka pipe pattern to send result back to actor for proper async handling. + * On failure, the actor schedules a retry with exponential backoff up to MAX_RESERVATION_UPSERT_RETRIES. + */ + void upsertReservationForStage( + final List stageWorkers, + final PriorityType priorityType, + final int stageNum, + final int attemptNumber) { + // Get stage target size from metadata or use current worker count + Optional stageMetaOp = mantisJobMetaData.getStageMetadata(stageNum); + if (stageMetaOp.isEmpty()) { + LOGGER.error("Stage {} not found while creating reservation request", stageNum); + return; + } + + int stageTargetSize = stageMetaOp.map(IMantisStageMetadata::getNumWorkers).orElse(stageWorkers.size()); + + // Create ScheduleRequests once for all workers - reuse for both constraints and allocation requests + List scheduleRequests = stageWorkers.stream() + .map(wm -> createSchedulingRequest(wm, Optional.empty())) + .toList(); + + // Extract scheduling constraints from first request (all workers in same stage share constraints) + SchedulingConstraints schedulingConstraints = scheduleRequests.get(0).getSchedulingConstraints(); + + // Create allocation requests from the schedule requests + Set allocationRequests = scheduleRequests.stream() + .map(sr -> TaskExecutorAllocationRequest.of( + sr.getWorkerId(), + sr.getSchedulingConstraints(), + sr.getJobMetadata(), + sr.getStageNum())) + .collect(Collectors.toSet()); + + // Build reservation request using shared UpsertReservation class + UpsertReservation request = UpsertReservation.builder() + .reservationKey(MantisResourceClusterReservationProto.ReservationKey.builder() + .jobId(jobId.getId()) + .stageNumber(stageNum) + .build()) + .schedulingConstraints(schedulingConstraints) + .allocationRequests(allocationRequests) + .stageTargetSize(stageTargetSize) + .priority(buildPriority(priorityType)) + .build(); + + LOGGER.info("Upserting reservation for job {} stage {} with {} workers (priority={}, attempt={})", + jobId, stageNum, allocationRequests.size(), priorityType, attemptNumber); + + // Use Akka pipe pattern: transform the future result to a message and pipe it to self + // This ensures proper actor message handling instead of callback-based state mutation + java.util.concurrent.CompletableFuture resultFuture = + scheduler.upsertReservation(request) + .handle((ack, ex) -> { + if (ex != null) { + return JobProto.ReservationUpsertResult.failure( + stageWorkers, priorityType, stageNum, attemptNumber, ex); + } else { + return JobProto.ReservationUpsertResult.success( + stageWorkers, priorityType, stageNum, attemptNumber); + } + }); + + pipe(resultFuture, getContext().getDispatcher()).to(getContext().getSelf()); + } + + /** + * Build priority using shared ReservationPriority class. + * Priority tier is derived from job duration type. + */ + private ReservationPriority buildPriority(PriorityType type) { + // Get tier from job SLA (durationType ordinal gives tier) + int tier = mantisJobMetaData.getJobDefinition().getJobSla().getDurationType().ordinal(); + + return ReservationPriority.builder() + .type(type) + .tier(tier) + .timestamp(System.currentTimeMillis()) + .build(); + } + + /** + * Queue a single worker for scheduling. + * Convenience method that wraps single worker in a list. + */ private void queueTask(final IMantisWorkerMetadata workerRequest) { - queueTasks(Collections.singletonList(workerRequest), empty()); + queueWorkers(Collections.singletonList(workerRequest), PriorityType.NEW_JOB, empty()); } private ScheduleRequest createSchedulingRequest( @@ -1895,7 +2183,30 @@ private IMantisWorkerMetadata addWorker(SchedulingInfo schedulingInfo, int stage @Override public void shutdown() { - scheduler.unscheduleJob(jobId.getId()); + if (useReservationScheduling) { + // Cancel all pending reservations for each stage using shared CancelReservation class + for (IMantisStageMetadata stage : mantisJobMetaData.getStageMetadata().values()) { + CancelReservation cancelRequest = CancelReservation.builder() + .reservationKey(ReservationKey.builder() + .jobId(jobId.getId()) + .stageNumber(stage.getStageNum()) + .build()) + .build(); + + scheduler.cancelReservation(cancelRequest) + .whenComplete((resp, ex) -> { + if (ex != null) { + LOGGER.warn("Failed to cancel reservation for job {} stage {}", + jobId, stage.getStageNum(), ex); + } else { + LOGGER.debug("Cancelled reservation for job {} stage {}", + jobId, stage.getStageNum()); + } + }); + } + } else { + scheduler.unscheduleJob(jobId.getId()); + } // if workers have not already completed if (!allWorkerCompleted()) { // kill workers @@ -2202,7 +2513,6 @@ private void terminateUnknownWorkerIfNonTerminal(final WorkerEvent event) { @Override public void processEvent(WorkerEvent event, JobState jobState) { try { - Optional stageMetaOp = getStageForWorker(event); if (!stageMetaOp.isPresent()) { terminateUnknownWorkerIfNonTerminal(event); @@ -2283,8 +2593,8 @@ public void processEvent(WorkerEvent event, JobState jobState) { if (!(event instanceof WorkerHeartbeat)) { // Use immediate refresh for terminal worker events in running jobs for faster recovery - boolean immediateRefresh = (event instanceof WorkerTerminate || - WorkerState.isTerminalState(wm.getState())) + boolean immediateRefresh = (event instanceof WorkerTerminate || + WorkerState.isTerminalState(wm.getState())) && jobState == JobState.Launched; markStageAssignmentsChanged(immediateRefresh); } @@ -2444,13 +2754,18 @@ private void resubmitWorker(JobWorker oldWorker) throws Exception { oldWorkerMetadata.getWorkerId(), Optional.ofNullable(oldWorkerMetadata.getSlave())); - long workerResubmitTime = resubmitRateLimiter.getWorkerResubmitTime( - newWorker.getMetadata().getWorkerId(), stageMeta.getStageNum()); - Optional delayDuration = of(workerResubmitTime); // publish a refresh before enqueuing new Task to Scheduler markStageAssignmentsChanged(true); // queue the new worker for execution - queueTasks(Collections.singletonList(newWorker.getMetadata()), delayDuration); + // Use REPLACE priority (highest) for worker resubmission in reservation system + // apply rate limiting delay + long workerResubmitTime = resubmitRateLimiter.getWorkerResubmitTime( + newWorker.getMetadata().getWorkerId(), stageMeta.getStageNum()); + Optional readyAt = of(workerResubmitTime); + queueWorkers( + Collections.singletonList(newWorker.getMetadata()), + PriorityType.REPLACE, + readyAt); LOGGER.info("Worker {} successfully queued for scheduling", newWorker); numWorkerResubmissions.increment(); } else { @@ -2502,6 +2817,7 @@ public int scaleStage( getJobId(), getJobState())); throw new RuntimeException(error); } + List workerRequests = new ArrayList<>(); if (newNumWorkerCount > oldNumWorkers) { for (int i = 0; i < newNumWorkerCount - oldNumWorkers; i++) { try { @@ -2510,7 +2826,8 @@ public int scaleStage( IMantisWorkerMetadata workerRequest = addWorker(schedInfo, stageMetaData.getStageNum(), newWorkerIndex); jobStore.storeNewWorker(workerRequest); - queueTask(workerRequest); + markStageAssignmentsChanged(true); + workerRequests.add(workerRequest); } catch (Exception e) { // creating a worker failed but expected no of workers was set successfully, // during heartbeat check we will @@ -2518,6 +2835,9 @@ public int scaleStage( LOGGER.warn("Exception adding new worker for {}", stageMetaData.getJobId().getId(), e); } } + // Queue all new workers + // Use SCALE priority (medium) for scaling operations in reservation system + queueWorkers(workerRequests, PriorityType.SCALE, empty()); } else { // potential bulk removal opportunity? for (int i = 0; i < oldNumWorkers - newNumWorkerCount; i++) { diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/job/worker/JobWorker.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/job/worker/JobWorker.java index 929a6dc5b..e3ea80666 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/job/worker/JobWorker.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/job/worker/JobWorker.java @@ -249,10 +249,7 @@ private boolean onTerminate(WorkerTerminate workerEvent) throws InvalidWorkerSta * our state doesn't match Mesos) */ private boolean onWorkerLaunched(WorkerLaunched workerEvent) throws InvalidWorkerStateChangeException { - if (LOGGER.isDebugEnabled()) { - LOGGER.debug("Processing for worker {} with id {}", workerEvent, metadata.getWorkerId()); - } - + LOGGER.info("JobActor: on WorkerLaunched for {} on {}", workerEvent, metadata.getWorkerId()); setSlave(workerEvent.getHostname()); addPorts(workerEvent.getPorts()); setSlaveID(workerEvent.getVmId()); diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobProto.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobProto.java index e9e8f2e45..dca1a5177 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobProto.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobProto.java @@ -17,8 +17,11 @@ package io.mantisrx.master.jobcluster.proto; import akka.actor.ActorRef; +import io.mantisrx.master.jobcluster.job.worker.IMantisWorkerMetadata; import io.mantisrx.server.master.domain.JobId; +import io.mantisrx.server.master.resourcecluster.proto.MantisResourceClusterReservationProto.ReservationPriority.PriorityType; import java.time.Instant; +import java.util.List; public class JobProto { @@ -118,6 +121,116 @@ public MigrateDisabledVmWorkersRequest() { } } + /** + * Message for delayed reservation requests. + * Used when workers need to be queued via reservation system but with a delay (rate limiting). + */ + public static final class DelayedReservationRequest { + public final List workerRequests; + public final PriorityType priorityType; + + public DelayedReservationRequest(List workerRequests, PriorityType priorityType) { + this.workerRequests = workerRequests; + this.priorityType = priorityType; + } + + @Override + public String toString() { + return "DelayedReservationRequest{" + + "workerCount=" + (workerRequests != null ? workerRequests.size() : 0) + + ", priorityType=" + priorityType + + '}'; + } + } + + /** + * Result of a reservation upsert operation, piped back to the JobActor. + * Contains either success (Ack) or failure (exception) along with context for retry. + */ + public static final class ReservationUpsertResult { + public final List workerRequests; + public final PriorityType priorityType; + public final int stageNum; + public final int attemptNumber; + public final boolean success; + public final Throwable error; + + private ReservationUpsertResult( + List workerRequests, + PriorityType priorityType, + int stageNum, + int attemptNumber, + boolean success, + Throwable error) { + this.workerRequests = workerRequests; + this.priorityType = priorityType; + this.stageNum = stageNum; + this.attemptNumber = attemptNumber; + this.success = success; + this.error = error; + } + + public static ReservationUpsertResult success( + List workerRequests, + PriorityType priorityType, + int stageNum, + int attemptNumber) { + return new ReservationUpsertResult(workerRequests, priorityType, stageNum, attemptNumber, true, null); + } + + public static ReservationUpsertResult failure( + List workerRequests, + PriorityType priorityType, + int stageNum, + int attemptNumber, + Throwable error) { + return new ReservationUpsertResult(workerRequests, priorityType, stageNum, attemptNumber, false, error); + } + + @Override + public String toString() { + return "ReservationUpsertResult{" + + "workerCount=" + (workerRequests != null ? workerRequests.size() : 0) + + ", priorityType=" + priorityType + + ", stageNum=" + stageNum + + ", attemptNumber=" + attemptNumber + + ", success=" + success + + ", error=" + (error != null ? error.getMessage() : "null") + + '}'; + } + } + + /** + * Message to trigger a retry of reservation upsert after a delay. + * Sent to self via Akka scheduler when a reservation upsert fails. + */ + public static final class ReservationRetryRequest { + public final List workerRequests; + public final PriorityType priorityType; + public final int stageNum; + public final int attemptNumber; + + public ReservationRetryRequest( + List workerRequests, + PriorityType priorityType, + int stageNum, + int attemptNumber) { + this.workerRequests = workerRequests; + this.priorityType = priorityType; + this.stageNum = stageNum; + this.attemptNumber = attemptNumber; + } + + @Override + public String toString() { + return "ReservationRetryRequest{" + + "workerCount=" + (workerRequests != null ? workerRequests.size() : 0) + + ", priorityType=" + priorityType + + ", stageNum=" + stageNum + + ", attemptNumber=" + attemptNumber + + '}'; + } + } public static class SelfDestructRequest { } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/AssignmentHandlerActor.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/AssignmentHandlerActor.java new file mode 100644 index 000000000..855271cbf --- /dev/null +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/AssignmentHandlerActor.java @@ -0,0 +1,361 @@ +package io.mantisrx.master.resourcecluster; + +import akka.actor.AbstractActorWithTimers; +import akka.actor.Props; +import akka.pattern.Patterns; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.TaskExecutorGatewayRequest; +import io.mantisrx.server.master.resourcecluster.ClusterID; +import io.mantisrx.server.master.resourcecluster.TaskExecutorAllocationRequest; +import io.mantisrx.server.master.resourcecluster.TaskExecutorID; +import io.mantisrx.server.master.resourcecluster.TaskExecutorRegistration; +import io.mantisrx.server.master.ExecuteStageRequestFactory; +import io.mantisrx.server.master.scheduler.JobMessageRouter; +import io.mantisrx.server.worker.TaskExecutorGateway; +import io.mantisrx.shaded.com.fasterxml.jackson.annotation.JsonCreator; +import io.mantisrx.shaded.com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.flink.util.ExceptionUtils; +import java.time.Duration; +import java.time.Instant; +import java.util.Objects; +import java.util.concurrent.CompletableFuture; +import javax.annotation.Nullable; +import lombok.Value; +import lombok.extern.slf4j.Slf4j; +import scala.compat.java8.FutureConverters; + +@Slf4j +public class AssignmentHandlerActor extends AbstractActorWithTimers { + + private final ClusterID clusterID; + private final JobMessageRouter jobMessageRouter; + private final Duration assignmentTimeout; + private final int maxAssignmentRetries; + private final Duration intervalBetweenRetries; + private final ExecuteStageRequestFactory executeStageRequestFactory; + + public static Props props( + ClusterID clusterID, + JobMessageRouter jobMessageRouter, + Duration assignmentTimeout, + ExecuteStageRequestFactory executeStageRequestFactory + ) { + return props(clusterID, jobMessageRouter, assignmentTimeout, executeStageRequestFactory, 3, assignmentTimeout); + } + + public static Props props( + ClusterID clusterID, + JobMessageRouter jobMessageRouter, + Duration assignmentTimeout, + ExecuteStageRequestFactory executeStageRequestFactory, + int maxAssignmentRetries, + Duration intervalBetweenRetries + ) { + Objects.requireNonNull(clusterID, "clusterID"); + Objects.requireNonNull(jobMessageRouter, "jobMessageRouter"); + Objects.requireNonNull(assignmentTimeout, "assignmentTimeout"); + Objects.requireNonNull(executeStageRequestFactory, "executeStageRequestFactory"); + return Props.create( + AssignmentHandlerActor.class, + clusterID, + jobMessageRouter, + assignmentTimeout, + executeStageRequestFactory, + maxAssignmentRetries, + intervalBetweenRetries + ); + } + + AssignmentHandlerActor( + ClusterID clusterID, + JobMessageRouter jobMessageRouter, + Duration assignmentTimeout, + ExecuteStageRequestFactory executeStageRequestFactory, + int maxAssignmentRetries, + Duration intervalBetweenRetries + ) { + this.clusterID = clusterID; + this.jobMessageRouter = jobMessageRouter; + this.assignmentTimeout = assignmentTimeout; + this.executeStageRequestFactory = executeStageRequestFactory; + this.maxAssignmentRetries = maxAssignmentRetries; + this.intervalBetweenRetries = intervalBetweenRetries; + } + + @Override + public Receive createReceive() { + return receiveBuilder() + .match(TaskExecutorAssignmentRequest.class, this::onTaskExecutorAssignmentRequest) + .match(TaskExecutorAssignmentSucceededEvent.class, this::onAssignmentSucceeded) + .match(TaskExecutorAssignmentFailedEvent.class, this::onAssignmentFailed) + .match(ScheduleRetryWithFreshGateway.class, this::onScheduleRetryWithFreshGateway) + .build(); + } + + private void onTaskExecutorAssignmentRequest(TaskExecutorAssignmentRequest request) { + log.info("Received task executor assignment request: {} (attempt {}/{})", + request, request.getAttempt(), maxAssignmentRetries); + try { + TaskExecutorRegistration registration = request.getRegistration(); + // Use the gateway future from the request + CompletableFuture gatewayFut = request.getGatewayFuture(); + + CompletableFuture ackFuture = + gatewayFut + .thenComposeAsync(gateway -> { + log.debug("Successfully obtained gateway for task executor {}", + registration.getTaskExecutorID()); + return gateway + .submitTask( + executeStageRequestFactory.of( + registration, + request.getAllocationRequest())) + .thenApplyAsync( + dontCare -> { + log.debug("[Submit Task] succeeded for {}", registration.getTaskExecutorID()); + return new TaskExecutorAssignmentSucceededEvent(request); + }) + .exceptionally( + throwable -> { + log.error("[Submit Task] failed for {}: {}", + registration.getTaskExecutorID(), throwable.getMessage()); + return new TaskExecutorAssignmentFailedEvent( + request, ExceptionUtils.stripCompletionException(throwable)); + }); + }) + .exceptionally(throwable -> { + log.warn("Failed to obtain gateway for task executor {}", + registration.getTaskExecutorID(), throwable); + return new TaskExecutorAssignmentFailedEvent( + request, + ExceptionUtils.stripCompletionException(throwable)); + }) + .toCompletableFuture() + .orTimeout( + assignmentTimeout.toMillis(), + java.util.concurrent.TimeUnit.MILLISECONDS) + .exceptionally(throwable -> { + if (throwable instanceof java.util.concurrent.TimeoutException) { + log.warn("Assignment timeout for task executor {} after {}ms", + registration.getTaskExecutorID(), assignmentTimeout.toMillis()); + return new TaskExecutorAssignmentFailedEvent( + request, + throwable); + } + return new TaskExecutorAssignmentFailedEvent( + request, + ExceptionUtils.stripCompletionException(throwable)); + }); + + akka.pattern.Patterns.pipe(ackFuture, getContext().getDispatcher()).to(self()); + } catch (Exception e) { + log.error("Exception during task executor assignment for {}", + request.getRegistration().getTaskExecutorID(), e); + self().tell(new TaskExecutorAssignmentFailedEvent(request, e), self()); + } + } + + private void onAssignmentSucceeded(TaskExecutorAssignmentSucceededEvent event) { + TaskExecutorAssignmentRequest request = event.getRequest(); + TaskExecutorRegistration registration = request.getRegistration(); + log.info("Task executor assignment succeeded for {}", registration.getTaskExecutorID()); + } + + private void onAssignmentFailed(TaskExecutorAssignmentFailedEvent event) { + TaskExecutorAssignmentRequest request = event.getRequest(); + TaskExecutorRegistration registration = request.getRegistration(); + + log.warn("Task executor assignment failed for {} (attempt {}/{}: {})", + registration.getTaskExecutorID(), + request.getAttempt(), + maxAssignmentRetries, + event.getThrowable().getMessage()); + + if (request.getAttempt() >= maxAssignmentRetries) { + log.error("Assignment failed for {} after {} attempts, giving up", + registration.getTaskExecutorID(), maxAssignmentRetries); + + // Send assignmentFailure event to parent after max retries + getContext().parent().tell(new TaskExecutorAssignmentFailAndTerminate( + registration.getTaskExecutorID(), + request.getAllocationRequest(), + event.getThrowable(), + request.getAttempt() + ), self()); + } else { + log.info("Retrying assignment for {} in {} (attempt {}/{})", + registration.getTaskExecutorID(), intervalBetweenRetries, request.getAttempt(), maxAssignmentRetries); + + // Request a fresh gateway future from the parent actor to avoid reusing a failed future + TaskExecutorGatewayRequest gatewayRequest = new TaskExecutorGatewayRequest( + registration.getTaskExecutorID(), + clusterID + ); + + CompletableFuture gatewayFutureResponse = FutureConverters.toJava( + Patterns.ask(getContext().parent(), gatewayRequest, assignmentTimeout.toMillis()) + ).toCompletableFuture(); + + // Pipe the result back to self as a message to schedule the retry + CompletableFuture retryScheduleFuture = gatewayFutureResponse + .thenApply(result -> { + if (result instanceof CompletableFuture) { + @SuppressWarnings("unchecked") + CompletableFuture gatewayFuture = + (CompletableFuture) result; + return new ScheduleRetryWithFreshGateway(request, gatewayFuture); + } else if (result instanceof Throwable) { + log.error("Failed to get fresh gateway for retry: {}", result); + // Create a failed future to avoid reusing the old one + CompletableFuture failedFuture = new CompletableFuture<>(); + failedFuture.completeExceptionally((Throwable) result); + return new ScheduleRetryWithFreshGateway(request, failedFuture); + } else { + log.error("Unexpected response type when requesting gateway: {}", result.getClass()); + CompletableFuture failedFuture = new CompletableFuture<>(); + failedFuture.completeExceptionally( + new RuntimeException("Unexpected response type: " + result.getClass())); + return new ScheduleRetryWithFreshGateway(request, failedFuture); + } + }) + .exceptionally(throwable -> { + log.error("Exception while requesting fresh gateway for retry: {}", throwable.getMessage(), throwable); + // Fallback: create retry request with a new failed future + // This ensures we don't reuse the old failed future + CompletableFuture failedFuture = new CompletableFuture<>(); + failedFuture.completeExceptionally(throwable); + return new ScheduleRetryWithFreshGateway(request, failedFuture); + }); + + Patterns.pipe(retryScheduleFuture, getContext().getDispatcher()).to(self()); + } + } + + private void onScheduleRetryWithFreshGateway(ScheduleRetryWithFreshGateway message) { + TaskExecutorAssignmentRequest originalRequest = message.getOriginalRequest(); + TaskExecutorRegistration registration = originalRequest.getRegistration(); + + TaskExecutorAssignmentRequest retryRequest = originalRequest.onRetry(message.getFreshGatewayFuture()); + + getTimers().startSingleTimer( + getRetryTimerKeyFor(registration.getTaskExecutorID()), + retryRequest, + intervalBetweenRetries + ); + } + + private String getRetryTimerKeyFor(TaskExecutorID taskExecutorID) { + return "Retry-" + taskExecutorID.getResourceId(); + } + + // Event classes + + @Value + public static class TaskExecutorAssignmentRequest { + TaskExecutorAllocationRequest allocationRequest; + TaskExecutorID taskExecutorID; + TaskExecutorRegistration registration; + CompletableFuture gatewayFuture; + int attempt; + + /* + Deprecated field. + */ + @Deprecated + @Nullable + Throwable previousFailure; + Instant requestTime; + + @JsonCreator + public TaskExecutorAssignmentRequest( + @JsonProperty("allocationRequest") TaskExecutorAllocationRequest allocationRequest, + @JsonProperty("taskExecutorID") TaskExecutorID taskExecutorID, + @JsonProperty("registration") TaskExecutorRegistration registration, + @JsonProperty("gatewayFuture") CompletableFuture gatewayFuture, + @JsonProperty("attempt") int attempt, + @JsonProperty("previousFailure") @Nullable Throwable previousFailure, + @JsonProperty("requestTime") Instant requestTime + ) { + this.allocationRequest = allocationRequest; + this.taskExecutorID = taskExecutorID; + this.registration = registration; + this.gatewayFuture = gatewayFuture; + this.attempt = attempt; + this.previousFailure = previousFailure; + this.requestTime = requestTime; + } + + public static TaskExecutorAssignmentRequest of( + TaskExecutorAllocationRequest allocationRequest, + TaskExecutorID taskExecutorID, + TaskExecutorRegistration registration, + CompletableFuture gatewayFuture + ) { + return new TaskExecutorAssignmentRequest( + allocationRequest, + taskExecutorID, + registration, + gatewayFuture, + 1, + null, + Instant.now() + ); + } + + /** + * Creates a retry request with a fresh gateway future to avoid reusing a failed future. + * This prevents race conditions where a failed gateway future from a previous attempt + * would cause all retries to immediately fail. + * + * @param freshGatewayFuture A new gateway future obtained from TaskExecutorState.getGatewayAsync() + * @return A new TaskExecutorAssignmentRequest with incremented attempt count and fresh gateway future + */ + public TaskExecutorAssignmentRequest onRetry(CompletableFuture freshGatewayFuture) { + return new TaskExecutorAssignmentRequest( + allocationRequest, + taskExecutorID, + registration, + freshGatewayFuture, // Use fresh future instead of reusing the old one + attempt + 1, + null, + requestTime + ); + } + + public boolean isRetry() { + return attempt > 1; + } + + public io.mantisrx.server.core.domain.WorkerId getWorkerId() { + return allocationRequest.getWorkerId(); + } + + public int getStageNum() { + return allocationRequest.getStageNum(); + } + } + + @Value + private static class TaskExecutorAssignmentSucceededEvent { + TaskExecutorAssignmentRequest request; + } + + @Value + private static class TaskExecutorAssignmentFailedEvent { + TaskExecutorAssignmentRequest request; + Throwable throwable; + } + + @Value + public static class TaskExecutorAssignmentFailAndTerminate { + TaskExecutorID taskExecutorID; + TaskExecutorAllocationRequest allocationRequest; + Throwable throwable; + int attemptCount; + } + + @Value + private static class ScheduleRetryWithFreshGateway { + TaskExecutorAssignmentRequest originalRequest; + CompletableFuture freshGatewayFuture; + } +} diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ExecutorStateManager.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ExecutorStateManager.java index 3e1bce7d9..c3949ec71 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ExecutorStateManager.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ExecutorStateManager.java @@ -19,15 +19,19 @@ import io.mantisrx.master.resourcecluster.ResourceClusterActor.BestFit; import io.mantisrx.master.resourcecluster.ResourceClusterActor.GetActiveJobsRequest; import io.mantisrx.master.resourcecluster.ResourceClusterActor.GetClusterUsageRequest; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.PendingReservationInfo; import io.mantisrx.master.resourcecluster.ResourceClusterActor.TaskExecutorBatchAssignmentRequest; import io.mantisrx.master.resourcecluster.proto.GetClusterIdleInstancesRequest; import io.mantisrx.master.resourcecluster.proto.GetClusterUsageResponse; +import io.mantisrx.server.master.resourcecluster.ClusterID; import io.mantisrx.server.master.resourcecluster.ResourceCluster.ResourceOverview; import io.mantisrx.server.master.resourcecluster.TaskExecutorID; +import io.mantisrx.server.master.resourcecluster.TaskExecutorRegistration; import java.util.List; import java.util.Map.Entry; import java.util.Optional; import java.util.Set; +import java.util.function.Function; import java.util.function.Predicate; import javax.annotation.Nullable; @@ -69,6 +73,20 @@ interface ExecutorStateManager { GetClusterUsageResponse getClusterUsage(GetClusterUsageRequest req); + /** + * Compute cluster usage with pending reservation counts. + * Uses actual SchedulingConstraints from reservations to map them to SKUs. + * + * @param clusterID The cluster ID + * @param groupKeyFunc Function to extract SKU key from TaskExecutorRegistration + * @param pendingReservations Pending reservations with actual SchedulingConstraints + * @return Usage response with pending reservation counts per SKU + */ + GetClusterUsageResponse getClusterUsageWithReservations( + ClusterID clusterID, + Function> groupKeyFunc, + List pendingReservations); + List getIdleInstanceList(GetClusterIdleInstancesRequest req); List getTaskExecutors(Predicate> predicate); diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ExecutorStateManagerActor.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ExecutorStateManagerActor.java new file mode 100644 index 000000000..a0002b585 --- /dev/null +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ExecutorStateManagerActor.java @@ -0,0 +1,1124 @@ +package io.mantisrx.master.resourcecluster; + +import akka.actor.AbstractActorWithTimers; +import akka.actor.ActorRef; +import akka.actor.Props; +import akka.actor.Status; +import com.netflix.spectator.api.Tag; +import com.netflix.spectator.api.TagList; +import io.mantisrx.common.Ack; +import io.mantisrx.common.WorkerConstants; +import io.mantisrx.server.master.resourcecluster.ResourceCluster.NoResourceAvailableException; +import io.mantisrx.server.master.resourcecluster.ResourceCluster.TaskExecutorNotFoundException; +import io.mantisrx.server.master.resourcecluster.ResourceCluster.ResourceOverview; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.AddNewJobArtifactsToCacheRequest; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.ArtifactList; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.BestFit; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.CacheJobArtifactsOnTaskExecutorRequest; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.CheckDisabledTaskExecutors; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.GetActiveJobsRequest; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.GetAssignedTaskExecutorRequest; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.GetAvailableTaskExecutorsRequest; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.GetBusyTaskExecutorsRequest; +import io.mantisrx.master.resourcecluster.proto.GetClusterIdleInstancesRequest; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.GetClusterUsageRequest; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.GetClusterUsageWithReservationsRequest; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.GetDisabledTaskExecutorsRequest; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.GetJobArtifactsToCacheRequest; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.GetRegisteredTaskExecutorsRequest; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.GetTaskExecutorStatusRequest; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.GetTaskExecutorWorkerMappingRequest; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.GetUnregisteredTaskExecutorsRequest; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.HeartbeatTimeout; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.InitializeTaskExecutorRequest; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.MarkExecutorTaskCancelledRequest; +import io.mantisrx.server.master.resourcecluster.PagedActiveJobOverview; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.PublishResourceOverviewMetricsRequest; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.RemoveJobArtifactsToCacheRequest; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.ResourceOverviewRequest; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.TaskExecutorBatchAssignmentRequest; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.TaskExecutorGatewayRequest; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.TaskExecutorInfoRequest; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.TaskExecutorsAllocation; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.TaskExecutorsList; +import io.mantisrx.master.resourcecluster.proto.GetClusterIdleInstancesResponse; +import io.mantisrx.master.scheduler.FitnessCalculator; +import io.mantisrx.server.core.CacheJobArtifactsRequest; +import io.mantisrx.server.core.domain.ArtifactID; +import io.mantisrx.server.core.domain.WorkerId; +import io.mantisrx.server.master.persistence.MantisJobStore; +import io.mantisrx.server.master.resourcecluster.ClusterID; +import io.mantisrx.server.master.resourcecluster.TaskExecutorAllocationRequest; +import io.mantisrx.server.master.resourcecluster.TaskExecutorDisconnection; +import io.mantisrx.server.master.resourcecluster.TaskExecutorHeartbeat; +import io.mantisrx.server.master.resourcecluster.TaskExecutorID; +import io.mantisrx.server.master.resourcecluster.TaskExecutorRegistration; +import io.mantisrx.server.master.resourcecluster.TaskExecutorReport; +import io.mantisrx.server.master.resourcecluster.ResourceCluster.TaskExecutorStatus; +import io.mantisrx.server.master.ExecuteStageRequestFactory; +import io.mantisrx.server.master.resourcecluster.TaskExecutorStatusChange; +import io.mantisrx.server.master.scheduler.JobMessageRouter; +import io.mantisrx.server.master.scheduler.WorkerLaunchFailed; +import io.mantisrx.server.master.scheduler.WorkerLaunched; +import io.mantisrx.server.worker.TaskExecutorGateway; +import io.mantisrx.server.worker.TaskExecutorGateway.TaskNotFoundException; +import io.mantisrx.shaded.com.google.common.base.Preconditions; +import io.mantisrx.shaded.com.google.common.collect.ImmutableMap; +import io.mantisrx.shaded.com.google.common.collect.Comparators; +import java.io.IOException; +import java.net.URI; +import java.time.Clock; +import java.time.Duration; +import java.time.Instant; +import java.util.AbstractMap; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import javax.annotation.Nullable; +import lombok.Value; +import lombok.extern.slf4j.Slf4j; +import org.apache.flink.runtime.rpc.RpcService; + + +/** + * [Changes TODO] + * - add a child actor "AssignmentHandlerActor" which takes props of clusterID/jobMessageRouter/assignmentTimeout + * - in child actor AssignmentHandlerActor support behavior "onTaskExecutorAssignementRequest" which gets a message + * from the parent actor and contains the future to connect to TE gateway and TE registration info. The actual logic + * is the same as onAssignedScheduleRequestEvent in ResourceClusterAwareSchedulerActor and the retry should be + * treated the same. + * - on successful assignment, route message to JobActor to notify worker starting state. + */ + +/** + * Akka actor wrapper around {@link ExecutorStateManager}. The actor provides an asynchronous façade that mirrors the + * current synchronous API so callers can be migrated incrementally. Once fully integrated, the actor will be the single + * owner of executor state mutations as described in {@code plan-reservation-registry-v3.md}. + */ +@Slf4j +public class ExecutorStateManagerActor extends AbstractActorWithTimers { + + @Value + static class ExpireDisableTaskExecutorsRequest { + DisableTaskExecutorsRequest request; + } + + @Value + static class UpdateDisabledState { + Set attributeRequests; + Set disabledExecutors; + } + + @Value + static class UpdateJobArtifactsToCache { + Set artifacts; + } + + static class RefreshTaskExecutorJobArtifactCache { } + + private final ExecutorStateManagerImpl delegate; + private final Clock clock; + private final RpcService rpcService; + private final JobMessageRouter jobMessageRouter; + private final MantisJobStore mantisJobStore; + private final Duration heartbeatTimeout; + private final Duration assignmentTimeout; + private final Duration disabledTaskExecutorsCheckInterval; + private final ClusterID clusterID; + private final boolean isJobArtifactCachingEnabled; + private final ResourceClusterActorMetrics metrics; + private final String jobClustersWithArtifactCachingEnabled; + private final Set activeDisableTaskExecutorsByAttributesRequests; + private final Set disabledTaskExecutors; + private final Set jobArtifactsToCache; + private final ActorRef assignmentHandlerActor; + private final ExecuteStageRequestFactory executeStageRequestFactory; + + // NEW: Flag to enable reservation-aware usage computation + private final boolean reservationSchedulingEnabled; + + public static Props props( + Map schedulingAttributes, + FitnessCalculator fitnessCalculator, + Duration schedulerLeaseExpirationDuration, + @Nullable AvailableTaskExecutorMutatorHook availableTaskExecutorMutatorHook, + Clock clock, + RpcService rpcService, + JobMessageRouter jobMessageRouter, + MantisJobStore mantisJobStore, + Duration heartbeatTimeout, + Duration assignmentTimeout, + Duration disabledTaskExecutorsCheckInterval, + ClusterID clusterID, + boolean isJobArtifactCachingEnabled, + String jobClustersWithArtifactCachingEnabled, + ResourceClusterActorMetrics metrics, + ExecuteStageRequestFactory executeStageRequestFactory, + boolean reservationSchedulingEnabled + ) { + Objects.requireNonNull(schedulingAttributes, "schedulingAttributes"); + Objects.requireNonNull(fitnessCalculator, "fitnessCalculator"); + Objects.requireNonNull(schedulerLeaseExpirationDuration, "schedulerLeaseExpirationDuration"); + Objects.requireNonNull(clock, "clock"); + Objects.requireNonNull(rpcService, "rpcService"); + Objects.requireNonNull(jobMessageRouter, "jobMessageRouter"); + Objects.requireNonNull(mantisJobStore, "mantisJobStore"); + Objects.requireNonNull(heartbeatTimeout, "heartbeatTimeout"); + Objects.requireNonNull(assignmentTimeout, "assignmentTimeout"); + Objects.requireNonNull(disabledTaskExecutorsCheckInterval, "disabledTaskExecutorsCheckInterval"); + Objects.requireNonNull(clusterID, "clusterID"); + Objects.requireNonNull(jobClustersWithArtifactCachingEnabled, "jobClustersWithArtifactCachingEnabled"); + Objects.requireNonNull(metrics, "metrics"); + Objects.requireNonNull(executeStageRequestFactory, "executeStageRequestFactory"); + return Props.create( + ExecutorStateManagerActor.class, + schedulingAttributes, + fitnessCalculator, + schedulerLeaseExpirationDuration, + availableTaskExecutorMutatorHook, + clock, + rpcService, + jobMessageRouter, + mantisJobStore, + heartbeatTimeout, + assignmentTimeout, + disabledTaskExecutorsCheckInterval, + clusterID, + isJobArtifactCachingEnabled, + jobClustersWithArtifactCachingEnabled, + metrics, + executeStageRequestFactory, + reservationSchedulingEnabled + ) + .withMailbox("akka.actor.metered-mailbox"); + } + + public static Props props( + ExecutorStateManagerImpl delegate, + Clock clock, + RpcService rpcService, + JobMessageRouter jobMessageRouter, + MantisJobStore mantisJobStore, + Duration heartbeatTimeout, + Duration assignmentTimeout, + Duration disabledTaskExecutorsCheckInterval, + ClusterID clusterID, + boolean isJobArtifactCachingEnabled, + String jobClustersWithArtifactCachingEnabled, + ResourceClusterActorMetrics metrics, + ExecuteStageRequestFactory executeStageRequestFactory, + boolean reservationSchedulingEnabled + ) { + Objects.requireNonNull(delegate, "delegate"); + Objects.requireNonNull(clock, "clock"); + Objects.requireNonNull(rpcService, "rpcService"); + Objects.requireNonNull(jobMessageRouter, "jobMessageRouter"); + Objects.requireNonNull(mantisJobStore, "mantisJobStore"); + Objects.requireNonNull(heartbeatTimeout, "heartbeatTimeout"); + Objects.requireNonNull(assignmentTimeout, "assignmentTimeout"); + Objects.requireNonNull(disabledTaskExecutorsCheckInterval, "disabledTaskExecutorsCheckInterval"); + Objects.requireNonNull(clusterID, "clusterID"); + Objects.requireNonNull(jobClustersWithArtifactCachingEnabled, "jobClustersWithArtifactCachingEnabled"); + Objects.requireNonNull(metrics, "metrics"); + Objects.requireNonNull(executeStageRequestFactory, "executeStageRequestFactory"); + return Props.create( + ExecutorStateManagerActor.class, + delegate, + clock, + rpcService, + jobMessageRouter, + mantisJobStore, + heartbeatTimeout, + assignmentTimeout, + disabledTaskExecutorsCheckInterval, + clusterID, + isJobArtifactCachingEnabled, + jobClustersWithArtifactCachingEnabled, + metrics, + executeStageRequestFactory, + reservationSchedulingEnabled + ) + .withMailbox("akka.actor.metered-mailbox"); + } + + ExecutorStateManagerActor( + Map schedulingAttributes, + FitnessCalculator fitnessCalculator, + Duration schedulerLeaseExpirationDuration, + @Nullable AvailableTaskExecutorMutatorHook availableTaskExecutorMutatorHook, + Clock clock, + RpcService rpcService, + JobMessageRouter jobMessageRouter, + MantisJobStore mantisJobStore, + Duration heartbeatTimeout, + Duration assignmentTimeout, + Duration disabledTaskExecutorsCheckInterval, + ClusterID clusterID, + boolean isJobArtifactCachingEnabled, + String jobClustersWithArtifactCachingEnabled, + ResourceClusterActorMetrics metrics, + ExecuteStageRequestFactory executeStageRequestFactory, + boolean reservationSchedulingEnabled + ) { + this( + new ExecutorStateManagerImpl( + schedulingAttributes, + fitnessCalculator, + schedulerLeaseExpirationDuration, + availableTaskExecutorMutatorHook, + reservationSchedulingEnabled), + clock, + rpcService, + jobMessageRouter, + mantisJobStore, + heartbeatTimeout, + assignmentTimeout, + disabledTaskExecutorsCheckInterval, + clusterID, + isJobArtifactCachingEnabled, + jobClustersWithArtifactCachingEnabled, + metrics, + executeStageRequestFactory, + reservationSchedulingEnabled); + } + + ExecutorStateManagerActor( + ExecutorStateManagerImpl delegate, + Clock clock, + RpcService rpcService, + JobMessageRouter jobMessageRouter, + MantisJobStore mantisJobStore, + Duration heartbeatTimeout, + Duration assignmentTimeout, + Duration disabledTaskExecutorsCheckInterval, + ClusterID clusterID, + boolean isJobArtifactCachingEnabled, + String jobClustersWithArtifactCachingEnabled, + ResourceClusterActorMetrics metrics, + ExecuteStageRequestFactory executeStageRequestFactory, + boolean reservationSchedulingEnabled + ) { + this.delegate = delegate; + this.clock = clock; + this.rpcService = rpcService; + this.jobMessageRouter = jobMessageRouter; + this.mantisJobStore = mantisJobStore; + this.heartbeatTimeout = heartbeatTimeout; + this.assignmentTimeout = assignmentTimeout; + this.disabledTaskExecutorsCheckInterval = disabledTaskExecutorsCheckInterval; + this.clusterID = clusterID; + this.isJobArtifactCachingEnabled = isJobArtifactCachingEnabled; + this.jobClustersWithArtifactCachingEnabled = jobClustersWithArtifactCachingEnabled; + this.metrics = metrics; + this.executeStageRequestFactory = executeStageRequestFactory; + this.reservationSchedulingEnabled = reservationSchedulingEnabled; + this.activeDisableTaskExecutorsByAttributesRequests = new HashSet<>(); + this.disabledTaskExecutors = new HashSet<>(); + this.jobArtifactsToCache = new HashSet<>(); + + // Create child AssignmentHandlerActor + this.assignmentHandlerActor = getContext().actorOf( + AssignmentHandlerActor.props(clusterID, jobMessageRouter, assignmentTimeout, executeStageRequestFactory), + "assignment-handler" + ); + } + + @Override + public void preStart() throws Exception { + super.preStart(); + List activeRequests = + mantisJobStore.loadAllDisableTaskExecutorsRequests(clusterID); + for (DisableTaskExecutorsRequest request : activeRequests) { + onNewDisableTaskExecutorsRequest(request); + } + + timers().startTimerWithFixedDelay( + String.format("periodic-disabled-task-executors-test-for-%s", clusterID.getResourceID()), + new CheckDisabledTaskExecutors("periodic"), + disabledTaskExecutorsCheckInterval); + } + + @Override + public Receive createReceive() { + return receiveBuilder() + .match(InitializeTaskExecutorRequest.class, this::onTaskExecutorInitialization) + .match(TaskExecutorRegistration.class, this::onTaskExecutorRegistration) + .match(TaskExecutorHeartbeat.class, this::onHeartbeat) + .match(TaskExecutorStatusChange.class, this::onTaskExecutorStatusChange) + .match(TaskExecutorBatchAssignmentRequest.class, this::onTaskExecutorBatchAssignmentRequest) + .match(AssignmentHandlerActor.TaskExecutorAssignmentFailAndTerminate.class, this::onTaskExecutorAssignmentFailure) + .match(TaskExecutorDisconnection.class, this::onTaskExecutorDisconnection) + .match(HeartbeatTimeout.class, this::onTaskExecutorHeartbeatTimeout) + .match(CacheJobArtifactsOnTaskExecutorRequest.class, this::onCacheJobArtifactsOnTaskExecutorRequest) + .match(TaskExecutorInfoRequest.class, this::onTaskExecutorInfoRequest) + .match(TaskExecutorGatewayRequest.class, this::onTaskExecutorGatewayRequest) + .match(GetTaskExecutorStatusRequest.class, this::onGetTaskExecutorStatus) + .match(GetRegisteredTaskExecutorsRequest.class, req -> onGetTaskExecutors(req, ExecutorStateManager.isRegistered)) + .match(GetBusyTaskExecutorsRequest.class, req -> onGetTaskExecutors(req, ExecutorStateManager.isBusy)) + .match(GetAvailableTaskExecutorsRequest.class, req -> onGetTaskExecutors(req, ExecutorStateManager.isAvailable)) + .match(GetDisabledTaskExecutorsRequest.class, req -> onGetTaskExecutors(req, ExecutorStateManager.isDisabled)) + .match(GetUnregisteredTaskExecutorsRequest.class, req -> onGetTaskExecutors(req, ExecutorStateManager.unregistered)) + .match(GetActiveJobsRequest.class, this::onGetActiveJobs) + .match(GetClusterUsageRequest.class, this::onGetClusterUsage) + .match(GetClusterUsageWithReservationsRequest.class, this::onGetClusterUsageWithReservations) // NEW + .match(GetClusterIdleInstancesRequest.class, this::onGetClusterIdleInstancesRequest) + .match(GetAssignedTaskExecutorRequest.class, this::onGetAssignedTaskExecutorRequest) + .match(MarkExecutorTaskCancelledRequest.class, this::onMarkExecutorTaskCancelledRequest) + .match(ResourceOverviewRequest.class, this::onResourceOverviewRequest) + .match(GetTaskExecutorWorkerMappingRequest.class, this::onGetTaskExecutorWorkerMappingRequest) + .match(PublishResourceOverviewMetricsRequest.class, this::onPublishResourceOverviewMetricsRequest) + .match(AddNewJobArtifactsToCacheRequest.class, this::onAddNewJobArtifactsToCacheRequest) + .match(RemoveJobArtifactsToCacheRequest.class, this::onRemoveJobArtifactsToCacheRequest) + .match(GetJobArtifactsToCacheRequest.class, this::onGetJobArtifactsToCacheRequest) + .match(RefreshTaskExecutorJobArtifactCache.class, refresh -> refreshTaskExecutorJobArtifactCache()) + .match(CheckDisabledTaskExecutors.class, this::onCheckDisabledTaskExecutors) + .match(ExpireDisableTaskExecutorsRequest.class, this::onDisableTaskExecutorsRequestExpiry) + .match(UpdateJobArtifactsToCache.class, this::onUpdateJobArtifactsToCache) + .match(DisableTaskExecutorsRequest.class, this::onNewDisableTaskExecutorsRequest) + .build(); + } + + private void onTaskExecutorInitialization(InitializeTaskExecutorRequest request) { + log.info("Initializing taskExecutor {} for the resource cluster {}", request.getTaskExecutorID(), clusterID); + try { + TaskExecutorRegistration registration = + mantisJobStore.getTaskExecutor(request.getTaskExecutorID()); + if (registration == null) { + sender().tell(new Status.Failure(new TaskExecutorNotFoundException(request.getTaskExecutorID())), self()); + return; + } + handleTaskExecutorRegistration(registration); + self().tell( + new TaskExecutorStatusChange( + registration.getTaskExecutorID(), + registration.getClusterID(), + TaskExecutorReport.occupied(request.getWorkerId())), + self()); + sender().tell(Ack.getInstance(), self()); + } catch (Exception e) { + log.error("Failed to initialize taskExecutor {}; all retries exhausted", request.getTaskExecutorID(), e); + sender().tell(new Status.Failure(e), self()); + } + } + + private void onTaskExecutorRegistration(TaskExecutorRegistration registration) { + try { + handleTaskExecutorRegistration(registration); + sender().tell(Ack.getInstance(), self()); + } catch (Exception e) { + sender().tell(new Status.Failure(e), self()); + } + } + + private void handleTaskExecutorRegistration(TaskExecutorRegistration registration) throws Exception { + setupTaskExecutorStateIfNecessary(registration.getTaskExecutorID()); + log.info("Request for registering on resource cluster {}: {}.", clusterID, registration); + final TaskExecutorID taskExecutorID = registration.getTaskExecutorID(); + final TaskExecutorState state = this.delegate.get(taskExecutorID); + boolean stateChange = state.onRegistration(registration); + mantisJobStore.storeNewTaskExecutor(registration); + if (stateChange) { + if (state.isAvailable()) { + this.delegate.tryMarkAvailable(taskExecutorID); + } + if (isTaskExecutorDisabled(registration)) { + log.info("Newly registered task executor {} was already marked for disabling.", registration.getTaskExecutorID()); + state.onNodeDisabled(); + } + updateHeartbeatTimeout(registration.getTaskExecutorID()); + } + log.info("Successfully registered {} with the resource cluster {}", registration.getTaskExecutorID(), clusterID); + if (!jobArtifactsToCache.isEmpty() && isJobArtifactCachingEnabled) { + self().tell(new CacheJobArtifactsOnTaskExecutorRequest(taskExecutorID, clusterID), self()); + } + } + + private void onHeartbeat(TaskExecutorHeartbeat heartbeat) { + log.debug("Received heartbeat {} from task executor {}", heartbeat, heartbeat.getTaskExecutorID()); + setupTaskExecutorStateIfNecessary(heartbeat.getTaskExecutorID()); + try { + final TaskExecutorID taskExecutorID = heartbeat.getTaskExecutorID(); + final TaskExecutorState state = this.delegate.get(taskExecutorID); + if (state.getRegistration() == null || !state.isRegistered()) { + TaskExecutorRegistration registration = this.mantisJobStore.getTaskExecutor(heartbeat.getTaskExecutorID()); + if (registration != null) { + log.debug("Found registration {} for task executor {}", registration, heartbeat.getTaskExecutorID()); + Preconditions.checkState(state.onRegistration(registration)); + + if (isTaskExecutorDisabled(registration)) { + log.info("Reconnected task executor {} was already marked for disabling.", registration.getTaskExecutorID()); + state.onNodeDisabled(); + } + } else { + log.warn("Received heartbeat from unknown task executor {}", heartbeat.getTaskExecutorID()); + sender().tell(new Status.Failure(new TaskExecutorNotFoundException(taskExecutorID)), self()); + return; + } + } else { + log.debug("Found registration {} for registered task executor {}", + state.getRegistration(), heartbeat.getTaskExecutorID()); + } + boolean stateChange = state.onHeartbeat(heartbeat); + if (stateChange && state.isAvailable()) { + this.delegate.tryMarkAvailable(taskExecutorID); + } + + updateHeartbeatTimeout(heartbeat.getTaskExecutorID()); + log.debug("Successfully processed heartbeat {} from task executor {}", heartbeat, heartbeat.getTaskExecutorID()); + sender().tell(Ack.getInstance(), self()); + } catch (Exception e) { + sender().tell(new Status.Failure(e), self()); + } + } + + private void onTaskExecutorStatusChange(TaskExecutorStatusChange statusChange) { + setupTaskExecutorStateIfNecessary(statusChange.getTaskExecutorID()); + try { + final TaskExecutorID taskExecutorID = statusChange.getTaskExecutorID(); + final TaskExecutorState state = this.delegate.get(taskExecutorID); + boolean stateChange = state.onTaskExecutorStatusChange(statusChange); + if (stateChange) { + if (state.isAvailable()) { + this.delegate.tryMarkAvailable(taskExecutorID); + } else { + this.delegate.tryMarkUnavailable(taskExecutorID); + } + } + + updateHeartbeatTimeout(statusChange.getTaskExecutorID()); + sender().tell(Ack.getInstance(), self()); + } catch (IllegalStateException e) { + sender().tell(new Status.Failure(e), self()); + } + } + + private void onTaskExecutorBatchAssignmentRequest(TaskExecutorBatchAssignmentRequest request) { + Optional matchedExecutors = this.delegate.findBestFit(request); + + if (matchedExecutors.isPresent()) { + log.info("Matched all executors {} for request {}", matchedExecutors.get(), request); + + // first inform registry the allocation has been fulfilled. + sender().tell( + new TaskExecutorsAllocation( + matchedExecutors.get().getRequestToTaskExecutorMap(), + request.getReservation()), + self()); + + // start worker assignment process + matchedExecutors.get().getBestFit().forEach( + (allocationRequest, taskExecutorToState) + -> assignTaskExecutor( + allocationRequest, taskExecutorToState.getLeft(), taskExecutorToState.getRight(), request)); + } else { + request.getAllocationRequests().forEach(req -> metrics.incrementCounter( + ResourceClusterActorMetrics.NO_RESOURCES_AVAILABLE, + createTagListFrom(req))); + String constraintKey = request.getReservation() != null + ? request.getReservation().getCanonicalConstraintKey() + : null; + sender().tell(new Status.Failure(new NoResourceAvailableException( + String.format("No resource available for request %s: resource overview: %s", request, + getResourceOverview()), constraintKey)), self()); + } + } + + private void assignTaskExecutor(TaskExecutorAllocationRequest allocationRequest, TaskExecutorID taskExecutorID, TaskExecutorState taskExecutorState, TaskExecutorBatchAssignmentRequest request) { + + // Mark task executor as assigned + taskExecutorState.onAssignment(allocationRequest.getWorkerId()); + + // Get task executor registration info + TaskExecutorRegistration registration = taskExecutorState.getRegistration(); + if (registration == null) { + log.error("Cannot assign task executor {} - no registration found", taskExecutorID); + return; + } + + // inform job actor the worker is being assigned to target TE. + jobMessageRouter.routeWorkerEvent( + new WorkerLaunched( + allocationRequest.getWorkerId(), + allocationRequest.getStageNum(), + registration.getHostname(), + taskExecutorID.getResourceId(), + Optional.ofNullable(registration.getClusterID().getResourceID()), + Optional.of(registration.getClusterID()), + registration.getWorkerPorts())); + + // Get the gateway future from the TaskExecutorState + CompletableFuture gatewayFuture = taskExecutorState.getGatewayAsync(); + + // Delegate actual assignment logic to AssignmentHandlerActor + AssignmentHandlerActor.TaskExecutorAssignmentRequest assignmentRequest = + AssignmentHandlerActor.TaskExecutorAssignmentRequest.of( + allocationRequest, + taskExecutorID, + registration, + gatewayFuture + ); + + assignmentHandlerActor.tell(assignmentRequest, self()); + } + + private void onTaskExecutorAssignmentFailure(AssignmentHandlerActor.TaskExecutorAssignmentFailAndTerminate request) { + TaskExecutorState state = this.delegate.get(request.getTaskExecutorID()); + + // Increment metrics with clusterID and jobId tags + metrics.incrementCounter( + ResourceClusterActorMetrics.TASK_EXECUTOR_ASSIGNMENT_FAILURE, + TagList.create(ImmutableMap.of( + "resourceCluster", clusterID.getResourceID(), + "jobId", request.getAllocationRequest().getWorkerId().getJobId()))); + + if (state == null) { + log.error("[TaskExecutorAssignmentFailure] TaskExecutor lost during task assignment: {}", request); + } + else if (state.isRunningTask()) { + log.warn("[onTaskExecutorAssignmentFailure] TaskExecutor {} entered running state already; no need to act", + request.getTaskExecutorID()); + } else { + log.error("[onTaskExecutorAssignmentFailure] TaskExecutor {} failed to accept assignment: {}", + request.getTaskExecutorID(), request.getAllocationRequest()); + try + { + jobMessageRouter.routeWorkerEvent(new WorkerLaunchFailed( + request.getAllocationRequest().getWorkerId(), + request.getAllocationRequest().getStageNum(), + "Failed to assign worker to task executor " + request.getTaskExecutorID())); + state.onUnassignment(); + // disconnect the TE since it cannot be assigned. + disconnectTaskExecutor(request.getTaskExecutorID()); + } catch (IllegalStateException e) { + log.error("Failed to un-assign taskExecutor {}", request.getTaskExecutorID(), e); + } + } + } + + private void onTaskExecutorDisconnection(TaskExecutorDisconnection disconnection) { + setupTaskExecutorStateIfNecessary(disconnection.getTaskExecutorID()); + try { + disconnectTaskExecutor(disconnection.getTaskExecutorID()); + sender().tell(Ack.getInstance(), self()); + } catch (IllegalStateException e) { + sender().tell(new Status.Failure(e), self()); + } + } + + private void disconnectTaskExecutor(TaskExecutorID taskExecutorID) { + final TaskExecutorState state = this.delegate.get(taskExecutorID); + boolean stateChange = state.onDisconnection(); + if (stateChange) { + this.delegate.archive(taskExecutorID); + getTimers().cancel(getHeartbeatTimerFor(taskExecutorID)); + } + } + + private void onTaskExecutorHeartbeatTimeout(HeartbeatTimeout timeout) { + setupTaskExecutorStateIfNecessary(timeout.getTaskExecutorID()); + try { + metrics.incrementCounter( + ResourceClusterActorMetrics.HEARTBEAT_TIMEOUT, + TagList.create(ImmutableMap.of("resourceCluster", clusterID.getResourceID(), "taskExecutorID", timeout.getTaskExecutorID().getResourceId()))); + log.info("heartbeat timeout received for {}", timeout.getTaskExecutorID()); + final TaskExecutorID taskExecutorID = timeout.getTaskExecutorID(); + final TaskExecutorState state = this.delegate.get(taskExecutorID); + if (state.getLastActivity().compareTo(timeout.getLastActivity()) <= 0) { + log.info("Disconnecting task executor {}", timeout.getTaskExecutorID()); + disconnectTaskExecutor(timeout.getTaskExecutorID()); + } + + } catch (IllegalStateException e) { + sender().tell(new Status.Failure(e), self()); + } + } + + private void setupTaskExecutorStateIfNecessary(TaskExecutorID taskExecutorID) { + this.delegate.trackIfAbsent( + taskExecutorID, + TaskExecutorState.of(clock, rpcService, jobMessageRouter)); + } + + private void updateHeartbeatTimeout(TaskExecutorID taskExecutorID) { + final TaskExecutorState state = this.delegate.get(taskExecutorID); + getTimers().startSingleTimer( + getHeartbeatTimerFor(taskExecutorID), + new HeartbeatTimeout(taskExecutorID, state.getLastActivity()), + heartbeatTimeout); + } + + private String getHeartbeatTimerFor(TaskExecutorID taskExecutorID) { + return "Heartbeat-" + taskExecutorID; + } + + private boolean isTaskExecutorDisabled(TaskExecutorRegistration registration) { + for (DisableTaskExecutorsRequest request : activeDisableTaskExecutorsByAttributesRequests) { + if (request.covers(registration)) { + return true; + } + } + return disabledTaskExecutors.contains(registration.getTaskExecutorID()); + } + + private void onCacheJobArtifactsOnTaskExecutorRequest(CacheJobArtifactsOnTaskExecutorRequest request) { + TaskExecutorState state = this.delegate.get(request.getTaskExecutorID()); + if (state != null && state.isRegistered()) { + try { + state.getGatewayAsync() + .thenComposeAsync(taskExecutorGateway -> + taskExecutorGateway.cacheJobArtifacts(new CacheJobArtifactsRequest( + jobArtifactsToCache + .stream() + .map(artifactID -> URI.create(artifactID.getResourceID())) + .collect(Collectors.toList())))) + .whenComplete((res, throwable) -> { + if (throwable != null) { + log.error("failed to cache artifact on {}", request.getTaskExecutorID(), throwable); + } else { + log.debug("Acked from cacheJobArtifacts for {}", request.getTaskExecutorID()); + } + }); + } catch (Exception ex) { + log.warn("Failed to cache job artifacts in task executor {}", request.getTaskExecutorID(), ex); + } + } else { + log.debug("no valid TE state for CacheJobArtifactsOnTaskExecutorRequest: {}", request); + } + sender().tell(Ack.getInstance(), self()); + } + + private boolean shouldCacheJobArtifacts(TaskExecutorAllocationRequest allocationRequest) { + final WorkerId workerId = allocationRequest.getWorkerId(); + final boolean isFirstWorkerOfFirstStage = allocationRequest.getStageNum() == 1 && workerId.getWorkerIndex() == 0; + if (isFirstWorkerOfFirstStage) { + final Set jobClusters = getJobClustersWithArtifactCachingEnabled(); + return jobClusters.contains(workerId.getJobCluster()); + } + return false; + } + + private Set getJobClustersWithArtifactCachingEnabled() { + return new HashSet<>(Arrays.asList(jobClustersWithArtifactCachingEnabled.split(","))); + } + + private void onTaskExecutorGatewayRequest(TaskExecutorGatewayRequest request) { + TaskExecutorState state = this.delegate.get(request.getTaskExecutorID()); + if (state == null) { + sender().tell(new NullPointerException("Null TaskExecutorState for: " + request.getTaskExecutorID()), self()); + } else { + try { + if (state.isRegistered()) { + sender().tell(state.getGatewayAsync(), self()); + } else { + sender().tell( + new Status.Failure(new IllegalStateException("Unregistered TaskExecutor: " + request.getTaskExecutorID())), + self()); + } + } catch (Exception e) { + log.error("onTaskExecutorGatewayRequest error: {}", request, e); + metrics.incrementCounter( + ResourceClusterActorMetrics.TE_CONNECTION_FAILURE, + TagList.create(ImmutableMap.of( + "resourceCluster", + clusterID.getResourceID(), + "taskExecutor", + request.getTaskExecutorID().getResourceId()))); + } + } + } + + private void onTaskExecutorInfoRequest(TaskExecutorInfoRequest request) { + if (request.getTaskExecutorID() != null) { + TaskExecutorState state = + this.delegate.getIncludeArchived(request.getTaskExecutorID()); + if (state != null && state.getRegistration() != null) { + sender().tell(state.getRegistration(), self()); + } else { + sender().tell(new Status.Failure(new Exception(String.format("No task executor state for %s", + request.getTaskExecutorID()))), self()); + } + } else { + Optional taskExecutorRegistration = + this.delegate + .findFirst( + kv -> kv.getValue().getRegistration() != null && + kv.getValue().getRegistration().getHostname().equals(request.getHostName())) + .map(Entry::getValue) + .map(TaskExecutorState::getRegistration); + if (taskExecutorRegistration.isPresent()) { + sender().tell(taskExecutorRegistration.get(), self()); + } else { + sender().tell(new Status.Failure(new Exception(String.format("Unknown task executor for hostname %s", request.getHostName()))), self()); + } + } + } + + private void onGetTaskExecutorStatus(GetTaskExecutorStatusRequest req) { + TaskExecutorID taskExecutorID = req.getTaskExecutorID(); + final TaskExecutorState state = this.delegate.get(taskExecutorID); + if (state == null) { + log.info("Unknown executorID: {}", taskExecutorID); + sender().tell( + new Status.Failure(new TaskExecutorNotFoundException(taskExecutorID)), + self()); + } else { + sender().tell( + new TaskExecutorStatus( + state.getRegistration(), + state.isRegistered(), + state.isRunningTask(), + state.isAssigned(), + state.isDisabled(), + state.getWorkerId(), + state.getLastActivity().toEpochMilli(), + state.getCancelledWorkerId()), + self()); + } + } + + private void onGetTaskExecutors(ResourceClusterActor.HasAttributes request, + Predicate> predicate) { + Predicate> combined = filterByAttrs(request).and(predicate); + sender().tell(new TaskExecutorsList(this.delegate.getTaskExecutors(combined)), self()); + } + + private Predicate> filterByAttrs(ResourceClusterActor.HasAttributes hasAttributes) { + if (hasAttributes.getAttributes().isEmpty()) { + return e -> true; + } else { + return e -> e.getValue().containsAttributes(hasAttributes.getAttributes()); + } + } + + private void onGetActiveJobs(GetActiveJobsRequest req) { + List pagedList = this.delegate.getActiveJobs(req); + + PagedActiveJobOverview res = + new PagedActiveJobOverview( + pagedList, + req.getStartingIndex().orElse(0) + pagedList.size() + ); + + sender().tell(res, self()); + } + + private void onGetClusterUsage(GetClusterUsageRequest req) { + sender().tell(this.delegate.getClusterUsage(req), self()); + } + + private void onGetClusterUsageWithReservations(GetClusterUsageWithReservationsRequest req) { + sender().tell( + this.delegate.getClusterUsageWithReservations( + req.getClusterID(), + req.getGroupKeyFunc(), + req.getPendingReservations()), + self()); + } + + private void onGetClusterIdleInstancesRequest(GetClusterIdleInstancesRequest req) { + if (!req.getClusterID().equals(this.clusterID)) { + sender().tell(new Status.Failure( + new IllegalArgumentException(String.format("Mismatch cluster ids %s, %s", req.getClusterID(), this.clusterID))), + self()); + return; + } + + List instanceList = this.delegate.getIdleInstanceList(req); + + GetClusterIdleInstancesResponse res = GetClusterIdleInstancesResponse.builder() + .instanceIds(instanceList) + .clusterId(this.clusterID) + .skuId(req.getSkuId()) + .build(); + sender().tell(res, self()); + } + + private void onGetAssignedTaskExecutorRequest(GetAssignedTaskExecutorRequest request) { + Optional matchedTaskExecutor = + this.delegate.findFirst( + e -> e.getValue().isRunningOrAssigned(request.getWorkerId())).map(Entry::getKey); + + if (matchedTaskExecutor.isPresent()) { + sender().tell(matchedTaskExecutor.get(), self()); + } else { + sender().tell(new Status.Failure(new TaskNotFoundException(request.getWorkerId())), self()); + } + } + + private void onMarkExecutorTaskCancelledRequest(MarkExecutorTaskCancelledRequest request) { + Optional> matchedTaskExecutor = + this.delegate.findFirst(e -> e.getValue().isRunningOrAssigned(request.getWorkerId())); + + if (matchedTaskExecutor.isPresent()) { + log.info("Setting executor {} to cancelled workerID: {}", matchedTaskExecutor.get().getKey(), request); + matchedTaskExecutor.get().getValue().setCancelledWorkerOnTask(request.getWorkerId()); + sender().tell(Ack.getInstance(), self()); + } else { + log.info("Cannot find executor to mark worker {} as cancelled", request); + sender().tell(new Status.Failure(new TaskNotFoundException(request.getWorkerId())), self()); + } + } + + private void onResourceOverviewRequest(ResourceOverviewRequest request) { + sender().tell(getResourceOverview(), self()); + } + + private void onGetTaskExecutorWorkerMappingRequest(GetTaskExecutorWorkerMappingRequest request) { + sender().tell(getTaskExecutorWorkerMapping(request.getAttributes()), self()); + } + + private void onPublishResourceOverviewMetricsRequest(PublishResourceOverviewMetricsRequest request) { + publishResourceClusterMetricBySKU( + new TaskExecutorsList(this.delegate.getTaskExecutors(ExecutorStateManager.isRegistered)), + ResourceClusterActorMetrics.NUM_REGISTERED_TE); + publishResourceClusterMetricBySKU( + new TaskExecutorsList(this.delegate.getTaskExecutors(ExecutorStateManager.isBusy)), + ResourceClusterActorMetrics.NUM_BUSY_TE); + publishResourceClusterMetricBySKU( + new TaskExecutorsList(this.delegate.getTaskExecutors(ExecutorStateManager.isAvailable)), + ResourceClusterActorMetrics.NUM_AVAILABLE_TE); + publishResourceClusterMetricBySKU( + new TaskExecutorsList(this.delegate.getTaskExecutors(ExecutorStateManager.isDisabled)), + ResourceClusterActorMetrics.NUM_DISABLED_TE); + publishResourceClusterMetricBySKU( + new TaskExecutorsList(this.delegate.getTaskExecutors(ExecutorStateManager.unregistered)), + ResourceClusterActorMetrics.NUM_UNREGISTERED_TE); + publishResourceClusterMetricBySKU( + new TaskExecutorsList(this.delegate.getTaskExecutors(ExecutorStateManager.isAssigned)), + ResourceClusterActorMetrics.NUM_ASSIGNED_TE); + // Note: Not sending Ack here since this message is triggered by a timer + // and no actor is waiting for the response. Sending Ack to timer-originated + // messages results in dead letters. + } + + private void publishResourceClusterMetricBySKU(TaskExecutorsList taskExecutorsList, String metricName) { + try { + taskExecutorsList.getTaskExecutors() + .stream() + .map(this.delegate::get) + .filter(Objects::nonNull) + .map(TaskExecutorState::getRegistration) + .filter(Objects::nonNull) + .filter(registration -> registration.getTaskExecutorContainerDefinitionId().isPresent() && registration.getAttributeByKey(WorkerConstants.AUTO_SCALE_GROUP_KEY).isPresent()) + .collect(Collectors.groupingBy( + registration -> new AbstractMap.SimpleEntry<>( + registration.getTaskExecutorContainerDefinitionId().get(), + registration.getAttributeByKey(WorkerConstants.AUTO_SCALE_GROUP_KEY).get()), + Collectors.counting())) + .forEach((keys, count) -> metrics.setGauge( + metricName, + count, + TagList.create(ImmutableMap.of( + "resourceCluster", + clusterID.getResourceID(), + "sku", + keys.getKey().getResourceID(), + "autoScaleGroup", + keys.getValue())))); + } catch (Exception e) { + log.warn("Error while publishing resource cluster metrics by sku. RC: {}, Metric: {}.", clusterID.getResourceID(), metricName, e); + } + } + + private void onAddNewJobArtifactsToCacheRequest(AddNewJobArtifactsToCacheRequest req) { + jobArtifactsToCache.addAll(req.getArtifacts()); + refreshTaskExecutorJobArtifactCache(); + sender().tell(Ack.getInstance(), self()); + } + + private void onRemoveJobArtifactsToCacheRequest(RemoveJobArtifactsToCacheRequest req) { + req.getArtifacts().forEach(jobArtifactsToCache::remove); + sender().tell(Ack.getInstance(), self()); + } + + private void onGetJobArtifactsToCacheRequest(GetJobArtifactsToCacheRequest req) { + sender().tell(new ArtifactList(new ArrayList<>(jobArtifactsToCache)), self()); + } + + private void onUpdateJobArtifactsToCache(UpdateJobArtifactsToCache update) { + this.jobArtifactsToCache.clear(); + this.jobArtifactsToCache.addAll(update.getArtifacts()); + } + + // custom equals function to check if the existing set already has the request under consideration. + private boolean addNewDisableTaskExecutorsRequest(DisableTaskExecutorsRequest newRequest) { + if (newRequest.isRequestByAttributes()) { + log.info("Req with attributes {}", newRequest); + for (DisableTaskExecutorsRequest existing: activeDisableTaskExecutorsByAttributesRequests) { + if (existing.targetsSameTaskExecutorsAs(newRequest)) { + return false; + } + } + + Preconditions.checkState(activeDisableTaskExecutorsByAttributesRequests.add(newRequest), "activeDisableTaskExecutorRequests cannot contain %s", newRequest); + return true; + } else if (newRequest.getTaskExecutorID().isPresent() && !disabledTaskExecutors.contains(newRequest.getTaskExecutorID().get())) { + log.info("Req with id {}", newRequest); + disabledTaskExecutors.add(newRequest.getTaskExecutorID().get()); + return true; + } + log.info("No Req {}", newRequest); + return false; + } + + private void onNewDisableTaskExecutorsRequest(DisableTaskExecutorsRequest request) { + ActorRef sender = sender(); + if (addNewDisableTaskExecutorsRequest(request)) { + try { + log.info("New req to add {}", request); + // store the request in a persistent store in order to retrieve it if the node goes down + mantisJobStore.storeNewDisabledTaskExecutorsRequest(request); + // figure out the time to expire the current request + Duration toExpiry = Comparators.max(Duration.between(clock.instant(), request.getExpiry()), Duration.ZERO); + // setup a timer to clear it after a given period + getTimers().startSingleTimer( + getExpiryKeyFor(request), + new ExpireDisableTaskExecutorsRequest(request), + toExpiry); + findAndMarkDisabledTaskExecutorsFor(request); + if (sender != null && !sender.equals(self())) { + sender.tell(Ack.getInstance(), self()); + } + } catch (IOException e) { + if (sender != null && !sender.equals(self())) { + sender.tell(new Status.Failure(e), self()); + } else { + log.error("Failed to store disable request: {}", request, e); + } + } + } else { + if (sender != null && !sender.equals(self())) { + sender.tell(Ack.getInstance(), self()); + } + } + } + + private String getExpiryKeyFor(DisableTaskExecutorsRequest request) { + return "ExpireDisableTaskExecutorsRequest-" + request; + } + + private void findAndMarkDisabledTaskExecutorsFor(DisableTaskExecutorsRequest request) { + if (request.isRequestByAttributes()) { + self().tell(new CheckDisabledTaskExecutors("new_request"), self()); + } else if (request.getTaskExecutorID().isPresent()) { + self().tell(new CheckDisabledTaskExecutors("targeted_request"), self()); + } + } + + private void onCheckDisabledTaskExecutors(CheckDisabledTaskExecutors request) { + final Instant now = clock.instant(); + for (DisableTaskExecutorsRequest disableRequest : activeDisableTaskExecutorsByAttributesRequests) { + if (disableRequest.isExpired(now)) { + self().tell(new ExpireDisableTaskExecutorsRequest(disableRequest), self()); + } else { + this.delegate.getActiveExecutorEntry().forEach(idAndState -> { + if (disableRequest.covers(idAndState.getValue().getRegistration())) { + idAndState.getValue().onNodeDisabled(); + } + }); + } + } + + for (TaskExecutorID taskExecutorID : disabledTaskExecutors) { + TaskExecutorState state = this.delegate.get(taskExecutorID); + if (state != null) { + state.onNodeDisabled(); + } + } + } + + private void onDisableTaskExecutorsRequestExpiry(ExpireDisableTaskExecutorsRequest request) { + try { + DisableTaskExecutorsRequest expiredRequest = request.getRequest(); + log.debug("Expiring Disable Task Executors Request {}", expiredRequest); + getTimers().cancel(getExpiryKeyFor(expiredRequest)); + + boolean removed = activeDisableTaskExecutorsByAttributesRequests.remove(expiredRequest) || + (expiredRequest.getTaskExecutorID().isPresent() && disabledTaskExecutors.remove(expiredRequest.getTaskExecutorID().get())); + + if (removed) { + log.info("Removed active disable task executors for request {}", request); + mantisJobStore.deleteExpiredDisableTaskExecutorsRequest(expiredRequest); + } + + if (expiredRequest.getTaskExecutorID().isPresent()) { + // Handle TaskExecutorID-based disable request expiry + final TaskExecutorID taskExecutorID = expiredRequest.getTaskExecutorID().get(); + final TaskExecutorState state = this.delegate.get(taskExecutorID); + if (state != null) { + // Only re-enable if not still disabled by other requests + if (!isTaskExecutorDisabled(state.getRegistration())) { + state.onNodeEnabled(); + } + } + } else if (expiredRequest.isRequestByAttributes()) { + // Handle attribute-based disable request expiry + // Find all task executors that match the expired request's attributes + this.delegate.getActiveExecutorEntry().forEach(idAndState -> { + TaskExecutorRegistration registration = idAndState.getValue().getRegistration(); + if (registration != null && expiredRequest.covers(registration)) { + // Only re-enable if not still covered by other active disable requests + if (!isTaskExecutorDisabled(registration)) { + log.info("re-enable TE: {}", idAndState.getKey()); + idAndState.getValue().onNodeEnabled(); + } + } + }); + } + + // trigger a check to ensure everything is consistent + self().tell(new CheckDisabledTaskExecutors("expiry"), self()); + + } catch (Exception e) { + log.error("Failed to handle expired disable request {}", request.getRequest(), e); + } + } + + private Map getTaskExecutorWorkerMapping(Map attributes) { + final Map result = new HashMap<>(); + this.delegate.getActiveExecutorEntry().forEach(idAndState -> { + if (idAndState.getValue().getRegistration() != null && idAndState.getValue().getRegistration().containsAttributes(attributes)) { + if (idAndState.getValue().isRunningTask()) { + result.put(idAndState.getKey(), idAndState.getValue().getWorkerId()); + } + } + }); + return result; + } + + private void refreshTaskExecutorJobArtifactCache() { + this.delegate.getTaskExecutors(ExecutorStateManager.isAvailable) + .forEach(taskExecutorID -> + self().tell(new CacheJobArtifactsOnTaskExecutorRequest(taskExecutorID, clusterID), self())); + } + + private Iterable createTagListFrom(TaskExecutorAllocationRequest req) { + ImmutableMap.Builder tagsBuilder = ImmutableMap.builder() + .put("resourceCluster", clusterID.getResourceID()) + .put("workerId", req.getWorkerId().getId()) + .put("jobCluster", req.getWorkerId().getJobCluster()); + + if (req.getConstraints().getSizeName().isPresent()) { + tagsBuilder.put("sizeName", req.getConstraints().getSizeName().get()); + } else { + tagsBuilder.put("cpuCores", String.valueOf(req.getConstraints().getMachineDefinition().getCpuCores())) + .put("memoryMB", String.valueOf(req.getConstraints().getMachineDefinition().getMemoryMB())); + } + + return TagList.create(tagsBuilder.build()); + } + + private ResourceOverview getResourceOverview() { + return this.delegate.getResourceOverview(); + } +} diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ExecutorStateManagerImpl.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ExecutorStateManagerImpl.java index 372552278..e33122f1a 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ExecutorStateManagerImpl.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ExecutorStateManagerImpl.java @@ -16,10 +16,16 @@ package io.mantisrx.master.resourcecluster; +import com.netflix.spectator.api.Registry; +import com.netflix.spectator.api.Tag; +import com.netflix.spectator.api.Timer; import io.mantisrx.common.WorkerConstants; +import io.mantisrx.common.metrics.spectator.MetricId; +import io.mantisrx.common.metrics.spectator.SpectatorRegistryFactory; import io.mantisrx.master.resourcecluster.ResourceClusterActor.BestFit; import io.mantisrx.master.resourcecluster.ResourceClusterActor.GetActiveJobsRequest; import io.mantisrx.master.resourcecluster.ResourceClusterActor.GetClusterUsageRequest; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.PendingReservationInfo; import io.mantisrx.master.resourcecluster.ResourceClusterActor.TaskExecutorBatchAssignmentRequest; import io.mantisrx.master.resourcecluster.proto.GetClusterIdleInstancesRequest; import io.mantisrx.master.resourcecluster.proto.GetClusterUsageResponse; @@ -30,6 +36,7 @@ import io.mantisrx.runtime.MachineDefinition; import io.mantisrx.server.core.domain.WorkerId; import io.mantisrx.server.core.scheduler.SchedulingConstraints; +import io.mantisrx.server.master.resourcecluster.ClusterID; import io.mantisrx.server.master.resourcecluster.ContainerSkuID; import io.mantisrx.server.master.resourcecluster.ResourceCluster.ResourceOverview; import io.mantisrx.server.master.resourcecluster.TaskExecutorAllocationRequest; @@ -45,6 +52,7 @@ import java.util.ArrayList; import java.util.Comparator; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -54,6 +62,7 @@ import java.util.Set; import java.util.TreeSet; import java.util.concurrent.TimeUnit; +import java.util.function.Function; import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -67,7 +76,11 @@ @Slf4j public class ExecutorStateManagerImpl implements ExecutorStateManager { + private static final String METRIC_GROUP_ID = "ExecutorStateManager"; + private final Map taskExecutorStateMap = new HashMap<>(); + private final Registry registry; + private final Timer clusterUsageWithReservationsTimer; Cache pendingJobRequests = CacheBuilder.newBuilder() .maximumSize(1000) .expireAfterWrite(10, TimeUnit.MINUTES) @@ -128,31 +141,41 @@ private TaskExecutorGroupKey findBestFitGroupOrDefault(SchedulingConstraints con private final AvailableTaskExecutorMutatorHook availableTaskExecutorMutatorHook; + // NEW: Flag to enable reservation-aware usage computation + private final boolean reservationSchedulingEnabled; + ExecutorStateManagerImpl(Map schedulingAttributes) { - this.schedulingAttributes = schedulingAttributes; - this.fitnessCalculator = new CpuWeightedFitnessCalculator(); - this.schedulerLeaseExpirationDuration = Duration.ofMillis(100); - this.availableTaskExecutorMutatorHook = null; + this(schedulingAttributes, new CpuWeightedFitnessCalculator(), Duration.ofMillis(100), null, false); } ExecutorStateManagerImpl( Map schedulingAttributes, FitnessCalculator fitnessCalculator, Duration schedulerLeaseExpirationDuration) { - this.schedulingAttributes = schedulingAttributes; - this.fitnessCalculator = fitnessCalculator; - this.schedulerLeaseExpirationDuration = schedulerLeaseExpirationDuration; - this.availableTaskExecutorMutatorHook = null; + this(schedulingAttributes, fitnessCalculator, schedulerLeaseExpirationDuration, null, false); } ExecutorStateManagerImpl(Map schedulingAttributes, FitnessCalculator fitnessCalculator, Duration schedulerLeaseExpirationDuration, AvailableTaskExecutorMutatorHook availableTaskExecutorMutatorHook) { + this(schedulingAttributes, fitnessCalculator, schedulerLeaseExpirationDuration, availableTaskExecutorMutatorHook, false); + } + + ExecutorStateManagerImpl(Map schedulingAttributes, + FitnessCalculator fitnessCalculator, + Duration schedulerLeaseExpirationDuration, + AvailableTaskExecutorMutatorHook availableTaskExecutorMutatorHook, + boolean reservationSchedulingEnabled) { this.schedulingAttributes = schedulingAttributes; this.fitnessCalculator = fitnessCalculator; this.schedulerLeaseExpirationDuration = schedulerLeaseExpirationDuration; this.availableTaskExecutorMutatorHook = availableTaskExecutorMutatorHook; + this.reservationSchedulingEnabled = reservationSchedulingEnabled; + this.registry = SpectatorRegistryFactory.getRegistry(); + this.clusterUsageWithReservationsTimer = registry.timer( + new MetricId(METRIC_GROUP_ID, "clusterUsageWithReservationsLatency", Tag.of("class", "ExecutorStateManagerImpl")) + .getSpectatorId(registry)); } @Override @@ -526,6 +549,214 @@ public GetClusterUsageResponse getClusterUsage(GetClusterUsageRequest req) { return res; } + @Override + public GetClusterUsageResponse getClusterUsageWithReservations( + ClusterID clusterID, + Function> groupKeyFunc, + List pendingReservations) { + + final long startNanos = System.nanoTime(); + try { + // Step 1: Compute base usage by SKU (idle count, total count) + // Note: We do NOT use pendingJobRequests cache here because: + // - When reservation scheduling is enabled, pending reservations already contain the demand info + // - pendingJobRequests would be duplicate/conflicting information + // - pendingJobRequests is only used in the legacy getClusterUsage() path + Map> usageByGroupKey = computeBaseUsage(groupKeyFunc); + + // Step 2: Map pending reservations to SKU keys using actual SchedulingConstraints + Map reservationCountBySku = mapReservationsToSku(pendingReservations, groupKeyFunc); + + // Step 3: Build response with merged counts + GetClusterUsageResponseBuilder resBuilder = GetClusterUsageResponse.builder().clusterID(clusterID); + + // Collect all SKU keys (from both current usage and reservations) + Set allSkuKeys = new HashSet<>(); + allSkuKeys.addAll(usageByGroupKey.keySet()); + allSkuKeys.addAll(reservationCountBySku.keySet()); + + for (String skuKey : allSkuKeys) { + Pair usage = usageByGroupKey.getOrDefault(skuKey, Pair.of(0, 0)); + int pendingFromReservations = reservationCountBySku.getOrDefault(skuKey, 0); + + resBuilder.usage(UsageByGroupKey.builder() + .usageGroupKey(skuKey) + .idleCount(usage.getLeft()) // No pendingJobRequests deduction - reservations are the source of truth + .totalCount(usage.getRight()) + .pendingReservationCount(pendingFromReservations) + .build()); + } + + GetClusterUsageResponse res = resBuilder.build(); + log.info("Usage result with reservations: {}", res); + return res; + } finally { + clusterUsageWithReservationsTimer.record(System.nanoTime() - startNanos, TimeUnit.NANOSECONDS); + } + } + + /** + * Compute base usage by SKU (idle count, total count). + * + * @param groupKeyFunc Function to extract SKU key from TaskExecutorRegistration + * @return Map of SKU key to (idleCount, totalCount) + */ + private Map> computeBaseUsage( + Function> groupKeyFunc) { + + Map> usageByGroupKey = new HashMap<>(); + + taskExecutorStateMap.forEach((key, value) -> { + if (value == null || value.getRegistration() == null) { + return; + } + + Optional groupKeyO = groupKeyFunc.apply(value.getRegistration()); + if (!groupKeyO.isPresent()) { + return; + } + + String groupKey = groupKeyO.get(); + Pair kvState = Pair.of( + value.isAvailable() && !value.isDisabled() ? 1 : 0, + value.isRegistered() ? 1 : 0); + + usageByGroupKey.merge(groupKey, kvState, + (prev, curr) -> Pair.of(prev.getLeft() + curr.getLeft(), prev.getRight() + curr.getRight())); + }); + + return usageByGroupKey; + } + + /** + * Map pending reservations to SKU keys using actual SchedulingConstraints. + * Uses findBestGroupForUsage with real constraints - NO PARSING NEEDED! + * + * @param pendingReservations Pending reservations with actual SchedulingConstraints + * @param groupKeyFunc Function to extract SKU key from TaskExecutorRegistration + * @return Map of SKU key to pending reservation worker count + */ + private Map mapReservationsToSku( + List pendingReservations, + Function> groupKeyFunc) { + + Map reservationCountBySku = new HashMap<>(); + + if (pendingReservations == null || pendingReservations.isEmpty()) { + return reservationCountBySku; + } + + for (PendingReservationInfo reservation : pendingReservations) { + int totalRequestedWorkers = reservation.getTotalRequestedWorkers(); + if (totalRequestedWorkers <= 0) { + continue; + } + + // Use actual SchedulingConstraints directly - no parsing needed! + SchedulingConstraints constraints = reservation.getSchedulingConstraints(); + Optional bestGroup = findBestGroupForUsage(constraints); + + if (bestGroup.isPresent()) { + // Map TaskExecutorGroupKey to SKU using sample TE + Optional skuKey = mapGroupKeyToSkuViaGroupKeyFunc(bestGroup.get(), groupKeyFunc); + skuKey.ifPresent(sku -> + reservationCountBySku.merge(sku, totalRequestedWorkers, Integer::sum)); + + log.debug("Mapped reservation {} ({} workers) to SKU {} via group {}", + reservation.getCanonicalConstraintKey(), + totalRequestedWorkers, + skuKey.orElse("unknown"), + bestGroup.get()); + } else { + log.error("Cannot map reservation constraints {} to any TaskExecutorGroup; " + + "{} pending workers will not be counted in scaling", + constraints, totalRequestedWorkers); + //todo add metrics + } + } + + return reservationCountBySku; + } + + /** + * Find the best matching TaskExecutorGroupKey for given SchedulingConstraints. + * This is a relaxed version of findBestGroup - it doesn't require available TEs, + * just finds which group would be used for scheduling. + * + * Reuses existing matching logic from findBestGroupBySizeNameMatch and + * findBestGroupByFitnessCalculator but without availability requirements. + * + * @param constraints The actual SchedulingConstraints from the reservation + * @return Optional TaskExecutorGroupKey that would match these constraints + */ + private Optional findBestGroupForUsage(SchedulingConstraints constraints) { + // Try size name match first (same logic as findBestGroupBySizeNameMatch) + Optional bySizeName = executorsByGroup.keySet().stream() + .filter(group -> group.getSizeName().isPresent()) + .filter(group -> constraints.getSizeName().isPresent()) + .filter(group -> group.getSizeName().get().equalsIgnoreCase(constraints.getSizeName().get())) + .filter(group -> areSchedulingAttributeConstraintsSatisfied(constraints, group.getSchedulingAttributes())) + .findFirst(); + + if (bySizeName.isPresent()) { + return bySizeName; + } + + // Fall back to fitness calculator (same logic as findBestGroupByFitnessCalculator) + if (constraints.getMachineDefinition() != null) { + return executorsByGroup.keySet().stream() + // Filter out if both sizeName exist and are different + .filter(group -> { + Optional teGroupSizeName = group.getSizeName(); + Optional requestSizeName = constraints.getSizeName(); + return !(teGroupSizeName.isPresent() && requestSizeName.isPresent() + && !teGroupSizeName.get().equalsIgnoreCase(requestSizeName.get())); + }) + // Verify scheduling attribute constraints + .filter(group -> areSchedulingAttributeConstraintsSatisfied(constraints, group.getSchedulingAttributes())) + // Calculate fitness and filter positive scores + .map(group -> Pair.of(group, + fitnessCalculator.calculate(constraints.getMachineDefinition(), group.getMachineDefinition()))) + .filter(pair -> pair.getRight() > 0) + // Get highest fitness score + .max(Comparator.comparingDouble(Pair::getRight)) + .map(Pair::getLeft); + } + + return Optional.empty(); + } + + /** + * Map a TaskExecutorGroupKey to SKU key using the provided groupKeyFunc. + * Finds the first TE from the group with a valid state and registration for mapping. + * + * @param groupKey The TaskExecutorGroupKey to map + * @param groupKeyFunc Function to extract SKU key from TaskExecutorRegistration + * @return Optional SKU key + */ + private Optional mapGroupKeyToSkuViaGroupKeyFunc( + TaskExecutorGroupKey groupKey, + Function> groupKeyFunc) { + + NavigableSet holders = executorsByGroup.get(groupKey); + if (holders == null || holders.isEmpty()) { + log.debug("No TaskExecutors found for group {}", groupKey); + return Optional.empty(); + } + + // Find the first holder with valid state and registration + for (TaskExecutorHolder holder : holders) { + TaskExecutorState state = taskExecutorStateMap.get(holder.getId()); + if (state != null && state.getRegistration() != null) { + // Apply the same groupKeyFunc that scaler uses + return groupKeyFunc.apply(state.getRegistration()); + } + } + + log.debug("No TaskExecutor with valid registration found for group {}", groupKey); + return Optional.empty(); + } + /** * Calculates the total count of pending scheduling requests for a specific Task Executor group. * The function does this by summing over the job requests' group-to-task executor counts. diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ReservationRegistryActor.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ReservationRegistryActor.java new file mode 100644 index 000000000..8cecfa9bf --- /dev/null +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ReservationRegistryActor.java @@ -0,0 +1,684 @@ +package io.mantisrx.master.resourcecluster; + +import akka.actor.AbstractActorWithTimers; +import akka.actor.ActorRef; +import akka.actor.Props; +import akka.actor.Status; +import akka.pattern.Patterns; +import io.mantisrx.common.Ack; +import io.mantisrx.server.master.resourcecluster.ResourceCluster.NoResourceAvailableException; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.GetPendingReservationsView; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.PendingReservationGroupView; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.PendingReservationsView; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.ProcessReservationsTick; +import static io.mantisrx.server.master.resourcecluster.proto.MantisResourceClusterReservationProto.*; +import io.mantisrx.server.master.resourcecluster.ClusterID; +import io.mantisrx.runtime.MachineDefinition; +import io.mantisrx.server.core.scheduler.SchedulingConstraints; +import java.time.Clock; +import java.time.Duration; +import java.time.Instant; +import java.util.*; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionStage; +import java.util.stream.Collectors; +import javax.annotation.Nullable; + +import io.mantisrx.server.worker.TaskExecutorGateway; +import lombok.Builder; +import lombok.Value; +import lombok.extern.slf4j.Slf4j; +import scala.compat.java8.FutureConverters; +import com.netflix.spectator.api.TagList; +import io.mantisrx.shaded.com.google.common.collect.ImmutableMap; + +import static akka.pattern.Patterns.pipe; + +/** + * Actor responsible for tracking and prioritizing reservations per scheduling constraint. The actor keeps all + * reservation state in-memory under the resource cluster and notifies the parent when batches are ready to assign. + * in current lifecycle resource cluster actor (the parent) won't start until all job clusters finish initialization, + * thus it's safe to do the auto-mark-ready in this class. Be careful with this limitation when making changes to + * lifecycle between job cluster actors and resource cluster/registry actors. + */ +@Slf4j +public class ReservationRegistryActor extends AbstractActorWithTimers { + private static final String TIMER_KEY_PROCESS = "reservation-registry-process"; + private static final String TIMER_KEY_AUTO_MARK_READY = "reservation-registry-auto-mark-ready"; + private static final Duration DEFAULT_PROCESS_INTERVAL = Duration.ofMillis(1000); + private static final Duration DEFAULT_AUTO_MARK_READY_TIMEOUT = Duration.ofSeconds(5); + + private final ClusterID clusterID; + private final Clock clock; + private final Duration processingInterval; + private final Duration inFlightReservationTimeout; + private final Duration autoMarkReadyTimeout; + private final ResourceClusterActorMetrics metrics; + + private final Duration processingCooldown; + private final Comparator reservationComparator; + + private final Map> reservationsByKey; + private final Map reservationsByConstraint; + private final Map inFlightReservations; + private final Map inFlightReservationRequestTimestamps; + + private boolean ready; + private Instant lastProcessAt; + + public ReservationRegistryActor(ClusterID clusterID, Clock clock, Duration processingInterval, Duration inFlightReservationTimeout, Duration autoMarkReadyTimeout, ResourceClusterActorMetrics metrics) { + this.clusterID = clusterID; + this.clock = Objects.requireNonNull(clock, "clock"); + this.processingInterval = processingInterval == null ? DEFAULT_PROCESS_INTERVAL : processingInterval; + this.processingCooldown = this.processingInterval.dividedBy(2); + this.inFlightReservationTimeout = inFlightReservationTimeout == null ? this.processingInterval.multipliedBy(5) : inFlightReservationTimeout; + this.autoMarkReadyTimeout = autoMarkReadyTimeout == null ? DEFAULT_AUTO_MARK_READY_TIMEOUT : autoMarkReadyTimeout; + this.metrics = Objects.requireNonNull(metrics, "metrics"); + this.reservationComparator = Comparator + .comparing(Reservation::getPriority) + .thenComparingInt(System::identityHashCode); + this.reservationsByKey = new HashMap<>(); + this.reservationsByConstraint = new HashMap<>(); + this.inFlightReservations = new HashMap<>(); + this.inFlightReservationRequestTimestamps = new HashMap<>(); + this.lastProcessAt = Instant.EPOCH; + } + + public static Props props(ClusterID clusterID, Clock clock, Duration processingInterval, Duration inFlightReservationTimeout, Duration autoMarkReadyTimeout, ResourceClusterActorMetrics metrics) { + return Props.create(ReservationRegistryActor.class, clusterID, clock, processingInterval, inFlightReservationTimeout, autoMarkReadyTimeout, metrics); + } + + @Override + public void preStart() throws Exception { + super.preStart(); + getTimers().startTimerWithFixedDelay(TIMER_KEY_PROCESS, ProcessReservationsTick.INSTANCE, processingInterval); + getTimers().startSingleTimer(TIMER_KEY_AUTO_MARK_READY, AutoMarkReadyTick.INSTANCE, autoMarkReadyTimeout); + } + + @Override + public Receive createReceive() { + return receiveBuilder() + .match(UpsertReservation.class, this::onUpsertReservation) + .match(CancelReservation.class, this::onCancelReservation) + .match(GetPendingReservationsView.class, this::onGetPendingReservationsView) + .match(GetPendingReservationsForScaler.class, this::onGetPendingReservationsForScaler) + .match(MarkReady.class, message -> onMarkReady()) + .match(ProcessReservationsTick.class, message -> onProcessReservationsTick(false)) + .match(ResourceClusterActor.ForceProcessReservationsTick.class, message -> onProcessReservationsTick(true)) + .match(AutoMarkReadyTick.class, message -> onAutoMarkReadyTick()) + .match(ResourceClusterActor.TaskExecutorsAllocation.class, this::onTaskExecutorBatchAssignmentResult) + .match(Status.Failure.class, this::onStatusFailure) + .build(); + } + + private void onMarkReady() { + log.info("Mark ready for registry: {}", this.clusterID); + if (!ready) { + ready = true; + log.info("{}: Reservation registry marked ready; pending reservations={}", this.clusterID, reservationsByKey.size()); + // Cancel the auto-mark-ready timer since we're now ready + getTimers().cancel(TIMER_KEY_AUTO_MARK_READY); + } + sender().tell(Ack.getInstance(), self()); + triggerForcedProcessingLoop(); + } + + private void onAutoMarkReadyTick() { + if (!ready) { + ready = true; + log.info("{}: Reservation registry auto-marked ready; pending reservations={}", this.clusterID, reservationsByKey.size()); + triggerForcedProcessingLoop(); + } else { + log.debug("{}: Auto-mark-ready timer fired but registry is already ready; ignoring", this.clusterID); + } + } + + private void onUpsertReservation(UpsertReservation message) { + ReservationKey key = message.getReservationKey(); + LinkedList existingReservations = reservationsByKey.get(key); + String canonicalConstraintKey = canonicalize(message.getSchedulingConstraints()); + + Reservation reservation = Reservation.fromUpsertReservation( + message, + canonicalConstraintKey + ); + + Reservation sameShapeReservation = findReservationWithSameShape(existingReservations, reservation); + if (sameShapeReservation != null) { + log.warn("{}: Replacing existing reservation {} with new reservation {}", this.clusterID, sameShapeReservation, reservation); + replaceEntry(sameShapeReservation, reservation); + + // Metric for reservation update + metrics.incrementCounter( + ResourceClusterActorMetrics.RESERVATION_UPSERTED, + TagList.create(ImmutableMap.of( + "resourceCluster", + clusterID.getResourceID(), + "jobId", + key.getJobId(), + "operation", + "update"))); + + sender().tell(Ack.getInstance(), self()); + triggerProcessingLoop(); + return; + } + + // todo: this implies that if a job with pending scaleup got scaled down, it needs to reset worker list and + // re-submit a reservation with all to-be-created workers if any. + if (existingReservations != null && !existingReservations.isEmpty()) { + int latestTargetSize = existingReservations.getLast().getStageTargetSize(); + if (reservation.getStageTargetSize() < latestTargetSize) { + new ArrayList<>(existingReservations).forEach(this::removeEntryAndClearInFlight); + } + } + + addEntry(reservation); + + // Metric for reservation insert + metrics.incrementCounter( + ResourceClusterActorMetrics.RESERVATION_UPSERTED, + TagList.create(ImmutableMap.of( + "resourceCluster", + clusterID.getResourceID(), + "jobId", + key.getJobId(), + "operation", + "insert"))); + + log.info("{}: Upserted reservation {} (priority={}, requestedWorkers={})", + this.clusterID, key, reservation.getPriority(), reservation.getRequestedWorkersCount()); + sender().tell(Ack.getInstance(), self()); + triggerProcessingLoop(); + } + + private void onCancelReservation(CancelReservation cancel) { + LinkedList existingReservations = reservationsByKey.get(cancel.getReservationKey()); + if (existingReservations != null && !existingReservations.isEmpty()) { + new ArrayList<>(existingReservations).forEach(this::removeEntryAndClearInFlight); + log.info("{}: Cancelled reservation {}", this.clusterID, cancel.getReservationKey()); + sender().tell(Ack.getInstance(), self()); + triggerForcedProcessingLoop(); + } else { + sender().tell(Ack.getInstance(), self()); + } + } + + private void onGetPendingReservationsView(GetPendingReservationsView request) { + if (!ready) { + sender().tell( + PendingReservationsView.builder() + .ready(ready) + .build(), + self()); + return; + } + + Map groups = new LinkedHashMap<>(); + reservationsByConstraint.forEach((key, group) -> groups.put(key, group.snapshot())); + sender().tell( + PendingReservationsView.builder() + .ready(ready) + .groups(groups) + .build(), + self()); + } + + /** + * Handler for scaler integration - returns pending reservations with actual SchedulingConstraints. + */ + private void onGetPendingReservationsForScaler(GetPendingReservationsForScaler request) { + if (!ready) { + sender().tell( + PendingReservationsForScalerResponse.builder() + .ready(false) + .build(), + self()); + return; + } + + List reservationInfos = new ArrayList<>(); + + for (ConstraintGroup group : reservationsByConstraint.values()) { + // Get a sample reservation from this group to extract SchedulingConstraints + Optional sampleReservation = group.peekTop(); + if (sampleReservation.isEmpty()) { + continue; + } + + reservationInfos.add(PendingReservationInfoSnapshot.builder() + .canonicalConstraintKey(group.getCanonicalConstraintKey()) + .schedulingConstraints(sampleReservation.get().getSchedulingConstraints()) // Actual constraints! + .totalRequestedWorkers(group.getTotalRequestedWorkers()) + .reservationCount(group.size()) + .build()); + } + + sender().tell( + PendingReservationsForScalerResponse.builder() + .ready(true) + .reservations(reservationInfos) + .build(), + self()); + } + + private void onProcessReservationsTick(boolean forced) { + Instant now = clock.instant(); + Duration sinceLastProcess = Duration.between(lastProcessAt, now); + if (!forced && sinceLastProcess.compareTo(processingCooldown) < 0) { + log.trace("{}: Skipping reservation processing tick due to cooldown (sinceLast={}ms, cooldown={}ms)", + this.clusterID, + sinceLastProcess.toMillis(), + processingCooldown.toMillis()); + return; + } + + lastProcessAt = now; + log.trace("{}: Reservation registry tick - queued reservations", this.clusterID); + processReservation(); + } + + /* + Main process logic for each sku (constraintKey) is: + - take the first item in queue and track it as inflight if inflight is empty. + - send batch allocation request for the inflight reservation (only trigger once when it's insert as inflight). + - process loop ends on the group. Rely on callback from allocation (ReservationAllocationResponse) to update inflight. + - once inflight gets allocated the reservation is considered done. Any worker task submit error or worker init + error is going to be tracked by JobActor worker level heartbeat timeout. This also requires the executorManager + to send JobActor worker state update when the TEs are leased for allocation. This allows the reservation + registry to be free from tracking those failure. + */ + private void processReservation() { + if (!ready) { + log.warn("Reservation registry {} not ready; skipping processing", this.clusterID); + return; + } + + for (ConstraintGroup group : reservationsByConstraint.values()) { + String constraintKey = group.getCanonicalConstraintKey(); + Reservation inFlight = inFlightReservations.get(constraintKey); + + if (inFlight != null) { + // Check if the in-flight reservation has timed out and needs to be retried + Instant requestTimestamp = inFlightReservationRequestTimestamps.get(constraintKey); + if (requestTimestamp != null) { + Duration timeSinceRequest = Duration.between(requestTimestamp, clock.instant()); + if (timeSinceRequest.compareTo(inFlightReservationTimeout) >= 0) { + log.info("{}: In-flight reservation {} for constraint group {} has timed out ({}ms), retrying", + this.clusterID, inFlight.getKey(), constraintKey, timeSinceRequest.toMillis()); + + // Metric for inflight timeout + metrics.incrementCounter( + ResourceClusterActorMetrics.RESERVATION_INFLIGHT_TIMEOUT, + TagList.create(ImmutableMap.of( + "resourceCluster", + clusterID.getResourceID(), + "jobId", + inFlight.getKey().getJobId()))); + + // Clear the in-flight state to allow retry + inFlightReservations.remove(constraintKey); + inFlightReservationRequestTimestamps.remove(constraintKey); + // Continue processing to retry the reservation + } else { + log.info("{} Skipping constraint group {} - already has in-flight reservation {} (waiting for {}ms)", + this.clusterID, + constraintKey, + inFlight.getKey(), + inFlightReservationTimeout.minus(timeSinceRequest).toMillis()); + continue; + } + } else { + log.info("{} Skipping constraint group {} - already has in-flight reservation {}", + this.clusterID, constraintKey, inFlight.getKey()); + continue; + } + } + + // the top reservation is only removed from group from upsert change and success batch assignment reply. + Optional topReservationO = group.peekTop(); + if (topReservationO.isEmpty()) { + log.debug("{}: Skipping constraint group {} - no reservation to process", this.clusterID, constraintKey); + continue; + } + + log.info("{}: Sending batch assignment request for reservation {} in constraint group {}", + this.clusterID, topReservationO.get().getKey(), constraintKey); + + Reservation reservation = topReservationO.get(); + metrics.incrementCounter( + ResourceClusterActorMetrics.RESERVATION_PROCESSED, + TagList.create(ImmutableMap.of( + "resourceCluster", + clusterID.getResourceID(), + "jobId", + reservation.getKey().getJobId(), + "constraintKey", + constraintKey))); + + inFlightReservations.put(constraintKey, reservation); + inFlightReservationRequestTimestamps.put(constraintKey, clock.instant()); + + ResourceClusterActor.TaskExecutorBatchAssignmentRequest request = + new ResourceClusterActor.TaskExecutorBatchAssignmentRequest( + topReservationO.get().getAllocationRequests(), + this.clusterID, + topReservationO.get()); + + // todo: make ESM track allocation results. Dedupe on repeated requests and only clear after registry ack. + // todo: make batch request ack as reply. retry if allocation failed or inflight timeout. + +// CompletionStage askFut = FutureConverters.toJava( +// Patterns.ask( +// getContext().parent(), +// request, +// this.inFlightReservationTimeout.toMillis())) +// .thenApply( res -> new ReservationAllocationResponse( +// ((ResourceClusterActor.TaskExecutorsAllocation) res).getReservation(), +// ((ResourceClusterActor.TaskExecutorsAllocation) res), +// null)) +// .exceptionally(failure -> new ReservationAllocationResponse( +// topReservationO.get(), null, failure)) +// ; +// +// pipe( +// askFut, +// getContext().dispatcher()) +// .to(self()); + + getContext().parent().tell(request, self()); + } + } + + private void onTaskExecutorBatchAssignmentResult(ResourceClusterActor.TaskExecutorsAllocation result) { + if (result == null || result.getReservation() == null) { + log.warn("{}: Received null reservation allocation result, ignoring", this.clusterID); + return; + } + + Reservation reservation = result.getReservation(); + String constraintKey = reservation.getCanonicalConstraintKey(); + + log.info("{}: Received batch assignment result for reservation {} (constraintKey={})", + this.clusterID, reservation.getKey(), constraintKey); + + clearInFlightIfMatches(reservation); + removeEntry(reservation); + triggerForcedProcessingLoop(); + } + + private void clearInFlightIfMatches(Reservation reservation) { + String constraintKey = reservation.getCanonicalConstraintKey(); + Reservation inFlight = inFlightReservations.get(constraintKey); + + if (inFlight != null && inFlight.equals(reservation)) { + inFlightReservations.remove(constraintKey); + inFlightReservationRequestTimestamps.remove(constraintKey); + log.debug("{}: Cleared in-flight reservation for constraint group {}", this.clusterID, constraintKey); + } + } + + private void onStatusFailure(Status.Failure failure) { + Throwable cause = failure.cause(); + if (cause instanceof NoResourceAvailableException) { + NoResourceAvailableException exception = (NoResourceAvailableException) cause; + String exceptionConstraintKey = exception.getConstraintKey(); + + log.info("{}: Received NoResourceAvailableException: {} (constraintKey={})", + this.clusterID, exception.getMessage(), exceptionConstraintKey); + + if (exceptionConstraintKey != null) { + // Match the exact reservation by constraint key + Reservation matchingReservation = inFlightReservations.get(exceptionConstraintKey); + if (matchingReservation != null) { + // Update the timestamp for the matching in-flight reservation + // This tracks when the reservation last received a NoResourceAvailableException + // The reservation will be retried if this timestamp is older than the timeout + inFlightReservationRequestTimestamps.put(exceptionConstraintKey, clock.instant()); + log.info("{}: Updated request timestamp for in-flight reservation {} (constraintKey={}) due to NoResourceAvailableException", + this.clusterID, matchingReservation.getKey(), exceptionConstraintKey); + } else { + log.warn("{}: Received NoResourceAvailableException for constraintKey {} but no matching in-flight reservation found", + this.clusterID, exceptionConstraintKey); + } + } + } else { + log.warn("{}: Received Status.Failure with non-NoResourceAvailableException: {}", this.clusterID, cause != null ? cause.getClass().getName() : "null"); + } + } + + private void addEntry(Reservation reservation) { + reservationsByKey + .computeIfAbsent(reservation.getKey(), key -> new LinkedList<>()) + .addLast(reservation); + ConstraintGroup group = reservationsByConstraint.computeIfAbsent( + reservation.getCanonicalConstraintKey(), + key -> new ConstraintGroup(key, reservationComparator)); + group.add(reservation); + } + + private void replaceEntry(Reservation existing, Reservation replacement) { + removeEntryAndClearInFlight(existing); + addEntry(replacement); + } + + private void removeEntry(Reservation reservation) { + LinkedList reservations = reservationsByKey.get(reservation.getKey()); + if (reservations != null) { + reservations.remove(reservation); + if (reservations.isEmpty()) { + reservationsByKey.remove(reservation.getKey()); + } + else { + log.warn("reservation not found for removal: {}", reservation); + } + } + + ConstraintGroup group = reservationsByConstraint.get(reservation.getCanonicalConstraintKey()); + if (group != null) { + group.remove(reservation); + if (group.isEmpty()) { + reservationsByConstraint.remove(group.getCanonicalConstraintKey()); + } + else { + log.warn("reservation not found in group for removal: {}", reservation); + } + } + } + + private void removeEntryAndClearInFlight(Reservation reservation) { + removeEntry(reservation); + clearInFlightIfMatches(reservation); + } + + @Nullable + private Reservation findReservationWithSameShape( + @Nullable LinkedList reservations, + Reservation candidate + ) { + if (reservations == null || reservations.isEmpty()) { + return null; + } + + for (Iterator iterator = reservations.descendingIterator(); iterator.hasNext(); ) { + Reservation existing = iterator.next(); + if (existing.hasSameShape(candidate)) { + return existing; + } + } + return null; + } + + private void triggerProcessingLoop() { + self().tell(ProcessReservationsTick.INSTANCE, self()); + } + + private void triggerForcedProcessingLoop() { + self().tell(ResourceClusterActor.ForceProcessReservationsTick.INSTANCE, self()); + } + + private static String canonicalize(@Nullable SchedulingConstraints constraints) { + if (constraints == null) { + return "constraints:none"; + } + + StringBuilder builder = new StringBuilder(); + MachineDefinition machineDefinition = constraints.getMachineDefinition(); + if (machineDefinition != null) { + builder.append("md:") + .append(machineDefinition.getCpuCores()).append('/') + .append(machineDefinition.getMemoryMB()).append('/') + .append(machineDefinition.getDiskMB()).append('/') + .append(machineDefinition.getNetworkMbps()).append('/') + .append(machineDefinition.getNumPorts()); + } else { + builder.append("md:none"); + } + + builder.append(";size=").append(constraints.getSizeName().orElse("~")); + + Map attributes = constraints.getSchedulingAttributes(); + if (attributes != null && !attributes.isEmpty()) { + builder.append(";attr="); + attributes.entrySet() + .stream() + .sorted(Map.Entry.comparingByKey()) + .forEach(entry -> + builder.append(entry.getKey()) + .append('=') + .append(entry.getValue()) + .append(',')); + } else { + builder.append(";attr=-"); + } + + return builder.toString(); + } + + /** + * Timer message to trigger auto-mark-ready after timeout. + */ + enum AutoMarkReadyTick { + INSTANCE + } + + /** + * Request for pending reservations with full scheduling constraints (for scaler integration). + */ + enum GetPendingReservationsForScaler { + INSTANCE + } + + /** + * Response containing pending reservations with actual SchedulingConstraints. + */ + @Value + @Builder + public static class PendingReservationsForScalerResponse { + boolean ready; + @Builder.Default + List reservations = Collections.emptyList(); + } + + /** + * Snapshot of pending reservation info including actual SchedulingConstraints. + */ + @Value + @Builder + public static class PendingReservationInfoSnapshot { + String canonicalConstraintKey; + SchedulingConstraints schedulingConstraints; + int totalRequestedWorkers; + int reservationCount; + } + + private static final class ConstraintGroup { + private final String canonicalConstraintKey; + private final NavigableSet queue; + private int totalRequestedWorkers; + + private ConstraintGroup(String canonicalConstraintKey, Comparator comparator) { + this.canonicalConstraintKey = canonicalConstraintKey; + this.queue = new TreeSet<>(comparator); + this.totalRequestedWorkers = 0; + } + + void add(Reservation reservation) { + if (queue.add(reservation)) { + totalRequestedWorkers += reservation.getRequestedWorkersCount(); + } + } + + void remove(Reservation reservation) { + if (queue.remove(reservation)) { + totalRequestedWorkers -= reservation.getRequestedWorkersCount(); + } + } + + boolean isEmpty() { + return queue.isEmpty(); + } + + Optional peekTop() { + return queue.isEmpty() ? Optional.empty() : Optional.of(queue.first()); + } + + String getCanonicalConstraintKey() { + return canonicalConstraintKey; + } + + PendingReservationGroupView snapshot() { + List reservations = queue + .stream() + .map(ReservationSnapshot::fromReservation) + .collect(Collectors.toList()); + return PendingReservationGroupView.builder() + .canonicalConstraintKey(canonicalConstraintKey) + .reservationCount(queue.size()) + .totalRequestedWorkers(totalRequestedWorkers) + .reservations(reservations) + .build(); + } + + int size() { + return queue.size(); + } + + int getTotalRequestedWorkers() { + return totalRequestedWorkers; + } + } + + + @Value + @Builder + public static class ReservationSnapshot { + ReservationKey reservationKey; + int requestedWorkers; + int stageTargetSize; + Instant createdAt; + Instant lastUpdatedAt; + long priorityEpoch; + + static ReservationSnapshot fromReservation(Reservation reservation) { + return ReservationSnapshot.builder() + .reservationKey(reservation.getKey()) + .requestedWorkers(reservation.getRequestedWorkersCount()) + .stageTargetSize(reservation.getStageTargetSize()) + .priorityEpoch(reservation.getPriority().getTimestamp()) + .build(); + } + } + + @Value + @Builder + public static class ReservationAllocationResponse { + Reservation reservation; + ResourceClusterActor.TaskExecutorsAllocation taskExecutorsAllocation; + Throwable error; + + public boolean isSuccess() { + return error == null; + } + } +} diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClusterActor.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClusterActor.java index a2d16f16e..ced30d270 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClusterActor.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClusterActor.java @@ -16,35 +16,28 @@ package io.mantisrx.master.resourcecluster; -import static java.util.stream.Collectors.groupingBy; +import static akka.pattern.Patterns.pipe; import akka.actor.AbstractActorWithTimers; import akka.actor.ActorRef; -import akka.actor.OneForOneStrategy; import akka.actor.Props; import akka.actor.Status; import akka.actor.SupervisorStrategy; -import akka.japi.pf.DeciderBuilder; import akka.japi.pf.ReceiveBuilder; -import com.netflix.spectator.api.Tag; import com.netflix.spectator.api.TagList; import io.mantisrx.common.Ack; -import io.mantisrx.common.WorkerConstants; +import io.mantisrx.common.akka.MantisActorSupervisorStrategy; +import io.mantisrx.master.resourcecluster.ExecutorStateManagerActor.RefreshTaskExecutorJobArtifactCache; +import io.mantisrx.master.resourcecluster.ExecutorStateManagerActor.UpdateDisabledState; +import io.mantisrx.master.resourcecluster.ExecutorStateManagerActor.UpdateJobArtifactsToCache; import io.mantisrx.master.resourcecluster.proto.GetClusterIdleInstancesRequest; -import io.mantisrx.master.resourcecluster.proto.GetClusterIdleInstancesResponse; +import static io.mantisrx.server.master.resourcecluster.proto.MantisResourceClusterReservationProto.*; import io.mantisrx.master.scheduler.FitnessCalculator; -import io.mantisrx.server.core.CacheJobArtifactsRequest; -import io.mantisrx.server.core.JobCompletedReason; import io.mantisrx.server.core.domain.ArtifactID; import io.mantisrx.server.core.domain.WorkerId; import io.mantisrx.server.core.scheduler.SchedulingConstraints; import io.mantisrx.server.master.persistence.MantisJobStore; import io.mantisrx.server.master.resourcecluster.ClusterID; -import io.mantisrx.server.master.resourcecluster.PagedActiveJobOverview; -import io.mantisrx.server.master.resourcecluster.ResourceCluster.NoResourceAvailableException; -import io.mantisrx.server.master.resourcecluster.ResourceCluster.ResourceOverview; -import io.mantisrx.server.master.resourcecluster.ResourceCluster.TaskExecutorNotFoundException; -import io.mantisrx.server.master.resourcecluster.ResourceCluster.TaskExecutorStatus; import io.mantisrx.server.master.resourcecluster.TaskExecutorAllocationRequest; import io.mantisrx.server.master.resourcecluster.TaskExecutorDisconnection; import io.mantisrx.server.master.resourcecluster.TaskExecutorHeartbeat; @@ -53,23 +46,21 @@ import io.mantisrx.server.master.resourcecluster.TaskExecutorReport; import io.mantisrx.server.master.resourcecluster.TaskExecutorReport.Available; import io.mantisrx.server.master.resourcecluster.TaskExecutorReport.Occupied; +import io.mantisrx.server.master.ExecuteStageRequestFactory; import io.mantisrx.server.master.resourcecluster.TaskExecutorStatusChange; import io.mantisrx.master.jobcluster.job.worker.WorkerState; import io.mantisrx.master.jobcluster.job.worker.WorkerTerminate; import io.mantisrx.server.master.scheduler.JobMessageRouter; -import io.mantisrx.server.worker.TaskExecutorGateway.TaskNotFoundException; import io.mantisrx.shaded.com.google.common.base.Preconditions; import io.mantisrx.shaded.com.google.common.collect.Comparators; import io.mantisrx.shaded.com.google.common.collect.ImmutableList; import io.mantisrx.shaded.com.google.common.collect.ImmutableMap; -import io.vavr.Tuple; + import java.io.IOException; -import java.net.URI; import java.time.Clock; import java.time.Duration; import java.time.Instant; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -80,7 +71,6 @@ import java.util.Optional; import java.util.Set; import java.util.function.Function; -import java.util.function.Predicate; import java.util.stream.Collectors; import javax.annotation.Nullable; import lombok.AllArgsConstructor; @@ -90,6 +80,9 @@ import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.tuple.Pair; import org.apache.flink.runtime.rpc.RpcService; +import akka.pattern.Patterns; +import scala.Option; +import scala.compat.java8.FutureConverters; /** * Akka actor implementation of ResourceCluster. @@ -101,23 +94,9 @@ @ToString(of = {"clusterID"}) @Slf4j public class ResourceClusterActor extends AbstractActorWithTimers { - /** - * For ResourceClusterActor instances, we need to ensure they are always running after encountering error so that - * TaskExecutors can still remain connected. If there is a fatal error that needs to be escalated to terminate the - * whole system/leader you can define a fatal exception type and override its behavior to - * SupervisorStrategy.escalate() instead. - */ - private static SupervisorStrategy resourceClusterActorStrategy = - new OneForOneStrategy( - 3, - Duration.ofSeconds(60), - DeciderBuilder - .match(Exception.class, e -> SupervisorStrategy.restart()) - .build()); - @Override public SupervisorStrategy supervisorStrategy() { - return resourceClusterActorStrategy; + return MantisActorSupervisorStrategy.getInstance().create(); } private final Duration heartbeatTimeout; @@ -130,9 +109,8 @@ public SupervisorStrategy supervisorStrategy() { private final RpcService rpcService; private final ClusterID clusterID; private final MantisJobStore mantisJobStore; - private final Set activeDisableTaskExecutorsByAttributesRequests; - private final Set disabledTaskExecutors; private final JobMessageRouter jobMessageRouter; + private final ExecuteStageRequestFactory executeStageRequestFactory; private final ResourceClusterActorMetrics metrics; @@ -143,6 +121,14 @@ public SupervisorStrategy supervisorStrategy() { private final boolean isJobArtifactCachingEnabled; + private final String reservationRegistryActorName; + private ActorRef reservationRegistryActor; + + private final String executorStateManagerActorName; + private ActorRef executorStateManagerActor; + + private final boolean reservationSchedulingEnabled; + static Props props( final ClusterID clusterID, final Duration heartbeatTimeout, @@ -158,7 +144,9 @@ static Props props( boolean isJobArtifactCachingEnabled, Map schedulingAttributes, FitnessCalculator fitnessCalculator, - AvailableTaskExecutorMutatorHook availableTaskExecutorMutatorHook + AvailableTaskExecutorMutatorHook availableTaskExecutorMutatorHook, + ExecuteStageRequestFactory executeStageRequestFactory, + boolean reservationSchedulingEnabled ) { return Props.create( ResourceClusterActor.class, @@ -176,7 +164,49 @@ static Props props( isJobArtifactCachingEnabled, schedulingAttributes, fitnessCalculator, - availableTaskExecutorMutatorHook + availableTaskExecutorMutatorHook, + executeStageRequestFactory, + reservationSchedulingEnabled + ).withMailbox("akka.actor.metered-mailbox"); + } + + static Props props( + final ClusterID clusterID, + final Duration heartbeatTimeout, + Duration assignmentTimeout, + Duration disabledTaskExecutorsCheckInterval, + Duration schedulerLeaseExpirationDuration, + Clock clock, + RpcService rpcService, + MantisJobStore mantisJobStore, + JobMessageRouter jobMessageRouter, + int maxJobArtifactsToCache, + String jobClustersWithArtifactCachingEnabled, + boolean isJobArtifactCachingEnabled, + Map schedulingAttributes, + FitnessCalculator fitnessCalculator, + ExecuteStageRequestFactory executeStageRequestFactory, + boolean reservationSchedulingEnabled + ) { + return Props.create( + ResourceClusterActor.class, + clusterID, + heartbeatTimeout, + assignmentTimeout, + disabledTaskExecutorsCheckInterval, + schedulerLeaseExpirationDuration, + clock, + rpcService, + mantisJobStore, + jobMessageRouter, + maxJobArtifactsToCache, + jobClustersWithArtifactCachingEnabled, + isJobArtifactCachingEnabled, + schedulingAttributes, + fitnessCalculator, + null, + executeStageRequestFactory, + reservationSchedulingEnabled ).withMailbox("akka.actor.metered-mailbox"); } @@ -195,7 +225,9 @@ static Props props( boolean isJobArtifactCachingEnabled, Map schedulingAttributes, FitnessCalculator fitnessCalculator, - AvailableTaskExecutorMutatorHook availableTaskExecutorMutatorHook) { + AvailableTaskExecutorMutatorHook availableTaskExecutorMutatorHook, + ExecuteStageRequestFactory executeStageRequestFactory, + boolean reservationSchedulingEnabled) { this.clusterID = clusterID; this.heartbeatTimeout = heartbeatTimeout; this.assignmentTimeout = assignmentTimeout; @@ -207,15 +239,17 @@ static Props props( this.rpcService = rpcService; this.jobMessageRouter = jobMessageRouter; this.mantisJobStore = mantisJobStore; - this.activeDisableTaskExecutorsByAttributesRequests = new HashSet<>(); - this.disabledTaskExecutors = new HashSet<>(); + this.executeStageRequestFactory = executeStageRequestFactory; this.maxJobArtifactsToCache = maxJobArtifactsToCache; this.jobClustersWithArtifactCachingEnabled = jobClustersWithArtifactCachingEnabled; + this.reservationSchedulingEnabled = reservationSchedulingEnabled; this.executorStateManager = new ExecutorStateManagerImpl( - schedulingAttributes, fitnessCalculator, this.schedulerLeaseExpirationDuration, availableTaskExecutorMutatorHook); + schedulingAttributes, fitnessCalculator, this.schedulerLeaseExpirationDuration, availableTaskExecutorMutatorHook, reservationSchedulingEnabled); this.metrics = new ResourceClusterActorMetrics(); + this.reservationRegistryActorName = buildReservationRegistryActorName(clusterID); + this.executorStateManagerActorName = buildExecutorStateManagerActorName(clusterID); } @Override @@ -227,23 +261,42 @@ public void preStart() throws Exception { "resourceCluster", clusterID.getResourceID()))); - fetchJobArtifactsToCache(); + Option existingRegistry = getContext().child(reservationRegistryActorName); + if (existingRegistry.isDefined()) { + reservationRegistryActor = existingRegistry.get(); + } else { + Props registryProps = ReservationRegistryActor.props(this.clusterID, clock, null, null, null, metrics); + reservationRegistryActor = getContext().actorOf(registryProps, reservationRegistryActorName); + } - final Instant now = clock.instant(); - List activeRequests = - mantisJobStore.loadAllDisableTaskExecutorsRequests(clusterID); - for (DisableTaskExecutorsRequest request : activeRequests) { - if (!request.isExpired(now)) { - onNewDisableTaskExecutorsRequest(request); - } else { - onDisableTaskExecutorsRequestExpiry(new ExpireDisableTaskExecutorsRequest(request)); + Option existingExecutorStateManager = getContext().child(executorStateManagerActorName); + if (existingExecutorStateManager.isDefined()) { + executorStateManagerActor = existingExecutorStateManager.get(); + } else { + if (!(executorStateManager instanceof ExecutorStateManagerImpl)) { + throw new IllegalStateException("ExecutorStateManager is not an instance of ExecutorStateManagerImpl"); } - } + Props esmProps = ExecutorStateManagerActor.props( + (ExecutorStateManagerImpl) executorStateManager, + clock, + rpcService, + jobMessageRouter, + mantisJobStore, + heartbeatTimeout, + assignmentTimeout, + disabledTaskExecutorsCheckInterval, + clusterID, + isJobArtifactCachingEnabled, + jobClustersWithArtifactCachingEnabled, + metrics, + executeStageRequestFactory, + reservationSchedulingEnabled); + executorStateManagerActor = getContext().actorOf(esmProps, executorStateManagerActorName); + } + + syncExecutorJobArtifactsCache(); - timers().startTimerWithFixedDelay( - String.format("periodic-disabled-task-executors-test-for-%s", clusterID.getResourceID()), - new CheckDisabledTaskExecutors("periodic"), - disabledTaskExecutorsCheckInterval); + fetchJobArtifactsToCache(); timers().startTimerWithFixedDelay( "periodic-resource-overview-metrics-publisher", @@ -256,50 +309,154 @@ public Receive createReceive() { return ReceiveBuilder .create() + .match(UpsertReservation.class, this::forwardToReservationRegistry) + .match(CancelReservation.class, this::forwardToReservationRegistry) + .match(GetPendingReservationsView.class, this::forwardToReservationRegistry) + .match(MarkReady.class, this::forwardToReservationRegistry) .match(GetRegisteredTaskExecutorsRequest.class, - req -> { - sender().tell(getTaskExecutors(filterByAttrs(req).and(ExecutorStateManager.isRegistered)), self()); - }) - .match(GetBusyTaskExecutorsRequest.class, req -> sender().tell(getTaskExecutors(filterByAttrs(req).and(ExecutorStateManager.isBusy)), self())) - .match(GetAvailableTaskExecutorsRequest.class, req -> sender().tell(getTaskExecutors(filterByAttrs(req).and(ExecutorStateManager.isAvailable)), self())) - .match(GetDisabledTaskExecutorsRequest.class, req -> sender().tell(getTaskExecutors(filterByAttrs(req).and(ExecutorStateManager.isDisabled)), self())) - .match(GetUnregisteredTaskExecutorsRequest.class, req -> sender().tell(getTaskExecutors(filterByAttrs(req).and(ExecutorStateManager.unregistered)), self())) - .match(GetActiveJobsRequest.class, this::getActiveJobs) - .match(GetTaskExecutorStatusRequest.class, this::getTaskExecutorStatus) + metrics.withTracking(this::forwardToExecutorStateManager)) + .match(GetBusyTaskExecutorsRequest.class, + metrics.withTracking(this::forwardToExecutorStateManager)) + .match(GetAvailableTaskExecutorsRequest.class, + metrics.withTracking(this::forwardToExecutorStateManager)) + .match(GetDisabledTaskExecutorsRequest.class, + metrics.withTracking(this::forwardToExecutorStateManager)) + .match(GetUnregisteredTaskExecutorsRequest.class, + metrics.withTracking(this::forwardToExecutorStateManager)) + .match(GetActiveJobsRequest.class, + metrics.withTracking(this::forwardToExecutorStateManager)) + .match(GetTaskExecutorStatusRequest.class, + metrics.withTracking(this::forwardToExecutorStateManager)) .match(GetClusterUsageRequest.class, - metrics.withTracking(req -> - sender().tell(this.executorStateManager.getClusterUsage(req), self()))) + metrics.withTracking(this::forwardToExecutorStateManager)) + .match(GetReservationAwareClusterUsageRequest.class, + metrics.withTracking(this::onGetReservationAwareClusterUsage)) .match(GetClusterIdleInstancesRequest.class, - metrics.withTracking(req -> - sender().tell(onGetClusterIdleInstancesRequest(req), self()))) - .match(GetAssignedTaskExecutorRequest.class, this::onAssignedTaskExecutorRequest) - .match(MarkExecutorTaskCancelledRequest.class, this::onMarkExecutorTaskCancelledRequest) + metrics.withTracking(this::forwardToExecutorStateManager)) + .match(GetAssignedTaskExecutorRequest.class, + metrics.withTracking(this::forwardToExecutorStateManager)) + .match(MarkExecutorTaskCancelledRequest.class, + metrics.withTracking(this::forwardToExecutorStateManager)) .match(Ack.class, ack -> log.info("Received ack from {}", sender())) - .match(TaskExecutorAssignmentTimeout.class, this::onTaskExecutorAssignmentTimeout) - .match(TaskExecutorRegistration.class, metrics.withTracking(this::onTaskExecutorRegistration)) - .match(InitializeTaskExecutorRequest.class, metrics.withTracking(this::onTaskExecutorInitialization)) - .match(TaskExecutorHeartbeat.class, metrics.withTracking(this::onHeartbeat)) - .match(TaskExecutorStatusChange.class, this::onTaskExecutorStatusChange) - .match(TaskExecutorDisconnection.class, metrics.withTracking(this::onTaskExecutorDisconnection)) - .match(HeartbeatTimeout.class, metrics.withTracking(this::onTaskExecutorHeartbeatTimeout)) - .match(TaskExecutorBatchAssignmentRequest.class, metrics.withTracking(this::onTaskExecutorBatchAssignmentRequest)) - .match(ResourceOverviewRequest.class, this::onResourceOverviewRequest) - .match(TaskExecutorInfoRequest.class, this::onTaskExecutorInfoRequest) - .match(TaskExecutorGatewayRequest.class, metrics.withTracking(this::onTaskExecutorGatewayRequest)) - .match(DisableTaskExecutorsRequest.class, this::onNewDisableTaskExecutorsRequest) - .match(CheckDisabledTaskExecutors.class, this::findAndMarkDisabledTaskExecutors) - .match(ExpireDisableTaskExecutorsRequest.class, this::onDisableTaskExecutorsRequestExpiry) - .match(GetTaskExecutorWorkerMappingRequest.class, req -> sender().tell(getTaskExecutorWorkerMapping(req.getAttributes()), self())) - .match(PublishResourceOverviewMetricsRequest.class, this::onPublishResourceOverviewMetricsRequest) - .match(CacheJobArtifactsOnTaskExecutorRequest.class, metrics.withTracking(this::onCacheJobArtifactsOnTaskExecutorRequest)) + .match(TaskExecutorRegistration.class, metrics.withTracking(this::forwardToExecutorStateManager)) + .match(InitializeTaskExecutorRequest.class, metrics.withTracking(this::forwardToExecutorStateManager)) + .match(TaskExecutorHeartbeat.class, metrics.withTracking(this::forwardToExecutorStateManager)) + .match(TaskExecutorStatusChange.class, metrics.withTracking(this::forwardToExecutorStateManager)) + .match(TaskExecutorDisconnection.class, metrics.withTracking(this::forwardToExecutorStateManager)) + .match(TaskExecutorBatchAssignmentRequest.class, metrics.withTracking(this::forwardToExecutorStateManager)) + .match(ResourceOverviewRequest.class, + metrics.withTracking(this::forwardToExecutorStateManager)) + .match(TaskExecutorInfoRequest.class, + metrics.withTracking(this::forwardToExecutorStateManager)) + .match(TaskExecutorGatewayRequest.class, + metrics.withTracking(this::forwardToExecutorStateManager)) + .match(DisableTaskExecutorsRequest.class, metrics.withTracking(this::forwardToExecutorStateManager)) + .match(CheckDisabledTaskExecutors.class, metrics.withTracking(this::forwardToExecutorStateManager)) + .match(GetTaskExecutorWorkerMappingRequest.class, + metrics.withTracking(this::forwardToExecutorStateManager)) + .match(PublishResourceOverviewMetricsRequest.class, + metrics.withTracking(this::forwardToExecutorStateManager)) + .match(CacheJobArtifactsOnTaskExecutorRequest.class, metrics.withTracking(req -> + pipe( + FutureConverters.toJava(Patterns.ask( + executorStateManagerActor, + req, + assignmentTimeout.toMillis())), + getContext().dispatcher()) + .to(sender(), self()))) .match(AddNewJobArtifactsToCacheRequest.class, this::onAddNewJobArtifactsToCacheRequest) .match(RemoveJobArtifactsToCacheRequest.class, this::onRemoveJobArtifactsToCacheRequest) .match(GetJobArtifactsToCacheRequest.class, req -> sender().tell(new ArtifactList(new ArrayList<>(jobArtifactsToCache)), self())) .build(); } + private void forwardToReservationRegistry(Object message) { + if (reservationRegistryActor == null) { + log.warn("Reservation registry actor not initialized; dropping {}", message); + sender().tell(new Status.Failure(new IllegalStateException("reservation registry not available")), self()); + return; + } + // TODO (reservation-registry): job actor and scheduler interactions will route through this bridge. + reservationRegistryActor.forward(message, getContext()); + } + + private void forwardToExecutorStateManager(Object message) { + if (executorStateManagerActor == null) { + log.warn("ExecutorStateManagerActor not initialized; dropping {}", message); + sender().tell(new Status.Failure(new IllegalStateException("executor state manager actor not available")), self()); + return; + } + executorStateManagerActor.forward(message, getContext()); + } + + /** + * Handler for reservation-aware cluster usage request. + * Two-phase approach: + * 1. Ask ReservationRegistryActor for pending reservations with actual SchedulingConstraints + * 2. Forward to ExecutorStateManagerActor with the enriched reservation info + */ + private void onGetReservationAwareClusterUsage(GetReservationAwareClusterUsageRequest request) { + final ActorRef originalSender = sender(); + + if (reservationRegistryActor == null) { + log.warn("ReservationRegistryActor not initialized; falling back to regular usage request"); + sender().tell(new Status.Failure(new IllegalStateException("reservationRegistryActor not available")), self()); + return; + } + + // Phase 1: Get pending reservations with actual constraints from ReservationRegistryActor + FutureConverters.toJava(Patterns.ask( + reservationRegistryActor, + ReservationRegistryActor.GetPendingReservationsForScaler.INSTANCE, + Duration.ofSeconds(5).toMillis())) + .thenApply(result -> (ReservationRegistryActor.PendingReservationsForScalerResponse) result) + .whenComplete((reservationsResponse, error) -> { + if (error != null) { + log.error("Failed to get pending reservations for usage request", error); + // Fall back to regular usage request without reservations + sender().tell( + new Status.Failure(new RuntimeException("Failed to get pending reservations for usage", error)), + self()); + } else { + // Phase 2: Convert to PendingReservationInfo and forward to ExecutorStateManagerActor + if (!reservationsResponse.isReady()) { + sender().tell( + new Status.Failure(new IllegalStateException("Reservation registry is not ready")), + self()); + return; + } + + List pendingReservations = + reservationsResponse.getReservations().stream() + .map(snapshot -> PendingReservationInfo.builder() + .canonicalConstraintKey(snapshot.getCanonicalConstraintKey()) + .schedulingConstraints(snapshot.getSchedulingConstraints()) + .totalRequestedWorkers(snapshot.getTotalRequestedWorkers()) + .reservationCount(snapshot.getReservationCount()) + .build()) + .collect(Collectors.toList()); + + log.debug("Forwarding usage request with {} pending reservation groups", pendingReservations.size()); + + executorStateManagerActor.tell( + new GetClusterUsageWithReservationsRequest( + request.getClusterID(), + request.getGroupKeyFunc(), + pendingReservations), + originalSender); + } + }); + } + private void syncExecutorJobArtifactsCache() { + if (executorStateManagerActor == null) { + return; + } + executorStateManagerActor.tell( + new UpdateJobArtifactsToCache(new HashSet<>(jobArtifactsToCache)), + self()); + } private void onAddNewJobArtifactsToCacheRequest(AddNewJobArtifactsToCacheRequest req) { try { @@ -312,6 +469,7 @@ private void onAddNewJobArtifactsToCacheRequest(AddNewJobArtifactsToCacheRequest jobArtifactsToCache.addAll(newArtifacts); mantisJobStore.addNewJobArtifactsToCache(req.getClusterID(), ImmutableList.copyOf(jobArtifactsToCache)); + syncExecutorJobArtifactsCache(); refreshTaskExecutorJobArtifactCache(); } else { log.warn("Cannot enable caching for artifacts {}. Max number ({}) of job artifacts to cache reached.", newArtifacts, maxJobArtifactsToCache); @@ -330,15 +488,18 @@ private void onAddNewJobArtifactsToCacheRequest(AddNewJobArtifactsToCacheRequest } private void refreshTaskExecutorJobArtifactCache() { - // TODO: implement rate control to confirm we are not overwhelming the TEs with excessive caching requests - getTaskExecutors(ExecutorStateManager.isAvailable).getTaskExecutors().forEach(taskExecutorID -> - self().tell(new CacheJobArtifactsOnTaskExecutorRequest(taskExecutorID, clusterID), self())); + if (executorStateManagerActor == null) { + log.warn("ExecutorStateManagerActor not initialized; skipping artifact cache refresh"); + return; + } + executorStateManagerActor.tell(new RefreshTaskExecutorJobArtifactCache(), self()); } private void onRemoveJobArtifactsToCacheRequest(RemoveJobArtifactsToCacheRequest req) { try { mantisJobStore.removeJobArtifactsToCache(req.getClusterID(), req.getArtifacts()); req.artifacts.forEach(jobArtifactsToCache::remove); + syncExecutorJobArtifactsCache(); sender().tell(Ack.getInstance(), self()); } catch (IOException e) { log.warn("Cannot remove job artifacts {} to cache in cluster: {}", req.getArtifacts(), req.getClusterID(), e); @@ -351,644 +512,24 @@ private void fetchJobArtifactsToCache() { .stream() .map(ArtifactID::of) .forEach(jobArtifactsToCache::add); + syncExecutorJobArtifactsCache(); } catch (IOException e) { log.warn("Cannot refresh job artifacts to cache in cluster: {}", clusterID, e); } } - private GetClusterIdleInstancesResponse onGetClusterIdleInstancesRequest(GetClusterIdleInstancesRequest req) { - log.info("Computing idle instance list: {}", req); - if (!req.getClusterID().equals(this.clusterID)) { - throw new RuntimeException(String.format("Mismatch cluster ids %s, %s", req.getClusterID(), this.clusterID)); - } - - List instanceList = this.executorStateManager.getIdleInstanceList(req); - - GetClusterIdleInstancesResponse res = GetClusterIdleInstancesResponse.builder() - .instanceIds(instanceList) - .clusterId(this.clusterID) - .skuId(req.getSkuId()) - .build(); - log.info("Return idle instance list: {}", res); - return res; - } - - private TaskExecutorsList getTaskExecutors(Predicate> predicate) { - return new TaskExecutorsList(this.executorStateManager.getTaskExecutors(predicate)); - } - - private void getActiveJobs(GetActiveJobsRequest req) { - List pagedList = this.executorStateManager.getActiveJobs(req); - - PagedActiveJobOverview res = - new PagedActiveJobOverview( - pagedList, - req.getStartingIndex().orElse(0) + pagedList.size() - ); - - log.info("Returning getActiveJobs res starting at {}: {}", req.getStartingIndex(), res.getActiveJobs().size()); - sender().tell(res, self()); - } - - private void onTaskExecutorInfoRequest(TaskExecutorInfoRequest request) { - if (request.getTaskExecutorID() != null) { - TaskExecutorState state = - this.executorStateManager.getIncludeArchived(request.getTaskExecutorID()); - if (state != null && state.getRegistration() != null) { - sender().tell(state.getRegistration(), self()); - } else { - sender().tell(new Status.Failure(new Exception(String.format("No task executor state for %s", - request.getTaskExecutorID()))), self()); - } - } else { - Optional taskExecutorRegistration = - this.executorStateManager - .findFirst( - kv -> kv.getValue().getRegistration() != null && - kv.getValue().getRegistration().getHostname().equals(request.getHostName())) - .map(Entry::getValue) - .map(TaskExecutorState::getRegistration); - if (taskExecutorRegistration.isPresent()) { - sender().tell(taskExecutorRegistration.get(), self()); - } else { - sender().tell(new Status.Failure(new Exception(String.format("Unknown task executor for hostname %s", request.getHostName()))), self()); - } - } - } - - private void onAssignedTaskExecutorRequest(GetAssignedTaskExecutorRequest request) { - Optional matchedTaskExecutor = - this.executorStateManager.findFirst( - e -> e.getValue().isRunningOrAssigned(request.getWorkerId())).map(Entry::getKey); - - if (matchedTaskExecutor.isPresent()) { - sender().tell(matchedTaskExecutor.get(), self()); - } else { - sender().tell(new Status.Failure(new TaskNotFoundException(request.getWorkerId())), - self()); - } - } - - private void onMarkExecutorTaskCancelledRequest(MarkExecutorTaskCancelledRequest request) { - Optional> matchedTaskExecutor = - this.executorStateManager.findFirst(e -> e.getValue().isRunningOrAssigned(request.getWorkerId())); - - if (matchedTaskExecutor.isPresent()) { - log.info("Setting executor {} to cancelled workerID: {}", matchedTaskExecutor.get().getKey(), request); - matchedTaskExecutor.get().getValue().setCancelledWorkerOnTask(request.getWorkerId()); - sender().tell(Ack.getInstance(), self()); - } else { - log.info("Cannot find executor to mark worker {} as cancelled", request); - sender().tell(new Status.Failure(new TaskNotFoundException(request.getWorkerId())), self()); - } - } - - private void onTaskExecutorGatewayRequest(TaskExecutorGatewayRequest request) { - TaskExecutorState state = this.executorStateManager.get(request.getTaskExecutorID()); - if (state == null) { - sender().tell(new NullPointerException("Null TaskExecutorState for: " + request.getTaskExecutorID()), self()); - } else { - try { - if (state.isRegistered()) { - sender().tell(state.getGatewayAsync(), self()); - } else { - sender().tell( - new Status.Failure(new IllegalStateException("Unregistered TaskExecutor: " + request.getTaskExecutorID())), - self()); - } - } catch (Exception e) { - log.error("onTaskExecutorGatewayRequest error: {}", request, e); - metrics.incrementCounter( - ResourceClusterActorMetrics.TE_CONNECTION_FAILURE, - TagList.create(ImmutableMap.of( - "resourceCluster", - clusterID.getResourceID(), - "taskExecutor", - request.getTaskExecutorID().getResourceId()))); - } - } - } - - // custom equals function to check if the existing set already has the request under consideration. - private boolean addNewDisableTaskExecutorsRequest(DisableTaskExecutorsRequest newRequest) { - if (newRequest.isRequestByAttributes()) { - log.info("Req with attributes {}", newRequest); - for (DisableTaskExecutorsRequest existing: activeDisableTaskExecutorsByAttributesRequests) { - if (existing.targetsSameTaskExecutorsAs(newRequest)) { - return false; - } - } - - Preconditions.checkState(activeDisableTaskExecutorsByAttributesRequests.add(newRequest), "activeDisableTaskExecutorRequests cannot contain %s", newRequest); - return true; - } else if (newRequest.getTaskExecutorID().isPresent() && !disabledTaskExecutors.contains(newRequest.getTaskExecutorID().get())) { - log.info("Req with id {}", newRequest); - disabledTaskExecutors.add(newRequest.getTaskExecutorID().get()); - return true; - } - log.info("No Req {}", newRequest); - return false; - } - - private void onNewDisableTaskExecutorsRequest(DisableTaskExecutorsRequest request) { - ActorRef sender = sender(); - if (addNewDisableTaskExecutorsRequest(request)) { - try { - log.info("New req to add {}", request); - // store the request in a persistent store in order to retrieve it if the node goes down - mantisJobStore.storeNewDisabledTaskExecutorsRequest(request); - // figure out the time to expire the current request - Duration toExpiry = Comparators.max(Duration.between(clock.instant(), request.getExpiry()), Duration.ZERO); - // setup a timer to clear it after a given period - getTimers().startSingleTimer( - getExpiryKeyFor(request), - new ExpireDisableTaskExecutorsRequest(request), - toExpiry); - findAndMarkDisabledTaskExecutorsFor(request); - sender.tell(Ack.getInstance(), self()); - } catch (IOException e) { - sender().tell(new Status.Failure(e), self()); - } - } else { - sender.tell(Ack.getInstance(), self()); - } - } - - private String getExpiryKeyFor(DisableTaskExecutorsRequest request) { - return "ExpireDisableTaskExecutorsRequest-" + request; - } - - private void findAndMarkDisabledTaskExecutorsFor(DisableTaskExecutorsRequest request) { - if (request.isRequestByAttributes()) { - findAndMarkDisabledTaskExecutors(new CheckDisabledTaskExecutors("new_request")); - } else if (request.getTaskExecutorID().isPresent()) { - final TaskExecutorID taskExecutorID = request.getTaskExecutorID().get(); - final TaskExecutorState state = this.executorStateManager.get(taskExecutorID); - if (state == null) { - // If the TE is unknown by mantis, delete it from state - disabledTaskExecutors.remove(taskExecutorID); - self().tell(new ExpireDisableTaskExecutorsRequest(request), self()); - } else { - log.info("Marking task executor {} as disabled", taskExecutorID); - state.onNodeDisabled(); - } - } - } - - private void findAndMarkDisabledTaskExecutors(CheckDisabledTaskExecutors r) { - log.info( - "Checking disabled task executors for Cluster {} because of {}. Current disabled request size: {}", - clusterID.getResourceID(), r.getReason(), activeDisableTaskExecutorsByAttributesRequests.size()); - final Instant now = clock.instant(); - for (DisableTaskExecutorsRequest request : activeDisableTaskExecutorsByAttributesRequests) { - if (request.isExpired(now)) { - self().tell(new ExpireDisableTaskExecutorsRequest(request), self()); - } else { - // go and mark all task executors that match the filter as disabled - this.executorStateManager.getActiveExecutorEntry().forEach(idAndState -> { - if (request.covers(idAndState.getValue().getRegistration())) { - if (idAndState.getValue().onNodeDisabled()) { - log.info("Marking task executor {} as disabled", idAndState.getKey()); - } - } - }); - } - } - } - - private void onDisableTaskExecutorsRequestExpiry(ExpireDisableTaskExecutorsRequest request) { - try { - log.debug("Expiring Disable Task Executors Request {}", request.getRequest()); - getTimers().cancel(getExpiryKeyFor(request.getRequest())); - if (activeDisableTaskExecutorsByAttributesRequests.remove(request.getRequest()) || (request.getRequest().getTaskExecutorID().isPresent() && disabledTaskExecutors.remove(request.getRequest().getTaskExecutorID().get()))) { - mantisJobStore.deleteExpiredDisableTaskExecutorsRequest(request.getRequest()); - } - - // also re-enable the node if the state is still valid. - if (request.getRequest().getTaskExecutorID().isPresent()) { - final TaskExecutorState state = this.executorStateManager.get( - request.getRequest().getTaskExecutorID().get()); - if (state != null) { - state.onNodeEnabled(); - } - } - } catch (Exception e) { - log.error("Failed to delete expired {}", request.getRequest(), e); - } - } - - private Map getTaskExecutorWorkerMapping(Map attributes) { - final Map result = new HashMap<>(); - this.executorStateManager.getActiveExecutorEntry().forEach(idAndState -> { - if (idAndState.getValue().getRegistration() != null && idAndState.getValue().getRegistration().containsAttributes(attributes)) { - if (idAndState.getValue().isRunningTask()) { - result.put(idAndState.getKey(), idAndState.getValue().getWorkerId()); - } - } - }); - return result; - } - - private void onTaskExecutorInitialization(InitializeTaskExecutorRequest request) { - log.info("Initializing taskExecutor {} for the resource cluster {}", request.getTaskExecutorID(), this); - ActorRef sender = sender(); - try { - TaskExecutorRegistration registration = - mantisJobStore.getTaskExecutor(request.getTaskExecutorID()); - setupTaskExecutorStateIfNecessary(request.getTaskExecutorID()); - self().tell(registration, self()); - self().tell( - new TaskExecutorStatusChange( - registration.getTaskExecutorID(), - registration.getClusterID(), - TaskExecutorReport.occupied(request.getWorkerId())), - self()); - sender.tell(Ack.getInstance(), self()); - } catch (Exception e) { - log.error("Failed to initialize taskExecutor {}; all retries exhausted", request.getTaskExecutorID(), e); - sender.tell(new Status.Failure(e), self()); - } - } - - private void onTaskExecutorRegistration(TaskExecutorRegistration registration) { - setupTaskExecutorStateIfNecessary(registration.getTaskExecutorID()); - log.info("Request for registering on resource cluster {}: {}.", this, registration); - try { - final TaskExecutorID taskExecutorID = registration.getTaskExecutorID(); - final TaskExecutorState state = this.executorStateManager.get(taskExecutorID); - - boolean stateChange = state.onRegistration(registration); - mantisJobStore.storeNewTaskExecutor(registration); - if (stateChange) { - if (state.isAvailable()) { - this.executorStateManager.tryMarkAvailable(taskExecutorID); - } - // check if the task executor has been marked as 'Disabled' - if (isTaskExecutorDisabled(registration)) { - log.info("Newly registered task executor {} was already marked for disabling.", registration.getTaskExecutorID()); - state.onNodeDisabled(); - } - updateHeartbeatTimeout(registration.getTaskExecutorID()); - } - log.info("Successfully registered {} with the resource cluster {}", registration.getTaskExecutorID(), this); - if (!jobArtifactsToCache.isEmpty() && isJobArtifactCachingEnabled) { - self().tell(new CacheJobArtifactsOnTaskExecutorRequest(taskExecutorID, clusterID), self()); - } - sender().tell(Ack.getInstance(), self()); - } catch (Exception e) { - sender().tell(new Status.Failure(e), self()); - } - } - - private boolean isTaskExecutorDisabled(TaskExecutorRegistration registration) { - for (DisableTaskExecutorsRequest request: activeDisableTaskExecutorsByAttributesRequests) { - if (request.covers(registration)) { - return true; - } - } - return disabledTaskExecutors.contains(registration.getTaskExecutorID()); - } - - private void onHeartbeat(TaskExecutorHeartbeat heartbeat) { - log.debug("Received heartbeat {} from task executor {}", heartbeat, heartbeat.getTaskExecutorID()); - setupTaskExecutorStateIfNecessary(heartbeat.getTaskExecutorID()); - try { - final TaskExecutorID taskExecutorID = heartbeat.getTaskExecutorID(); - final TaskExecutorState state = this.executorStateManager.get(taskExecutorID); - if (state.getRegistration() == null || !state.isRegistered()) { - TaskExecutorRegistration registration = this.mantisJobStore.getTaskExecutor(heartbeat.getTaskExecutorID()); - if (registration != null) { - log.debug("Found registration {} for task executor {}", registration, heartbeat.getTaskExecutorID()); - Preconditions.checkState(state.onRegistration(registration)); - - // check if the task executor has been marked as 'Disabled' - if (isTaskExecutorDisabled(registration)) { - log.info("Reconnected task executor {} was already marked for disabling.", registration.getTaskExecutorID()); - state.onNodeDisabled(); - } - } else { -// TODO(sundaram): add a metric - log.warn("Received heartbeat from unknown task executor {}", heartbeat.getTaskExecutorID()); - sender().tell(new Status.Failure(new TaskExecutorNotFoundException(taskExecutorID)), self()); - return; - } - } else { - log.debug("Found registration {} for registered task executor {}", - state.getRegistration(), heartbeat.getTaskExecutorID()); - } - // Check if this TE was previously running a worker before it disconnected - WorkerId previousWorkerId = state.getPreviousWorkerId(); - - // Alternative detection: If disconnection didn't update previousWorkerId (e.g., segfault), - // check if current state thinks it's running a worker but heartbeat shows Available - if (previousWorkerId == null) { - WorkerId currentStateWorkerId = state.getWorkerId(); - TaskExecutorReport heartbeatReport = heartbeat.getTaskExecutorReport(); - - // If state thinks TE is running a worker, but heartbeat shows Available, - // this indicates the TE crashed and reconnected - if (currentStateWorkerId != null && heartbeatReport instanceof TaskExecutorReport.Available) { - previousWorkerId = currentStateWorkerId; - log.info("Detected TaskExecutor {} crash/reconnection: state shows running worker {} but heartbeat shows Available", - taskExecutorID, previousWorkerId); - } - } - - boolean stateChange = state.onHeartbeat(heartbeat); - if (stateChange && state.isAvailable()) { - // If this TE was previously running a worker, terminate the stale worker - if (previousWorkerId != null) { - log.info("Task executor {} reconnected via heartbeat, terminating stale worker {} due to crash/reconnection", - taskExecutorID, previousWorkerId); - WorkerTerminate terminateEvent = new WorkerTerminate(previousWorkerId, - WorkerState.Failed, JobCompletedReason.Lost); - jobMessageRouter.routeWorkerEvent(terminateEvent); - state.clearPreviousWorkerId(); - } - - this.executorStateManager.tryMarkAvailable(taskExecutorID); - } - - updateHeartbeatTimeout(heartbeat.getTaskExecutorID()); - log.debug("Successfully processed heartbeat {} from task executor {}", heartbeat, heartbeat.getTaskExecutorID()); - sender().tell(Ack.getInstance(), self()); - } catch (Exception e) { - sender().tell(new Status.Failure(e), self()); - } - } - - private void onTaskExecutorStatusChange(TaskExecutorStatusChange statusChange) { - setupTaskExecutorStateIfNecessary(statusChange.getTaskExecutorID()); - try { - final TaskExecutorID taskExecutorID = statusChange.getTaskExecutorID(); - final TaskExecutorState state = this.executorStateManager.get(taskExecutorID); - boolean stateChange = state.onTaskExecutorStatusChange(statusChange); - if (stateChange) { - if (state.isAvailable()) { - this.executorStateManager.tryMarkAvailable(taskExecutorID); - } else { - this.executorStateManager.tryMarkUnavailable(taskExecutorID); - } - } - - updateHeartbeatTimeout(statusChange.getTaskExecutorID()); - sender().tell(Ack.getInstance(), self()); - } catch (IllegalStateException e) { - sender().tell(new Status.Failure(e), self()); - } - } - - private void onTaskExecutorBatchAssignmentRequest(TaskExecutorBatchAssignmentRequest request) { - Optional matchedExecutors = this.executorStateManager.findBestFit(request); - - if (matchedExecutors.isPresent()) { - log.info("Matched all executors {} for request {}", matchedExecutors.get(), request); - matchedExecutors.get().getBestFit().forEach((allocationRequest, taskExecutorToState) -> assignTaskExecutor( - allocationRequest, taskExecutorToState.getLeft(), taskExecutorToState.getRight(), request)); - sender().tell(new TaskExecutorsAllocation(matchedExecutors.get().getRequestToTaskExecutorMap()), self()); - } else { - request.allocationRequests.forEach(req -> metrics.incrementCounter( - ResourceClusterActorMetrics.NO_RESOURCES_AVAILABLE, - createTagListFrom(req))); - sender().tell(new Status.Failure(new NoResourceAvailableException( - String.format("No resource available for request %s: resource overview: %s", request, - getResourceOverview()))), self()); - } - } - - private void assignTaskExecutor(TaskExecutorAllocationRequest allocationRequest, TaskExecutorID taskExecutorID, TaskExecutorState taskExecutorState, TaskExecutorBatchAssignmentRequest request) { - if(shouldCacheJobArtifacts(allocationRequest)) { - self().tell(new AddNewJobArtifactsToCacheRequest(clusterID, Collections.singletonList(allocationRequest.getJobMetadata().getJobArtifact())), self()); - } - - taskExecutorState.onAssignment(allocationRequest.getWorkerId()); - // let's give some time for the assigned executor to be scheduled work. otherwise, the assigned executor - // will be returned back to the pool. - getTimers().startSingleTimer( - "Assignment-" + taskExecutorID.toString(), - new TaskExecutorAssignmentTimeout(taskExecutorID), - assignmentTimeout); - } - - private void onTaskExecutorAssignmentTimeout(TaskExecutorAssignmentTimeout request) { - TaskExecutorState state = this.executorStateManager.get(request.getTaskExecutorID()); - if (state == null) { - log.error("TaskExecutor lost during task assignment: {}", request); - } - else if (state.isRunningTask()) { - log.debug("TaskExecutor {} entered running state already; no need to act", request.getTaskExecutorID()); - } else { - try - { - boolean stateChange = state.onUnassignment(); - if (stateChange) { - this.executorStateManager.tryMarkAvailable(request.getTaskExecutorID()); - } - } catch (IllegalStateException e) { - if (state.isRegistered()) { - log.error("Failed to un-assign registered taskExecutor {}", request.getTaskExecutorID(), e); - } else { - log.debug("Failed to un-assign unRegistered taskExecutor {}", request.getTaskExecutorID(), e); - } - } - } - } - - private void onResourceOverviewRequest(ResourceOverviewRequest request) { - sender().tell(getResourceOverview(), self()); - } - - private void onPublishResourceOverviewMetricsRequest(PublishResourceOverviewMetricsRequest request) { - publishResourceClusterMetricBySKU(getTaskExecutors(ExecutorStateManager.isRegistered), ResourceClusterActorMetrics.NUM_REGISTERED_TE); - publishResourceClusterMetricBySKU(getTaskExecutors(ExecutorStateManager.isBusy), ResourceClusterActorMetrics.NUM_BUSY_TE); - publishResourceClusterMetricBySKU(getTaskExecutors(ExecutorStateManager.isAvailable), ResourceClusterActorMetrics.NUM_AVAILABLE_TE); - publishResourceClusterMetricBySKU(getTaskExecutors(ExecutorStateManager.isDisabled), ResourceClusterActorMetrics.NUM_DISABLED_TE); - publishResourceClusterMetricBySKU(getTaskExecutors(ExecutorStateManager.unregistered), ResourceClusterActorMetrics.NUM_UNREGISTERED_TE); - publishResourceClusterMetricBySKU(getTaskExecutors(ExecutorStateManager.isAssigned), ResourceClusterActorMetrics.NUM_ASSIGNED_TE); - } - - private void publishResourceClusterMetricBySKU(TaskExecutorsList taskExecutorsList, String metricName) { - try { - taskExecutorsList.getTaskExecutors() - .stream() - .map(this::getTaskExecutorState) - .filter(Objects::nonNull) - .map(TaskExecutorState::getRegistration) - .filter(Objects::nonNull) - .filter(registration -> registration.getTaskExecutorContainerDefinitionId().isPresent() && registration.getAttributeByKey(WorkerConstants.AUTO_SCALE_GROUP_KEY).isPresent()) - .collect(groupingBy(registration -> Tuple.of(registration.getTaskExecutorContainerDefinitionId().get(), registration.getAttributeByKey(WorkerConstants.AUTO_SCALE_GROUP_KEY).get()), Collectors.counting())) - .forEach((keys, count) -> metrics.setGauge( - metricName, - count, - TagList.create(ImmutableMap.of("resourceCluster", clusterID.getResourceID(), "sku", keys._1.getResourceID(), "autoScaleGroup", keys._2)))); - } catch (Exception e) { - log.warn("Error while publishing resource cluster metrics by sku. RC: {}, Metric: {}.", clusterID.getResourceID(), metricName, e); - } - } - - private ResourceOverview getResourceOverview() { - return this.executorStateManager.getResourceOverview(); - } - - private void getTaskExecutorStatus(GetTaskExecutorStatusRequest req) { - TaskExecutorID taskExecutorID = req.getTaskExecutorID(); - final TaskExecutorState state = this.executorStateManager.get(taskExecutorID); - if (state == null) { - log.info("Unknown executorID: {}", taskExecutorID); - getSender().tell( - new Status.Failure(new TaskExecutorNotFoundException(taskExecutorID)), - self()); - } - else { - getSender().tell( - new TaskExecutorStatus( - state.getRegistration(), - state.isRegistered(), - state.isRunningTask(), - state.isAssigned(), - state.isDisabled(), - state.getWorkerId(), - state.getLastActivity().toEpochMilli(), - state.getCancelledWorkerId()), - self()); - } - } - - @Nullable - private TaskExecutorState getTaskExecutorState(TaskExecutorID taskExecutorID) { - return this.executorStateManager.get(taskExecutorID); - } - - private void onTaskExecutorDisconnection(TaskExecutorDisconnection disconnection) { - setupTaskExecutorStateIfNecessary(disconnection.getTaskExecutorID()); - try { - disconnectTaskExecutor(disconnection.getTaskExecutorID()); - sender().tell(Ack.getInstance(), self()); - } catch (IllegalStateException e) { - sender().tell(new Status.Failure(e), self()); - } - } - - private void disconnectTaskExecutor(TaskExecutorID taskExecutorID) { - final TaskExecutorState state = this.executorStateManager.get(taskExecutorID); - boolean stateChange = state.onDisconnection(); - if (stateChange) { - this.executorStateManager.archive(taskExecutorID); - getTimers().cancel(getHeartbeatTimerFor(taskExecutorID)); - } - } - - private String getHeartbeatTimerFor(TaskExecutorID taskExecutorID) { - return "Heartbeat-" + taskExecutorID.toString(); - } - - private void onTaskExecutorHeartbeatTimeout(HeartbeatTimeout timeout) { - setupTaskExecutorStateIfNecessary(timeout.getTaskExecutorID()); - try { - metrics.incrementCounter( - ResourceClusterActorMetrics.HEARTBEAT_TIMEOUT, - TagList.create(ImmutableMap.of("resourceCluster", clusterID.getResourceID(), "taskExecutorID", timeout.getTaskExecutorID().getResourceId()))); - log.info("heartbeat timeout received for {}", timeout.getTaskExecutorID()); - final TaskExecutorID taskExecutorID = timeout.getTaskExecutorID(); - final TaskExecutorState state = this.executorStateManager.get(taskExecutorID); - if (state.getLastActivity().compareTo(timeout.getLastActivity()) <= 0) { - log.info("Disconnecting task executor {}", timeout.getTaskExecutorID()); - disconnectTaskExecutor(timeout.getTaskExecutorID()); - } - - } catch (IllegalStateException e) { - sender().tell(new Status.Failure(e), self()); - } - } - - private void setupTaskExecutorStateIfNecessary(TaskExecutorID taskExecutorID) { - this.executorStateManager - .trackIfAbsent(taskExecutorID, TaskExecutorState.of(clock, rpcService, jobMessageRouter)); - } - - private void updateHeartbeatTimeout(TaskExecutorID taskExecutorID) { - final TaskExecutorState state = this.executorStateManager.get(taskExecutorID); - getTimers().startSingleTimer( - getHeartbeatTimerFor(taskExecutorID), - new HeartbeatTimeout(taskExecutorID, state.getLastActivity()), - heartbeatTimeout); - } - - private void onCacheJobArtifactsOnTaskExecutorRequest(CacheJobArtifactsOnTaskExecutorRequest request) { - TaskExecutorState state = this.executorStateManager.get(request.getTaskExecutorID()); - if (state != null && state.isRegistered()) { - try { - // TODO(fdichiara): store URI directly to avoid remapping for each TE - state.getGatewayAsync() - .thenComposeAsync(taskExecutorGateway -> - taskExecutorGateway.cacheJobArtifacts(new CacheJobArtifactsRequest( - jobArtifactsToCache - .stream() - .map(artifactID -> URI.create(artifactID.getResourceID())) - .collect(Collectors.toList())))) - .whenComplete((res, throwable) -> { - if (throwable != null) { - log.error("failed to cache artifact on {}", request.getTaskExecutorID(), throwable); - } - else { - log.debug("Acked from cacheJobArtifacts for {}", request.getTaskExecutorID()); - } - }); - } catch (Exception ex) { - log.warn("Failed to cache job artifacts in task executor {}", request.getTaskExecutorID(), ex); - } - } - else { - log.debug("no valid TE state for CacheJobArtifactsOnTaskExecutorRequest: {}", request); - } - } - /** * Artifact is added to the list of artifacts if it's the first worker of the first stage * (this is to reduce the work in master) and if the job cluster is enabled (via config * for now) */ - private boolean shouldCacheJobArtifacts(TaskExecutorAllocationRequest allocationRequest) { - final WorkerId workerId = allocationRequest.getWorkerId(); - final boolean isFirstWorkerOfFirstStage = allocationRequest.getStageNum() == 1 && workerId.getWorkerIndex() == 0; - if (isFirstWorkerOfFirstStage) { - final Set jobClusters = getJobClustersWithArtifactCachingEnabled(); - return jobClusters.contains(workerId.getJobCluster()); - } - return false; - } - - private Set getJobClustersWithArtifactCachingEnabled() { - return new HashSet<>(Arrays.asList(jobClustersWithArtifactCachingEnabled.split(","))); - } - /** * Creates a list of tags from the provided TaskExecutorAllocationRequest. * The list includes resource cluster, workerId, jobCluster, and either sizeName or cpuCores and memoryMB * based on whether sizeName is present in the request's constraints. * - * @param req The task executor allocation request from which the tag list will be generated. - * * @return An iterable list of tags created from the task executor allocation request. */ - private Iterable createTagListFrom(TaskExecutorAllocationRequest req) { - // Basic tags that will always be included - ImmutableMap.Builder tagsBuilder = ImmutableMap.builder() - .put("resourceCluster", clusterID.getResourceID()) - .put("workerId", req.getWorkerId().getId()) - .put("jobCluster", req.getWorkerId().getJobCluster()); - - // Add the sizeName tag if it exists, otherwise add the cpuCores and memoryMB tags - if (req.getConstraints().getSizeName().isPresent()) { - tagsBuilder.put("sizeName", req.getConstraints().getSizeName().get()); - } else { - tagsBuilder.put("cpuCores", String.valueOf(req.getConstraints().getMachineDefinition().getCpuCores())) - .put("memoryMB", String.valueOf(req.getConstraints().getMachineDefinition().getMemoryMB())); - } - - return TagList.create(tagsBuilder.build()); - } - @Value static class HeartbeatTimeout { @@ -1000,6 +541,7 @@ static class HeartbeatTimeout { public static class TaskExecutorBatchAssignmentRequest { Set allocationRequests; ClusterID clusterID; + Reservation reservation; public Map> getGroupedBySchedulingConstraints() { return allocationRequests @@ -1012,16 +554,6 @@ public String getJobId() { } } - @Value - static class TaskExecutorAssignmentTimeout { - TaskExecutorID taskExecutorID; - } - - @Value - static class ExpireDisableTaskExecutorsRequest { - DisableTaskExecutorsRequest request; - } - @Value static class InitializeTaskExecutorRequest { TaskExecutorID taskExecutorID; @@ -1125,6 +657,7 @@ static class TaskExecutorsList { @Value static class TaskExecutorsAllocation { Map allocations; + Reservation reservation; } @Value @@ -1138,8 +671,43 @@ static class GetClusterUsageRequest { Function> groupKeyFunc; } + /** + * Request from ScalerActor to get cluster usage with pending reservation counts. + * This triggers a two-phase query: first to ReservationRegistryActor, then to ExecutorStateManagerActor. + */ @Value - private static class CheckDisabledTaskExecutors { + static class GetReservationAwareClusterUsageRequest { + ClusterID clusterID; + Function> groupKeyFunc; + } + + /** + * Pending reservation info with actual SchedulingConstraints (not just constraint key string). + * This avoids the need to parse constraint keys back into machine definitions. + */ + @Value + @Builder + static class PendingReservationInfo { + String canonicalConstraintKey; + SchedulingConstraints schedulingConstraints; // Actual constraints - no parsing needed! + int totalRequestedWorkers; + int reservationCount; + } + + /** + * Internal message to pass pending reservations from ReservationRegistryActor + * to ExecutorStateManagerActor for final usage computation. + * Contains actual SchedulingConstraints for direct matching. + */ + @Value + static class GetClusterUsageWithReservationsRequest { + ClusterID clusterID; + Function> groupKeyFunc; + List pendingReservations; + } + + @Value + static class CheckDisabledTaskExecutors { String reason; } @@ -1149,7 +717,7 @@ static class GetTaskExecutorWorkerMappingRequest { } @Value - private static class PublishResourceOverviewMetricsRequest { + static class PublishResourceOverviewMetricsRequest { } @Value @@ -1216,6 +784,45 @@ public Map getRequestToTaskExecut } } + /** + * Reservation value objects are now in MantisResourceClusterReservationProto. + * These classes are shared between ResourceClusterActor, ReservationRegistryActor, + * MantisScheduler, and JobActor. + */ + + @Value + @Builder + static class PendingReservationsView { + boolean ready; + @Builder.Default + Map groups = Collections.emptyMap(); + } + + @Value + @Builder + static class PendingReservationGroupView { + String canonicalConstraintKey; + int reservationCount; + int totalRequestedWorkers; + @Builder.Default + List reservations = Collections.emptyList(); + } + + // CancelReservation, CancelReservationAck, UpsertReservation, and MarkReady + // are now in MantisResourceClusterReservationProto + + enum ProcessReservationsTick { + INSTANCE + } + + enum ForceProcessReservationsTick { + INSTANCE + } + + enum GetPendingReservationsView { + INSTANCE + } + /** @@ -1349,11 +956,31 @@ public AvailabilityState onTaskExecutorStatusChange(TaskExecutorReport report) { } } - private Predicate> filterByAttrs(HasAttributes hasAttributes) { - if (hasAttributes.getAttributes().isEmpty()) { - return e -> true; - } else { - return e -> e.getValue().containsAttributes(hasAttributes.getAttributes()); + private static String buildReservationRegistryActorName(ClusterID clusterID) { + String resourceId = clusterID != null ? clusterID.getResourceID() : ""; + if (resourceId == null) { + resourceId = ""; + } + String sanitized = resourceId.replaceAll("[^a-zA-Z0-9-_]", "_"); + if (sanitized.isEmpty()) { + sanitized = "default"; + } else if (sanitized.charAt(0) == '$') { + sanitized = "_" + sanitized.substring(1); + } + return "reservationRegistry-" + sanitized; + } + + private static String buildExecutorStateManagerActorName(ClusterID clusterID) { + String resourceId = clusterID != null ? clusterID.getResourceID() : ""; + if (resourceId == null) { + resourceId = ""; + } + String sanitized = resourceId.replaceAll("[^a-zA-Z0-9-_]", "_"); + if (sanitized.isEmpty()) { + sanitized = "default"; + } else if (sanitized.charAt(0) == '$') { + sanitized = "_" + sanitized.substring(1); } + return "executorStateManager-" + sanitized; } } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClusterActorMetrics.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClusterActorMetrics.java index dac1f6d11..595ba4d07 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClusterActorMetrics.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClusterActorMetrics.java @@ -55,9 +55,14 @@ class ResourceClusterActorMetrics { public static final String HEARTBEAT_TIMEOUT = "taskExecutorHeartbeatTimeout"; public static final String TE_CONNECTION_FAILURE = "taskExecutorConnectionFailure"; + public static final String TASK_EXECUTOR_ASSIGNMENT_FAILURE = "taskExecutorAssignmentFailure"; public static final String RC_ACTOR_RESTART = "resourceClusterActorRestart"; public static final String MAX_JOB_ARTIFACTS_TO_CACHE_REACHED = "maxJobArtifactsToCacheReached"; + public static final String RESERVATION_PROCESSED = "reservationProcessed"; + public static final String RESERVATION_UPSERTED = "reservationUpserted"; + public static final String RESERVATION_INFLIGHT_TIMEOUT = "reservationInFlightTimeout"; + public static final String CLUSTER_USAGE_WITH_RESERVATIONS_LATENCY = "clusterUsageWithReservationsLatency"; private final Registry registry; private final Map, Tuple2> messageMetrics; @@ -107,6 +112,11 @@ public void incrementCounter(final String metric, final Iterable tags) { .increment(); } + public void recordTimer(final String metric, final long nanos, final Iterable tags) { + registry.timer(new MetricId(METRIC_GROUP_ID, metric, tags).getSpectatorId(registry)) + .record(nanos, TimeUnit.NANOSECONDS); + } + public

FI.UnitApply

withTracking(final FI.UnitApply

apply) { return p -> { final long start = System.nanoTime(); diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClusterAkkaImpl.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClusterAkkaImpl.java index 2cadb4793..ddd169c0b 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClusterAkkaImpl.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClusterAkkaImpl.java @@ -42,7 +42,11 @@ import io.mantisrx.master.resourcecluster.ResourceClusterActor.TaskExecutorsAllocation; import io.mantisrx.master.resourcecluster.ResourceClusterActor.TaskExecutorsList; import io.mantisrx.master.resourcecluster.ResourceClusterScalerActor.QueueClusterRuleRefreshRequest; +import static io.mantisrx.server.master.resourcecluster.proto.MantisResourceClusterReservationProto.MarkReady; import io.mantisrx.master.resourcecluster.proto.SetResourceClusterScalerStatusRequest; +import io.mantisrx.server.master.resourcecluster.proto.MantisResourceClusterReservationProto.UpsertReservation; +import io.mantisrx.server.master.resourcecluster.proto.MantisResourceClusterReservationProto.CancelReservation; +import io.mantisrx.server.master.resourcecluster.proto.MantisResourceClusterReservationProto.CancelReservationAck; import io.mantisrx.server.core.domain.ArtifactID; import io.mantisrx.server.core.domain.WorkerId; import io.mantisrx.server.master.resourcecluster.ClusterID; @@ -194,11 +198,12 @@ public CompletableFuture> getJobArtifactsToCache() { .thenApply(ArtifactList::getArtifacts); } + // todo: deprecate @Override public CompletableFuture> getTaskExecutorsFor(Set allocationRequests) { return Patterns - .ask(resourceClusterManagerActor, new TaskExecutorBatchAssignmentRequest(allocationRequests, clusterID), askTimeout) + .ask(resourceClusterManagerActor, new TaskExecutorBatchAssignmentRequest(allocationRequests, clusterID, null), askTimeout) .thenApply(TaskExecutorsAllocation.class::cast) .toCompletableFuture() .thenApply(l -> l.getAllocations()); @@ -336,4 +341,39 @@ public CompletableFuture> getTaskExecutorWorkerMap .thenApply(obj -> (Map) obj) .toCompletableFuture(); } + + @Override + public CompletableFuture markRegistryReady() { + // MarkReady is broadcast to all cluster actors by ResourceClustersManagerActor + return Patterns + .ask(resourceClusterManagerActor, MarkReady.INSTANCE, askTimeout) + .thenApply(Ack.class::cast) + .toCompletableFuture(); + } + + @Override + public CompletableFuture upsertReservation(UpsertReservation request) { + return Patterns.ask( + resourceClusterManagerActor, + ResourceClustersManagerActor.UpsertReservationRequest.builder() + .clusterID(clusterID) + .upsertReservation(request) + .build(), + askTimeout) + .thenApply(Ack.class::cast) + .toCompletableFuture(); + } + + @Override + public CompletableFuture cancelReservation(CancelReservation request) { + return Patterns.ask( + resourceClusterManagerActor, + ResourceClustersManagerActor.CancelReservationRequest.builder() + .clusterID(clusterID) + .cancelReservation(request) + .build(), + askTimeout) + .thenApply(Ack.class::cast) + .toCompletableFuture(); + } } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClusterScalerActor.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClusterScalerActor.java index c2f531f2d..07e7fa874 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClusterScalerActor.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClusterScalerActor.java @@ -27,6 +27,7 @@ import io.mantisrx.common.metrics.MetricsRegistry; import io.mantisrx.common.metrics.spectator.MetricGroupId; import io.mantisrx.master.resourcecluster.ResourceClusterActor.GetClusterUsageRequest; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.GetReservationAwareClusterUsageRequest; import io.mantisrx.master.resourcecluster.proto.GetClusterIdleInstancesRequest; import io.mantisrx.master.resourcecluster.proto.GetClusterIdleInstancesResponse; import io.mantisrx.master.resourcecluster.proto.GetClusterUsageResponse; @@ -61,6 +62,44 @@ * [Notes] There can be two communication model between the scaler actor and resource cluster actor. If the state is * pushed from resource cluster actor to the scaler actor, the downside is we need to ensure all changes are properly * handled and can trigger the push, while pulling state from scaler actor requires explicit timer firing. + * + *

Reservation-Aware Scaling Data Flow

+ * When {@code reservationSchedulingEnabled} is true, the scaler uses a reservation-aware flow to factor + * pending reservations into scaling decisions: + * + *
+ * ScalerActor.onTriggerClusterUsageRequest()
+ *    │
+ *    └─ (reservationEnabled=true) ──> GetReservationAwareClusterUsageRequest
+ *                                         │
+ *                                         ▼
+ *    ResourceClusterActor.onGetReservationAwareClusterUsage()
+ *        │
+ *        ├─ Phase 1: Ask ReservationRegistryActor.GetPendingReservationsForScaler
+ *        │           → PendingReservationsForScalerResponse (with SchedulingConstraints)
+ *        │
+ *        └─ Phase 2: Forward GetClusterUsageWithReservationsRequest to ESMActor
+ *                         │
+ *                         ▼
+ *    ExecutorStateManagerActor.onGetClusterUsageWithReservations()
+ *        │
+ *        └─ delegate.getClusterUsageWithReservations()
+ *            → GetClusterUsageResponse (with pendingReservationCount per SKU)
+ *                         │
+ *                         ▼
+ *    ScalerActor.onGetClusterUsageResponse()
+ *        │
+ *        └─ ClusterAvailabilityRule.apply(usage)
+ *            │
+ *            ├─ effectiveIdleCount = idleCount - pendingReservationCount
+ *            ├─ Scale Down: Only if effectiveIdleCount > maxIdleToKeep
+ *            └─ Scale Up: If effectiveIdleCount < minIdleToKeep OR pendingReservations > 0
+ * 
+ * + *

This ensures the cluster proactively scales up to meet pending reservation demand and avoids + * premature scale-down when idle TEs are about to be consumed by pending reservations.

+ *

Include inflight reservation can potentially cause double booking (reservation + assigned but not yet + * signaled TEs. For now tolerate this via scale down later to avoid extra complexity in state.

*/ @Slf4j public class ResourceClusterScalerActor extends AbstractActorWithTimers { @@ -86,6 +125,8 @@ public class ResourceClusterScalerActor extends AbstractActorWithTimers { private final Counter numReachScaleMinLimit; private final Counter numScaleRuleTrigger; + private final boolean reservationSchedulingEnabled; + public static Props props( ClusterID clusterId, Clock clock, @@ -93,7 +134,8 @@ public static Props props( Duration ruleRefreshThreshold, IMantisPersistenceProvider storageProvider, ActorRef resourceClusterHostActor, - ActorRef resourceClusterActor) { + ActorRef resourceClusterActor, + boolean reservationSchedulingEnabled) { return Props.create( ResourceClusterScalerActor.class, clusterId, @@ -102,7 +144,8 @@ public static Props props( ruleRefreshThreshold, storageProvider, resourceClusterHostActor, - resourceClusterActor); + resourceClusterActor, + reservationSchedulingEnabled); } public ResourceClusterScalerActor( @@ -112,7 +155,8 @@ public ResourceClusterScalerActor( Duration ruleRefreshThreshold, IMantisPersistenceProvider storageProvider, ActorRef resourceClusterHostActor, - ActorRef resourceClusterActor) { + ActorRef resourceClusterActor, + boolean reservationSchedulingEnabled) { this.clusterId = clusterId; this.resourceClusterActor = resourceClusterActor; this.resourceClusterHostActor = resourceClusterHostActor; @@ -120,6 +164,7 @@ public ResourceClusterScalerActor( this.clock = clock; this.scalerPullThreshold = scalerPullThreshold; this.ruleSetRefreshThreshold = ruleRefreshThreshold; + this.reservationSchedulingEnabled = reservationSchedulingEnabled; MetricGroupId metricGroupId = new MetricGroupId( "ResourceClusterScalerActor", @@ -267,10 +312,20 @@ private void onTriggerClusterUsageRequest(TriggerClusterUsageRequest req) { log.info("{} scaler is disabled due to no rules", this.clusterId); return; } - this.resourceClusterActor.tell( - new GetClusterUsageRequest( - this.clusterId, ResourceClusterScalerActor.groupKeyFromTaskExecutorDefinitionIdFunc), - self()); + + if (reservationSchedulingEnabled) { + // Use reservation-aware usage request to factor in pending reservations + this.resourceClusterActor.tell( + new GetReservationAwareClusterUsageRequest( + this.clusterId, ResourceClusterScalerActor.groupKeyFromTaskExecutorDefinitionIdFunc), + self()); + } else { + // Legacy behavior without reservation awareness + this.resourceClusterActor.tell( + new GetClusterUsageRequest( + this.clusterId, ResourceClusterScalerActor.groupKeyFromTaskExecutorDefinitionIdFunc), + self()); + } } private void onTriggerClusterRuleRefreshRequest(TriggerClusterRuleRefreshRequest req) { @@ -441,15 +496,29 @@ public boolean isLastActionOlderThan(long secondsSinceLastAction) { public Optional apply(UsageByGroupKey usage) { Optional decision = Optional.empty(); - if (usage.getIdleCount() > scaleSpec.getMaxIdleToKeep()) { + + // Use effective idle count that accounts for pending reservations + int effectiveIdleCount = usage.getEffectiveIdleCount(); + int actualIdleCount = usage.getIdleCount(); + int pendingReservations = usage.getPendingReservationCount(); + + log.debug("Evaluating scale rule for SKU {}: idle={}, effectiveIdle={}, pending={}, total={}", + usage.getUsageGroupKey(), + usage.getIdleCount(), + effectiveIdleCount, + pendingReservations, + usage.getTotalCount()); + + // SCALE DOWN: Only if effective idle (after reservations) exceeds max + if (effectiveIdleCount > scaleSpec.getMaxIdleToKeep()) { // Cool down check: for scaling down we want to wait 5x the nominal cool down period if (isLastActionOlderThan(scaleSpec.getCoolDownSecs() * 5)) { log.debug("Scale Down CoolDown skip: {}, {}", this.scaleSpec.getClusterId(), this.scaleSpec.getSkuId()); return Optional.empty(); } - // too many idle agents, scale down. - int step = usage.getIdleCount() - scaleSpec.getMaxIdleToKeep(); + // too many idle agents (accounting for pending reservations), scale down. + int step = effectiveIdleCount - scaleSpec.getMaxIdleToKeep(); int newSize = Math.max( usage.getTotalCount() - step, this.scaleSpec.getMinSize()); decision = Optional.of( @@ -462,15 +531,17 @@ public Optional apply(UsageByGroupKey usage) { .type(newSize == usage.getTotalCount() ? ScaleType.NoOpReachMin : ScaleType.ScaleDown) .build()); } - else if (usage.getIdleCount() < scaleSpec.getMinIdleToKeep()) { + // SCALE UP: If real idle is below min + else if (effectiveIdleCount < scaleSpec.getMinIdleToKeep()) { // Cool down check if (isLastActionOlderThan(scaleSpec.getCoolDownSecs())) { log.debug("Scale Up CoolDown skip: {}, {}", this.scaleSpec.getClusterId(), this.scaleSpec.getSkuId()); return Optional.empty(); } - // scale up - int step = scaleSpec.getMinIdleToKeep() - usage.getIdleCount(); + // Scale up to cover both idle deficit and pending reservations + int step = (pendingReservations + scaleSpec.getMinIdleToKeep() - actualIdleCount) ; + int newSize = Math.min( usage.getTotalCount() + step, this.scaleSpec.getMaxSize()); decision = Optional.of( @@ -484,8 +555,12 @@ else if (usage.getIdleCount() < scaleSpec.getMinIdleToKeep()) { .build()); } - log.info("Scale Decision for {}-{}: {}", - this.scaleSpec.getClusterId(), this.scaleSpec.getSkuId(), decision); + log.info("Scale Decision for {}-{}: {} (effectiveIdle={}, pending={})", + this.scaleSpec.getClusterId(), + this.scaleSpec.getSkuId(), + decision, + effectiveIdleCount, + pendingReservations); // reset last action only if we decided to scale up or down if (decision.isPresent() && (decision.get().type.equals(ScaleType.ScaleDown) || decision.get().type.equals(ScaleType.ScaleUp))) { diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClustersAkkaImpl.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClustersAkkaImpl.java index 00535fc24..ddb1b65af 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClustersAkkaImpl.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClustersAkkaImpl.java @@ -19,6 +19,7 @@ import akka.actor.ActorRef; import akka.actor.ActorSystem; import akka.pattern.Patterns; +import io.mantisrx.common.Ack; import io.mantisrx.common.properties.MantisPropertiesLoader; import io.mantisrx.config.dynamic.LongDynamicProperty; import io.mantisrx.server.core.utils.ConfigUtils; @@ -77,6 +78,17 @@ public CompletableFuture> listActiveClusters() { .thenApply(clusterIdSet -> clusterIdSet.getClusterIDS()); } + @Override + public CompletableFuture markAllRegistriesReady() { + // Send MarkReady to manager actor, which will broadcast to all cluster actors + return Patterns + .ask(resourceClustersManagerActor, + io.mantisrx.server.master.resourcecluster.proto.MantisResourceClusterReservationProto.MarkReady.INSTANCE, + askTimeout) + .thenApply(Ack.class::cast) + .toCompletableFuture(); + } + public static ResourceClusters load( ConfigurationFactory masterConfiguration, RpcService rpcService, diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClustersManagerActor.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClustersManagerActor.java index 8d10bc0ce..5e2d7e6c7 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClustersManagerActor.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/ResourceClustersManagerActor.java @@ -38,9 +38,15 @@ import io.mantisrx.master.resourcecluster.ResourceClusterActor.TaskExecutorBatchAssignmentRequest; import io.mantisrx.master.resourcecluster.ResourceClusterActor.TaskExecutorGatewayRequest; import io.mantisrx.master.resourcecluster.ResourceClusterActor.TaskExecutorInfoRequest; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.GetPendingReservationsView; +import io.mantisrx.master.resourcecluster.ResourceClusterActor.GetReservationAwareClusterUsageRequest; import io.mantisrx.master.resourcecluster.ResourceClusterScalerActor.QueueClusterRuleRefreshRequest; import io.mantisrx.master.resourcecluster.ResourceClusterScalerActor.TriggerClusterRuleRefreshRequest; import io.mantisrx.master.resourcecluster.proto.SetResourceClusterScalerStatusRequest; +import static io.mantisrx.server.master.resourcecluster.proto.MantisResourceClusterReservationProto.MarkReady; +import static io.mantisrx.server.master.resourcecluster.proto.MantisResourceClusterReservationProto.CancelReservation; +import static io.mantisrx.server.master.resourcecluster.proto.MantisResourceClusterReservationProto.UpsertReservation; +import io.mantisrx.server.master.ExecuteStageRequestFactory; import io.mantisrx.server.master.config.MasterConfiguration; import io.mantisrx.server.master.persistence.IMantisPersistenceProvider; import io.mantisrx.server.master.persistence.MantisJobStore; @@ -77,6 +83,7 @@ class ResourceClustersManagerActor extends AbstractActor { private final ActorRef resourceClusterHostActor; private final IMantisPersistenceProvider mantisPersistenceProvider; private final JobMessageRouter jobMessageRouter; + private final ExecuteStageRequestFactory executeStageRequestFactory; public static Props props( MasterConfiguration masterConfiguration, @@ -111,6 +118,7 @@ public ResourceClustersManagerActor( this.resourceClusterHostActor = resourceClusterHostActorRef; this.mantisPersistenceProvider = mantisPersistenceProvider; this.jobMessageRouter = jobMessageRouter; + this.executeStageRequestFactory = new ExecuteStageRequestFactory(masterConfiguration); this.resourceClusterActorMap = new HashMap<>(); } @@ -162,6 +170,20 @@ public Receive createReceive() { getRCScalerActor(req.getClusterID()).forward(req, context())) .match(SetResourceClusterScalerStatusRequest.class, req -> getRCScalerActor(req.getClusterID()).forward(req, context())) + .match(UpsertReservationRequest.class, req -> + getRCActor(req.getClusterID()).forward(req.getUpsertReservation(), context())) + .match(CancelReservationRequest.class, req -> + getRCActor(req.getClusterID()).forward(req.getCancelReservation(), context())) + .match(GetPendingReservationsViewRequest.class, req -> + getRCActor(req.getClusterID()).forward(GetPendingReservationsView.INSTANCE, context())) + .match(GetReservationAwareClusterUsageRequest.class, req -> + getRCActor(req.getClusterID()).forward(req, context())) + .match(MarkReady.class, markReady -> { + // Broadcast MarkReady to all cluster actors + resourceClusterActorMap.values().forEach(holder -> + holder.getResourceClusterActor().forward(markReady, context())); + sender().tell(io.mantisrx.common.Ack.getInstance(), self()); + }) .build(); } @@ -184,7 +206,9 @@ private ActorRef createResourceClusterActorFor(ClusterID clusterID) { masterConfiguration.isJobArtifactCachingEnabled(), masterConfiguration.getSchedulingConstraints(), masterConfiguration.getFitnessCalculator(), - masterConfiguration.getAvailableTaskExecutorMutatorHook()), + masterConfiguration.getAvailableTaskExecutorMutatorHook(), + executeStageRequestFactory, + masterConfiguration.isReservationSchedulingEnabled()), "ResourceClusterActor-" + clusterID.getResourceID()); log.info("Created resource cluster actor for {}", clusterID); return clusterActor; @@ -201,7 +225,8 @@ private ActorRef createResourceClusterScalerActorFor(ClusterID clusterID, ActorR Duration.ofSeconds(masterConfiguration.getScalerRuleSetRefreshThresholdInSecs()), this.mantisPersistenceProvider, this.resourceClusterHostActor, - rcActor + rcActor, + masterConfiguration.isReservationSchedulingEnabled() ), "ResourceClusterScalerActor-" + clusterID.getResourceID()); log.info("Created resource cluster scaler actor for {}", clusterID); @@ -258,4 +283,33 @@ static class ActorHolder { ActorRef resourceClusterActor; ActorRef resourceClusterScalerActor; } + + /** + * Wrapper message for UpsertReservation that includes ClusterID for routing. + */ + @Value + @Builder + public static class UpsertReservationRequest { + ClusterID clusterID; + UpsertReservation upsertReservation; + } + + /** + * Wrapper message for CancelReservation that includes ClusterID for routing. + */ + @Value + @Builder + public static class CancelReservationRequest { + ClusterID clusterID; + CancelReservation cancelReservation; + } + + /** + * Wrapper message for GetPendingReservationsView that includes ClusterID for routing. + */ + @Value + @Builder + public static class GetPendingReservationsViewRequest { + ClusterID clusterID; + } } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/TaskAssignmentManagerActor.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/TaskAssignmentManagerActor.java new file mode 100644 index 000000000..a854f0c66 --- /dev/null +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/TaskAssignmentManagerActor.java @@ -0,0 +1,20 @@ +package io.mantisrx.master.resourcecluster; + +public class TaskAssignmentManagerActor { + /* todo: context + - AssignmentTrackerActor: tracks and manages actual worker assignments + - Host similar funcationality as onAssignedScheduleRequestEvent(AssignedScheduleRequestEvent event) in ResourceClusterAwareSchedulerActor; Since this is now a child actor in resourceClusterActor it doesn't need to go throgh resource cluster interface to get the connections etc. but can directly ask the parent actor for the needed objects. + - track ongoing IO tasks to each TE and handle retry similar to logic in scheduler actor. + - assignment tasks can be cancelled due to job kill and worker replacement. + + AssignmentTask needs to track: + - Original Reservation + - assigned workers + - pending workers w/ retry cnt + tracker timer on dispatcher; + assignment IO to TE on IO; + - route starting event to jobactor + + */ + +} diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/proto/GetClusterUsageResponse.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/proto/GetClusterUsageResponse.java index d70406024..553ff5962 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/proto/GetClusterUsageResponse.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/resourcecluster/proto/GetClusterUsageResponse.java @@ -36,5 +36,16 @@ public static class UsageByGroupKey { String usageGroupKey; int idleCount; int totalCount; + + @Builder.Default + int pendingReservationCount = 0; + + /** + * Effective idle count accounting for pending reservations. + * If pending reservations exist, those "idle" TEs will soon be consumed. + */ + public int getEffectiveIdleCount() { + return Math.max(0, idleCount - pendingReservationCount); + } } } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/ExecuteStageRequestFactory.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/ExecuteStageRequestFactory.java index 960e41e60..1051d3fee 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/ExecuteStageRequestFactory.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/ExecuteStageRequestFactory.java @@ -18,9 +18,12 @@ import io.mantisrx.server.core.ExecuteStageRequest; import io.mantisrx.server.master.config.MasterConfiguration; +import io.mantisrx.server.master.resourcecluster.TaskExecutorAllocationRequest; import io.mantisrx.server.master.resourcecluster.TaskExecutorRegistration; import io.mantisrx.server.master.scheduler.ScheduleRequest; import java.util.Optional; + +import io.mantisrx.shaded.com.google.common.base.Preconditions; import lombok.RequiredArgsConstructor; @RequiredArgsConstructor @@ -52,4 +55,32 @@ public ExecuteStageRequest of( scheduleRequest.getJobMetadata().getUser(), scheduleRequest.getJobMetadata().getJobVersion()); } + + public ExecuteStageRequest of( + TaskExecutorRegistration matchedTaskExecutorInfo, + TaskExecutorAllocationRequest allocationRequest) { + Preconditions.checkNotNull(allocationRequest); + Preconditions.checkNotNull(allocationRequest.getJobMetadata()); + return new ExecuteStageRequest( + allocationRequest.getWorkerId().getJobCluster(), + allocationRequest.getWorkerId().getJobId(), + allocationRequest.getWorkerId().getWorkerIndex(), + allocationRequest.getWorkerId().getWorkerNum(), + allocationRequest.getJobMetadata().getJobJarUrl(), + allocationRequest.getStageNum(), + allocationRequest.getJobMetadata().getTotalStages(), + matchedTaskExecutorInfo.getWorkerPorts().getPorts(), + masterConfiguration.getTimeoutSecondsToReportStart(), + matchedTaskExecutorInfo.getWorkerPorts().getMetricsPort(), + allocationRequest.getJobMetadata().getParameters(), + allocationRequest.getJobMetadata().getSchedulingInfo(), + allocationRequest.getDurationType(), + allocationRequest.getJobMetadata().getHeartbeatIntervalSecs(), + allocationRequest.getJobMetadata().getSubscriptionTimeoutSecs(), + allocationRequest.getJobMetadata().getMinRuntimeSecs() - (System.currentTimeMillis() - allocationRequest.getJobMetadata().getMinRuntimeSecs()), + matchedTaskExecutorInfo.getWorkerPorts(), + Optional.empty(), + allocationRequest.getJobMetadata().getUser(), + allocationRequest.getJobMetadata().getJobVersion()); + } } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/config/MasterConfiguration.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/config/MasterConfiguration.java index 50cedd175..77d7d49dd 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/config/MasterConfiguration.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/config/MasterConfiguration.java @@ -386,6 +386,10 @@ default Duration getSchedulerIntervalBetweenRetries() { @Default("true") boolean getSchedulerHandlesAllocationRetries(); + @Config("mantis.scheduling.reservation.enabled") + @Default("true") + boolean isReservationSchedulingEnabled(); + default Duration getHeartbeatInterval() { return Duration.ofMillis(getHeartbeatIntervalInMs()); } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/scheduler/LaunchTaskRequest.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/scheduler/LaunchTaskRequest.java deleted file mode 100644 index 0008b2bb9..000000000 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/scheduler/LaunchTaskRequest.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * Copyright 2019 Netflix, Inc. - * - * Licensed 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 io.mantisrx.server.master.scheduler; - - -import io.mantisrx.common.WorkerPorts; - - -public class LaunchTaskRequest { - - private final ScheduleRequest scheduleRequest; - private final WorkerPorts ports; - - public LaunchTaskRequest(ScheduleRequest scheduleRequest, WorkerPorts ports) { - this.scheduleRequest = scheduleRequest; - this.ports = ports; - } - - public ScheduleRequest getScheduleRequest() { - return scheduleRequest; - } - - public WorkerPorts getPorts() { - return ports; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - LaunchTaskRequest that = (LaunchTaskRequest) o; - - if (scheduleRequest != null ? !scheduleRequest.equals(that.scheduleRequest) : that.scheduleRequest != null) - return false; - return ports != null ? ports.equals(that.ports) : that.ports == null; - } - - @Override - public int hashCode() { - int result = scheduleRequest != null ? scheduleRequest.hashCode() : 0; - result = 31 * result + (ports != null ? ports.hashCode() : 0); - return result; - } - - @Override - public String toString() { - return "LaunchTaskRequest{" + - "scheduleRequest=" + scheduleRequest + - ", ports=" + ports + - '}'; - } -} diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/scheduler/MantisScheduler.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/scheduler/MantisScheduler.java index 00273121b..c2024d899 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/scheduler/MantisScheduler.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/scheduler/MantisScheduler.java @@ -16,8 +16,12 @@ package io.mantisrx.server.master.scheduler; +import io.mantisrx.common.Ack; +import io.mantisrx.server.master.resourcecluster.proto.MantisResourceClusterReservationProto.CancelReservation; +import io.mantisrx.server.master.resourcecluster.proto.MantisResourceClusterReservationProto.UpsertReservation; import io.mantisrx.server.core.domain.WorkerId; import java.util.Optional; +import java.util.concurrent.CompletableFuture; public interface MantisScheduler { @@ -83,4 +87,22 @@ public interface MantisScheduler { * the worker is assigned, then return true; otherwise, return false. */ boolean schedulerHandlesAllocationRetries(); + + /** + * Insert or update a reservation for workers. + * @param request The reservation request (reuses existing UpsertReservation class) + * @return Future that completes when the reservation is accepted + */ + default CompletableFuture upsertReservation(UpsertReservation request) { + throw new UnsupportedOperationException("Reservation-based scheduling not supported"); + } + + /** + * Cancel all pending reservations for a given job stage. + * @param request The cancel request (reuses existing CancelReservation class) + * @return Future containing acknowledgement + */ + default CompletableFuture cancelReservation(CancelReservation request) { + throw new UnsupportedOperationException("Reservation-based scheduling not supported"); + } } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/scheduler/MantisSchedulerFactory.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/scheduler/MantisSchedulerFactory.java index 522153fb3..00b42e497 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/scheduler/MantisSchedulerFactory.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/scheduler/MantisSchedulerFactory.java @@ -16,9 +16,11 @@ package io.mantisrx.server.master.scheduler; +import io.mantisrx.common.Ack; import io.mantisrx.server.master.domain.JobDefinition; import io.mantisrx.server.master.resourcecluster.ClusterID; import java.util.Optional; +import java.util.concurrent.CompletableFuture; import javax.annotation.Nullable; /** @@ -38,4 +40,15 @@ default MantisScheduler forJob(JobDefinition jobDefinition) { * @return MantisScheduler corresponding to the ClusterID. */ MantisScheduler forClusterID(@Nullable ClusterID clusterID); + + /** + * Mark all reservation registries as ready after master initialization. + * Should be called after all jobs have been recovered. + * + * @return Future that completes when all registries are ready + */ + default CompletableFuture markAllRegistriesReady() { + // Default implementation for factories that don't support reservation scheduling + return CompletableFuture.completedFuture(Ack.getInstance()); + } } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/scheduler/MantisSchedulerFactoryImpl.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/scheduler/MantisSchedulerFactoryImpl.java index 20e7ca285..6f2bcafb1 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/scheduler/MantisSchedulerFactoryImpl.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/scheduler/MantisSchedulerFactoryImpl.java @@ -17,6 +17,7 @@ package io.mantisrx.server.master.scheduler; import akka.actor.ActorSystem; +import io.mantisrx.common.Ack; import io.mantisrx.common.metrics.MetricsRegistry; import io.mantisrx.server.master.ExecuteStageRequestFactory; import io.mantisrx.server.master.config.MasterConfiguration; @@ -24,6 +25,7 @@ import io.mantisrx.server.master.resourcecluster.ResourceClusters; import io.mantisrx.shaded.com.google.common.base.Strings; import java.util.Map; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; @@ -42,33 +44,48 @@ public class MantisSchedulerFactoryImpl implements MantisSchedulerFactory { @Override public MantisScheduler forClusterID(ClusterID clusterID) { - if (clusterID != null) { - if (Strings.isNullOrEmpty(clusterID.getResourceID())) { - log.error("Received empty resource id: {}", clusterID); - throw new RuntimeException("Empty resourceID in clusterID for MantisScheduler"); - } + if (clusterID == null || Strings.isNullOrEmpty(clusterID.getResourceID())) { + throw new RuntimeException("Invalid clusterID for MantisScheduler"); + } + return actorRefMap.computeIfAbsent(clusterID, this::createScheduler); + } + + private MantisScheduler createScheduler(ClusterID clusterID) { + log.info("Creating scheduler for cluster: {} (reservationEnabled={})", + clusterID.getResourceID(), + masterConfiguration.isReservationSchedulingEnabled()); + + if (masterConfiguration.isReservationSchedulingEnabled()) { + return new ResourceClusterReservationAwareScheduler( + resourceClusters.getClusterFor(clusterID)); + } else { + return new ResourceClusterAwareScheduler(actorSystem.actorOf( + ResourceClusterAwareSchedulerActor.props( + masterConfiguration.getSchedulerMaxRetries(), + masterConfiguration.getSchedulerMaxRetries(), + masterConfiguration.getSchedulerIntervalBetweenRetries(), + resourceClusters.getClusterFor(clusterID), + executeStageRequestFactory, + jobMessageRouter, + metricsRegistry), + "scheduler-for-" + clusterID.getResourceID()), + masterConfiguration.getSchedulerHandlesAllocationRetries()); + } + } - return - actorRefMap.computeIfAbsent( - clusterID, - (cid) -> { - log.info("Created scheduler actor for cluster: {}", - clusterID.getResourceID()); - return new ResourceClusterAwareScheduler(actorSystem.actorOf( - ResourceClusterAwareSchedulerActor.props( - masterConfiguration.getSchedulerMaxRetries(), - masterConfiguration.getSchedulerMaxRetries(), - masterConfiguration.getSchedulerIntervalBetweenRetries(), - resourceClusters.getClusterFor(cid), - executeStageRequestFactory, - jobMessageRouter, - metricsRegistry), - "scheduler-for-" + cid.getResourceID()), - masterConfiguration.getSchedulerHandlesAllocationRetries()); - }); + /** + * Mark all reservation registries as ready after master initialization. + * Should be called after all jobs have been recovered. + * + * @return Future that completes when all registries are ready + */ + public CompletableFuture markAllRegistriesReady() { + if (masterConfiguration.isReservationSchedulingEnabled()) { + log.info("Marking all reservation registries as ready"); + return resourceClusters.markAllRegistriesReady(); } else { - log.error("Scheduler gets unexpected null clusterID"); - throw new RuntimeException("invalid null clusterID."); + log.debug("Reservation scheduling disabled, skipping markAllRegistriesReady"); + return CompletableFuture.completedFuture(Ack.getInstance()); } } } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/scheduler/ResourceClusterReservationAwareScheduler.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/scheduler/ResourceClusterReservationAwareScheduler.java new file mode 100644 index 000000000..10040e531 --- /dev/null +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/scheduler/ResourceClusterReservationAwareScheduler.java @@ -0,0 +1,120 @@ +/* + * Copyright 2025 Netflix, Inc. + * + * Licensed 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 io.mantisrx.server.master.scheduler; + +import io.mantisrx.common.Ack; +import io.mantisrx.server.master.resourcecluster.proto.MantisResourceClusterReservationProto.CancelReservation; +import io.mantisrx.server.master.resourcecluster.proto.MantisResourceClusterReservationProto.UpsertReservation; +import io.mantisrx.server.master.resourcecluster.proto.MantisResourceClusterReservationProto.ReservationKey; +import io.mantisrx.server.core.domain.WorkerId; +import io.mantisrx.server.master.resourcecluster.ResourceCluster; +import io.mantisrx.server.master.resourcecluster.TaskExecutorID; +import lombok.RequiredArgsConstructor; +import lombok.extern.slf4j.Slf4j; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; + +/** + * Scheduler implementation that uses the reservation-based scheduling system. + * + * This scheduler delegates scheduling decisions to the ReservationRegistryActor + * which processes reservations in priority order per constraint group. + * WorkerLaunched events are still routed via jobMessageRouter from ExecutorStateManagerActor. + */ +@Slf4j +@RequiredArgsConstructor +public class ResourceClusterReservationAwareScheduler implements MantisScheduler { + + private final ResourceCluster resourceCluster; + + // ==================== Reservation APIs ==================== + + @Override + public CompletableFuture upsertReservation(UpsertReservation request) { + ReservationKey key = request.getReservationKey(); + log.info("Upserting reservation for job {} stage {} with {} workers (priority={})", + key.getJobId(), + key.getStageNumber(), + request.getAllocationRequests() != null ? request.getAllocationRequests().size() : 0, + request.getPriority() != null ? request.getPriority().getType() : "UNKNOWN"); + return resourceCluster.upsertReservation(request); + } + + @Override + public CompletableFuture cancelReservation(CancelReservation request) { + ReservationKey key = request.getReservationKey(); + log.info("Cancelling reservation for job {} stage {}", key.getJobId(), key.getStageNumber()); + return resourceCluster.cancelReservation(request); + } + + @Override + public boolean schedulerHandlesAllocationRetries() { + // Reservation registry handles retries until the worker is allocated (launched) + return true; + } + + // ==================== Legacy APIs ==================== + + @Override + public void scheduleWorkers(BatchScheduleRequest scheduleRequest) { + throw new UnsupportedOperationException( + "Use upsertReservation() instead of scheduleWorkers() with reservation-based scheduling"); + } + + @Override + public void unscheduleJob(String jobId) { + // For reservation-based scheduling, caller should use cancelReservation per stage + log.warn("unscheduleJob({}) called - use cancelReservation() per stage instead", jobId); + } + + @Override + public void unscheduleWorker(WorkerId workerId, Optional hostname) { + throw new UnsupportedOperationException( + "unscheduleWorker not supported - use cancelReservation() for pending workers"); + } + + @Override + public void unscheduleAndTerminateWorker(WorkerId workerId, Optional hostname) { + // Still needed for terminating running workers + log.info("Terminating worker {}", workerId); + resourceCluster.markTaskExecutorWorkerCancelled(workerId) + .whenComplete((ack, ex) -> { + if (ex != null) { + log.warn("Failed to mark worker {} as cancelled", workerId, ex); + } + }); + } + + @Override + public void updateWorkerSchedulingReadyTime(WorkerId workerId, long when) { + throw new UnsupportedOperationException( + "Reservation registry handles retry timing internally"); + } + + @Override + public void initializeRunningWorker(ScheduleRequest scheduleRequest, String hostname, String hostID) { + // Still needed for leader switch recovery + log.info("Initializing running worker {} on host {}", scheduleRequest.getWorkerId(), hostname); + resourceCluster.initializeTaskExecutor(TaskExecutorID.of(hostID), scheduleRequest.getWorkerId()) + .whenComplete((ack, ex) -> { + if (ex != null) { + log.warn("Failed to initialize running worker {}", scheduleRequest.getWorkerId(), ex); + } + }); + } +} + diff --git a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/events/WorkerRegistryV2Test.java b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/events/WorkerRegistryV2Test.java index 1ba8229dd..8ee95746a 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/events/WorkerRegistryV2Test.java +++ b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/events/WorkerRegistryV2Test.java @@ -180,7 +180,7 @@ public void testJobScaleUp() throws Exception, InvalidJobException, io.mantisrx. new StageScalingPolicy(1, 0, 10, 1, 1, 0, smap, true)) .build(); String clusterName = "testJobScaleUp"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); ActorRef jobActor = JobTestHelper.submitSingleStageScalableJob(system,probe, clusterName, sInfo, schedulerMock, jobStoreMock, eventPublisher); @@ -232,7 +232,7 @@ public void testJobScaleDown() throws Exception { new StageScalingPolicy(1, 0, 10, 1, 1, 0, smap, true)) .build(); String clusterName = "testJobScaleDown"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); @@ -288,7 +288,7 @@ public void testJobShutdown() { new StageScalingPolicy(1, 0, 10, 1, 1, 0, smap, true)) .build(); String clusterName = "testJobShutdown"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); diff --git a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/jobcluster/JobClusterAkkaTest.java b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/jobcluster/JobClusterAkkaTest.java index be6941564..50ae801c0 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/jobcluster/JobClusterAkkaTest.java +++ b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/jobcluster/JobClusterAkkaTest.java @@ -373,7 +373,7 @@ public void testJobClusterEnable() { try { TestKit probe = new TestKit(system); String clusterName = "testJobClusterEnable"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); String jobId = clusterName + "-1"; JobDefinition jobDefn = createJob(clusterName); @@ -455,7 +455,7 @@ public void testJobClusterUpdateAndDelete() throws Exception { labels.add(l); String clusterName = "testJobClusterUpdateAndDelete"; String artifactName = "myart"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName, labels); ActorRef jobClusterActor = system.actorOf(props(clusterName, jobStoreMock, jobDfn -> schedulerMock, eventPublisher, costsCalculator, 0)); @@ -522,7 +522,7 @@ public void testJobClusterUpdateFailsIfArtifactNotUnique() throws Exception { Label l = new Label("labelname","labelvalue"); labels.add(l); String clusterName = "testJobClusterUpdateFailsIfArtifactNotUnique"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName, labels); ActorRef jobClusterActor = system.actorOf(props(clusterName, jobStoreMock, jobDfn -> schedulerMock, eventPublisher, costsCalculator, 0)); @@ -550,7 +550,7 @@ public void testJobClusterDeleteFailsIfJobsActive() throws Exception { Label l = new Label("labelname","labelvalue"); labels.add(l); String clusterName = "testJobClusterDeleteFailsIfJobsActive"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName, labels); ActorRef jobClusterActor = system.actorOf(props(clusterName, jobStoreMock, jobDfn -> schedulerMock, eventPublisher, costsCalculator, 0)); @@ -587,7 +587,7 @@ public void testJobClusterDeletePurgesCompletedJobs() throws Exception { Label l = new Label("labelname","labelvalue"); labels.add(l); String clusterName = "testJobClusterDeletePurgesCompletedJobs"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName, labels); ActorRef jobClusterActor = system.actorOf(props(clusterName, jobStoreMock, jobDfn -> schedulerMock, eventPublisher, costsCalculator, 0)); @@ -630,7 +630,7 @@ public void testJobClusterDisable() throws InterruptedException { TestKit probe = new TestKit(system); CountDownLatch storeCompletedCalled = new CountDownLatch(1); String clusterName = "testJobClusterDisable"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName); ActorRef jobClusterActor = system.actorOf(props(clusterName, jobStoreMock, jobDfn -> schedulerMock, eventPublisher, costsCalculator, 0)); @@ -725,7 +725,7 @@ public void testJobClusterDisable() throws InterruptedException { public void testJobClusterSLAUpdate() throws Exception { TestKit probe = new TestKit(system); String clusterName = "testJobClusterSLAUpdate"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName); ActorRef jobClusterActor = system.actorOf(props(clusterName, jobStoreMock, jobDfn -> schedulerMock, eventPublisher, costsCalculator, 0)); @@ -758,7 +758,7 @@ public void testJobClusterSLAUpdate() throws Exception { public void testJobClusterMigrationConfigUpdate() throws Exception { TestKit probe = new TestKit(system); String clusterName = "testJobClusterMigrationConfigUpdate"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName); ActorRef jobClusterActor = system.actorOf(props(clusterName, jobStoreMock, jobDfn -> schedulerMock, eventPublisher, costsCalculator, 0)); @@ -825,7 +825,7 @@ public void testJobClusterArtifactUpdateBackCompat() throws Exception { public void testJobClusterArtifactUpdate() throws Exception { TestKit probe = new TestKit(system); String clusterName = "testJobClusterArtifactUpdate"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName); ActorRef jobClusterActor = system.actorOf(props(clusterName, jobStoreMock, jobDfn -> schedulerMock, eventPublisher, costsCalculator, 0)); @@ -864,7 +864,7 @@ public void testJobClusterArtifactUpdate() throws Exception { public void testJobClusterArtifactUpdateNotUniqueFails() throws Exception { TestKit probe = new TestKit(system); String clusterName = "testJobClusterArtifactUpdateNotUniqueFails"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName); ActorRef jobClusterActor = system.actorOf(props(clusterName, jobStoreMock, jobDfn -> schedulerMock, eventPublisher, costsCalculator, 0)); @@ -901,7 +901,7 @@ public void testJobClusterArtifactUpdateNotUniqueFails() throws Exception { public void testJobClusterArtifactUpdateMultipleTimes() throws Exception { TestKit probe = new TestKit(system); String clusterName = "testJobClusterArtifactUpdateMultipleTimes"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName); ActorRef jobClusterActor = system.actorOf(props(clusterName, jobStoreMock, jobDfn -> schedulerMock, eventPublisher, costsCalculator, 0)); @@ -968,7 +968,7 @@ public void testJobClusterInvalidSLAUpdateIgnored() throws Exception { TestKit probe = new TestKit(system); String clusterName = "testJobClusterInvalidSLAUpdateIgnored"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName); @@ -1003,7 +1003,7 @@ public void testJobClusterInvalidSLAUpdateIgnored() throws Exception { public void testJobClusterLabelsUpdate() throws Exception { TestKit probe = new TestKit(system); String clusterName = "testJobClusterLabelsUpdate"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName); @@ -1059,7 +1059,7 @@ public void testJobClusterLabelsUpdate() throws Exception { public void testJobSubmit() { TestKit probe = new TestKit(system); String clusterName = "testJobSubmit"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName); @@ -1097,7 +1097,7 @@ public void testJobSubmit() { public void testJobSubmitWithNoJarAndSchedInfo() { TestKit probe = new TestKit(system); String clusterName = "testJobSubmitWithNoJarAndSchedInfo"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName); @@ -1150,7 +1150,7 @@ public void testJobSubmitWithVersionAndNoSchedInfo() { TestKit probe = new TestKit(system); String clusterName = "testJobSubmitWithVersionAndNoSchedInfo"; String artifactName = "myart2"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName); @@ -1258,7 +1258,7 @@ public void testJobSubmitWithVersionAndNoSchedInfo() { public void testJobComplete() { TestKit probe = new TestKit(system); String clusterName = "testJobComplete"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName); @@ -1297,7 +1297,7 @@ public void testJobKillTriggersSLAToLaunchNew() { TestKit probe = new TestKit(system); String clusterName = "testJobKillTriggersSLAToLaunchNew"; MantisSchedulerFactory schedulerMockFactory = mock(MantisSchedulerFactory.class); - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); when(schedulerMockFactory.forJob(any())).thenReturn(schedulerMock); MantisJobStore jobStoreMock = mock(MantisJobStore.class); @@ -1353,7 +1353,7 @@ public void testJobSubmitTriggersSLAToKillOld() { TestKit probe = new TestKit(system); String clusterName = "testJobSubmitTriggersSLAToKillOld"; MantisSchedulerFactory schedulerMockFactory = mock(MantisSchedulerFactory.class); - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); when(schedulerMockFactory.forJob(any())).thenReturn(schedulerMock); MantisJobStore jobStoreMock = mock(MantisJobStore.class); @@ -1436,7 +1436,7 @@ public void testJobSubmitTriggersSLAToKillOld() { public void testJobSubmitTriggersSLAToKillOldHandlesErrors() { TestKit probe = new TestKit(system); String clusterName = "testJobSubmitTriggersSLAToKillOldHandlesErrors"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); SLA sla = new SLA(1,1,null,null); @@ -1519,7 +1519,7 @@ public void testJobSubmitTriggersSLAToKillOldHandlesErrors() { public void testCronTriggersSLAToKillOld() { TestKit probe = new TestKit(system); String clusterName = "testJobSubmitTriggersSLAToKillOld"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); SLA sla = new SLA(1,1,"0/1 * * * * ?",IJobClusterDefinition.CronPolicy.KEEP_NEW); @@ -1570,7 +1570,7 @@ public void testCronTriggersSLAToKillOld() { public void testInvalidCronDefined() { TestKit probe = new TestKit(system); String clusterName = "testInvalidCronSubmit"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); SLA sla = new SLA(1,1,"a b * * * * * * *",IJobClusterDefinition.CronPolicy.KEEP_NEW); @@ -1581,7 +1581,7 @@ public void testInvalidCronDefined() { public void testInvalidCronSLAUpdate() throws Exception { TestKit probe = new TestKit(system); String clusterName = "testJobClusterInvalidSLAUpdateIgnored"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName); @@ -1616,7 +1616,7 @@ public void testInvalidCronSLAUpdate() throws Exception { public void testJobSubmitWithUnique() { TestKit probe = new TestKit(system); String clusterName = "testJobSubmitWithUnique"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName); @@ -1665,7 +1665,7 @@ public void testJobSubmitWithUnique() { public void testJobSubmitWithoutInheritInstance() { TestKit probe = new TestKit(system); String clusterName = "testJobSubmitWithInheritInstance"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName); @@ -1717,7 +1717,7 @@ public void testJobSubmitWithoutInheritInstance() { public void testJobSubmitWithInheritInstanceFlagsSingleStage() { TestKit probe = new TestKit(system); String clusterName = "testJobSubmitWithInheritInstance"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName); @@ -1773,7 +1773,7 @@ public void testJobSubmitWithInheritInstanceFlagsSingleStage() { public void testJobSubmitWithInheritInstanceFlagsMultiStage() { TestKit probe = new TestKit(system); String clusterName = "testJobSubmitWithInheritInstance"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); // default job with 3 stage == (2 worker) @@ -1851,7 +1851,7 @@ public void testJobSubmitWithInheritInstanceFlagsMultiStage() { public void testJobSubmitWithInheritInstanceFlagsScaled() { TestKit probe = new TestKit(system); String clusterName = "testJobSubmitWithInheritInstance"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); // default job with 3 stage == (2 worker) @@ -1927,7 +1927,7 @@ public void testJobSubmitWithInheritInstanceFlagsScaled() { public void testQuickJobSubmit() { TestKit probe = new TestKit(system); String clusterName = "testQuickJobSubmit"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName); @@ -1968,7 +1968,7 @@ public void testQuickJobSubmit() { public void testQuickJobSubmitWithNoSchedInfoInPreviousJob() { TestKit probe = new TestKit(system); String clusterName = "testQuickJobSubmitWithNoSchedInfoInPreviousJob"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName); @@ -2019,7 +2019,7 @@ public void testQuickJobSubmitWithNoSchedInfoInPreviousJob() { public void testJobSubmitWithNoSchedInfoUsesJobClusterValues() { TestKit probe = new TestKit(system); String clusterName = "testJobSubmitWithNoSchedInfoUsesJobClusterValues"; - MantisScheduler schedulerMock = mock(MantisScheduler.class); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); MantisJobStore jobStoreMock = mock(MantisJobStore.class); List