From 836f2f03c3aee2ca5e826832861bf39221a4e0b0 Mon Sep 17 00:00:00 2001 From: jlubin Date: Wed, 8 Apr 2026 18:24:49 -0400 Subject: [PATCH 01/15] Add healthcheck endpoint --- .../master/api/akka/MasterApiAkkaService.java | 43 +++++- .../handlers/JobClusterRouteHandler.java | 6 +- .../JobClusterRouteHandlerAkkaImpl.java | 25 ++++ .../api/akka/route/v1/HttpRequestMetrics.java | 4 +- .../api/akka/route/v1/JobClustersRoute.java | 36 +++++ .../master/jobcluster/HealthCheck.java | 42 ++++++ .../jobcluster/WorkerLaunchHealthCheck.java | 138 ++++++++++++++++++ .../jobcluster/proto/HealthCheckResponse.java | 79 ++++++++++ .../io/mantisrx/server/master/MasterMain.java | 14 +- 9 files changed, 382 insertions(+), 5 deletions(-) create mode 100644 mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/HealthCheck.java create mode 100644 mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/WorkerLaunchHealthCheck.java create mode 100644 mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/HealthCheckResponse.java diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/MasterApiAkkaService.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/MasterApiAkkaService.java index a42136f0d..1903b82fe 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/MasterApiAkkaService.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/MasterApiAkkaService.java @@ -64,7 +64,11 @@ import io.mantisrx.server.core.master.MasterMonitor; import io.mantisrx.server.master.LeaderRedirectionFilter; import io.mantisrx.server.master.persistence.IMantisPersistenceProvider; +import io.mantisrx.master.jobcluster.HealthCheck; +import io.mantisrx.master.jobcluster.WorkerLaunchHealthCheck; import io.mantisrx.server.master.resourcecluster.ResourceClusters; +import java.util.ArrayList; +import java.util.List; import java.util.concurrent.CompletionStage; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; @@ -94,6 +98,7 @@ public class MasterApiAkkaService extends BaseService { private final ExecutorService executorService; private final CountDownLatch serviceLatch = new CountDownLatch(1); private final HttpsConnectionContext httpsConnectionContext; + private final List healthChecks; public MasterApiAkkaService(final MasterMonitor masterMonitor, final MasterDescription masterDescription, @@ -117,7 +122,8 @@ public MasterApiAkkaService(final MasterMonitor masterMonitor, mantisStorageProvider, lifecycleEventPublisher, leadershipManager, - null + null, + List.of() ); } public MasterApiAkkaService(final MasterMonitor masterMonitor, @@ -131,6 +137,33 @@ public MasterApiAkkaService(final MasterMonitor masterMonitor, final LifecycleEventPublisher lifecycleEventPublisher, final ILeadershipManager leadershipManager, final HttpsConnectionContext httpsConnectionContext) { + this( + masterMonitor, + masterDescription, + jobClustersManagerActor, + statusEventBrokerActor, + resourceClusters, + resourceClustersHostManagerActor, + serverPort, + mantisStorageProvider, + lifecycleEventPublisher, + leadershipManager, + httpsConnectionContext, + List.of() + ); + } + public MasterApiAkkaService(final MasterMonitor masterMonitor, + final MasterDescription masterDescription, + final ActorRef jobClustersManagerActor, + final ActorRef statusEventBrokerActor, + final ResourceClusters resourceClusters, + final ActorRef resourceClustersHostManagerActor, + final int serverPort, + final IMantisPersistenceProvider mantisStorageProvider, + final LifecycleEventPublisher lifecycleEventPublisher, + final ILeadershipManager leadershipManager, + final HttpsConnectionContext httpsConnectionContext, + final List healthChecks) { super(true); Preconditions.checkNotNull(masterMonitor, "MasterMonitor"); Preconditions.checkNotNull(masterDescription, "masterDescription"); @@ -149,6 +182,7 @@ public MasterApiAkkaService(final MasterMonitor masterMonitor, this.storageProvider = mantisStorageProvider; this.lifecycleEventPublisher = lifecycleEventPublisher; this.leadershipManager = leadershipManager; + this.healthChecks = healthChecks != null ? healthChecks : List.of(); this.system = ActorSystem.create("MasterApiActorSystem"); this.materializer = Materializer.createMaterializer(system); this.mantisMasterRoute = configureApiRoutes(this.system); @@ -170,9 +204,14 @@ public MasterApiAkkaService(final MasterMonitor masterMonitor, private MantisMasterRoute configureApiRoutes(final ActorSystem actorSystem) { // Setup API routes - final JobClusterRouteHandler jobClusterRouteHandler = new JobClusterRouteHandlerAkkaImpl(jobClustersManagerActor); final JobRouteHandler jobRouteHandler = new JobRouteHandlerAkkaImpl(jobClustersManagerActor); + final List allHealthChecks = new ArrayList<>(); + allHealthChecks.add(new WorkerLaunchHealthCheck(jobRouteHandler)); + allHealthChecks.addAll(this.healthChecks); + + final JobClusterRouteHandler jobClusterRouteHandler = new JobClusterRouteHandlerAkkaImpl(jobClustersManagerActor, allHealthChecks); + final MasterDescriptionRoute masterDescriptionRoute = new MasterDescriptionRoute(masterDescription); final JobRoute v0JobRoute = new JobRoute(jobRouteHandler, actorSystem); diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandler.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandler.java index 6369478a9..c6285e45e 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandler.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandler.java @@ -16,11 +16,13 @@ package io.mantisrx.master.api.akka.route.handlers; +import io.mantisrx.master.jobcluster.proto.HealthCheckResponse; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.UpdateSchedulingInfoRequest; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.UpdateSchedulingInfoResponse; import io.mantisrx.master.jobcluster.proto.JobClusterScalerRuleProto; - +import java.util.List; +import java.util.Map; import java.util.concurrent.CompletionStage; public interface JobClusterRouteHandler { @@ -59,4 +61,6 @@ CompletionStage updateSchedulingInfo( CompletionStage getScalerRules(JobClusterScalerRuleProto.GetScalerRulesRequest request); CompletionStage deleteScalerRule(JobClusterScalerRuleProto.DeleteScalerRuleRequest request); + + CompletionStage healthCheck(String clusterName, List jobIds, Map context); } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java index 0f2b311c7..28615804a 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java @@ -23,13 +23,18 @@ import io.mantisrx.common.metrics.Metrics; import io.mantisrx.common.metrics.MetricsRegistry; import io.mantisrx.master.JobClustersManagerActor.UpdateSchedulingInfo; +import io.mantisrx.master.jobcluster.HealthCheck; +import io.mantisrx.master.jobcluster.proto.HealthCheckResponse; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.UpdateSchedulingInfoRequest; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.UpdateSchedulingInfoResponse; import io.mantisrx.master.jobcluster.proto.JobClusterScalerRuleProto; import io.mantisrx.server.master.config.ConfigurationProvider; import java.time.Duration; +import java.util.List; +import java.util.Map; import java.util.Optional; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,9 +45,15 @@ public class JobClusterRouteHandlerAkkaImpl implements JobClusterRouteHandler { private final ActorRef jobClustersManagerActor; private final Counter allJobClustersGET; private final Duration timeout; + private final List healthChecks; public JobClusterRouteHandlerAkkaImpl(ActorRef jobClusterManagerActor) { + this(jobClusterManagerActor, List.of()); + } + + public JobClusterRouteHandlerAkkaImpl(ActorRef jobClusterManagerActor, List healthChecks) { this.jobClustersManagerActor = jobClusterManagerActor; + this.healthChecks = healthChecks; long timeoutMs = Optional.ofNullable(ConfigurationProvider.getConfig().getMasterApiAskTimeoutMs()).orElse(1000L); this.timeout = Duration.ofMillis(timeoutMs); Metrics m = new Metrics.Builder() @@ -177,4 +188,18 @@ public CompletionStage .thenApply(JobClusterManagerProto.GetLatestJobDiscoveryInfoResponse.class::cast); return response; } + + @Override + public CompletionStage healthCheck( + String clusterName, List jobIds, Map context) { + return CompletableFuture.supplyAsync(() -> { + for (HealthCheck check : healthChecks) { + HealthCheckResponse result = check.check(clusterName, jobIds, context); + if (!result.isHealthy()) { + return result; + } + } + return HealthCheckResponse.healthy(0L); + }); + } } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/HttpRequestMetrics.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/HttpRequestMetrics.java index 772f56610..c43dfb0d8 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/HttpRequestMetrics.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/HttpRequestMetrics.java @@ -67,6 +67,7 @@ public static class Endpoints { public static final String RESOURCE_CLUSTERS = "api.v1.resourceClusters"; public static final String JOB_CLUSTER_SCALER_RULES = "api.v1.jobClusters.scalerRules"; + public static final String JOB_CLUSTER_INSTANCE_HEALTHCHECK = "api.v1.jobClusters.instance.healthcheck"; private static String[] endpoints = new String[]{ JOB_ARTIFACTS, @@ -99,7 +100,8 @@ public static class Endpoints { JOB_DISCOVERY_STREAM, LAST_SUBMITTED_JOB_ID_STREAM, RESOURCE_CLUSTERS, - JOB_CLUSTER_SCALER_RULES + JOB_CLUSTER_SCALER_RULES, + JOB_CLUSTER_INSTANCE_HEALTHCHECK }; private static Set endpointSet = Sets.newHashSet(endpoints); diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java index 9c33738b6..6dbb098fe 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java @@ -56,12 +56,17 @@ import io.mantisrx.master.api.akka.route.handlers.JobClusterRouteHandler; import io.mantisrx.master.api.akka.route.proto.JobClusterProtoAdapter; import io.mantisrx.master.jobcluster.proto.BaseResponse; +import io.mantisrx.master.jobcluster.proto.HealthCheckResponse; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto; import io.mantisrx.master.jobcluster.proto.JobClusterScalerRuleProto; import io.mantisrx.runtime.NamedJobDefinition; import io.mantisrx.server.master.config.ConfigurationProvider; import io.mantisrx.server.master.config.MasterConfiguration; import io.mantisrx.shaded.com.google.common.base.Strings; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; @@ -211,6 +216,14 @@ public Route constructRoutes() { )) ), + // api/v1/jobClusters/{}/healthcheck + path( + PathMatchers.segment().slash("healthcheck"), + (clusterName) -> pathEndOrSingleSlash(() -> + get(() -> healthCheckRoute(clusterName)) + ) + ), + // api/v1/jobClusters/{}/scalerRules path( PathMatchers.segment().slash("scalerRules"), @@ -775,6 +788,29 @@ private Route getScalerRulesRoute(String clusterName) { ); } + private Route healthCheckRoute(String clusterName) { + logger.trace("GET /api/v1/jobClusters/{}/healthcheck called", clusterName); + + return parameterMap(params -> { + String jobIdsParam = params.get("job-ids"); + List jobIds = (jobIdsParam != null && !jobIdsParam.isBlank()) + ? Arrays.asList(jobIdsParam.split(",")) + : null; + + Map context = new HashMap<>(params); + + CompletionStage response = + jobClusterRouteHandler.healthCheck(clusterName, jobIds, context); + + return completeAsync( + response, + resp -> complete(StatusCodes.OK, resp, Jackson.marshaller()), + HttpRequestMetrics.Endpoints.JOB_CLUSTER_INSTANCE_HEALTHCHECK, + HttpRequestMetrics.HttpVerb.GET + ); + }); + } + private Route deleteScalerRuleRoute(String clusterName, String ruleId) { logger.info("DELETE /api/v1/jobClusters/{}/scalerRules/{} called", clusterName, ruleId); diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/HealthCheck.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/HealthCheck.java new file mode 100644 index 000000000..f489ebae0 --- /dev/null +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/HealthCheck.java @@ -0,0 +1,42 @@ +/* + * Copyright 2026 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.master.jobcluster; + +import io.mantisrx.master.jobcluster.proto.HealthCheckResponse; +import java.util.List; +import java.util.Map; + +/** + * Extension point for job cluster health checks. Implementations are composed into an ordered + * list and executed sequentially — the chain stops at the first failure. + * + *

The {@code context} map carries caller-provided parameters (e.g., alert group names) + * without the interface needing to know about caller-specific concepts. + */ +public interface HealthCheck { + + /** + * Run a health check against the given job cluster. + * + * @param clusterName the job cluster name + * @param jobIds specific job IDs to check, or empty to use the latest active job + * @param context arbitrary caller-provided parameters + * @return a healthy response, or an unhealthy response with the appropriate {@link + * HealthCheckResponse.FailureReason} + */ + HealthCheckResponse check(String clusterName, List jobIds, Map context); +} diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/WorkerLaunchHealthCheck.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/WorkerLaunchHealthCheck.java new file mode 100644 index 000000000..f953ed116 --- /dev/null +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/WorkerLaunchHealthCheck.java @@ -0,0 +1,138 @@ +/* + * Copyright 2026 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.master.jobcluster; + +import io.mantisrx.master.api.akka.route.handlers.JobRouteHandler; +import io.mantisrx.master.jobcluster.job.FilterableMantisWorkerMetadataWritable; +import io.mantisrx.master.jobcluster.job.MantisJobMetadataView; +import io.mantisrx.master.jobcluster.proto.HealthCheckResponse; +import io.mantisrx.master.jobcluster.proto.HealthCheckResponse.FailedWorker; +import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.GetJobDetailsRequest; +import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.GetJobDetailsResponse; +import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.ListJobCriteria; +import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.ListJobsRequest; +import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.ListJobsResponse; +import io.mantisrx.runtime.MantisJobState; +import io.mantisrx.shaded.com.google.common.collect.Lists; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Default health check that verifies all workers in a job cluster are in the {@link + * MantisJobState#Started} state. + */ +public class WorkerLaunchHealthCheck implements HealthCheck { + + private static final Logger logger = LoggerFactory.getLogger(WorkerLaunchHealthCheck.class); + + private final JobRouteHandler jobRouteHandler; + + public WorkerLaunchHealthCheck(JobRouteHandler jobRouteHandler) { + this.jobRouteHandler = jobRouteHandler; + } + + @Override + public HealthCheckResponse check( + String clusterName, List jobIds, Map context) { + List jobs = fetchJobs(clusterName, jobIds); + List failedWorkers = findUnhealthyWorkers(jobs); + + if (!failedWorkers.isEmpty()) { + logger.info( + "Health check failed for cluster {}: {} unhealthy workers", + clusterName, + failedWorkers.size()); + return HealthCheckResponse.unhealthyWorkers(0L, failedWorkers); + } + + return HealthCheckResponse.healthy(0L); + } + + private List fetchJobs(String clusterName, List jobIds) { + if (jobIds != null && !jobIds.isEmpty()) { + List jobs = new ArrayList<>(); + for (String jobId : jobIds) { + try { + GetJobDetailsResponse response = + jobRouteHandler + .getJobDetails(new GetJobDetailsRequest("healthcheck", jobId)) + .toCompletableFuture() + .join(); + response.getJobMetadata() + .ifPresent(meta -> jobs.add(new MantisJobMetadataView( + meta, -1, + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList(), + false))); + } catch (Exception e) { + logger.warn("Failed to fetch job details for {}", jobId, e); + } + } + return jobs; + } + + try { + ListJobCriteria criteria = new ListJobCriteria( + Optional.of(1), + Optional.empty(), + Lists.newArrayList(), + Lists.newArrayList(), + Lists.newArrayList(), + Lists.newArrayList(), + Optional.of(true), + Optional.of(clusterName), + Optional.empty(), + Optional.empty(), + Optional.empty()); + ListJobsResponse response = + jobRouteHandler + .listJobs(new ListJobsRequest(criteria)) + .toCompletableFuture() + .join(); + return response.getJobList(); + } catch (Exception e) { + logger.warn("Failed to list jobs for cluster {}", clusterName, e); + return List.of(); + } + } + + private List findUnhealthyWorkers(List jobs) { + List failedWorkers = new ArrayList<>(); + for (MantisJobMetadataView job : jobs) { + if (job.getWorkerMetadataList() == null) { + continue; + } + for (FilterableMantisWorkerMetadataWritable worker : job.getWorkerMetadataList()) { + if (worker.getState() != MantisJobState.Started) { + failedWorkers.add( + new FailedWorker( + worker.getWorkerIndex(), + worker.getWorkerNumber(), + worker.getState().name())); + } + } + } + return failedWorkers; + } +} diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/HealthCheckResponse.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/HealthCheckResponse.java new file mode 100644 index 000000000..334c17c2f --- /dev/null +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/HealthCheckResponse.java @@ -0,0 +1,79 @@ +/* + * Copyright 2026 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.master.jobcluster.proto; + +import io.mantisrx.shaded.com.fasterxml.jackson.annotation.JsonCreator; +import io.mantisrx.shaded.com.fasterxml.jackson.annotation.JsonInclude; +import io.mantisrx.shaded.com.fasterxml.jackson.annotation.JsonProperty; +import io.mantisrx.shaded.com.fasterxml.jackson.annotation.JsonSubTypes; +import io.mantisrx.shaded.com.fasterxml.jackson.annotation.JsonTypeInfo; +import java.util.List; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.ToString; + +@Getter +@EqualsAndHashCode(callSuper = true) +@ToString +@JsonInclude(JsonInclude.Include.ALWAYS) +public class HealthCheckResponse extends BaseResponse { + + private final boolean isHealthy; + private final FailureReason failureReason; + + @JsonCreator + public HealthCheckResponse( + @JsonProperty("requestId") long requestId, + @JsonProperty("responseCode") ResponseCode responseCode, + @JsonProperty("message") String message, + @JsonProperty("isHealthy") boolean isHealthy, + @JsonProperty("failureReason") FailureReason failureReason) { + super(requestId, responseCode, message); + this.isHealthy = isHealthy; + this.failureReason = failureReason; + } + + public static HealthCheckResponse healthy(long requestId) { + return new HealthCheckResponse( + requestId, ResponseCode.SUCCESS, "OK", true, null); + } + + public static HealthCheckResponse unhealthyWorkers(long requestId, List failedWorkers) { + return new HealthCheckResponse( + requestId, ResponseCode.SUCCESS, "unhealthy workers", false, + new WorkerFailure(failedWorkers)); + } + + public static HealthCheckResponse unhealthyAlerts(long requestId, List alerts) { + return new HealthCheckResponse( + requestId, ResponseCode.SUCCESS, "alerts firing", false, + new AlertFailure(alerts)); + } + + @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") + @JsonSubTypes({ + @JsonSubTypes.Type(value = WorkerFailure.class, name = "workerStatus"), + @JsonSubTypes.Type(value = AlertFailure.class, name = "alertsFiring") + }) + public sealed interface FailureReason permits WorkerFailure, AlertFailure {} + + public record WorkerFailure(List failedWorkers) implements FailureReason {} + + public record AlertFailure(List alerts) implements FailureReason {} + + public record FailedWorker(int workerIndex, int workerNumber, String state) {} +} diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/MasterMain.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/MasterMain.java index 273172052..ba086f89a 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/MasterMain.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/MasterMain.java @@ -36,6 +36,8 @@ import io.mantisrx.master.api.akka.MasterApiAkkaService; import io.mantisrx.master.events.AuditEventBrokerActor; import io.mantisrx.master.events.AuditEventSubscriber; +import io.mantisrx.master.jobcluster.HealthCheck; +import java.util.List; import io.mantisrx.master.events.AuditEventSubscriberAkkaImpl; import io.mantisrx.master.events.AuditEventSubscriberLoggingImpl; import io.mantisrx.master.events.LifecycleEventPublisher; @@ -103,11 +105,21 @@ public class MasterMain implements Service { private MasterConfiguration config; private ILeadershipManager leadershipManager; private MasterMonitor monitor; + private final List healthChecks; public MasterMain( ConfigurationFactory configFactory, MantisPropertiesLoader dynamicPropertiesLoader, AuditEventSubscriber auditEventSubscriber) { + this(configFactory, dynamicPropertiesLoader, auditEventSubscriber, List.of()); + } + + public MasterMain( + ConfigurationFactory configFactory, + MantisPropertiesLoader dynamicPropertiesLoader, + AuditEventSubscriber auditEventSubscriber, + List healthChecks) { + this.healthChecks = healthChecks != null ? healthChecks : List.of(); String test = "{\"jobId\":\"sine-function-1\",\"status\":{\"jobId\":\"sine-function-1\",\"stageNum\":1,\"workerIndex\":0,\"workerNumber\":2,\"type\":\"HEARTBEAT\",\"message\":\"heartbeat\",\"state\":\"Noop\",\"hostname\":null,\"timestamp\":1525813363585,\"reason\":\"Normal\",\"payloads\":[{\"type\":\"SubscriptionState\",\"data\":\"false\"},{\"type\":\"IncomingDataDrop\",\"data\":\"{\\\"onNextCount\\\":0,\\\"droppedCount\\\":0}\"}]}}"; Metrics metrics = new Metrics.Builder() @@ -217,7 +229,7 @@ public MasterMain( monitor.start(); mantisServices.addService(leaderFactory.createLeaderElector(config, leadershipManager)); mantisServices.addService(new MasterApiAkkaService(monitor, leadershipManager.getDescription(), jobClusterManagerActor, statusEventBrokerActor, - resourceClusters, resourceClustersHostActor, config.getApiPort(), storageProvider, lifecycleEventPublisher, leadershipManager)); + resourceClusters, resourceClustersHostActor, config.getApiPort(), storageProvider, lifecycleEventPublisher, leadershipManager, null, healthChecks)); if (leaderFactory instanceof LocalLeaderFactory && !config.isLocalMode()) { logger.error("local mode is [ {} ] and leader factory is {} this configuration is unsafe", config.isLocalMode(), leaderFactory.getClass().getSimpleName()); From 0c66a84d99c634e8b8aa9762dd3c95c0ee42f06e Mon Sep 17 00:00:00 2001 From: jlubin Date: Thu, 9 Apr 2026 11:13:05 -0400 Subject: [PATCH 02/15] Refactor --- .../master/JobClustersManagerActor.java | 13 ++ .../master/api/akka/MasterApiAkkaService.java | 12 +- .../JobClusterRouteHandlerAkkaImpl.java | 40 +++-- .../api/akka/route/v1/JobClustersRoute.java | 8 +- .../master/jobcluster/HealthCheck.java | 18 ++- .../master/jobcluster/JobClusterActor.java | 78 ++++++++++ .../jobcluster/WorkerLaunchHealthCheck.java | 138 ------------------ .../jobcluster/proto/HealthCheckResponse.java | 4 +- .../proto/JobClusterManagerProto.java | 31 ++++ 9 files changed, 176 insertions(+), 166 deletions(-) delete mode 100644 mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/WorkerLaunchHealthCheck.java 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 00b2ebbd6..5b47c5ae4 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 @@ -32,6 +32,7 @@ import static io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.EnableJobClusterRequest; import static io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.EnableJobClusterResponse; import static io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.GetJobClusterRequest; +import static io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.HealthCheckRequest; import static io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.GetJobClusterResponse; import static io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.GetJobDetailsResponse; import static io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.GetJobSchedInfoRequest; @@ -93,6 +94,7 @@ import io.mantisrx.master.jobcluster.job.JobHelper; import io.mantisrx.master.jobcluster.job.JobState; import io.mantisrx.master.jobcluster.proto.BaseResponse; +import io.mantisrx.master.jobcluster.proto.HealthCheckResponse; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.GetJobDetailsRequest; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.ResubmitWorkerRequest; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.UpdateJobClusterArtifactRequest; @@ -241,6 +243,7 @@ private Receive getInitializedBehavior() { .match(EnableJobClusterRequest.class, this::onJobClusterEnable) .match(DisableJobClusterRequest.class, this::onJobClusterDisable) .match(GetJobClusterRequest.class, this::onJobClusterGet) + .match(HealthCheckRequest.class, this::onHealthCheck) .match(ListCompletedJobsInClusterRequest.class, this::onJobListCompleted) .match(GetLastSubmittedJobIdStreamRequest.class, this::onGetLastSubmittedJobIdSubject) .match(ListArchivedWorkersRequest.class, this::onListArchivedWorkers) @@ -561,6 +564,16 @@ public void onJobClusterGet(GetJobClusterRequest r) { sender.tell(new GetJobClusterResponse(r.requestId, CLIENT_ERROR_NOT_FOUND, "No such Job cluster " + r.getJobClusterName(), empty()), getSelf()); } } + + public void onHealthCheck(HealthCheckRequest r) { + Optional jobClusterInfo = jobClusterInfoManager.getJobClusterInfo(r.getClusterName()); + ActorRef sender = getSender(); + if (jobClusterInfo.isPresent()) { + jobClusterInfo.get().jobClusterActor.forward(r, getContext()); + } else { + sender.tell(new HealthCheckResponse(r.requestId, CLIENT_ERROR_NOT_FOUND, "No such Job cluster " + r.getClusterName(), false, null), getSelf()); + } + } @Override public void onGetLastSubmittedJobIdSubject(GetLastSubmittedJobIdStreamRequest r) { Optional jobClusterInfo = jobClusterInfoManager.getJobClusterInfo(r.getClusterName()); diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/MasterApiAkkaService.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/MasterApiAkkaService.java index 1903b82fe..25f6b3985 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/MasterApiAkkaService.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/MasterApiAkkaService.java @@ -65,10 +65,9 @@ import io.mantisrx.server.master.LeaderRedirectionFilter; import io.mantisrx.server.master.persistence.IMantisPersistenceProvider; import io.mantisrx.master.jobcluster.HealthCheck; -import io.mantisrx.master.jobcluster.WorkerLaunchHealthCheck; import io.mantisrx.server.master.resourcecluster.ResourceClusters; -import java.util.ArrayList; import java.util.List; +import java.util.Objects; import java.util.concurrent.CompletionStage; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; @@ -182,7 +181,7 @@ public MasterApiAkkaService(final MasterMonitor masterMonitor, this.storageProvider = mantisStorageProvider; this.lifecycleEventPublisher = lifecycleEventPublisher; this.leadershipManager = leadershipManager; - this.healthChecks = healthChecks != null ? healthChecks : List.of(); + this.healthChecks = Objects.requireNonNullElseGet(healthChecks, List::of); this.system = ActorSystem.create("MasterApiActorSystem"); this.materializer = Materializer.createMaterializer(system); this.mantisMasterRoute = configureApiRoutes(this.system); @@ -204,14 +203,9 @@ public MasterApiAkkaService(final MasterMonitor masterMonitor, private MantisMasterRoute configureApiRoutes(final ActorSystem actorSystem) { // Setup API routes + final JobClusterRouteHandler jobClusterRouteHandler = new JobClusterRouteHandlerAkkaImpl(jobClustersManagerActor, this.healthChecks); final JobRouteHandler jobRouteHandler = new JobRouteHandlerAkkaImpl(jobClustersManagerActor); - final List allHealthChecks = new ArrayList<>(); - allHealthChecks.add(new WorkerLaunchHealthCheck(jobRouteHandler)); - allHealthChecks.addAll(this.healthChecks); - - final JobClusterRouteHandler jobClusterRouteHandler = new JobClusterRouteHandlerAkkaImpl(jobClustersManagerActor, allHealthChecks); - final MasterDescriptionRoute masterDescriptionRoute = new MasterDescriptionRoute(masterDescription); final JobRoute v0JobRoute = new JobRoute(jobRouteHandler, actorSystem); diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java index 28615804a..67d4f6fb8 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java @@ -31,10 +31,10 @@ import io.mantisrx.master.jobcluster.proto.JobClusterScalerRuleProto; import io.mantisrx.server.master.config.ConfigurationProvider; import java.time.Duration; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -192,14 +192,36 @@ public CompletionStage @Override public CompletionStage healthCheck( String clusterName, List jobIds, Map context) { - return CompletableFuture.supplyAsync(() -> { - for (HealthCheck check : healthChecks) { - HealthCheckResponse result = check.check(clusterName, jobIds, context); - if (!result.isHealthy()) { - return result; - } + JobClusterManagerProto.HealthCheckRequest request = + new JobClusterManagerProto.HealthCheckRequest(clusterName, jobIds); + return ask(jobClustersManagerActor, request, timeout) + .thenApply(HealthCheckResponse.class::cast) + .thenApply(actorResponse -> { + if (!actorResponse.isHealthy() || healthChecks.isEmpty()) { + return actorResponse; + } + for (HealthCheck healthCheck : healthChecks) { + Map scopedContext = extractContext(context, healthCheck.contextId()); + HealthCheckResponse result = healthCheck.checkHealth(clusterName, jobIds, scopedContext); + if (!result.isHealthy()) { + return result; + } + } + return actorResponse; + }); + } + + private static Map extractContext(Map context, String contextId) { + if (contextId == null || contextId.isEmpty()) { + return context; + } + String prefix = contextId + "."; + Map scoped = new HashMap<>(); + for (Map.Entry entry : context.entrySet()) { + if (entry.getKey().startsWith(prefix)) { + scoped.put(entry.getKey().substring(prefix.length()), entry.getValue()); } - return HealthCheckResponse.healthy(0L); - }); + } + return scoped; } } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java index 6dbb098fe..65d6beefc 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java @@ -798,13 +798,13 @@ private Route healthCheckRoute(String clusterName) { : null; Map context = new HashMap<>(params); - - CompletionStage response = - jobClusterRouteHandler.healthCheck(clusterName, jobIds, context); + CompletionStage response = jobClusterRouteHandler.healthCheck(clusterName, jobIds, context); return completeAsync( response, - resp -> complete(StatusCodes.OK, resp, Jackson.marshaller()), + resp -> complete( + resp.isHealthy() ? StatusCodes.OK : StatusCodes.SERVICE_UNAVAILABLE, + resp, Jackson.marshaller()), HttpRequestMetrics.Endpoints.JOB_CLUSTER_INSTANCE_HEALTHCHECK, HttpRequestMetrics.HttpVerb.GET ); diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/HealthCheck.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/HealthCheck.java index f489ebae0..53d826ade 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/HealthCheck.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/HealthCheck.java @@ -24,19 +24,29 @@ * Extension point for job cluster health checks. Implementations are composed into an ordered * list and executed sequentially — the chain stops at the first failure. * - *

The {@code context} map carries caller-provided parameters (e.g., alert group names) - * without the interface needing to know about caller-specific concepts. + *

Each implementation declares a {@link #contextId()} that acts as a namespace prefix. + * The handler extracts query params prefixed with this ID (e.g., {@code radar.alertGroups}) + * and passes them to the check with the prefix stripped. */ public interface HealthCheck { + /** + * Namespace prefix for context parameters belonging to this health check. + * For example, a check with contextId "radar" receives query params like + * {@code radar.alertGroups} as {@code alertGroups} in its context map. + * + * @return the context namespace, or empty string to receive unnamespaced params + */ + String contextId(); + /** * Run a health check against the given job cluster. * * @param clusterName the job cluster name * @param jobIds specific job IDs to check, or empty to use the latest active job - * @param context arbitrary caller-provided parameters + * @param context parameters namespaced to this check (prefix already stripped) * @return a healthy response, or an unhealthy response with the appropriate {@link * HealthCheckResponse.FailureReason} */ - HealthCheckResponse check(String clusterName, List jobIds, Map context); + HealthCheckResponse checkHealth(String clusterName, List jobIds, Map context); } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java index d2c238e8f..0cd64c702 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java @@ -61,8 +61,11 @@ import io.mantisrx.master.jobcluster.job.MantisJobMetadataImpl; import io.mantisrx.master.jobcluster.job.MantisJobMetadataView; import io.mantisrx.master.jobcluster.job.worker.IMantisWorkerMetadata; +import io.mantisrx.master.jobcluster.proto.HealthCheckResponse; +import io.mantisrx.master.jobcluster.proto.HealthCheckResponse.FailedWorker; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.DeleteJobClusterResponse; +import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.HealthCheckRequest; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.DisableJobClusterRequest; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.DisableJobClusterResponse; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.EnableJobClusterRequest; @@ -118,6 +121,7 @@ import io.mantisrx.master.jobcluster.scaler.JobClusterScalerRuleDataFactory; import io.mantisrx.master.jobcluster.scaler.JobClusterScalerRuleDataImplWritable; import io.mantisrx.runtime.JobSla; +import io.mantisrx.runtime.MantisJobState; import io.mantisrx.runtime.command.InvalidJobException; import io.mantisrx.server.core.JobCompletedReason; import io.mantisrx.server.master.InvalidJobRequestException; @@ -617,6 +621,7 @@ private Receive buildInitializedBehavior() { new EnableJobClusterResponse(x.requestId, SUCCESS, genUnexpectedMsg(x.toString(), this.name, state)), getSelf())) .match(GetJobClusterRequest.class, this::onJobClusterGet) + .match(HealthCheckRequest.class, this::onHealthCheck) .match(JobClusterProto.DeleteJobClusterRequest.class, this::onJobClusterDelete) .match(ListArchivedWorkersRequest.class, this::onListArchivedWorkers) .match(ListCompletedJobsInClusterRequest.class, this::onJobListCompleted) @@ -1361,6 +1366,79 @@ public void onJobClusterGet(final GetJobClusterRequest request) { if(logger.isTraceEnabled()) { logger.trace("Exit onJobClusterGet"); } } + public void onHealthCheck(final HealthCheckRequest request) { + final ActorRef sender = getSender(); + final ActorRef self = getSelf(); + Duration timeout = Duration.ofMillis(500); + + List jobInfoList; + if (request.getJobIds() != null && !request.getJobIds().isEmpty()) { + jobInfoList = request.getJobIds().stream() + .map(id -> JobId.fromId(id)) + .filter(Optional::isPresent) + .map(Optional::get) + .map(jId -> jobManager.getJobInfoForNonTerminalJob(jId)) + .filter(Optional::isPresent) + .map(Optional::get) + .collect(Collectors.toList()); + } else { + jobInfoList = jobManager.getAllNonTerminalJobsList(); + if (jobInfoList.size() > 1) { + jobInfoList = jobInfoList.subList(0, 1); + } + } + + if (jobInfoList.isEmpty()) { + sender.tell(HealthCheckResponse.healthy(request.requestId), self); + return; + } + + List failedWorkers = new ArrayList<>(); + Observable.from(jobInfoList) + .flatMap(jInfo -> { + GetJobDetailsRequest req = new GetJobDetailsRequest("healthcheck", jInfo.jobId); + CompletionStage respCS = ask(jInfo.jobActor, req, timeout) + .thenApply(GetJobDetailsResponse.class::cast); + return Observable.from(respCS.toCompletableFuture(), Schedulers.io()) + .onErrorResumeNext(ex -> { + logger.warn("Health check failed to get job details for {}", jInfo.jobId, ex); + return Observable.empty(); + }); + }) + .filter(resp -> resp != null && resp.getJobMetadata().isPresent()) + .map(resp -> new MantisJobMetadataView(resp.getJobMetadata().get(), + Collections.emptyList(), Collections.emptyList(), + Collections.emptyList(), Collections.emptyList(), false)) + .doOnNext(view -> { + if (view.getWorkerMetadataList() != null) { + for (var worker : view.getWorkerMetadataList()) { + if (worker.getState() != MantisJobState.Started) { + failedWorkers.add(new FailedWorker( + worker.getWorkerIndex(), + worker.getWorkerNumber(), + worker.getState().name())); + } + } + } + }) + .toList() + .subscribeOn(Schedulers.computation()) + .subscribe( + views -> { + if (failedWorkers.isEmpty()) { + sender.tell(HealthCheckResponse.healthy(request.requestId), self); + } else { + sender.tell(HealthCheckResponse.unhealthyWorkers(request.requestId, failedWorkers), self); + } + }, + error -> { + logger.error("Health check failed for cluster {}", name, error); + sender.tell(new HealthCheckResponse(request.requestId, SERVER_ERROR, + "Health check failed: " + error.getMessage(), false, null), self); + } + ); + } + private MantisJobClusterMetadataView generateJobClusterMetadataView(IJobClusterMetadata jobClusterMetadata, boolean isDisabled, boolean cronActive) { return new MantisJobClusterMetadataView.Builder() .withName(jobClusterMetadata.getJobClusterDefinition().getName()) diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/WorkerLaunchHealthCheck.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/WorkerLaunchHealthCheck.java deleted file mode 100644 index f953ed116..000000000 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/WorkerLaunchHealthCheck.java +++ /dev/null @@ -1,138 +0,0 @@ -/* - * Copyright 2026 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.master.jobcluster; - -import io.mantisrx.master.api.akka.route.handlers.JobRouteHandler; -import io.mantisrx.master.jobcluster.job.FilterableMantisWorkerMetadataWritable; -import io.mantisrx.master.jobcluster.job.MantisJobMetadataView; -import io.mantisrx.master.jobcluster.proto.HealthCheckResponse; -import io.mantisrx.master.jobcluster.proto.HealthCheckResponse.FailedWorker; -import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.GetJobDetailsRequest; -import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.GetJobDetailsResponse; -import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.ListJobCriteria; -import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.ListJobsRequest; -import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.ListJobsResponse; -import io.mantisrx.runtime.MantisJobState; -import io.mantisrx.shaded.com.google.common.collect.Lists; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Default health check that verifies all workers in a job cluster are in the {@link - * MantisJobState#Started} state. - */ -public class WorkerLaunchHealthCheck implements HealthCheck { - - private static final Logger logger = LoggerFactory.getLogger(WorkerLaunchHealthCheck.class); - - private final JobRouteHandler jobRouteHandler; - - public WorkerLaunchHealthCheck(JobRouteHandler jobRouteHandler) { - this.jobRouteHandler = jobRouteHandler; - } - - @Override - public HealthCheckResponse check( - String clusterName, List jobIds, Map context) { - List jobs = fetchJobs(clusterName, jobIds); - List failedWorkers = findUnhealthyWorkers(jobs); - - if (!failedWorkers.isEmpty()) { - logger.info( - "Health check failed for cluster {}: {} unhealthy workers", - clusterName, - failedWorkers.size()); - return HealthCheckResponse.unhealthyWorkers(0L, failedWorkers); - } - - return HealthCheckResponse.healthy(0L); - } - - private List fetchJobs(String clusterName, List jobIds) { - if (jobIds != null && !jobIds.isEmpty()) { - List jobs = new ArrayList<>(); - for (String jobId : jobIds) { - try { - GetJobDetailsResponse response = - jobRouteHandler - .getJobDetails(new GetJobDetailsRequest("healthcheck", jobId)) - .toCompletableFuture() - .join(); - response.getJobMetadata() - .ifPresent(meta -> jobs.add(new MantisJobMetadataView( - meta, -1, - Collections.emptyList(), - Collections.emptyList(), - Collections.emptyList(), - Collections.emptyList(), - false))); - } catch (Exception e) { - logger.warn("Failed to fetch job details for {}", jobId, e); - } - } - return jobs; - } - - try { - ListJobCriteria criteria = new ListJobCriteria( - Optional.of(1), - Optional.empty(), - Lists.newArrayList(), - Lists.newArrayList(), - Lists.newArrayList(), - Lists.newArrayList(), - Optional.of(true), - Optional.of(clusterName), - Optional.empty(), - Optional.empty(), - Optional.empty()); - ListJobsResponse response = - jobRouteHandler - .listJobs(new ListJobsRequest(criteria)) - .toCompletableFuture() - .join(); - return response.getJobList(); - } catch (Exception e) { - logger.warn("Failed to list jobs for cluster {}", clusterName, e); - return List.of(); - } - } - - private List findUnhealthyWorkers(List jobs) { - List failedWorkers = new ArrayList<>(); - for (MantisJobMetadataView job : jobs) { - if (job.getWorkerMetadataList() == null) { - continue; - } - for (FilterableMantisWorkerMetadataWritable worker : job.getWorkerMetadataList()) { - if (worker.getState() != MantisJobState.Started) { - failedWorkers.add( - new FailedWorker( - worker.getWorkerIndex(), - worker.getWorkerNumber(), - worker.getState().name())); - } - } - } - return failedWorkers; - } -} diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/HealthCheckResponse.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/HealthCheckResponse.java index 334c17c2f..a0de73ddd 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/HealthCheckResponse.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/HealthCheckResponse.java @@ -54,13 +54,13 @@ public static HealthCheckResponse healthy(long requestId) { public static HealthCheckResponse unhealthyWorkers(long requestId, List failedWorkers) { return new HealthCheckResponse( - requestId, ResponseCode.SUCCESS, "unhealthy workers", false, + requestId, ResponseCode.SERVER_ERROR, "unhealthy workers", false, new WorkerFailure(failedWorkers)); } public static HealthCheckResponse unhealthyAlerts(long requestId, List alerts) { return new HealthCheckResponse( - requestId, ResponseCode.SUCCESS, "alerts firing", false, + requestId, ResponseCode.SERVER_ERROR, "alerts firing", false, new AlertFailure(alerts)); } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java index 6612b9c79..630d66a2b 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java @@ -883,6 +883,37 @@ public String toString() { } } + public static final class HealthCheckRequest extends BaseRequest { + private final String clusterName; + private final List jobIds; + + public HealthCheckRequest(final String clusterName, final List jobIds) { + super(); + Preconditions.checkArg( + clusterName != null && !clusterName.isEmpty(), + "Cluster name cannot be null or empty"); + this.clusterName = clusterName; + this.jobIds = jobIds; + } + + public String getClusterName() { + return clusterName; + } + + public List getJobIds() { + return jobIds; + } + + @Override + public String toString() { + return "HealthCheckRequest{" + + "clusterName='" + clusterName + '\'' + + ", jobIds=" + jobIds + + ", requestId=" + requestId + + '}'; + } + } + public static final class ListJobCriteria { private final Optional limit; private final Optional jobState; From 5f240aa570c639492fe67d63cbf1cbad6b7af2fc Mon Sep 17 00:00:00 2001 From: jlubin Date: Thu, 9 Apr 2026 12:52:27 -0400 Subject: [PATCH 03/15] Additional tweaks --- .../master/JobClustersManagerActor.java | 20 +-- .../JobClusterRouteHandlerAkkaImpl.java | 5 + .../api/akka/route/v1/JobClustersRoute.java | 38 +++--- .../master/jobcluster/HealthCheck.java | 15 +-- .../master/jobcluster/JobClusterActor.java | 117 +++++++----------- .../proto/JobClusterManagerProto.java | 60 +++++---- .../io/mantisrx/server/master/MasterMain.java | 2 +- 7 files changed, 117 insertions(+), 140 deletions(-) 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 5b47c5ae4..9abe41266 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 @@ -565,15 +565,6 @@ public void onJobClusterGet(GetJobClusterRequest r) { } } - public void onHealthCheck(HealthCheckRequest r) { - Optional jobClusterInfo = jobClusterInfoManager.getJobClusterInfo(r.getClusterName()); - ActorRef sender = getSender(); - if (jobClusterInfo.isPresent()) { - jobClusterInfo.get().jobClusterActor.forward(r, getContext()); - } else { - sender.tell(new HealthCheckResponse(r.requestId, CLIENT_ERROR_NOT_FOUND, "No such Job cluster " + r.getClusterName(), false, null), getSelf()); - } - } @Override public void onGetLastSubmittedJobIdSubject(GetLastSubmittedJobIdStreamRequest r) { Optional jobClusterInfo = jobClusterInfoManager.getJobClusterInfo(r.getClusterName()); @@ -678,6 +669,17 @@ public void onGetJobScalerRuleStream(final JobClusterScalerRuleProto.GetJobScale } } + public void onHealthCheck(HealthCheckRequest request) { + Optional jobClusterInfo = jobClusterInfoManager.getJobClusterInfo(request.getClusterName()); + + if (jobClusterInfo.isPresent()) { + jobClusterInfo.get().jobClusterActor.forward(request, getContext()); + } else { + ActorRef sender = getSender(); + sender.tell(new HealthCheckResponse(request.requestId, CLIENT_ERROR_NOT_FOUND, "No such Job cluster " + request.getClusterName(), false, null), getSelf()); + } + } + private void onTerminated(final Terminated terminated) { logger.warn("onTerminated {}", terminated.actor()); } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java index 67d4f6fb8..3c2348665 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java @@ -194,12 +194,14 @@ public CompletionStage healthCheck( String clusterName, List jobIds, Map context) { JobClusterManagerProto.HealthCheckRequest request = new JobClusterManagerProto.HealthCheckRequest(clusterName, jobIds); + return ask(jobClustersManagerActor, request, timeout) .thenApply(HealthCheckResponse.class::cast) .thenApply(actorResponse -> { if (!actorResponse.isHealthy() || healthChecks.isEmpty()) { return actorResponse; } + for (HealthCheck healthCheck : healthChecks) { Map scopedContext = extractContext(context, healthCheck.contextId()); HealthCheckResponse result = healthCheck.checkHealth(clusterName, jobIds, scopedContext); @@ -207,6 +209,7 @@ public CompletionStage healthCheck( return result; } } + return actorResponse; }); } @@ -215,6 +218,7 @@ private static Map extractContext(Map context, S if (contextId == null || contextId.isEmpty()) { return context; } + String prefix = contextId + "."; Map scoped = new HashMap<>(); for (Map.Entry entry : context.entrySet()) { @@ -222,6 +226,7 @@ private static Map extractContext(Map context, S scoped.put(entry.getKey().substring(prefix.length()), entry.getValue()); } } + return scoped; } } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java index 65d6beefc..453d8e54c 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java @@ -788,37 +788,37 @@ private Route getScalerRulesRoute(String clusterName) { ); } + private Route deleteScalerRuleRoute(String clusterName, String ruleId) { + logger.info("DELETE /api/v1/jobClusters/{}/scalerRules/{} called", clusterName, ruleId); + + return completeAsync( + jobClusterRouteHandler.deleteScalerRule(new JobClusterScalerRuleProto.DeleteScalerRuleRequest(clusterName, ruleId)), + resp -> complete(StatusCodes.NO_CONTENT, ""), + HttpRequestMetrics.Endpoints.JOB_CLUSTER_SCALER_RULES, + HttpRequestMetrics.HttpVerb.DELETE + ); + } + private Route healthCheckRoute(String clusterName) { logger.trace("GET /api/v1/jobClusters/{}/healthcheck called", clusterName); return parameterMap(params -> { String jobIdsParam = params.get("job-ids"); List jobIds = (jobIdsParam != null && !jobIdsParam.isBlank()) - ? Arrays.asList(jobIdsParam.split(",")) - : null; + ? Arrays.asList(jobIdsParam.split(",")) + : null; Map context = new HashMap<>(params); CompletionStage response = jobClusterRouteHandler.healthCheck(clusterName, jobIds, context); return completeAsync( - response, - resp -> complete( - resp.isHealthy() ? StatusCodes.OK : StatusCodes.SERVICE_UNAVAILABLE, - resp, Jackson.marshaller()), - HttpRequestMetrics.Endpoints.JOB_CLUSTER_INSTANCE_HEALTHCHECK, - HttpRequestMetrics.HttpVerb.GET + response, + resp -> complete( + resp.isHealthy() ? StatusCodes.OK : StatusCodes.SERVICE_UNAVAILABLE, + resp, Jackson.marshaller()), + HttpRequestMetrics.Endpoints.JOB_CLUSTER_INSTANCE_HEALTHCHECK, + HttpRequestMetrics.HttpVerb.GET ); }); } - - private Route deleteScalerRuleRoute(String clusterName, String ruleId) { - logger.info("DELETE /api/v1/jobClusters/{}/scalerRules/{} called", clusterName, ruleId); - - return completeAsync( - jobClusterRouteHandler.deleteScalerRule(new JobClusterScalerRuleProto.DeleteScalerRuleRequest(clusterName, ruleId)), - resp -> complete(StatusCodes.NO_CONTENT, ""), - HttpRequestMetrics.Endpoints.JOB_CLUSTER_SCALER_RULES, - HttpRequestMetrics.HttpVerb.DELETE - ); - } } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/HealthCheck.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/HealthCheck.java index 53d826ade..4cd6d8664 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/HealthCheck.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/HealthCheck.java @@ -25,26 +25,27 @@ * list and executed sequentially — the chain stops at the first failure. * *

Each implementation declares a {@link #contextId()} that acts as a namespace prefix. - * The handler extracts query params prefixed with this ID (e.g., {@code radar.alertGroups}) + * The handler extracts query params prefixed with this ID (e.g., {@code alertSystem.names}) * and passes them to the check with the prefix stripped. */ public interface HealthCheck { /** * Namespace prefix for context parameters belonging to this health check. - * For example, a check with contextId "radar" receives query params like - * {@code radar.alertGroups} as {@code alertGroups} in its context map. + * For example, a check with contextId "alertSystem" receives query params like + * {@code alertSystem.names} as {@code names} in its context map. * - * @return the context namespace, or empty string to receive unnamespaced params + * @return the context namespace. */ String contextId(); /** * Run a health check against the given job cluster. * - * @param clusterName the job cluster name - * @param jobIds specific job IDs to check, or empty to use the latest active job - * @param context parameters namespaced to this check (prefix already stripped) + * @param clusterName the job cluster name. + * @param jobIds specific job IDs to check. Used to overwrite the default behavior of checking all active jobs. + * @param context parameters namespaced to this check (prefix already stripped). + * * @return a healthy response, or an unhealthy response with the appropriate {@link * HealthCheckResponse.FailureReason} */ diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java index 0cd64c702..86209ab0d 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java @@ -54,6 +54,7 @@ import io.mantisrx.master.events.LifecycleEventPublisher; import io.mantisrx.master.events.LifecycleEventsProto; import io.mantisrx.master.jobcluster.job.CostsCalculator; +import io.mantisrx.master.jobcluster.job.FilterableMantisWorkerMetadataWritable; import io.mantisrx.master.jobcluster.job.IMantisJobMetadata; import io.mantisrx.master.jobcluster.job.JobActor; import io.mantisrx.master.jobcluster.job.JobHelper; @@ -1366,79 +1367,6 @@ public void onJobClusterGet(final GetJobClusterRequest request) { if(logger.isTraceEnabled()) { logger.trace("Exit onJobClusterGet"); } } - public void onHealthCheck(final HealthCheckRequest request) { - final ActorRef sender = getSender(); - final ActorRef self = getSelf(); - Duration timeout = Duration.ofMillis(500); - - List jobInfoList; - if (request.getJobIds() != null && !request.getJobIds().isEmpty()) { - jobInfoList = request.getJobIds().stream() - .map(id -> JobId.fromId(id)) - .filter(Optional::isPresent) - .map(Optional::get) - .map(jId -> jobManager.getJobInfoForNonTerminalJob(jId)) - .filter(Optional::isPresent) - .map(Optional::get) - .collect(Collectors.toList()); - } else { - jobInfoList = jobManager.getAllNonTerminalJobsList(); - if (jobInfoList.size() > 1) { - jobInfoList = jobInfoList.subList(0, 1); - } - } - - if (jobInfoList.isEmpty()) { - sender.tell(HealthCheckResponse.healthy(request.requestId), self); - return; - } - - List failedWorkers = new ArrayList<>(); - Observable.from(jobInfoList) - .flatMap(jInfo -> { - GetJobDetailsRequest req = new GetJobDetailsRequest("healthcheck", jInfo.jobId); - CompletionStage respCS = ask(jInfo.jobActor, req, timeout) - .thenApply(GetJobDetailsResponse.class::cast); - return Observable.from(respCS.toCompletableFuture(), Schedulers.io()) - .onErrorResumeNext(ex -> { - logger.warn("Health check failed to get job details for {}", jInfo.jobId, ex); - return Observable.empty(); - }); - }) - .filter(resp -> resp != null && resp.getJobMetadata().isPresent()) - .map(resp -> new MantisJobMetadataView(resp.getJobMetadata().get(), - Collections.emptyList(), Collections.emptyList(), - Collections.emptyList(), Collections.emptyList(), false)) - .doOnNext(view -> { - if (view.getWorkerMetadataList() != null) { - for (var worker : view.getWorkerMetadataList()) { - if (worker.getState() != MantisJobState.Started) { - failedWorkers.add(new FailedWorker( - worker.getWorkerIndex(), - worker.getWorkerNumber(), - worker.getState().name())); - } - } - } - }) - .toList() - .subscribeOn(Schedulers.computation()) - .subscribe( - views -> { - if (failedWorkers.isEmpty()) { - sender.tell(HealthCheckResponse.healthy(request.requestId), self); - } else { - sender.tell(HealthCheckResponse.unhealthyWorkers(request.requestId, failedWorkers), self); - } - }, - error -> { - logger.error("Health check failed for cluster {}", name, error); - sender.tell(new HealthCheckResponse(request.requestId, SERVER_ERROR, - "Health check failed: " + error.getMessage(), false, null), self); - } - ); - } - private MantisJobClusterMetadataView generateJobClusterMetadataView(IJobClusterMetadata jobClusterMetadata, boolean isDisabled, boolean cronActive) { return new MantisJobClusterMetadataView.Builder() .withName(jobClusterMetadata.getJobClusterDefinition().getName()) @@ -2711,6 +2639,49 @@ public void onJobScalerRuleStream(JobClusterScalerRuleProto.GetJobScalerRuleStre } } + public void onHealthCheck(final HealthCheckRequest request) { + final ActorRef sender = getSender(); + final ActorRef self = getSelf(); + + Set prefilteredJobIds = new HashSet<>(); + if (request.getJobIds() != null && !request.getJobIds().isEmpty()) { + request.getJobIds().stream() + .map(JobId::fromId) + .filter(Optional::isPresent) + .map(Optional::get) + .forEach(prefilteredJobIds::add); + } + + ListJobCriteria criteria = new ListJobCriteria(); + getFilteredNonTerminalJobList(criteria, prefilteredJobIds) + .collect(Lists::newArrayList, (failedWorkers, view) -> { + if (view.getWorkerMetadataList() != null) { + for (FilterableMantisWorkerMetadataWritable worker : view.getWorkerMetadataList()) { + if (worker.getState() != MantisJobState.Started) { + failedWorkers.add(new FailedWorker( + worker.getWorkerIndex(), + worker.getWorkerNumber(), + worker.getState().name())); + } + } + } + }) + .subscribe( + failedWorkers -> { + if (failedWorkers.isEmpty()) { + sender.tell(HealthCheckResponse.healthy(request.requestId), self); + } else { + sender.tell(HealthCheckResponse.unhealthyWorkers(request.requestId, failedWorkers), self); + } + }, + error -> { + logger.error("Health check failed for cluster {}", name, error); + sender.tell(new HealthCheckResponse(request.requestId, SERVER_ERROR, + "Health check failed: " + error.getMessage(), false, null), self); + } + ); + } + static final class JobInfo { final long submittedAt; diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java index 630d66a2b..ff238ebdd 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java @@ -883,37 +883,6 @@ public String toString() { } } - public static final class HealthCheckRequest extends BaseRequest { - private final String clusterName; - private final List jobIds; - - public HealthCheckRequest(final String clusterName, final List jobIds) { - super(); - Preconditions.checkArg( - clusterName != null && !clusterName.isEmpty(), - "Cluster name cannot be null or empty"); - this.clusterName = clusterName; - this.jobIds = jobIds; - } - - public String getClusterName() { - return clusterName; - } - - public List getJobIds() { - return jobIds; - } - - @Override - public String toString() { - return "HealthCheckRequest{" + - "clusterName='" + clusterName + '\'' + - ", jobIds=" + jobIds + - ", requestId=" + requestId + - '}'; - } - } - public static final class ListJobCriteria { private final Optional limit; private final Optional jobState; @@ -2284,5 +2253,34 @@ public String toString() { } } + public static final class HealthCheckRequest extends BaseRequest { + private final String clusterName; + private final List jobIds; + + public HealthCheckRequest(final String clusterName, final List jobIds) { + super(); + Preconditions.checkArg( + clusterName != null && !clusterName.isEmpty(), + "Cluster name cannot be null or empty"); + this.clusterName = clusterName; + this.jobIds = jobIds; + } + + public String getClusterName() { + return clusterName; + } + + public List getJobIds() { + return jobIds; + } + @Override + public String toString() { + return "HealthCheckRequest{" + + "clusterName='" + clusterName + '\'' + + ", jobIds=" + jobIds + + ", requestId=" + requestId + + '}'; + } + } } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/MasterMain.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/MasterMain.java index ba086f89a..085f496db 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/MasterMain.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/MasterMain.java @@ -119,7 +119,7 @@ public MasterMain( MantisPropertiesLoader dynamicPropertiesLoader, AuditEventSubscriber auditEventSubscriber, List healthChecks) { - this.healthChecks = healthChecks != null ? healthChecks : List.of(); + this.healthChecks = healthChecks == null ? List.of() : healthChecks; String test = "{\"jobId\":\"sine-function-1\",\"status\":{\"jobId\":\"sine-function-1\",\"stageNum\":1,\"workerIndex\":0,\"workerNumber\":2,\"type\":\"HEARTBEAT\",\"message\":\"heartbeat\",\"state\":\"Noop\",\"hostname\":null,\"timestamp\":1525813363585,\"reason\":\"Normal\",\"payloads\":[{\"type\":\"SubscriptionState\",\"data\":\"false\"},{\"type\":\"IncomingDataDrop\",\"data\":\"{\\\"onNextCount\\\":0,\\\"droppedCount\\\":0}\"}]}}"; Metrics metrics = new Metrics.Builder() From 153ca32af6b04ef34a2ee0a63be3f3287f85bad8 Mon Sep 17 00:00:00 2001 From: jlubin Date: Thu, 9 Apr 2026 17:01:18 -0400 Subject: [PATCH 04/15] Add tests --- .../master/api/akka/MasterApiAkkaService.java | 10 +- .../JobClusterRouteHandlerAkkaImpl.java | 16 +- .../api/akka/route/v1/JobClustersRoute.java | 4 +- ...thCheck.java => HealthCheckExtension.java} | 2 +- .../io/mantisrx/server/master/MasterMain.java | 10 +- .../JobClusterRouteHandlerAkkaImplTest.java | 264 ++++++++++++++++++ .../akka/route/v1/JobClustersRouteTest.java | 26 ++ .../master/jobcluster/JobClusterAkkaTest.java | 128 +++++++++ 8 files changed, 438 insertions(+), 22 deletions(-) rename mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/{HealthCheck.java => HealthCheckExtension.java} (98%) create mode 100644 mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImplTest.java diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/MasterApiAkkaService.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/MasterApiAkkaService.java index 25f6b3985..b87199539 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/MasterApiAkkaService.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/MasterApiAkkaService.java @@ -64,7 +64,7 @@ import io.mantisrx.server.core.master.MasterMonitor; import io.mantisrx.server.master.LeaderRedirectionFilter; import io.mantisrx.server.master.persistence.IMantisPersistenceProvider; -import io.mantisrx.master.jobcluster.HealthCheck; +import io.mantisrx.master.jobcluster.HealthCheckExtension; import io.mantisrx.server.master.resourcecluster.ResourceClusters; import java.util.List; import java.util.Objects; @@ -97,7 +97,7 @@ public class MasterApiAkkaService extends BaseService { private final ExecutorService executorService; private final CountDownLatch serviceLatch = new CountDownLatch(1); private final HttpsConnectionContext httpsConnectionContext; - private final List healthChecks; + private final List healthCheckExtensions; public MasterApiAkkaService(final MasterMonitor masterMonitor, final MasterDescription masterDescription, @@ -162,7 +162,7 @@ public MasterApiAkkaService(final MasterMonitor masterMonitor, final LifecycleEventPublisher lifecycleEventPublisher, final ILeadershipManager leadershipManager, final HttpsConnectionContext httpsConnectionContext, - final List healthChecks) { + final List healthCheckExtensions) { super(true); Preconditions.checkNotNull(masterMonitor, "MasterMonitor"); Preconditions.checkNotNull(masterDescription, "masterDescription"); @@ -181,7 +181,7 @@ public MasterApiAkkaService(final MasterMonitor masterMonitor, this.storageProvider = mantisStorageProvider; this.lifecycleEventPublisher = lifecycleEventPublisher; this.leadershipManager = leadershipManager; - this.healthChecks = Objects.requireNonNullElseGet(healthChecks, List::of); + this.healthCheckExtensions = Objects.requireNonNullElseGet(healthCheckExtensions, List::of); this.system = ActorSystem.create("MasterApiActorSystem"); this.materializer = Materializer.createMaterializer(system); this.mantisMasterRoute = configureApiRoutes(this.system); @@ -203,7 +203,7 @@ public MasterApiAkkaService(final MasterMonitor masterMonitor, private MantisMasterRoute configureApiRoutes(final ActorSystem actorSystem) { // Setup API routes - final JobClusterRouteHandler jobClusterRouteHandler = new JobClusterRouteHandlerAkkaImpl(jobClustersManagerActor, this.healthChecks); + final JobClusterRouteHandler jobClusterRouteHandler = new JobClusterRouteHandlerAkkaImpl(jobClustersManagerActor, this.healthCheckExtensions); final JobRouteHandler jobRouteHandler = new JobRouteHandlerAkkaImpl(jobClustersManagerActor); final MasterDescriptionRoute masterDescriptionRoute = new MasterDescriptionRoute(masterDescription); diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java index 3c2348665..081bd0768 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java @@ -23,7 +23,7 @@ import io.mantisrx.common.metrics.Metrics; import io.mantisrx.common.metrics.MetricsRegistry; import io.mantisrx.master.JobClustersManagerActor.UpdateSchedulingInfo; -import io.mantisrx.master.jobcluster.HealthCheck; +import io.mantisrx.master.jobcluster.HealthCheckExtension; import io.mantisrx.master.jobcluster.proto.HealthCheckResponse; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.UpdateSchedulingInfoRequest; @@ -45,15 +45,15 @@ public class JobClusterRouteHandlerAkkaImpl implements JobClusterRouteHandler { private final ActorRef jobClustersManagerActor; private final Counter allJobClustersGET; private final Duration timeout; - private final List healthChecks; + private final List healthCheckExtensions; public JobClusterRouteHandlerAkkaImpl(ActorRef jobClusterManagerActor) { this(jobClusterManagerActor, List.of()); } - public JobClusterRouteHandlerAkkaImpl(ActorRef jobClusterManagerActor, List healthChecks) { + public JobClusterRouteHandlerAkkaImpl(ActorRef jobClusterManagerActor, List healthCheckExtensions) { this.jobClustersManagerActor = jobClusterManagerActor; - this.healthChecks = healthChecks; + this.healthCheckExtensions = healthCheckExtensions; long timeoutMs = Optional.ofNullable(ConfigurationProvider.getConfig().getMasterApiAskTimeoutMs()).orElse(1000L); this.timeout = Duration.ofMillis(timeoutMs); Metrics m = new Metrics.Builder() @@ -198,13 +198,13 @@ public CompletionStage healthCheck( return ask(jobClustersManagerActor, request, timeout) .thenApply(HealthCheckResponse.class::cast) .thenApply(actorResponse -> { - if (!actorResponse.isHealthy() || healthChecks.isEmpty()) { + if (!actorResponse.isHealthy() || healthCheckExtensions.isEmpty()) { return actorResponse; } - for (HealthCheck healthCheck : healthChecks) { - Map scopedContext = extractContext(context, healthCheck.contextId()); - HealthCheckResponse result = healthCheck.checkHealth(clusterName, jobIds, scopedContext); + for (HealthCheckExtension healthCheckExtension : healthCheckExtensions) { + Map scopedContext = extractContext(context, healthCheckExtension.contextId()); + HealthCheckResponse result = healthCheckExtension.checkHealth(clusterName, jobIds, scopedContext); if (!result.isHealthy()) { return result; } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java index 453d8e54c..178df9863 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java @@ -813,9 +813,7 @@ private Route healthCheckRoute(String clusterName) { return completeAsync( response, - resp -> complete( - resp.isHealthy() ? StatusCodes.OK : StatusCodes.SERVICE_UNAVAILABLE, - resp, Jackson.marshaller()), + resp -> complete(StatusCodes.OK, resp, Jackson.marshaller()), HttpRequestMetrics.Endpoints.JOB_CLUSTER_INSTANCE_HEALTHCHECK, HttpRequestMetrics.HttpVerb.GET ); diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/HealthCheck.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/HealthCheckExtension.java similarity index 98% rename from mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/HealthCheck.java rename to mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/HealthCheckExtension.java index 4cd6d8664..e6cdb2d7b 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/HealthCheck.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/HealthCheckExtension.java @@ -28,7 +28,7 @@ * The handler extracts query params prefixed with this ID (e.g., {@code alertSystem.names}) * and passes them to the check with the prefix stripped. */ -public interface HealthCheck { +public interface HealthCheckExtension { /** * Namespace prefix for context parameters belonging to this health check. diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/MasterMain.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/MasterMain.java index 085f496db..0c04d80cc 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/MasterMain.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/MasterMain.java @@ -36,7 +36,7 @@ import io.mantisrx.master.api.akka.MasterApiAkkaService; import io.mantisrx.master.events.AuditEventBrokerActor; import io.mantisrx.master.events.AuditEventSubscriber; -import io.mantisrx.master.jobcluster.HealthCheck; +import io.mantisrx.master.jobcluster.HealthCheckExtension; import java.util.List; import io.mantisrx.master.events.AuditEventSubscriberAkkaImpl; import io.mantisrx.master.events.AuditEventSubscriberLoggingImpl; @@ -105,7 +105,7 @@ public class MasterMain implements Service { private MasterConfiguration config; private ILeadershipManager leadershipManager; private MasterMonitor monitor; - private final List healthChecks; + private final List healthCheckExtensions; public MasterMain( ConfigurationFactory configFactory, @@ -118,8 +118,8 @@ public MasterMain( ConfigurationFactory configFactory, MantisPropertiesLoader dynamicPropertiesLoader, AuditEventSubscriber auditEventSubscriber, - List healthChecks) { - this.healthChecks = healthChecks == null ? List.of() : healthChecks; + List healthCheckExtensions) { + this.healthCheckExtensions = healthCheckExtensions == null ? List.of() : healthCheckExtensions; String test = "{\"jobId\":\"sine-function-1\",\"status\":{\"jobId\":\"sine-function-1\",\"stageNum\":1,\"workerIndex\":0,\"workerNumber\":2,\"type\":\"HEARTBEAT\",\"message\":\"heartbeat\",\"state\":\"Noop\",\"hostname\":null,\"timestamp\":1525813363585,\"reason\":\"Normal\",\"payloads\":[{\"type\":\"SubscriptionState\",\"data\":\"false\"},{\"type\":\"IncomingDataDrop\",\"data\":\"{\\\"onNextCount\\\":0,\\\"droppedCount\\\":0}\"}]}}"; Metrics metrics = new Metrics.Builder() @@ -229,7 +229,7 @@ public MasterMain( monitor.start(); mantisServices.addService(leaderFactory.createLeaderElector(config, leadershipManager)); mantisServices.addService(new MasterApiAkkaService(monitor, leadershipManager.getDescription(), jobClusterManagerActor, statusEventBrokerActor, - resourceClusters, resourceClustersHostActor, config.getApiPort(), storageProvider, lifecycleEventPublisher, leadershipManager, null, healthChecks)); + resourceClusters, resourceClustersHostActor, config.getApiPort(), storageProvider, lifecycleEventPublisher, leadershipManager, null, healthCheckExtensions)); if (leaderFactory instanceof LocalLeaderFactory && !config.isLocalMode()) { logger.error("local mode is [ {} ] and leader factory is {} this configuration is unsafe", config.isLocalMode(), leaderFactory.getClass().getSimpleName()); diff --git a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImplTest.java b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImplTest.java new file mode 100644 index 000000000..e867560c3 --- /dev/null +++ b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImplTest.java @@ -0,0 +1,264 @@ +/* + * Copyright 2026 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.master.api.akka.route.handlers; + +import static org.junit.Assert.*; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import akka.actor.AbstractActor; +import akka.actor.ActorRef; +import akka.actor.ActorSystem; +import akka.actor.Props; +import akka.testkit.javadsl.TestKit; +import com.netflix.mantis.master.scheduler.TestHelpers; +import io.mantisrx.master.JobClustersManagerActor; +import io.mantisrx.master.events.AuditEventSubscriberLoggingImpl; +import io.mantisrx.master.events.LifecycleEventPublisher; +import io.mantisrx.master.events.LifecycleEventPublisherImpl; +import io.mantisrx.master.events.StatusEventSubscriberLoggingImpl; +import io.mantisrx.master.events.WorkerEventSubscriberLoggingImpl; +import io.mantisrx.master.jobcluster.HealthCheckExtension; +import io.mantisrx.master.jobcluster.job.CostsCalculator; +import io.mantisrx.master.jobcluster.proto.HealthCheckResponse; +import io.mantisrx.master.jobcluster.proto.HealthCheckResponse.FailedWorker; +import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto; +import io.mantisrx.master.scheduler.FakeMantisScheduler; +import io.mantisrx.server.master.persistence.FileBasedPersistenceProvider; +import io.mantisrx.server.master.persistence.MantisJobStore; +import io.mantisrx.server.master.scheduler.MantisScheduler; +import io.mantisrx.server.master.scheduler.MantisSchedulerFactory; +import java.io.File; +import java.nio.file.Files; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.commons.io.FileUtils; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +public class JobClusterRouteHandlerAkkaImplTest { + + private static ActorSystem system; + private static ActorRef jobClustersManagerActor; + private static File stateDirectory; + + @BeforeClass + public static void setup() throws Exception { + TestHelpers.setupMasterConfig(); + system = ActorSystem.create("JobClusterRouteHandlerTest"); + stateDirectory = Files.createTempDirectory("test-handler").toFile(); + + LifecycleEventPublisher lifecycleEventPublisher = new LifecycleEventPublisherImpl( + new AuditEventSubscriberLoggingImpl(), + new StatusEventSubscriberLoggingImpl(), + new WorkerEventSubscriberLoggingImpl()); + + jobClustersManagerActor = system.actorOf( + JobClustersManagerActor.props( + new MantisJobStore(new FileBasedPersistenceProvider(stateDirectory, true)), + lifecycleEventPublisher, + CostsCalculator.noop(), + 0), + "jobClustersManager"); + + MantisSchedulerFactory mantisSchedulerFactory = mock(MantisSchedulerFactory.class); + MantisScheduler fakeScheduler = new FakeMantisScheduler(jobClustersManagerActor); + when(mantisSchedulerFactory.forJob(any())).thenReturn(fakeScheduler); + jobClustersManagerActor.tell( + new JobClusterManagerProto.JobClustersManagerInitialize( + mantisSchedulerFactory, false), ActorRef.noSender()); + + Thread.sleep(1000); + } + + @AfterClass + public static void tearDown() throws Exception { + TestKit.shutdownActorSystem(system); + system = null; + FileUtils.deleteDirectory(stateDirectory); + } + + @Test + public void testHealthCheckNoExtensions() throws Exception { + JobClusterRouteHandlerAkkaImpl handler = new JobClusterRouteHandlerAkkaImpl(jobClustersManagerActor); + + HealthCheckResponse response = handler.healthCheck("nonExistentCluster", null, Map.of()) + .toCompletableFuture().get(5, TimeUnit.SECONDS); + + assertNotNull(response); + } + + @Test + public void testHealthCheckExtensionChainingStopsOnFailure() throws Exception { + HealthCheckExtension passingExtension = new HealthCheckExtension() { + @Override + public String contextId() { return "passing"; } + + @Override + public HealthCheckResponse checkHealth(String clusterName, List jobIds, Map context) { + return HealthCheckResponse.healthy(0); + } + }; + + HealthCheckExtension failingExtension = new HealthCheckExtension() { + @Override + public String contextId() { return "alertSystem"; } + + @Override + public HealthCheckResponse checkHealth(String clusterName, List jobIds, Map context) { + return HealthCheckResponse.unhealthyAlerts(0, List.of("alert1")); + } + }; + + HealthCheckExtension neverCalledExtension = new HealthCheckExtension() { + @Override + public String contextId() { return "neverCalled"; } + + @Override + public HealthCheckResponse checkHealth(String clusterName, List jobIds, Map context) { + throw new AssertionError("This extension should not be called"); + } + }; + + ActorRef mockActor = system.actorOf(Props.create(HealthyActorStub.class)); + JobClusterRouteHandlerAkkaImpl handler = new JobClusterRouteHandlerAkkaImpl( + mockActor, List.of(passingExtension, failingExtension, neverCalledExtension)); + + HealthCheckResponse response = handler.healthCheck("testCluster", null, Map.of()) + .toCompletableFuture().get(5, TimeUnit.SECONDS); + + assertFalse(response.isHealthy()); + assertTrue(response.getFailureReason() instanceof HealthCheckResponse.AlertFailure); + HealthCheckResponse.AlertFailure alertFailure = (HealthCheckResponse.AlertFailure) response.getFailureReason(); + assertEquals(List.of("alert1"), alertFailure.alerts()); + } + + @Test + public void testHealthCheckExtensionReceivesScopedContext() throws Exception { + final Map[] capturedContext = new Map[1]; + + HealthCheckExtension contextCapturingExtension = new HealthCheckExtension() { + @Override + public String contextId() { return "myCheck"; } + + @Override + public HealthCheckResponse checkHealth(String clusterName, List jobIds, Map context) { + capturedContext[0] = context; + return HealthCheckResponse.healthy(0); + } + }; + + ActorRef mockActor = system.actorOf(Props.create(HealthyActorStub.class)); + JobClusterRouteHandlerAkkaImpl handler = new JobClusterRouteHandlerAkkaImpl( + mockActor, List.of(contextCapturingExtension)); + + Map fullContext = Map.of( + "myCheck.names", "foo,bar", + "myCheck.threshold", "5", + "otherCheck.something", "ignored", + "job-ids", "job-1" + ); + + HealthCheckResponse response = handler.healthCheck("testCluster", null, fullContext) + .toCompletableFuture().get(5, TimeUnit.SECONDS); + + assertTrue(response.isHealthy()); + assertNotNull(capturedContext[0]); + assertEquals("foo,bar", capturedContext[0].get("names")); + assertEquals("5", capturedContext[0].get("threshold")); + assertNull(capturedContext[0].get("otherCheck.something")); + assertNull(capturedContext[0].get("job-ids")); + } + + @Test + public void testHealthCheckAllExtensionsPassReturnsHealthy() throws Exception { + HealthCheckExtension ext1 = new HealthCheckExtension() { + @Override + public String contextId() { return "ext1"; } + + @Override + public HealthCheckResponse checkHealth(String clusterName, List jobIds, Map context) { + return HealthCheckResponse.healthy(0); + } + }; + + HealthCheckExtension ext2 = new HealthCheckExtension() { + @Override + public String contextId() { return "ext2"; } + + @Override + public HealthCheckResponse checkHealth(String clusterName, List jobIds, Map context) { + return HealthCheckResponse.healthy(0); + } + }; + + ActorRef mockActor = system.actorOf(Props.create(HealthyActorStub.class)); + JobClusterRouteHandlerAkkaImpl handler = new JobClusterRouteHandlerAkkaImpl( + mockActor, List.of(ext1, ext2)); + + HealthCheckResponse response = handler.healthCheck("testCluster", null, Map.of()) + .toCompletableFuture().get(5, TimeUnit.SECONDS); + + assertTrue(response.isHealthy()); + } + + @Test + public void testHealthCheckActorUnhealthySkipsExtensions() throws Exception { + HealthCheckExtension shouldNotBeCalled = new HealthCheckExtension() { + @Override + public String contextId() { return "ext"; } + + @Override + public HealthCheckResponse checkHealth(String clusterName, List jobIds, Map context) { + throw new AssertionError("Extension should not be called when actor reports unhealthy"); + } + }; + + ActorRef mockActor = system.actorOf(Props.create(UnhealthyActorStub.class)); + JobClusterRouteHandlerAkkaImpl handler = new JobClusterRouteHandlerAkkaImpl( + mockActor, List.of(shouldNotBeCalled)); + + HealthCheckResponse response = handler.healthCheck("testCluster", null, Map.of()) + .toCompletableFuture().get(5, TimeUnit.SECONDS); + + assertFalse(response.isHealthy()); + } + + public static class HealthyActorStub extends AbstractActor { + @Override + public Receive createReceive() { + return receiveBuilder() + .match(JobClusterManagerProto.HealthCheckRequest.class, req -> + getSender().tell(HealthCheckResponse.healthy(req.requestId), getSelf())) + .build(); + } + } + + public static class UnhealthyActorStub extends AbstractActor { + @Override + public Receive createReceive() { + return receiveBuilder() + .match(JobClusterManagerProto.HealthCheckRequest.class, req -> + getSender().tell(HealthCheckResponse.unhealthyWorkers(req.requestId, + List.of(new FailedWorker(0, 1, "Accepted"))), getSelf())) + .build(); + } + } +} diff --git a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/v1/JobClustersRouteTest.java b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/v1/JobClustersRouteTest.java index 076439a5d..39bc5e5ce 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/v1/JobClustersRouteTest.java +++ b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/v1/JobClustersRouteTest.java @@ -18,6 +18,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -155,6 +156,8 @@ public void testIt() throws Exception { testJobClustersDelete(); testJobClustersPut(); testJobClusterInstanceGET(); + testJobClusterHealthCheck(); + testJobClusterHealthCheckNonExistent(); testNonExistentJobClusterInstanceGET(); testJobClusterInstancePOSTNotAllowed(); testJobClusterInstanceValidUpdate(); @@ -679,6 +682,29 @@ private void testJobClusterActionDisableDELETENotAllowed() throws InterruptedExc testDelete(getJobClusterDisableEp(JobClusterPayloads.CLUSTER_NAME), StatusCodes.METHOD_NOT_ALLOWED, null); } + private void testJobClusterHealthCheck() throws InterruptedException { + testGet( + getJobClusterInstanceEndpoint(JobClusterPayloads.CLUSTER_NAME) + "/healthcheck", + StatusCodes.OK, + response -> { + try { + ObjectMapper mapper = new ObjectMapper(); + JsonNode node = mapper.readTree(response); + assertTrue("Expected 'healthy' field in response: " + response, node.has("healthy")); + assertTrue(node.get("healthy").asBoolean()); + } catch (IOException e) { + fail("Failed to parse health check response: " + e.getMessage()); + } + }); + } + + private void testJobClusterHealthCheckNonExistent() throws InterruptedException { + testGet( + getJobClusterInstanceEndpoint("nonExistentCluster") + "/healthcheck", + StatusCodes.NOT_FOUND, + null); + } + private void testJobClusterDeleteWithoutRequiredParam() throws InterruptedException { testDelete( getJobClusterInstanceEndpoint(JobClusterPayloads.CLUSTER_NAME), 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 854b4dde3..428279bb0 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 @@ -91,6 +91,7 @@ import io.mantisrx.master.jobcluster.job.worker.WorkerState; import io.mantisrx.master.jobcluster.job.worker.WorkerTerminate; import io.mantisrx.master.jobcluster.proto.BaseResponse; +import io.mantisrx.master.jobcluster.proto.HealthCheckResponse; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.UpdateJobClusterArtifactRequest; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.UpdateJobClusterLabelsRequest; @@ -3377,6 +3378,133 @@ public void testExpireOldJobs() { //TODO } + // HEALTH CHECK TESTS ////////////////////////////////////////////////////////////////////////// + @Test + public void testHealthCheckAllWorkersStarted() { + try { + TestKit probe = new TestKit(system); + String clusterName = "testHealthCheckAllWorkersStarted"; + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); + MantisJobStore jobStoreMock = mock(MantisJobStore.class); + String jobId = clusterName + "-1"; + JobDefinition jobDefn = createJob(clusterName); + + final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName); + ActorRef jobClusterActor = system.actorOf(props(clusterName, jobStoreMock, jobDfn -> schedulerMock, eventPublisher, costsCalculator, 0)); + jobClusterActor.tell(new JobClusterProto.InitializeJobClusterRequest(fakeJobCluster, user, probe.getRef()), probe.getRef()); + JobClusterProto.InitializeJobClusterResponse createResp = probe.expectMsgClass(JobClusterProto.InitializeJobClusterResponse.class); + assertEquals(SUCCESS, createResp.responseCode); + + JobTestHelper.submitJobAndVerifySuccess(probe, clusterName, jobClusterActor, jobDefn, jobId); + JobTestHelper.getJobDetailsAndVerify(probe, jobClusterActor, jobId, SUCCESS, JobState.Accepted); + JobTestHelper.sendLaunchedInitiatedStartedEventsToWorker(probe, jobClusterActor, jobId, 1, new WorkerId(clusterName, jobId, 0, 1)); + JobTestHelper.getJobDetailsAndVerify(probe, jobClusterActor, jobId, SUCCESS, JobState.Launched); + + jobClusterActor.tell(new JobClusterManagerProto.HealthCheckRequest(clusterName, null), probe.getRef()); + HealthCheckResponse healthResp = probe.expectMsgClass(HealthCheckResponse.class); + + assertTrue(healthResp.isHealthy()); + assertEquals(SUCCESS, healthResp.responseCode); + assertNull(healthResp.getFailureReason()); + + probe.getSystem().stop(jobClusterActor); + } catch (Exception e) { + e.printStackTrace(); + fail(); + } + } + + @Test + public void testHealthCheckWithUnstartedWorkers() { + try { + TestKit probe = new TestKit(system); + String clusterName = "testHealthCheckWithUnstartedWorkers"; + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); + MantisJobStore jobStoreMock = mock(MantisJobStore.class); + String jobId = clusterName + "-1"; + JobDefinition jobDefn = createJob(clusterName); + + final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName); + ActorRef jobClusterActor = system.actorOf(props(clusterName, jobStoreMock, jobDfn -> schedulerMock, eventPublisher, costsCalculator, 0)); + jobClusterActor.tell(new JobClusterProto.InitializeJobClusterRequest(fakeJobCluster, user, probe.getRef()), probe.getRef()); + JobClusterProto.InitializeJobClusterResponse createResp = probe.expectMsgClass(JobClusterProto.InitializeJobClusterResponse.class); + assertEquals(SUCCESS, createResp.responseCode); + + JobTestHelper.submitJobAndVerifySuccess(probe, clusterName, jobClusterActor, jobDefn, jobId); + JobTestHelper.getJobDetailsAndVerify(probe, jobClusterActor, jobId, SUCCESS, JobState.Accepted); + + jobClusterActor.tell(new JobClusterManagerProto.HealthCheckRequest(clusterName, null), probe.getRef()); + HealthCheckResponse healthResp = probe.expectMsgClass(HealthCheckResponse.class); + + assertFalse(healthResp.isHealthy()); + assertEquals(SERVER_ERROR, healthResp.responseCode); + assertNotNull(healthResp.getFailureReason()); + assertTrue(healthResp.getFailureReason() instanceof HealthCheckResponse.WorkerFailure); + + HealthCheckResponse.WorkerFailure workerFailure = + (HealthCheckResponse.WorkerFailure) healthResp.getFailureReason(); + assertFalse(workerFailure.failedWorkers().isEmpty()); + + probe.getSystem().stop(jobClusterActor); + } catch (Exception e) { + e.printStackTrace(); + fail(); + } + } + + @Test + public void testHealthCheckWithJobIdFilter() { + try { + TestKit probe = new TestKit(system); + String clusterName = "testHealthCheckWithJobIdFilter"; + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); + MantisJobStore jobStoreMock = mock(MantisJobStore.class); + String jobId = clusterName + "-1"; + JobDefinition jobDefn = createJob(clusterName); + + final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName); + ActorRef jobClusterActor = system.actorOf(props(clusterName, jobStoreMock, jobDfn -> schedulerMock, eventPublisher, costsCalculator, 0)); + jobClusterActor.tell(new JobClusterProto.InitializeJobClusterRequest(fakeJobCluster, user, probe.getRef()), probe.getRef()); + JobClusterProto.InitializeJobClusterResponse createResp = probe.expectMsgClass(JobClusterProto.InitializeJobClusterResponse.class); + assertEquals(SUCCESS, createResp.responseCode); + + JobTestHelper.submitJobAndVerifySuccess(probe, clusterName, jobClusterActor, jobDefn, jobId); + JobTestHelper.sendLaunchedInitiatedStartedEventsToWorker(probe, jobClusterActor, jobId, 1, new WorkerId(clusterName, jobId, 0, 1)); + + jobClusterActor.tell(new JobClusterManagerProto.HealthCheckRequest(clusterName, ImmutableList.of(jobId)), probe.getRef()); + HealthCheckResponse healthResp = probe.expectMsgClass(HealthCheckResponse.class); + + assertTrue(healthResp.isHealthy()); + assertEquals(SUCCESS, healthResp.responseCode); + + probe.getSystem().stop(jobClusterActor); + } catch (Exception e) { + e.printStackTrace(); + fail(); + } + } + + @Test + public void testHealthCheckNoActiveJobs() throws Exception { + TestKit probe = new TestKit(system); + String clusterName = "testHealthCheckNoActiveJobs"; + MantisSchedulerFactory schedulerMock = mock(MantisSchedulerFactory.class); + MantisJobStore jobStoreMock = mock(MantisJobStore.class); + + final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName); + ActorRef jobClusterActor = system.actorOf(props(clusterName, jobStoreMock, schedulerMock, eventPublisher, costsCalculator, 0)); + jobClusterActor.tell(new JobClusterProto.InitializeJobClusterRequest(fakeJobCluster, user, probe.getRef()), probe.getRef()); + JobClusterProto.InitializeJobClusterResponse createResp = probe.expectMsgClass(JobClusterProto.InitializeJobClusterResponse.class); + assertEquals(SUCCESS, createResp.responseCode); + + jobClusterActor.tell(new JobClusterManagerProto.HealthCheckRequest(clusterName, null), probe.getRef()); + HealthCheckResponse healthResp = probe.expectMsgClass(HealthCheckResponse.class); + + assertTrue(healthResp.isHealthy()); + assertEquals(SUCCESS, healthResp.responseCode); + + probe.getSystem().stop(jobClusterActor); + } } From 8aefc73ac5353a837ee99f5efcdd01f59e2b9d6c Mon Sep 17 00:00:00 2001 From: jlubin Date: Thu, 9 Apr 2026 19:36:50 -0400 Subject: [PATCH 05/15] No health check extension interface --- .../master/JobClustersManagerActor.java | 3 +- .../master/api/akka/MasterApiAkkaService.java | 38 +--- .../handlers/JobClusterRouteHandler.java | 5 +- .../JobClusterRouteHandlerAkkaImpl.java | 49 +---- .../api/akka/route/v1/JobClustersRoute.java | 8 +- .../jobcluster/HealthCheckExtension.java | 53 ------ .../master/jobcluster/JobClusterActor.java | 9 +- .../jobcluster/proto/HealthCheckResponse.java | 79 -------- .../jobcluster/proto/JobClusterProto.java | 49 +++++ .../io/mantisrx/server/master/MasterMain.java | 17 +- .../JobClusterRouteHandlerAkkaImplTest.java | 169 +----------------- .../master/jobcluster/JobClusterAkkaTest.java | 27 ++- 12 files changed, 84 insertions(+), 422 deletions(-) delete mode 100644 mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/HealthCheckExtension.java delete mode 100644 mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/HealthCheckResponse.java 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 9abe41266..e2c72f70a 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 @@ -94,7 +94,6 @@ import io.mantisrx.master.jobcluster.job.JobHelper; import io.mantisrx.master.jobcluster.job.JobState; import io.mantisrx.master.jobcluster.proto.BaseResponse; -import io.mantisrx.master.jobcluster.proto.HealthCheckResponse; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.GetJobDetailsRequest; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.ResubmitWorkerRequest; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.UpdateJobClusterArtifactRequest; @@ -676,7 +675,7 @@ public void onHealthCheck(HealthCheckRequest request) { jobClusterInfo.get().jobClusterActor.forward(request, getContext()); } else { ActorRef sender = getSender(); - sender.tell(new HealthCheckResponse(request.requestId, CLIENT_ERROR_NOT_FOUND, "No such Job cluster " + request.getClusterName(), false, null), getSelf()); + sender.tell(new JobClusterProto.HealthCheckResponse(request.requestId, CLIENT_ERROR_NOT_FOUND, "No such Job cluster " + request.getClusterName(), false, null), getSelf()); } } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/MasterApiAkkaService.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/MasterApiAkkaService.java index b87199539..d007e1a2c 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/MasterApiAkkaService.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/MasterApiAkkaService.java @@ -64,10 +64,7 @@ import io.mantisrx.server.core.master.MasterMonitor; import io.mantisrx.server.master.LeaderRedirectionFilter; import io.mantisrx.server.master.persistence.IMantisPersistenceProvider; -import io.mantisrx.master.jobcluster.HealthCheckExtension; import io.mantisrx.server.master.resourcecluster.ResourceClusters; -import java.util.List; -import java.util.Objects; import java.util.concurrent.CompletionStage; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; @@ -97,8 +94,6 @@ public class MasterApiAkkaService extends BaseService { private final ExecutorService executorService; private final CountDownLatch serviceLatch = new CountDownLatch(1); private final HttpsConnectionContext httpsConnectionContext; - private final List healthCheckExtensions; - public MasterApiAkkaService(final MasterMonitor masterMonitor, final MasterDescription masterDescription, final ActorRef jobClustersManagerActor, @@ -121,8 +116,7 @@ public MasterApiAkkaService(final MasterMonitor masterMonitor, mantisStorageProvider, lifecycleEventPublisher, leadershipManager, - null, - List.of() + null ); } public MasterApiAkkaService(final MasterMonitor masterMonitor, @@ -136,33 +130,6 @@ public MasterApiAkkaService(final MasterMonitor masterMonitor, final LifecycleEventPublisher lifecycleEventPublisher, final ILeadershipManager leadershipManager, final HttpsConnectionContext httpsConnectionContext) { - this( - masterMonitor, - masterDescription, - jobClustersManagerActor, - statusEventBrokerActor, - resourceClusters, - resourceClustersHostManagerActor, - serverPort, - mantisStorageProvider, - lifecycleEventPublisher, - leadershipManager, - httpsConnectionContext, - List.of() - ); - } - public MasterApiAkkaService(final MasterMonitor masterMonitor, - final MasterDescription masterDescription, - final ActorRef jobClustersManagerActor, - final ActorRef statusEventBrokerActor, - final ResourceClusters resourceClusters, - final ActorRef resourceClustersHostManagerActor, - final int serverPort, - final IMantisPersistenceProvider mantisStorageProvider, - final LifecycleEventPublisher lifecycleEventPublisher, - final ILeadershipManager leadershipManager, - final HttpsConnectionContext httpsConnectionContext, - final List healthCheckExtensions) { super(true); Preconditions.checkNotNull(masterMonitor, "MasterMonitor"); Preconditions.checkNotNull(masterDescription, "masterDescription"); @@ -181,7 +148,6 @@ public MasterApiAkkaService(final MasterMonitor masterMonitor, this.storageProvider = mantisStorageProvider; this.lifecycleEventPublisher = lifecycleEventPublisher; this.leadershipManager = leadershipManager; - this.healthCheckExtensions = Objects.requireNonNullElseGet(healthCheckExtensions, List::of); this.system = ActorSystem.create("MasterApiActorSystem"); this.materializer = Materializer.createMaterializer(system); this.mantisMasterRoute = configureApiRoutes(this.system); @@ -203,7 +169,7 @@ public MasterApiAkkaService(final MasterMonitor masterMonitor, private MantisMasterRoute configureApiRoutes(final ActorSystem actorSystem) { // Setup API routes - final JobClusterRouteHandler jobClusterRouteHandler = new JobClusterRouteHandlerAkkaImpl(jobClustersManagerActor, this.healthCheckExtensions); + final JobClusterRouteHandler jobClusterRouteHandler = new JobClusterRouteHandlerAkkaImpl(jobClustersManagerActor); final JobRouteHandler jobRouteHandler = new JobRouteHandlerAkkaImpl(jobClustersManagerActor); final MasterDescriptionRoute masterDescriptionRoute = new MasterDescriptionRoute(masterDescription); diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandler.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandler.java index c6285e45e..64637092b 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandler.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandler.java @@ -16,13 +16,12 @@ package io.mantisrx.master.api.akka.route.handlers; -import io.mantisrx.master.jobcluster.proto.HealthCheckResponse; +import io.mantisrx.master.jobcluster.proto.JobClusterProto; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.UpdateSchedulingInfoRequest; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.UpdateSchedulingInfoResponse; import io.mantisrx.master.jobcluster.proto.JobClusterScalerRuleProto; import java.util.List; -import java.util.Map; import java.util.concurrent.CompletionStage; public interface JobClusterRouteHandler { @@ -62,5 +61,5 @@ CompletionStage updateSchedulingInfo( CompletionStage deleteScalerRule(JobClusterScalerRuleProto.DeleteScalerRuleRequest request); - CompletionStage healthCheck(String clusterName, List jobIds, Map context); + CompletionStage healthCheck(String clusterName, List jobIds); } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java index 081bd0768..2f7fff857 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java @@ -23,17 +23,14 @@ import io.mantisrx.common.metrics.Metrics; import io.mantisrx.common.metrics.MetricsRegistry; import io.mantisrx.master.JobClustersManagerActor.UpdateSchedulingInfo; -import io.mantisrx.master.jobcluster.HealthCheckExtension; -import io.mantisrx.master.jobcluster.proto.HealthCheckResponse; +import io.mantisrx.master.jobcluster.proto.JobClusterProto; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.UpdateSchedulingInfoRequest; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.UpdateSchedulingInfoResponse; import io.mantisrx.master.jobcluster.proto.JobClusterScalerRuleProto; import io.mantisrx.server.master.config.ConfigurationProvider; import java.time.Duration; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletionStage; import org.slf4j.Logger; @@ -45,15 +42,8 @@ public class JobClusterRouteHandlerAkkaImpl implements JobClusterRouteHandler { private final ActorRef jobClustersManagerActor; private final Counter allJobClustersGET; private final Duration timeout; - private final List healthCheckExtensions; - public JobClusterRouteHandlerAkkaImpl(ActorRef jobClusterManagerActor) { - this(jobClusterManagerActor, List.of()); - } - - public JobClusterRouteHandlerAkkaImpl(ActorRef jobClusterManagerActor, List healthCheckExtensions) { this.jobClustersManagerActor = jobClusterManagerActor; - this.healthCheckExtensions = healthCheckExtensions; long timeoutMs = Optional.ofNullable(ConfigurationProvider.getConfig().getMasterApiAskTimeoutMs()).orElse(1000L); this.timeout = Duration.ofMillis(timeoutMs); Metrics m = new Metrics.Builder() @@ -190,43 +180,12 @@ public CompletionStage } @Override - public CompletionStage healthCheck( - String clusterName, List jobIds, Map context) { + public CompletionStage healthCheck( + String clusterName, List jobIds) { JobClusterManagerProto.HealthCheckRequest request = new JobClusterManagerProto.HealthCheckRequest(clusterName, jobIds); return ask(jobClustersManagerActor, request, timeout) - .thenApply(HealthCheckResponse.class::cast) - .thenApply(actorResponse -> { - if (!actorResponse.isHealthy() || healthCheckExtensions.isEmpty()) { - return actorResponse; - } - - for (HealthCheckExtension healthCheckExtension : healthCheckExtensions) { - Map scopedContext = extractContext(context, healthCheckExtension.contextId()); - HealthCheckResponse result = healthCheckExtension.checkHealth(clusterName, jobIds, scopedContext); - if (!result.isHealthy()) { - return result; - } - } - - return actorResponse; - }); - } - - private static Map extractContext(Map context, String contextId) { - if (contextId == null || contextId.isEmpty()) { - return context; - } - - String prefix = contextId + "."; - Map scoped = new HashMap<>(); - for (Map.Entry entry : context.entrySet()) { - if (entry.getKey().startsWith(prefix)) { - scoped.put(entry.getKey().substring(prefix.length()), entry.getValue()); - } - } - - return scoped; + .thenApply(JobClusterProto.HealthCheckResponse.class::cast); } } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java index 178df9863..fe7f125aa 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java @@ -56,15 +56,14 @@ import io.mantisrx.master.api.akka.route.handlers.JobClusterRouteHandler; import io.mantisrx.master.api.akka.route.proto.JobClusterProtoAdapter; import io.mantisrx.master.jobcluster.proto.BaseResponse; -import io.mantisrx.master.jobcluster.proto.HealthCheckResponse; +import io.mantisrx.master.jobcluster.proto.JobClusterProto; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto; import io.mantisrx.master.jobcluster.proto.JobClusterScalerRuleProto; import io.mantisrx.runtime.NamedJobDefinition; import io.mantisrx.server.master.config.ConfigurationProvider; import io.mantisrx.server.master.config.MasterConfiguration; -import io.mantisrx.shaded.com.google.common.base.Strings; import java.util.Arrays; -import java.util.HashMap; + import java.util.List; import java.util.Map; import java.util.Optional; @@ -808,8 +807,7 @@ private Route healthCheckRoute(String clusterName) { ? Arrays.asList(jobIdsParam.split(",")) : null; - Map context = new HashMap<>(params); - CompletionStage response = jobClusterRouteHandler.healthCheck(clusterName, jobIds, context); + CompletionStage response = jobClusterRouteHandler.healthCheck(clusterName, jobIds); return completeAsync( response, diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/HealthCheckExtension.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/HealthCheckExtension.java deleted file mode 100644 index e6cdb2d7b..000000000 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/HealthCheckExtension.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Copyright 2026 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.master.jobcluster; - -import io.mantisrx.master.jobcluster.proto.HealthCheckResponse; -import java.util.List; -import java.util.Map; - -/** - * Extension point for job cluster health checks. Implementations are composed into an ordered - * list and executed sequentially — the chain stops at the first failure. - * - *

Each implementation declares a {@link #contextId()} that acts as a namespace prefix. - * The handler extracts query params prefixed with this ID (e.g., {@code alertSystem.names}) - * and passes them to the check with the prefix stripped. - */ -public interface HealthCheckExtension { - - /** - * Namespace prefix for context parameters belonging to this health check. - * For example, a check with contextId "alertSystem" receives query params like - * {@code alertSystem.names} as {@code names} in its context map. - * - * @return the context namespace. - */ - String contextId(); - - /** - * Run a health check against the given job cluster. - * - * @param clusterName the job cluster name. - * @param jobIds specific job IDs to check. Used to overwrite the default behavior of checking all active jobs. - * @param context parameters namespaced to this check (prefix already stripped). - * - * @return a healthy response, or an unhealthy response with the appropriate {@link - * HealthCheckResponse.FailureReason} - */ - HealthCheckResponse checkHealth(String clusterName, List jobIds, Map context); -} diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java index 86209ab0d..b8d5105fa 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java @@ -62,8 +62,7 @@ import io.mantisrx.master.jobcluster.job.MantisJobMetadataImpl; import io.mantisrx.master.jobcluster.job.MantisJobMetadataView; import io.mantisrx.master.jobcluster.job.worker.IMantisWorkerMetadata; -import io.mantisrx.master.jobcluster.proto.HealthCheckResponse; -import io.mantisrx.master.jobcluster.proto.HealthCheckResponse.FailedWorker; +import io.mantisrx.master.jobcluster.proto.JobClusterProto.FailedWorker; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.DeleteJobClusterResponse; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.HealthCheckRequest; @@ -2669,14 +2668,14 @@ public void onHealthCheck(final HealthCheckRequest request) { .subscribe( failedWorkers -> { if (failedWorkers.isEmpty()) { - sender.tell(HealthCheckResponse.healthy(request.requestId), self); + sender.tell(JobClusterProto.healthy(request.requestId), self); } else { - sender.tell(HealthCheckResponse.unhealthyWorkers(request.requestId, failedWorkers), self); + sender.tell(JobClusterProto.unhealthyWorkers(request.requestId, failedWorkers), self); } }, error -> { logger.error("Health check failed for cluster {}", name, error); - sender.tell(new HealthCheckResponse(request.requestId, SERVER_ERROR, + sender.tell(new JobClusterProto.HealthCheckResponse(request.requestId, SERVER_ERROR, "Health check failed: " + error.getMessage(), false, null), self); } ); diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/HealthCheckResponse.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/HealthCheckResponse.java deleted file mode 100644 index a0de73ddd..000000000 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/HealthCheckResponse.java +++ /dev/null @@ -1,79 +0,0 @@ -/* - * Copyright 2026 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.master.jobcluster.proto; - -import io.mantisrx.shaded.com.fasterxml.jackson.annotation.JsonCreator; -import io.mantisrx.shaded.com.fasterxml.jackson.annotation.JsonInclude; -import io.mantisrx.shaded.com.fasterxml.jackson.annotation.JsonProperty; -import io.mantisrx.shaded.com.fasterxml.jackson.annotation.JsonSubTypes; -import io.mantisrx.shaded.com.fasterxml.jackson.annotation.JsonTypeInfo; -import java.util.List; -import lombok.EqualsAndHashCode; -import lombok.Getter; -import lombok.ToString; - -@Getter -@EqualsAndHashCode(callSuper = true) -@ToString -@JsonInclude(JsonInclude.Include.ALWAYS) -public class HealthCheckResponse extends BaseResponse { - - private final boolean isHealthy; - private final FailureReason failureReason; - - @JsonCreator - public HealthCheckResponse( - @JsonProperty("requestId") long requestId, - @JsonProperty("responseCode") ResponseCode responseCode, - @JsonProperty("message") String message, - @JsonProperty("isHealthy") boolean isHealthy, - @JsonProperty("failureReason") FailureReason failureReason) { - super(requestId, responseCode, message); - this.isHealthy = isHealthy; - this.failureReason = failureReason; - } - - public static HealthCheckResponse healthy(long requestId) { - return new HealthCheckResponse( - requestId, ResponseCode.SUCCESS, "OK", true, null); - } - - public static HealthCheckResponse unhealthyWorkers(long requestId, List failedWorkers) { - return new HealthCheckResponse( - requestId, ResponseCode.SERVER_ERROR, "unhealthy workers", false, - new WorkerFailure(failedWorkers)); - } - - public static HealthCheckResponse unhealthyAlerts(long requestId, List alerts) { - return new HealthCheckResponse( - requestId, ResponseCode.SERVER_ERROR, "alerts firing", false, - new AlertFailure(alerts)); - } - - @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") - @JsonSubTypes({ - @JsonSubTypes.Type(value = WorkerFailure.class, name = "workerStatus"), - @JsonSubTypes.Type(value = AlertFailure.class, name = "alertsFiring") - }) - public sealed interface FailureReason permits WorkerFailure, AlertFailure {} - - public record WorkerFailure(List failedWorkers) implements FailureReason {} - - public record AlertFailure(List alerts) implements FailureReason {} - - public record FailedWorker(int workerIndex, int workerNumber, String state) {} -} diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterProto.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterProto.java index 594b1e528..6ad75ac67 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterProto.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterProto.java @@ -21,10 +21,13 @@ import com.netflix.spectator.impl.Preconditions; import io.mantisrx.master.jobcluster.job.IMantisJobMetadata; import io.mantisrx.master.jobcluster.job.JobState; +import io.mantisrx.master.jobcluster.proto.BaseResponse.ResponseCode; import io.mantisrx.server.core.JobCompletedReason; import io.mantisrx.server.master.domain.JobClusterDefinitionImpl; import io.mantisrx.server.master.domain.JobDefinition; import io.mantisrx.server.master.domain.JobId; +import io.mantisrx.shaded.com.fasterxml.jackson.annotation.JsonSubTypes; +import io.mantisrx.shaded.com.fasterxml.jackson.annotation.JsonTypeInfo; import io.mantisrx.shaded.com.google.common.collect.Lists; import java.time.Instant; import java.util.List; @@ -277,4 +280,50 @@ public TriggerCronRequest() { } + public static final class HealthCheckResponse extends BaseResponse { + + public final boolean isHealthy; + public final FailureReason failureReason; + + public HealthCheckResponse(long requestId, ResponseCode responseCode, String message, boolean isHealthy, FailureReason failureReason) { + super(requestId, responseCode, message); + this.isHealthy = isHealthy; + this.failureReason = failureReason; + } + + public HealthCheckResponse(ResponseCode responseCode, String message, boolean isHealthy, FailureReason failureReason) { + this(0, responseCode, message, isHealthy, failureReason); + } + } + + public static HealthCheckResponse healthy(long requestId) { + return new HealthCheckResponse( + requestId, ResponseCode.SUCCESS, "OK", true, null); + } + + public static HealthCheckResponse unhealthyWorkers(long requestId, List failedWorkers) { + return new HealthCheckResponse( + requestId, ResponseCode.SERVER_ERROR, "unhealthy workers", false, + new WorkerFailure(failedWorkers)); + } + + public static HealthCheckResponse unhealthyAlerts(long requestId, List alerts) { + return new HealthCheckResponse( + requestId, ResponseCode.SERVER_ERROR, "alerts firing", false, + new AlertFailure(alerts)); + } + + @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") + @JsonSubTypes({ + @JsonSubTypes.Type(value = WorkerFailure.class, name = "workerStatus"), + @JsonSubTypes.Type(value = AlertFailure.class, name = "alertsFiring") + }) + public sealed interface FailureReason permits WorkerFailure, AlertFailure {} + + public record WorkerFailure(List failedWorkers) implements FailureReason {} + + public record AlertFailure(List alerts) implements FailureReason {} + + public record FailedWorker(int workerIndex, int workerNumber, String state) {} + } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/MasterMain.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/MasterMain.java index 0c04d80cc..176f741ca 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/MasterMain.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/MasterMain.java @@ -36,8 +36,8 @@ import io.mantisrx.master.api.akka.MasterApiAkkaService; import io.mantisrx.master.events.AuditEventBrokerActor; import io.mantisrx.master.events.AuditEventSubscriber; -import io.mantisrx.master.jobcluster.HealthCheckExtension; -import java.util.List; + + import io.mantisrx.master.events.AuditEventSubscriberAkkaImpl; import io.mantisrx.master.events.AuditEventSubscriberLoggingImpl; import io.mantisrx.master.events.LifecycleEventPublisher; @@ -105,21 +105,10 @@ public class MasterMain implements Service { private MasterConfiguration config; private ILeadershipManager leadershipManager; private MasterMonitor monitor; - private final List healthCheckExtensions; - public MasterMain( ConfigurationFactory configFactory, MantisPropertiesLoader dynamicPropertiesLoader, AuditEventSubscriber auditEventSubscriber) { - this(configFactory, dynamicPropertiesLoader, auditEventSubscriber, List.of()); - } - - public MasterMain( - ConfigurationFactory configFactory, - MantisPropertiesLoader dynamicPropertiesLoader, - AuditEventSubscriber auditEventSubscriber, - List healthCheckExtensions) { - this.healthCheckExtensions = healthCheckExtensions == null ? List.of() : healthCheckExtensions; String test = "{\"jobId\":\"sine-function-1\",\"status\":{\"jobId\":\"sine-function-1\",\"stageNum\":1,\"workerIndex\":0,\"workerNumber\":2,\"type\":\"HEARTBEAT\",\"message\":\"heartbeat\",\"state\":\"Noop\",\"hostname\":null,\"timestamp\":1525813363585,\"reason\":\"Normal\",\"payloads\":[{\"type\":\"SubscriptionState\",\"data\":\"false\"},{\"type\":\"IncomingDataDrop\",\"data\":\"{\\\"onNextCount\\\":0,\\\"droppedCount\\\":0}\"}]}}"; Metrics metrics = new Metrics.Builder() @@ -229,7 +218,7 @@ public MasterMain( monitor.start(); mantisServices.addService(leaderFactory.createLeaderElector(config, leadershipManager)); mantisServices.addService(new MasterApiAkkaService(monitor, leadershipManager.getDescription(), jobClusterManagerActor, statusEventBrokerActor, - resourceClusters, resourceClustersHostActor, config.getApiPort(), storageProvider, lifecycleEventPublisher, leadershipManager, null, healthCheckExtensions)); + resourceClusters, resourceClustersHostActor, config.getApiPort(), storageProvider, lifecycleEventPublisher, leadershipManager)); if (leaderFactory instanceof LocalLeaderFactory && !config.isLocalMode()) { logger.error("local mode is [ {} ] and leader factory is {} this configuration is unsafe", config.isLocalMode(), leaderFactory.getClass().getSimpleName()); diff --git a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImplTest.java b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImplTest.java index e867560c3..6d467e81e 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImplTest.java +++ b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImplTest.java @@ -21,10 +21,8 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -import akka.actor.AbstractActor; import akka.actor.ActorRef; import akka.actor.ActorSystem; -import akka.actor.Props; import akka.testkit.javadsl.TestKit; import com.netflix.mantis.master.scheduler.TestHelpers; import io.mantisrx.master.JobClustersManagerActor; @@ -33,10 +31,8 @@ import io.mantisrx.master.events.LifecycleEventPublisherImpl; import io.mantisrx.master.events.StatusEventSubscriberLoggingImpl; import io.mantisrx.master.events.WorkerEventSubscriberLoggingImpl; -import io.mantisrx.master.jobcluster.HealthCheckExtension; import io.mantisrx.master.jobcluster.job.CostsCalculator; -import io.mantisrx.master.jobcluster.proto.HealthCheckResponse; -import io.mantisrx.master.jobcluster.proto.HealthCheckResponse.FailedWorker; +import io.mantisrx.master.jobcluster.proto.JobClusterProto; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto; import io.mantisrx.master.scheduler.FakeMantisScheduler; import io.mantisrx.server.master.persistence.FileBasedPersistenceProvider; @@ -45,8 +41,6 @@ import io.mantisrx.server.master.scheduler.MantisSchedulerFactory; import java.io.File; import java.nio.file.Files; -import java.util.List; -import java.util.Map; import java.util.concurrent.TimeUnit; import org.apache.commons.io.FileUtils; import org.junit.AfterClass; @@ -96,169 +90,12 @@ public static void tearDown() throws Exception { } @Test - public void testHealthCheckNoExtensions() throws Exception { + public void testHealthCheck() throws Exception { JobClusterRouteHandlerAkkaImpl handler = new JobClusterRouteHandlerAkkaImpl(jobClustersManagerActor); - HealthCheckResponse response = handler.healthCheck("nonExistentCluster", null, Map.of()) + JobClusterProto.HealthCheckResponse response = handler.healthCheck("nonExistentCluster", null) .toCompletableFuture().get(5, TimeUnit.SECONDS); assertNotNull(response); } - - @Test - public void testHealthCheckExtensionChainingStopsOnFailure() throws Exception { - HealthCheckExtension passingExtension = new HealthCheckExtension() { - @Override - public String contextId() { return "passing"; } - - @Override - public HealthCheckResponse checkHealth(String clusterName, List jobIds, Map context) { - return HealthCheckResponse.healthy(0); - } - }; - - HealthCheckExtension failingExtension = new HealthCheckExtension() { - @Override - public String contextId() { return "alertSystem"; } - - @Override - public HealthCheckResponse checkHealth(String clusterName, List jobIds, Map context) { - return HealthCheckResponse.unhealthyAlerts(0, List.of("alert1")); - } - }; - - HealthCheckExtension neverCalledExtension = new HealthCheckExtension() { - @Override - public String contextId() { return "neverCalled"; } - - @Override - public HealthCheckResponse checkHealth(String clusterName, List jobIds, Map context) { - throw new AssertionError("This extension should not be called"); - } - }; - - ActorRef mockActor = system.actorOf(Props.create(HealthyActorStub.class)); - JobClusterRouteHandlerAkkaImpl handler = new JobClusterRouteHandlerAkkaImpl( - mockActor, List.of(passingExtension, failingExtension, neverCalledExtension)); - - HealthCheckResponse response = handler.healthCheck("testCluster", null, Map.of()) - .toCompletableFuture().get(5, TimeUnit.SECONDS); - - assertFalse(response.isHealthy()); - assertTrue(response.getFailureReason() instanceof HealthCheckResponse.AlertFailure); - HealthCheckResponse.AlertFailure alertFailure = (HealthCheckResponse.AlertFailure) response.getFailureReason(); - assertEquals(List.of("alert1"), alertFailure.alerts()); - } - - @Test - public void testHealthCheckExtensionReceivesScopedContext() throws Exception { - final Map[] capturedContext = new Map[1]; - - HealthCheckExtension contextCapturingExtension = new HealthCheckExtension() { - @Override - public String contextId() { return "myCheck"; } - - @Override - public HealthCheckResponse checkHealth(String clusterName, List jobIds, Map context) { - capturedContext[0] = context; - return HealthCheckResponse.healthy(0); - } - }; - - ActorRef mockActor = system.actorOf(Props.create(HealthyActorStub.class)); - JobClusterRouteHandlerAkkaImpl handler = new JobClusterRouteHandlerAkkaImpl( - mockActor, List.of(contextCapturingExtension)); - - Map fullContext = Map.of( - "myCheck.names", "foo,bar", - "myCheck.threshold", "5", - "otherCheck.something", "ignored", - "job-ids", "job-1" - ); - - HealthCheckResponse response = handler.healthCheck("testCluster", null, fullContext) - .toCompletableFuture().get(5, TimeUnit.SECONDS); - - assertTrue(response.isHealthy()); - assertNotNull(capturedContext[0]); - assertEquals("foo,bar", capturedContext[0].get("names")); - assertEquals("5", capturedContext[0].get("threshold")); - assertNull(capturedContext[0].get("otherCheck.something")); - assertNull(capturedContext[0].get("job-ids")); - } - - @Test - public void testHealthCheckAllExtensionsPassReturnsHealthy() throws Exception { - HealthCheckExtension ext1 = new HealthCheckExtension() { - @Override - public String contextId() { return "ext1"; } - - @Override - public HealthCheckResponse checkHealth(String clusterName, List jobIds, Map context) { - return HealthCheckResponse.healthy(0); - } - }; - - HealthCheckExtension ext2 = new HealthCheckExtension() { - @Override - public String contextId() { return "ext2"; } - - @Override - public HealthCheckResponse checkHealth(String clusterName, List jobIds, Map context) { - return HealthCheckResponse.healthy(0); - } - }; - - ActorRef mockActor = system.actorOf(Props.create(HealthyActorStub.class)); - JobClusterRouteHandlerAkkaImpl handler = new JobClusterRouteHandlerAkkaImpl( - mockActor, List.of(ext1, ext2)); - - HealthCheckResponse response = handler.healthCheck("testCluster", null, Map.of()) - .toCompletableFuture().get(5, TimeUnit.SECONDS); - - assertTrue(response.isHealthy()); - } - - @Test - public void testHealthCheckActorUnhealthySkipsExtensions() throws Exception { - HealthCheckExtension shouldNotBeCalled = new HealthCheckExtension() { - @Override - public String contextId() { return "ext"; } - - @Override - public HealthCheckResponse checkHealth(String clusterName, List jobIds, Map context) { - throw new AssertionError("Extension should not be called when actor reports unhealthy"); - } - }; - - ActorRef mockActor = system.actorOf(Props.create(UnhealthyActorStub.class)); - JobClusterRouteHandlerAkkaImpl handler = new JobClusterRouteHandlerAkkaImpl( - mockActor, List.of(shouldNotBeCalled)); - - HealthCheckResponse response = handler.healthCheck("testCluster", null, Map.of()) - .toCompletableFuture().get(5, TimeUnit.SECONDS); - - assertFalse(response.isHealthy()); - } - - public static class HealthyActorStub extends AbstractActor { - @Override - public Receive createReceive() { - return receiveBuilder() - .match(JobClusterManagerProto.HealthCheckRequest.class, req -> - getSender().tell(HealthCheckResponse.healthy(req.requestId), getSelf())) - .build(); - } - } - - public static class UnhealthyActorStub extends AbstractActor { - @Override - public Receive createReceive() { - return receiveBuilder() - .match(JobClusterManagerProto.HealthCheckRequest.class, req -> - getSender().tell(HealthCheckResponse.unhealthyWorkers(req.requestId, - List.of(new FailedWorker(0, 1, "Accepted"))), getSelf())) - .build(); - } - } } 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 428279bb0..6cc8a54d1 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 @@ -91,7 +91,6 @@ import io.mantisrx.master.jobcluster.job.worker.WorkerState; import io.mantisrx.master.jobcluster.job.worker.WorkerTerminate; import io.mantisrx.master.jobcluster.proto.BaseResponse; -import io.mantisrx.master.jobcluster.proto.HealthCheckResponse; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.UpdateJobClusterArtifactRequest; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.UpdateJobClusterLabelsRequest; @@ -3402,11 +3401,11 @@ public void testHealthCheckAllWorkersStarted() { JobTestHelper.getJobDetailsAndVerify(probe, jobClusterActor, jobId, SUCCESS, JobState.Launched); jobClusterActor.tell(new JobClusterManagerProto.HealthCheckRequest(clusterName, null), probe.getRef()); - HealthCheckResponse healthResp = probe.expectMsgClass(HealthCheckResponse.class); + JobClusterProto.HealthCheckResponse healthResp = probe.expectMsgClass(JobClusterProto.HealthCheckResponse.class); - assertTrue(healthResp.isHealthy()); + assertTrue(healthResp.isHealthy); assertEquals(SUCCESS, healthResp.responseCode); - assertNull(healthResp.getFailureReason()); + assertNull(healthResp.failureReason); probe.getSystem().stop(jobClusterActor); } catch (Exception e) { @@ -3435,15 +3434,15 @@ public void testHealthCheckWithUnstartedWorkers() { JobTestHelper.getJobDetailsAndVerify(probe, jobClusterActor, jobId, SUCCESS, JobState.Accepted); jobClusterActor.tell(new JobClusterManagerProto.HealthCheckRequest(clusterName, null), probe.getRef()); - HealthCheckResponse healthResp = probe.expectMsgClass(HealthCheckResponse.class); + JobClusterProto.HealthCheckResponse healthResp = probe.expectMsgClass(JobClusterProto.HealthCheckResponse.class); - assertFalse(healthResp.isHealthy()); + assertFalse(healthResp.isHealthy); assertEquals(SERVER_ERROR, healthResp.responseCode); - assertNotNull(healthResp.getFailureReason()); - assertTrue(healthResp.getFailureReason() instanceof HealthCheckResponse.WorkerFailure); + assertNotNull(healthResp.failureReason); + assertTrue(healthResp.failureReason instanceof JobClusterProto.WorkerFailure); - HealthCheckResponse.WorkerFailure workerFailure = - (HealthCheckResponse.WorkerFailure) healthResp.getFailureReason(); + JobClusterProto.WorkerFailure workerFailure = + (JobClusterProto.WorkerFailure) healthResp.failureReason; assertFalse(workerFailure.failedWorkers().isEmpty()); probe.getSystem().stop(jobClusterActor); @@ -3473,9 +3472,9 @@ public void testHealthCheckWithJobIdFilter() { JobTestHelper.sendLaunchedInitiatedStartedEventsToWorker(probe, jobClusterActor, jobId, 1, new WorkerId(clusterName, jobId, 0, 1)); jobClusterActor.tell(new JobClusterManagerProto.HealthCheckRequest(clusterName, ImmutableList.of(jobId)), probe.getRef()); - HealthCheckResponse healthResp = probe.expectMsgClass(HealthCheckResponse.class); + JobClusterProto.HealthCheckResponse healthResp = probe.expectMsgClass(JobClusterProto.HealthCheckResponse.class); - assertTrue(healthResp.isHealthy()); + assertTrue(healthResp.isHealthy); assertEquals(SUCCESS, healthResp.responseCode); probe.getSystem().stop(jobClusterActor); @@ -3499,9 +3498,9 @@ public void testHealthCheckNoActiveJobs() throws Exception { assertEquals(SUCCESS, createResp.responseCode); jobClusterActor.tell(new JobClusterManagerProto.HealthCheckRequest(clusterName, null), probe.getRef()); - HealthCheckResponse healthResp = probe.expectMsgClass(HealthCheckResponse.class); + JobClusterProto.HealthCheckResponse healthResp = probe.expectMsgClass(JobClusterProto.HealthCheckResponse.class); - assertTrue(healthResp.isHealthy()); + assertTrue(healthResp.isHealthy); assertEquals(SUCCESS, healthResp.responseCode); probe.getSystem().stop(jobClusterActor); From 828ff61003f497cc71efbc514bfccb5fb965f955 Mon Sep 17 00:00:00 2001 From: jlubin Date: Thu, 9 Apr 2026 19:53:02 -0400 Subject: [PATCH 06/15] Tweaks --- .../master/api/akka/MasterApiAkkaService.java | 1 + .../handlers/JobClusterRouteHandlerAkkaImpl.java | 11 ++++------- .../api/akka/route/v1/JobClustersRoute.java | 1 - .../master/jobcluster/proto/JobClusterProto.java | 15 +-------------- .../io/mantisrx/server/master/MasterMain.java | 3 +-- 5 files changed, 7 insertions(+), 24 deletions(-) diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/MasterApiAkkaService.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/MasterApiAkkaService.java index d007e1a2c..9e42f8e94 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/MasterApiAkkaService.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/MasterApiAkkaService.java @@ -119,6 +119,7 @@ public MasterApiAkkaService(final MasterMonitor masterMonitor, null ); } + public MasterApiAkkaService(final MasterMonitor masterMonitor, final MasterDescription masterDescription, final ActorRef jobClustersManagerActor, diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java index 2f7fff857..b5bfc2fde 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java @@ -42,6 +42,7 @@ public class JobClusterRouteHandlerAkkaImpl implements JobClusterRouteHandler { private final ActorRef jobClustersManagerActor; private final Counter allJobClustersGET; private final Duration timeout; + public JobClusterRouteHandlerAkkaImpl(ActorRef jobClusterManagerActor) { this.jobClustersManagerActor = jobClusterManagerActor; long timeoutMs = Optional.ofNullable(ConfigurationProvider.getConfig().getMasterApiAskTimeoutMs()).orElse(1000L); @@ -180,12 +181,8 @@ public CompletionStage } @Override - public CompletionStage healthCheck( - String clusterName, List jobIds) { - JobClusterManagerProto.HealthCheckRequest request = - new JobClusterManagerProto.HealthCheckRequest(clusterName, jobIds); - - return ask(jobClustersManagerActor, request, timeout) - .thenApply(JobClusterProto.HealthCheckResponse.class::cast); + public CompletionStage healthCheck(String clusterName, List jobIds) { + JobClusterManagerProto.HealthCheckRequest request = new JobClusterManagerProto.HealthCheckRequest(clusterName, jobIds); + return ask(jobClustersManagerActor, request, timeout).thenApply(JobClusterProto.HealthCheckResponse.class::cast); } } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java index fe7f125aa..86c03772b 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java @@ -808,7 +808,6 @@ private Route healthCheckRoute(String clusterName) { : null; CompletionStage response = jobClusterRouteHandler.healthCheck(clusterName, jobIds); - return completeAsync( response, resp -> complete(StatusCodes.OK, resp, Jackson.marshaller()), diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterProto.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterProto.java index 6ad75ac67..1c066dead 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterProto.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterProto.java @@ -290,10 +290,6 @@ public HealthCheckResponse(long requestId, ResponseCode responseCode, String mes this.isHealthy = isHealthy; this.failureReason = failureReason; } - - public HealthCheckResponse(ResponseCode responseCode, String message, boolean isHealthy, FailureReason failureReason) { - this(0, responseCode, message, isHealthy, failureReason); - } } public static HealthCheckResponse healthy(long requestId) { @@ -307,23 +303,14 @@ public static HealthCheckResponse unhealthyWorkers(long requestId, List alerts) { - return new HealthCheckResponse( - requestId, ResponseCode.SERVER_ERROR, "alerts firing", false, - new AlertFailure(alerts)); - } - @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes({ @JsonSubTypes.Type(value = WorkerFailure.class, name = "workerStatus"), - @JsonSubTypes.Type(value = AlertFailure.class, name = "alertsFiring") }) - public sealed interface FailureReason permits WorkerFailure, AlertFailure {} + public sealed interface FailureReason permits WorkerFailure {} public record WorkerFailure(List failedWorkers) implements FailureReason {} - public record AlertFailure(List alerts) implements FailureReason {} - public record FailedWorker(int workerIndex, int workerNumber, String state) {} } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/MasterMain.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/MasterMain.java index 176f741ca..273172052 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/MasterMain.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/server/master/MasterMain.java @@ -36,8 +36,6 @@ import io.mantisrx.master.api.akka.MasterApiAkkaService; import io.mantisrx.master.events.AuditEventBrokerActor; import io.mantisrx.master.events.AuditEventSubscriber; - - import io.mantisrx.master.events.AuditEventSubscriberAkkaImpl; import io.mantisrx.master.events.AuditEventSubscriberLoggingImpl; import io.mantisrx.master.events.LifecycleEventPublisher; @@ -105,6 +103,7 @@ public class MasterMain implements Service { private MasterConfiguration config; private ILeadershipManager leadershipManager; private MasterMonitor monitor; + public MasterMain( ConfigurationFactory configFactory, MantisPropertiesLoader dynamicPropertiesLoader, From 2feb89540db1c3f2abfa9651c875c3596f123d0d Mon Sep 17 00:00:00 2001 From: jlubin Date: Fri, 10 Apr 2026 12:13:38 -0400 Subject: [PATCH 07/15] Refactor --- .../master/JobClustersManagerActor.java | 3 +- .../handlers/JobClusterRouteHandler.java | 4 +-- .../JobClusterRouteHandlerAkkaImpl.java | 6 ++-- .../api/akka/route/v1/JobClustersRoute.java | 4 +-- .../master/jobcluster/JobClusterActor.java | 20 ++++++++--- .../proto/JobClusterManagerProto.java | 23 +++++++++++++ .../jobcluster/proto/JobClusterProto.java | 34 ------------------- .../JobClusterRouteHandlerAkkaImplTest.java | 4 +-- .../akka/route/v1/JobClustersRouteTest.java | 4 +-- .../master/jobcluster/JobClusterAkkaTest.java | 18 ++++------ 10 files changed, 59 insertions(+), 61 deletions(-) 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 e2c72f70a..a7cf80443 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 @@ -33,6 +33,7 @@ import static io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.EnableJobClusterResponse; import static io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.GetJobClusterRequest; import static io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.HealthCheckRequest; +import static io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.HealthCheckResponse; import static io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.GetJobClusterResponse; import static io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.GetJobDetailsResponse; import static io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.GetJobSchedInfoRequest; @@ -675,7 +676,7 @@ public void onHealthCheck(HealthCheckRequest request) { jobClusterInfo.get().jobClusterActor.forward(request, getContext()); } else { ActorRef sender = getSender(); - sender.tell(new JobClusterProto.HealthCheckResponse(request.requestId, CLIENT_ERROR_NOT_FOUND, "No such Job cluster " + request.getClusterName(), false, null), getSelf()); + sender.tell(new HealthCheckResponse(request.requestId, CLIENT_ERROR_NOT_FOUND, "No such Job cluster " + request.getClusterName(), false, null), getSelf()); } } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandler.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandler.java index 64637092b..70ba99b3a 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandler.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandler.java @@ -16,7 +16,7 @@ package io.mantisrx.master.api.akka.route.handlers; -import io.mantisrx.master.jobcluster.proto.JobClusterProto; + import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.UpdateSchedulingInfoRequest; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.UpdateSchedulingInfoResponse; @@ -61,5 +61,5 @@ CompletionStage updateSchedulingInfo( CompletionStage deleteScalerRule(JobClusterScalerRuleProto.DeleteScalerRuleRequest request); - CompletionStage healthCheck(String clusterName, List jobIds); + CompletionStage healthCheck(String clusterName, List jobIds); } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java index b5bfc2fde..a9174fa0b 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImpl.java @@ -23,7 +23,7 @@ import io.mantisrx.common.metrics.Metrics; import io.mantisrx.common.metrics.MetricsRegistry; import io.mantisrx.master.JobClustersManagerActor.UpdateSchedulingInfo; -import io.mantisrx.master.jobcluster.proto.JobClusterProto; + import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.UpdateSchedulingInfoRequest; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.UpdateSchedulingInfoResponse; @@ -181,8 +181,8 @@ public CompletionStage } @Override - public CompletionStage healthCheck(String clusterName, List jobIds) { + public CompletionStage healthCheck(String clusterName, List jobIds) { JobClusterManagerProto.HealthCheckRequest request = new JobClusterManagerProto.HealthCheckRequest(clusterName, jobIds); - return ask(jobClustersManagerActor, request, timeout).thenApply(JobClusterProto.HealthCheckResponse.class::cast); + return ask(jobClustersManagerActor, request, timeout).thenApply(JobClusterManagerProto.HealthCheckResponse.class::cast); } } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java index 86c03772b..cf7c6cd69 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/route/v1/JobClustersRoute.java @@ -56,7 +56,7 @@ import io.mantisrx.master.api.akka.route.handlers.JobClusterRouteHandler; import io.mantisrx.master.api.akka.route.proto.JobClusterProtoAdapter; import io.mantisrx.master.jobcluster.proto.BaseResponse; -import io.mantisrx.master.jobcluster.proto.JobClusterProto; + import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto; import io.mantisrx.master.jobcluster.proto.JobClusterScalerRuleProto; import io.mantisrx.runtime.NamedJobDefinition; @@ -807,7 +807,7 @@ private Route healthCheckRoute(String clusterName) { ? Arrays.asList(jobIdsParam.split(",")) : null; - CompletionStage response = jobClusterRouteHandler.healthCheck(clusterName, jobIds); + CompletionStage response = jobClusterRouteHandler.healthCheck(clusterName, jobIds); return completeAsync( response, resp -> complete(StatusCodes.OK, resp, Jackson.marshaller()), diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java index b8d5105fa..31a88e324 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java @@ -62,7 +62,8 @@ import io.mantisrx.master.jobcluster.job.MantisJobMetadataImpl; import io.mantisrx.master.jobcluster.job.MantisJobMetadataView; import io.mantisrx.master.jobcluster.job.worker.IMantisWorkerMetadata; -import io.mantisrx.master.jobcluster.proto.JobClusterProto.FailedWorker; +import io.mantisrx.master.jobcluster.proto.BaseResponse.ResponseCode; +import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.FailedWorker; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.DeleteJobClusterResponse; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.HealthCheckRequest; @@ -113,8 +114,10 @@ import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.UpdateJobClusterWorkerMigrationStrategyResponse; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.UpdateSchedulingInfoResponse; import io.mantisrx.master.jobcluster.proto.JobClusterProto; +import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.HealthCheckResponse; import io.mantisrx.master.jobcluster.proto.JobClusterProto.JobStartedEvent; import io.mantisrx.master.jobcluster.proto.JobClusterProto.KillJobRequest; +import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.WorkerFailure; import io.mantisrx.master.jobcluster.proto.JobClusterScalerRuleProto; import io.mantisrx.master.jobcluster.proto.JobProto; import io.mantisrx.master.jobcluster.scaler.IJobClusterScalerRuleData; @@ -2668,14 +2671,23 @@ public void onHealthCheck(final HealthCheckRequest request) { .subscribe( failedWorkers -> { if (failedWorkers.isEmpty()) { - sender.tell(JobClusterProto.healthy(request.requestId), self); + HealthCheckResponse response + = new HealthCheckResponse(request.requestId, ResponseCode.SUCCESS, "OK", true, null); + sender.tell(response, self); } else { - sender.tell(JobClusterProto.unhealthyWorkers(request.requestId, failedWorkers), self); + HealthCheckResponse response + = new HealthCheckResponse( + request.requestId, + ResponseCode.SERVER_ERROR, + "unhealthy workers", + false, + new WorkerFailure(failedWorkers)); + sender.tell(response, self); } }, error -> { logger.error("Health check failed for cluster {}", name, error); - sender.tell(new JobClusterProto.HealthCheckResponse(request.requestId, SERVER_ERROR, + sender.tell(new JobClusterManagerProto.HealthCheckResponse(request.requestId, SERVER_ERROR, "Health check failed: " + error.getMessage(), false, null), self); } ); diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java index ff238ebdd..91293b645 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java @@ -2283,4 +2283,27 @@ public String toString() { '}'; } } + + public static final class HealthCheckResponse extends BaseResponse { + + public final boolean isHealthy; + public final WorkerFailure workerFailure; + + @JsonCreator + @JsonIgnoreProperties(ignoreUnknown = true) + public HealthCheckResponse( + @JsonProperty("requestId") long requestId, + @JsonProperty("responseCode") ResponseCode responseCode, + @JsonProperty("message") String message, + @JsonProperty("isHealthy") boolean isHealthy, + @JsonProperty("workerFailure") WorkerFailure workerFailure) { + super(requestId, responseCode, message); + this.isHealthy = isHealthy; + this.workerFailure = workerFailure; + } + } + + public record WorkerFailure(List failedWorkers) {} + + public record FailedWorker(int workerIndex, int workerNumber, String state) {} } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterProto.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterProto.java index 1c066dead..da685b1b5 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterProto.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterProto.java @@ -279,38 +279,4 @@ public TriggerCronRequest() { } } - - public static final class HealthCheckResponse extends BaseResponse { - - public final boolean isHealthy; - public final FailureReason failureReason; - - public HealthCheckResponse(long requestId, ResponseCode responseCode, String message, boolean isHealthy, FailureReason failureReason) { - super(requestId, responseCode, message); - this.isHealthy = isHealthy; - this.failureReason = failureReason; - } - } - - public static HealthCheckResponse healthy(long requestId) { - return new HealthCheckResponse( - requestId, ResponseCode.SUCCESS, "OK", true, null); - } - - public static HealthCheckResponse unhealthyWorkers(long requestId, List failedWorkers) { - return new HealthCheckResponse( - requestId, ResponseCode.SERVER_ERROR, "unhealthy workers", false, - new WorkerFailure(failedWorkers)); - } - - @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") - @JsonSubTypes({ - @JsonSubTypes.Type(value = WorkerFailure.class, name = "workerStatus"), - }) - public sealed interface FailureReason permits WorkerFailure {} - - public record WorkerFailure(List failedWorkers) implements FailureReason {} - - public record FailedWorker(int workerIndex, int workerNumber, String state) {} - } diff --git a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImplTest.java b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImplTest.java index 6d467e81e..450a1b857 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImplTest.java +++ b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImplTest.java @@ -32,7 +32,7 @@ import io.mantisrx.master.events.StatusEventSubscriberLoggingImpl; import io.mantisrx.master.events.WorkerEventSubscriberLoggingImpl; import io.mantisrx.master.jobcluster.job.CostsCalculator; -import io.mantisrx.master.jobcluster.proto.JobClusterProto; + import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto; import io.mantisrx.master.scheduler.FakeMantisScheduler; import io.mantisrx.server.master.persistence.FileBasedPersistenceProvider; @@ -93,7 +93,7 @@ public static void tearDown() throws Exception { public void testHealthCheck() throws Exception { JobClusterRouteHandlerAkkaImpl handler = new JobClusterRouteHandlerAkkaImpl(jobClustersManagerActor); - JobClusterProto.HealthCheckResponse response = handler.healthCheck("nonExistentCluster", null) + JobClusterManagerProto.HealthCheckResponse response = handler.healthCheck("nonExistentCluster", null) .toCompletableFuture().get(5, TimeUnit.SECONDS); assertNotNull(response); diff --git a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/v1/JobClustersRouteTest.java b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/v1/JobClustersRouteTest.java index 39bc5e5ce..3db61f412 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/v1/JobClustersRouteTest.java +++ b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/v1/JobClustersRouteTest.java @@ -690,8 +690,8 @@ private void testJobClusterHealthCheck() throws InterruptedException { try { ObjectMapper mapper = new ObjectMapper(); JsonNode node = mapper.readTree(response); - assertTrue("Expected 'healthy' field in response: " + response, node.has("healthy")); - assertTrue(node.get("healthy").asBoolean()); + assertTrue("Expected 'isHealthy' field in response: " + response, node.has("isHealthy")); + assertTrue(node.get("isHealthy").asBoolean()); } catch (IOException e) { fail("Failed to parse health check response: " + e.getMessage()); } 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 6cc8a54d1..15328c717 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 @@ -3401,11 +3401,11 @@ public void testHealthCheckAllWorkersStarted() { JobTestHelper.getJobDetailsAndVerify(probe, jobClusterActor, jobId, SUCCESS, JobState.Launched); jobClusterActor.tell(new JobClusterManagerProto.HealthCheckRequest(clusterName, null), probe.getRef()); - JobClusterProto.HealthCheckResponse healthResp = probe.expectMsgClass(JobClusterProto.HealthCheckResponse.class); + JobClusterManagerProto.HealthCheckResponse healthResp = probe.expectMsgClass(JobClusterManagerProto.HealthCheckResponse.class); assertTrue(healthResp.isHealthy); assertEquals(SUCCESS, healthResp.responseCode); - assertNull(healthResp.failureReason); + assertNull(healthResp.workerFailure); probe.getSystem().stop(jobClusterActor); } catch (Exception e) { @@ -3434,16 +3434,12 @@ public void testHealthCheckWithUnstartedWorkers() { JobTestHelper.getJobDetailsAndVerify(probe, jobClusterActor, jobId, SUCCESS, JobState.Accepted); jobClusterActor.tell(new JobClusterManagerProto.HealthCheckRequest(clusterName, null), probe.getRef()); - JobClusterProto.HealthCheckResponse healthResp = probe.expectMsgClass(JobClusterProto.HealthCheckResponse.class); + JobClusterManagerProto.HealthCheckResponse healthResp = probe.expectMsgClass(JobClusterManagerProto.HealthCheckResponse.class); assertFalse(healthResp.isHealthy); assertEquals(SERVER_ERROR, healthResp.responseCode); - assertNotNull(healthResp.failureReason); - assertTrue(healthResp.failureReason instanceof JobClusterProto.WorkerFailure); - - JobClusterProto.WorkerFailure workerFailure = - (JobClusterProto.WorkerFailure) healthResp.failureReason; - assertFalse(workerFailure.failedWorkers().isEmpty()); + assertNotNull(healthResp.workerFailure); + assertFalse(healthResp.workerFailure.failedWorkers().isEmpty()); probe.getSystem().stop(jobClusterActor); } catch (Exception e) { @@ -3472,7 +3468,7 @@ public void testHealthCheckWithJobIdFilter() { JobTestHelper.sendLaunchedInitiatedStartedEventsToWorker(probe, jobClusterActor, jobId, 1, new WorkerId(clusterName, jobId, 0, 1)); jobClusterActor.tell(new JobClusterManagerProto.HealthCheckRequest(clusterName, ImmutableList.of(jobId)), probe.getRef()); - JobClusterProto.HealthCheckResponse healthResp = probe.expectMsgClass(JobClusterProto.HealthCheckResponse.class); + JobClusterManagerProto.HealthCheckResponse healthResp = probe.expectMsgClass(JobClusterManagerProto.HealthCheckResponse.class); assertTrue(healthResp.isHealthy); assertEquals(SUCCESS, healthResp.responseCode); @@ -3498,7 +3494,7 @@ public void testHealthCheckNoActiveJobs() throws Exception { assertEquals(SUCCESS, createResp.responseCode); jobClusterActor.tell(new JobClusterManagerProto.HealthCheckRequest(clusterName, null), probe.getRef()); - JobClusterProto.HealthCheckResponse healthResp = probe.expectMsgClass(JobClusterProto.HealthCheckResponse.class); + JobClusterManagerProto.HealthCheckResponse healthResp = probe.expectMsgClass(JobClusterManagerProto.HealthCheckResponse.class); assertTrue(healthResp.isHealthy); assertEquals(SUCCESS, healthResp.responseCode); From 2d1b5dae7d6974a975fb5ce74aa14da144631d94 Mon Sep 17 00:00:00 2001 From: jlubin Date: Fri, 10 Apr 2026 12:53:35 -0400 Subject: [PATCH 08/15] More tweaks --- .../master/api/akka/MasterApiAkkaService.java | 2 +- .../jobcluster/proto/JobClusterManagerProto.java | 13 ++----------- 2 files changed, 3 insertions(+), 12 deletions(-) diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/MasterApiAkkaService.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/MasterApiAkkaService.java index 9e42f8e94..a42136f0d 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/MasterApiAkkaService.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/api/akka/MasterApiAkkaService.java @@ -94,6 +94,7 @@ public class MasterApiAkkaService extends BaseService { private final ExecutorService executorService; private final CountDownLatch serviceLatch = new CountDownLatch(1); private final HttpsConnectionContext httpsConnectionContext; + public MasterApiAkkaService(final MasterMonitor masterMonitor, final MasterDescription masterDescription, final ActorRef jobClustersManagerActor, @@ -119,7 +120,6 @@ public MasterApiAkkaService(final MasterMonitor masterMonitor, null ); } - public MasterApiAkkaService(final MasterMonitor masterMonitor, final MasterDescription masterDescription, final ActorRef jobClustersManagerActor, diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java index 91293b645..f4604f93e 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java @@ -2254,8 +2254,8 @@ public String toString() { } public static final class HealthCheckRequest extends BaseRequest { - private final String clusterName; - private final List jobIds; + public final String clusterName; + public final List jobIds; public HealthCheckRequest(final String clusterName, final List jobIds) { super(); @@ -2266,14 +2266,6 @@ public HealthCheckRequest(final String clusterName, final List jobIds) { this.jobIds = jobIds; } - public String getClusterName() { - return clusterName; - } - - public List getJobIds() { - return jobIds; - } - @Override public String toString() { return "HealthCheckRequest{" + @@ -2285,7 +2277,6 @@ public String toString() { } public static final class HealthCheckResponse extends BaseResponse { - public final boolean isHealthy; public final WorkerFailure workerFailure; From 5abb3c484851e8e26c93cce25cea304c28092c43 Mon Sep 17 00:00:00 2001 From: jlubin Date: Fri, 10 Apr 2026 13:18:36 -0400 Subject: [PATCH 09/15] Modify tests --- .../master/JobClustersManagerActor.java | 4 +- .../master/jobcluster/JobClusterActor.java | 12 +- .../jobcluster/proto/JobClusterProto.java | 4 +- .../JobClusterRouteHandlerAkkaImplTest.java | 101 -------------- .../akka/route/v1/JobClustersRouteTest.java | 46 +++---- .../master/jobcluster/JobClusterAkkaTest.java | 127 ++++++------------ 6 files changed, 75 insertions(+), 219 deletions(-) delete mode 100644 mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImplTest.java 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 a7cf80443..14716584f 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 @@ -670,13 +670,13 @@ public void onGetJobScalerRuleStream(final JobClusterScalerRuleProto.GetJobScale } public void onHealthCheck(HealthCheckRequest request) { - Optional jobClusterInfo = jobClusterInfoManager.getJobClusterInfo(request.getClusterName()); + Optional jobClusterInfo = jobClusterInfoManager.getJobClusterInfo(request.clusterName); if (jobClusterInfo.isPresent()) { jobClusterInfo.get().jobClusterActor.forward(request, getContext()); } else { ActorRef sender = getSender(); - sender.tell(new HealthCheckResponse(request.requestId, CLIENT_ERROR_NOT_FOUND, "No such Job cluster " + request.getClusterName(), false, null), getSelf()); + sender.tell(new HealthCheckResponse(request.requestId, CLIENT_ERROR_NOT_FOUND, "No such Job cluster " + request.clusterName, false, null), getSelf()); } } diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java index 31a88e324..b1a0bbe57 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java @@ -2646,8 +2646,8 @@ public void onHealthCheck(final HealthCheckRequest request) { final ActorRef self = getSelf(); Set prefilteredJobIds = new HashSet<>(); - if (request.getJobIds() != null && !request.getJobIds().isEmpty()) { - request.getJobIds().stream() + if (request.jobIds != null && !request.jobIds.isEmpty()) { + request.jobIds.stream() .map(JobId::fromId) .filter(Optional::isPresent) .map(Optional::get) @@ -2678,10 +2678,10 @@ public void onHealthCheck(final HealthCheckRequest request) { HealthCheckResponse response = new HealthCheckResponse( request.requestId, - ResponseCode.SERVER_ERROR, - "unhealthy workers", - false, - new WorkerFailure(failedWorkers)); + ResponseCode.SERVER_ERROR, + "unhealthy workers", + false, + new WorkerFailure(failedWorkers)); sender.tell(response, self); } }, diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterProto.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterProto.java index da685b1b5..594b1e528 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterProto.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterProto.java @@ -21,13 +21,10 @@ import com.netflix.spectator.impl.Preconditions; import io.mantisrx.master.jobcluster.job.IMantisJobMetadata; import io.mantisrx.master.jobcluster.job.JobState; -import io.mantisrx.master.jobcluster.proto.BaseResponse.ResponseCode; import io.mantisrx.server.core.JobCompletedReason; import io.mantisrx.server.master.domain.JobClusterDefinitionImpl; import io.mantisrx.server.master.domain.JobDefinition; import io.mantisrx.server.master.domain.JobId; -import io.mantisrx.shaded.com.fasterxml.jackson.annotation.JsonSubTypes; -import io.mantisrx.shaded.com.fasterxml.jackson.annotation.JsonTypeInfo; import io.mantisrx.shaded.com.google.common.collect.Lists; import java.time.Instant; import java.util.List; @@ -279,4 +276,5 @@ public TriggerCronRequest() { } } + } diff --git a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImplTest.java b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImplTest.java deleted file mode 100644 index 450a1b857..000000000 --- a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/handlers/JobClusterRouteHandlerAkkaImplTest.java +++ /dev/null @@ -1,101 +0,0 @@ -/* - * Copyright 2026 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.master.api.akka.route.handlers; - -import static org.junit.Assert.*; -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -import akka.actor.ActorRef; -import akka.actor.ActorSystem; -import akka.testkit.javadsl.TestKit; -import com.netflix.mantis.master.scheduler.TestHelpers; -import io.mantisrx.master.JobClustersManagerActor; -import io.mantisrx.master.events.AuditEventSubscriberLoggingImpl; -import io.mantisrx.master.events.LifecycleEventPublisher; -import io.mantisrx.master.events.LifecycleEventPublisherImpl; -import io.mantisrx.master.events.StatusEventSubscriberLoggingImpl; -import io.mantisrx.master.events.WorkerEventSubscriberLoggingImpl; -import io.mantisrx.master.jobcluster.job.CostsCalculator; - -import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto; -import io.mantisrx.master.scheduler.FakeMantisScheduler; -import io.mantisrx.server.master.persistence.FileBasedPersistenceProvider; -import io.mantisrx.server.master.persistence.MantisJobStore; -import io.mantisrx.server.master.scheduler.MantisScheduler; -import io.mantisrx.server.master.scheduler.MantisSchedulerFactory; -import java.io.File; -import java.nio.file.Files; -import java.util.concurrent.TimeUnit; -import org.apache.commons.io.FileUtils; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; - -public class JobClusterRouteHandlerAkkaImplTest { - - private static ActorSystem system; - private static ActorRef jobClustersManagerActor; - private static File stateDirectory; - - @BeforeClass - public static void setup() throws Exception { - TestHelpers.setupMasterConfig(); - system = ActorSystem.create("JobClusterRouteHandlerTest"); - stateDirectory = Files.createTempDirectory("test-handler").toFile(); - - LifecycleEventPublisher lifecycleEventPublisher = new LifecycleEventPublisherImpl( - new AuditEventSubscriberLoggingImpl(), - new StatusEventSubscriberLoggingImpl(), - new WorkerEventSubscriberLoggingImpl()); - - jobClustersManagerActor = system.actorOf( - JobClustersManagerActor.props( - new MantisJobStore(new FileBasedPersistenceProvider(stateDirectory, true)), - lifecycleEventPublisher, - CostsCalculator.noop(), - 0), - "jobClustersManager"); - - MantisSchedulerFactory mantisSchedulerFactory = mock(MantisSchedulerFactory.class); - MantisScheduler fakeScheduler = new FakeMantisScheduler(jobClustersManagerActor); - when(mantisSchedulerFactory.forJob(any())).thenReturn(fakeScheduler); - jobClustersManagerActor.tell( - new JobClusterManagerProto.JobClustersManagerInitialize( - mantisSchedulerFactory, false), ActorRef.noSender()); - - Thread.sleep(1000); - } - - @AfterClass - public static void tearDown() throws Exception { - TestKit.shutdownActorSystem(system); - system = null; - FileUtils.deleteDirectory(stateDirectory); - } - - @Test - public void testHealthCheck() throws Exception { - JobClusterRouteHandlerAkkaImpl handler = new JobClusterRouteHandlerAkkaImpl(jobClustersManagerActor); - - JobClusterManagerProto.HealthCheckResponse response = handler.healthCheck("nonExistentCluster", null) - .toCompletableFuture().get(5, TimeUnit.SECONDS); - - assertNotNull(response); - } -} diff --git a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/v1/JobClustersRouteTest.java b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/v1/JobClustersRouteTest.java index 3db61f412..c6d7e9d66 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/v1/JobClustersRouteTest.java +++ b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/api/akka/route/v1/JobClustersRouteTest.java @@ -682,29 +682,6 @@ private void testJobClusterActionDisableDELETENotAllowed() throws InterruptedExc testDelete(getJobClusterDisableEp(JobClusterPayloads.CLUSTER_NAME), StatusCodes.METHOD_NOT_ALLOWED, null); } - private void testJobClusterHealthCheck() throws InterruptedException { - testGet( - getJobClusterInstanceEndpoint(JobClusterPayloads.CLUSTER_NAME) + "/healthcheck", - StatusCodes.OK, - response -> { - try { - ObjectMapper mapper = new ObjectMapper(); - JsonNode node = mapper.readTree(response); - assertTrue("Expected 'isHealthy' field in response: " + response, node.has("isHealthy")); - assertTrue(node.get("isHealthy").asBoolean()); - } catch (IOException e) { - fail("Failed to parse health check response: " + e.getMessage()); - } - }); - } - - private void testJobClusterHealthCheckNonExistent() throws InterruptedException { - testGet( - getJobClusterInstanceEndpoint("nonExistentCluster") + "/healthcheck", - StatusCodes.NOT_FOUND, - null); - } - private void testJobClusterDeleteWithoutRequiredParam() throws InterruptedException { testDelete( getJobClusterInstanceEndpoint(JobClusterPayloads.CLUSTER_NAME), @@ -803,4 +780,27 @@ private void compareClustersPayload(String clusterListResponse) { assert ex == null; } } + + private void testJobClusterHealthCheck() throws InterruptedException { + testGet( + getJobClusterInstanceEndpoint(JobClusterPayloads.CLUSTER_NAME) + "/healthcheck", + StatusCodes.OK, + response -> { + try { + ObjectMapper mapper = new ObjectMapper(); + JsonNode node = mapper.readTree(response); + assertTrue("Expected 'isHealthy' field in response: " + response, node.has("isHealthy")); + assertTrue(node.get("isHealthy").asBoolean()); + } catch (IOException e) { + fail("Failed to parse health check response: " + e.getMessage()); + } + }); + } + + private void testJobClusterHealthCheckNonExistent() throws InterruptedException { + testGet( + getJobClusterInstanceEndpoint("nonExistentCluster") + "/healthcheck", + StatusCodes.NOT_FOUND, + null); + } } 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 15328c717..f27028953 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 @@ -3379,105 +3379,64 @@ public void testExpireOldJobs() { // HEALTH CHECK TESTS ////////////////////////////////////////////////////////////////////////// - @Test - public void testHealthCheckAllWorkersStarted() { - try { - TestKit probe = new TestKit(system); - String clusterName = "testHealthCheckAllWorkersStarted"; - MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); - MantisJobStore jobStoreMock = mock(MantisJobStore.class); - String jobId = clusterName + "-1"; - JobDefinition jobDefn = createJob(clusterName); + /** + * Submits a job, optionally starts workers, sends a health check request, and returns the response. + */ + private JobClusterManagerProto.HealthCheckResponse submitJobAndHealthCheck( + String clusterName, boolean startWorkers, List jobIds) throws Exception { + TestKit probe = new TestKit(system); + MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); + MantisJobStore jobStoreMock = mock(MantisJobStore.class); + String jobId = clusterName + "-1"; + JobDefinition jobDefinition = createJob(clusterName); - final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName); - ActorRef jobClusterActor = system.actorOf(props(clusterName, jobStoreMock, jobDfn -> schedulerMock, eventPublisher, costsCalculator, 0)); - jobClusterActor.tell(new JobClusterProto.InitializeJobClusterRequest(fakeJobCluster, user, probe.getRef()), probe.getRef()); - JobClusterProto.InitializeJobClusterResponse createResp = probe.expectMsgClass(JobClusterProto.InitializeJobClusterResponse.class); - assertEquals(SUCCESS, createResp.responseCode); + final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName); + ActorRef jobClusterActor = system.actorOf(props(clusterName, jobStoreMock, jobDfn -> schedulerMock, eventPublisher, costsCalculator, 0)); + jobClusterActor.tell(new JobClusterProto.InitializeJobClusterRequest(fakeJobCluster, user, probe.getRef()), probe.getRef()); + JobClusterProto.InitializeJobClusterResponse createResp = probe.expectMsgClass(JobClusterProto.InitializeJobClusterResponse.class); + assertEquals(SUCCESS, createResp.responseCode); - JobTestHelper.submitJobAndVerifySuccess(probe, clusterName, jobClusterActor, jobDefn, jobId); - JobTestHelper.getJobDetailsAndVerify(probe, jobClusterActor, jobId, SUCCESS, JobState.Accepted); + JobTestHelper.submitJobAndVerifySuccess(probe, clusterName, jobClusterActor, jobDefinition, jobId); + if (startWorkers) { JobTestHelper.sendLaunchedInitiatedStartedEventsToWorker(probe, jobClusterActor, jobId, 1, new WorkerId(clusterName, jobId, 0, 1)); - JobTestHelper.getJobDetailsAndVerify(probe, jobClusterActor, jobId, SUCCESS, JobState.Launched); - - jobClusterActor.tell(new JobClusterManagerProto.HealthCheckRequest(clusterName, null), probe.getRef()); - JobClusterManagerProto.HealthCheckResponse healthResp = probe.expectMsgClass(JobClusterManagerProto.HealthCheckResponse.class); + } - assertTrue(healthResp.isHealthy); - assertEquals(SUCCESS, healthResp.responseCode); - assertNull(healthResp.workerFailure); + jobClusterActor.tell(new JobClusterManagerProto.HealthCheckRequest(clusterName, jobIds), probe.getRef()); + JobClusterManagerProto.HealthCheckResponse healthResp = probe.expectMsgClass(JobClusterManagerProto.HealthCheckResponse.class); - probe.getSystem().stop(jobClusterActor); - } catch (Exception e) { - e.printStackTrace(); - fail(); - } + probe.getSystem().stop(jobClusterActor); + return healthResp; } @Test - public void testHealthCheckWithUnstartedWorkers() { - try { - TestKit probe = new TestKit(system); - String clusterName = "testHealthCheckWithUnstartedWorkers"; - MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); - MantisJobStore jobStoreMock = mock(MantisJobStore.class); - String jobId = clusterName + "-1"; - JobDefinition jobDefn = createJob(clusterName); - - final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName); - ActorRef jobClusterActor = system.actorOf(props(clusterName, jobStoreMock, jobDfn -> schedulerMock, eventPublisher, costsCalculator, 0)); - jobClusterActor.tell(new JobClusterProto.InitializeJobClusterRequest(fakeJobCluster, user, probe.getRef()), probe.getRef()); - JobClusterProto.InitializeJobClusterResponse createResp = probe.expectMsgClass(JobClusterProto.InitializeJobClusterResponse.class); - assertEquals(SUCCESS, createResp.responseCode); - - JobTestHelper.submitJobAndVerifySuccess(probe, clusterName, jobClusterActor, jobDefn, jobId); - JobTestHelper.getJobDetailsAndVerify(probe, jobClusterActor, jobId, SUCCESS, JobState.Accepted); + public void testHealthCheckAllWorkersStarted() throws Exception { + JobClusterManagerProto.HealthCheckResponse resp + = submitJobAndHealthCheck("testHealthCheckAllWorkersStarted", true, null); - jobClusterActor.tell(new JobClusterManagerProto.HealthCheckRequest(clusterName, null), probe.getRef()); - JobClusterManagerProto.HealthCheckResponse healthResp = probe.expectMsgClass(JobClusterManagerProto.HealthCheckResponse.class); - - assertFalse(healthResp.isHealthy); - assertEquals(SERVER_ERROR, healthResp.responseCode); - assertNotNull(healthResp.workerFailure); - assertFalse(healthResp.workerFailure.failedWorkers().isEmpty()); - - probe.getSystem().stop(jobClusterActor); - } catch (Exception e) { - e.printStackTrace(); - fail(); - } + assertTrue(resp.isHealthy); + assertEquals(SUCCESS, resp.responseCode); + assertNull(resp.workerFailure); } @Test - public void testHealthCheckWithJobIdFilter() { - try { - TestKit probe = new TestKit(system); - String clusterName = "testHealthCheckWithJobIdFilter"; - MantisScheduler schedulerMock = JobTestHelper.createMockScheduler(); - MantisJobStore jobStoreMock = mock(MantisJobStore.class); - String jobId = clusterName + "-1"; - JobDefinition jobDefn = createJob(clusterName); - - final JobClusterDefinitionImpl fakeJobCluster = createFakeJobClusterDefn(clusterName); - ActorRef jobClusterActor = system.actorOf(props(clusterName, jobStoreMock, jobDfn -> schedulerMock, eventPublisher, costsCalculator, 0)); - jobClusterActor.tell(new JobClusterProto.InitializeJobClusterRequest(fakeJobCluster, user, probe.getRef()), probe.getRef()); - JobClusterProto.InitializeJobClusterResponse createResp = probe.expectMsgClass(JobClusterProto.InitializeJobClusterResponse.class); - assertEquals(SUCCESS, createResp.responseCode); + public void testHealthCheckWithUnstartedWorkers() throws Exception { + JobClusterManagerProto.HealthCheckResponse resp = + submitJobAndHealthCheck("testHealthCheckWithUnstartedWorkers", false, null); - JobTestHelper.submitJobAndVerifySuccess(probe, clusterName, jobClusterActor, jobDefn, jobId); - JobTestHelper.sendLaunchedInitiatedStartedEventsToWorker(probe, jobClusterActor, jobId, 1, new WorkerId(clusterName, jobId, 0, 1)); - - jobClusterActor.tell(new JobClusterManagerProto.HealthCheckRequest(clusterName, ImmutableList.of(jobId)), probe.getRef()); - JobClusterManagerProto.HealthCheckResponse healthResp = probe.expectMsgClass(JobClusterManagerProto.HealthCheckResponse.class); + assertFalse(resp.isHealthy); + assertEquals(SERVER_ERROR, resp.responseCode); + assertNotNull(resp.workerFailure); + assertFalse(resp.workerFailure.failedWorkers().isEmpty()); + } - assertTrue(healthResp.isHealthy); - assertEquals(SUCCESS, healthResp.responseCode); + @Test + public void testHealthCheckWithJobIdFilter() throws Exception { + String clusterName = "testHealthCheckWithJobIdFilter"; + JobClusterManagerProto.HealthCheckResponse resp = + submitJobAndHealthCheck(clusterName, true, ImmutableList.of(clusterName + "-1")); - probe.getSystem().stop(jobClusterActor); - } catch (Exception e) { - e.printStackTrace(); - fail(); - } + assertTrue(resp.isHealthy); + assertEquals(SUCCESS, resp.responseCode); } @Test From 01240fd245bd0e5033e72052daa746ef20d707df Mon Sep 17 00:00:00 2001 From: jlubin Date: Fri, 10 Apr 2026 14:09:06 -0400 Subject: [PATCH 10/15] Have unhealthy workers return 2xx status code --- .../java/io/mantisrx/master/jobcluster/JobClusterActor.java | 2 +- .../java/io/mantisrx/master/jobcluster/JobClusterAkkaTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java index b1a0bbe57..9335bf611 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java @@ -2678,7 +2678,7 @@ public void onHealthCheck(final HealthCheckRequest request) { HealthCheckResponse response = new HealthCheckResponse( request.requestId, - ResponseCode.SERVER_ERROR, + ResponseCode.SUCCESS, "unhealthy workers", false, new WorkerFailure(failedWorkers)); 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 f27028953..9dd2e0725 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 @@ -3424,7 +3424,7 @@ public void testHealthCheckWithUnstartedWorkers() throws Exception { submitJobAndHealthCheck("testHealthCheckWithUnstartedWorkers", false, null); assertFalse(resp.isHealthy); - assertEquals(SERVER_ERROR, resp.responseCode); + assertEquals(SUCCESS, resp.responseCode); assertNotNull(resp.workerFailure); assertFalse(resp.workerFailure.failedWorkers().isEmpty()); } From cb81244b8c2aef5544226c9888c1708be7b0e8e5 Mon Sep 17 00:00:00 2001 From: jlubin Date: Mon, 13 Apr 2026 19:01:11 -0400 Subject: [PATCH 11/15] Rename workerFailure -> workersUnready --- .../master/jobcluster/JobClusterActor.java | 16 ++++++++-------- .../jobcluster/proto/JobClusterManagerProto.java | 10 +++++----- .../master/jobcluster/JobClusterAkkaTest.java | 6 +++--- 3 files changed, 16 insertions(+), 16 deletions(-) diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java index 9335bf611..c1d62340a 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java @@ -63,7 +63,7 @@ import io.mantisrx.master.jobcluster.job.MantisJobMetadataView; import io.mantisrx.master.jobcluster.job.worker.IMantisWorkerMetadata; import io.mantisrx.master.jobcluster.proto.BaseResponse.ResponseCode; -import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.FailedWorker; +import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.UnreadyWorker; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.DeleteJobClusterResponse; import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.HealthCheckRequest; @@ -117,7 +117,7 @@ import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.HealthCheckResponse; import io.mantisrx.master.jobcluster.proto.JobClusterProto.JobStartedEvent; import io.mantisrx.master.jobcluster.proto.JobClusterProto.KillJobRequest; -import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.WorkerFailure; +import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.WorkersUnready; import io.mantisrx.master.jobcluster.proto.JobClusterScalerRuleProto; import io.mantisrx.master.jobcluster.proto.JobProto; import io.mantisrx.master.jobcluster.scaler.IJobClusterScalerRuleData; @@ -2656,11 +2656,11 @@ public void onHealthCheck(final HealthCheckRequest request) { ListJobCriteria criteria = new ListJobCriteria(); getFilteredNonTerminalJobList(criteria, prefilteredJobIds) - .collect(Lists::newArrayList, (failedWorkers, view) -> { + .collect(Lists::newArrayList, (unreadyWorkers, view) -> { if (view.getWorkerMetadataList() != null) { for (FilterableMantisWorkerMetadataWritable worker : view.getWorkerMetadataList()) { if (worker.getState() != MantisJobState.Started) { - failedWorkers.add(new FailedWorker( + unreadyWorkers.add(new UnreadyWorker( worker.getWorkerIndex(), worker.getWorkerNumber(), worker.getState().name())); @@ -2669,8 +2669,8 @@ public void onHealthCheck(final HealthCheckRequest request) { } }) .subscribe( - failedWorkers -> { - if (failedWorkers.isEmpty()) { + unreadyWorkers -> { + if (unreadyWorkers.isEmpty()) { HealthCheckResponse response = new HealthCheckResponse(request.requestId, ResponseCode.SUCCESS, "OK", true, null); sender.tell(response, self); @@ -2679,9 +2679,9 @@ public void onHealthCheck(final HealthCheckRequest request) { = new HealthCheckResponse( request.requestId, ResponseCode.SUCCESS, - "unhealthy workers", + "unready workers", false, - new WorkerFailure(failedWorkers)); + new WorkersUnready(unreadyWorkers)); sender.tell(response, self); } }, diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java index f4604f93e..83122bd59 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java @@ -2278,7 +2278,7 @@ public String toString() { public static final class HealthCheckResponse extends BaseResponse { public final boolean isHealthy; - public final WorkerFailure workerFailure; + public final WorkersUnready workersUnready; @JsonCreator @JsonIgnoreProperties(ignoreUnknown = true) @@ -2287,14 +2287,14 @@ public HealthCheckResponse( @JsonProperty("responseCode") ResponseCode responseCode, @JsonProperty("message") String message, @JsonProperty("isHealthy") boolean isHealthy, - @JsonProperty("workerFailure") WorkerFailure workerFailure) { + @JsonProperty("workersUnready") WorkersUnready workersUnready) { super(requestId, responseCode, message); this.isHealthy = isHealthy; - this.workerFailure = workerFailure; + this.workersUnready = workersUnready; } } - public record WorkerFailure(List failedWorkers) {} + public record WorkersUnready(List unreadyWorkers) {} - public record FailedWorker(int workerIndex, int workerNumber, String state) {} + public record UnreadyWorker(int workerIndex, int workerNumber, String state) {} } 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 9dd2e0725..7744e6311 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 @@ -3415,7 +3415,7 @@ public void testHealthCheckAllWorkersStarted() throws Exception { assertTrue(resp.isHealthy); assertEquals(SUCCESS, resp.responseCode); - assertNull(resp.workerFailure); + assertNull(resp.workersUnready); } @Test @@ -3425,8 +3425,8 @@ public void testHealthCheckWithUnstartedWorkers() throws Exception { assertFalse(resp.isHealthy); assertEquals(SUCCESS, resp.responseCode); - assertNotNull(resp.workerFailure); - assertFalse(resp.workerFailure.failedWorkers().isEmpty()); + assertNotNull(resp.workersUnready); + assertFalse(resp.workersUnready.unreadyWorkers().isEmpty()); } @Test From b27700ce5c76f8354113b3710845fcddf6921767 Mon Sep 17 00:00:00 2001 From: jlubin Date: Mon, 13 Apr 2026 19:04:43 -0400 Subject: [PATCH 12/15] UnreadyWorkers --- .../io/mantisrx/master/jobcluster/JobClusterActor.java | 4 ++-- .../master/jobcluster/proto/JobClusterManagerProto.java | 8 ++++---- .../io/mantisrx/master/jobcluster/JobClusterAkkaTest.java | 6 +++--- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java index c1d62340a..1c2c21fe7 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/JobClusterActor.java @@ -117,7 +117,7 @@ import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.HealthCheckResponse; import io.mantisrx.master.jobcluster.proto.JobClusterProto.JobStartedEvent; import io.mantisrx.master.jobcluster.proto.JobClusterProto.KillJobRequest; -import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.WorkersUnready; +import io.mantisrx.master.jobcluster.proto.JobClusterManagerProto.UnreadyWorkers; import io.mantisrx.master.jobcluster.proto.JobClusterScalerRuleProto; import io.mantisrx.master.jobcluster.proto.JobProto; import io.mantisrx.master.jobcluster.scaler.IJobClusterScalerRuleData; @@ -2681,7 +2681,7 @@ public void onHealthCheck(final HealthCheckRequest request) { ResponseCode.SUCCESS, "unready workers", false, - new WorkersUnready(unreadyWorkers)); + new UnreadyWorkers(unreadyWorkers)); sender.tell(response, self); } }, diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java index 83122bd59..8a4456b29 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java @@ -2278,7 +2278,7 @@ public String toString() { public static final class HealthCheckResponse extends BaseResponse { public final boolean isHealthy; - public final WorkersUnready workersUnready; + public final UnreadyWorkers unreadyWorkers; @JsonCreator @JsonIgnoreProperties(ignoreUnknown = true) @@ -2287,14 +2287,14 @@ public HealthCheckResponse( @JsonProperty("responseCode") ResponseCode responseCode, @JsonProperty("message") String message, @JsonProperty("isHealthy") boolean isHealthy, - @JsonProperty("workersUnready") WorkersUnready workersUnready) { + @JsonProperty("workersUnready") UnreadyWorkers unreadyWorkers) { super(requestId, responseCode, message); this.isHealthy = isHealthy; - this.workersUnready = workersUnready; + this.unreadyWorkers = unreadyWorkers; } } - public record WorkersUnready(List unreadyWorkers) {} + public record UnreadyWorkers(List workers) {} public record UnreadyWorker(int workerIndex, int workerNumber, String state) {} } 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 7744e6311..05ea1960a 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 @@ -3415,7 +3415,7 @@ public void testHealthCheckAllWorkersStarted() throws Exception { assertTrue(resp.isHealthy); assertEquals(SUCCESS, resp.responseCode); - assertNull(resp.workersUnready); + assertNull(resp.unreadyWorkers); } @Test @@ -3425,8 +3425,8 @@ public void testHealthCheckWithUnstartedWorkers() throws Exception { assertFalse(resp.isHealthy); assertEquals(SUCCESS, resp.responseCode); - assertNotNull(resp.workersUnready); - assertFalse(resp.workersUnready.unreadyWorkers().isEmpty()); + assertNotNull(resp.unreadyWorkers); + assertFalse(resp.unreadyWorkers.workers().isEmpty()); } @Test From ecab32378f7bd9763de2d311211e3fb4aeb0666b Mon Sep 17 00:00:00 2001 From: jlubin Date: Mon, 13 Apr 2026 19:06:04 -0400 Subject: [PATCH 13/15] Update JsonProperty argument --- .../master/jobcluster/proto/JobClusterManagerProto.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java index 8a4456b29..7ad367a29 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java +++ b/mantis-control-plane/mantis-control-plane-server/src/main/java/io/mantisrx/master/jobcluster/proto/JobClusterManagerProto.java @@ -2287,7 +2287,7 @@ public HealthCheckResponse( @JsonProperty("responseCode") ResponseCode responseCode, @JsonProperty("message") String message, @JsonProperty("isHealthy") boolean isHealthy, - @JsonProperty("workersUnready") UnreadyWorkers unreadyWorkers) { + @JsonProperty("unreadyWorkers") UnreadyWorkers unreadyWorkers) { super(requestId, responseCode, message); this.isHealthy = isHealthy; this.unreadyWorkers = unreadyWorkers; From fb7bdf18b545b8d0f467ac77ea4a3633bdc64e73 Mon Sep 17 00:00:00 2001 From: jlubin Date: Mon, 13 Apr 2026 20:27:31 -0400 Subject: [PATCH 14/15] Force preStart to finish by calling getRegisteredTaskExecutors --- .../master/resourcecluster/ResourceClusterActorTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/resourcecluster/ResourceClusterActorTest.java b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/resourcecluster/ResourceClusterActorTest.java index 880ccd4f3..d6b93d7d3 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/resourcecluster/ResourceClusterActorTest.java +++ b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/resourcecluster/ResourceClusterActorTest.java @@ -956,6 +956,7 @@ public void testPreStartRetriesAndSucceeds() throws Exception { @Test public void testTaskExecutorIsDisabledEvenAfterRestart() throws Exception { + resourceCluster.getRegisteredTaskExecutors().get(); when(mantisJobStore.getTaskExecutor(TASK_EXECUTOR_ID)).thenReturn(TASK_EXECUTOR_REGISTRATION); doReturn(ImmutableList.of()).when(mantisJobStore).getJobArtifactsToCache(CLUSTER_ID); From 6a3a17446e034e04f90771ea2b3cd6aea3b8f195 Mon Sep 17 00:00:00 2001 From: jlubin Date: Tue, 14 Apr 2026 08:58:49 -0400 Subject: [PATCH 15/15] Move getRegisteredTaskExecutors().get() to setupActor() --- .../master/resourcecluster/ResourceClusterActorTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/resourcecluster/ResourceClusterActorTest.java b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/resourcecluster/ResourceClusterActorTest.java index d6b93d7d3..aa069d284 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/resourcecluster/ResourceClusterActorTest.java +++ b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/resourcecluster/ResourceClusterActorTest.java @@ -223,7 +223,7 @@ private void setupRpcService() throws Exception { .getJobArtifactsToCache(CLUSTER_ID); } - private void setupActor() { + private void setupActor() throws ExecutionException, InterruptedException { MasterConfiguration masterConfig = mock(MasterConfiguration.class); when(masterConfig.getTimeoutSecondsToReportStart()).thenReturn(1); ExecuteStageRequestFactory executeStageRequestFactory = new ExecuteStageRequestFactory(masterConfig); @@ -253,6 +253,7 @@ private void setupActor() { Duration.ofSeconds(15), CLUSTER_ID, new LongDynamicProperty(propertiesLoader, "resourcecluster.gateway.maxConcurrentRequests.test", 100000L)); + resourceCluster.getRegisteredTaskExecutors().get(); } @Test @@ -956,7 +957,6 @@ public void testPreStartRetriesAndSucceeds() throws Exception { @Test public void testTaskExecutorIsDisabledEvenAfterRestart() throws Exception { - resourceCluster.getRegisteredTaskExecutors().get(); when(mantisJobStore.getTaskExecutor(TASK_EXECUTOR_ID)).thenReturn(TASK_EXECUTOR_REGISTRATION); doReturn(ImmutableList.of()).when(mantisJobStore).getJobArtifactsToCache(CLUSTER_ID);