From fe8ad58344c41c5c05161f967a0e1d9174bcb5d5 Mon Sep 17 00:00:00 2001 From: Harshit Mittal Date: Thu, 11 Apr 2024 12:59:39 -0700 Subject: [PATCH 1/8] Prevent scale downs for evacuated regions during region failover --- .../runtime/descriptor/SchedulingInfo.java | 4 +- .../descriptor/StageScalingPolicy.java | 51 +++++-------- .../descriptor/SchedulingInfoTest.java | 12 ++- .../descriptor/StageScalingPolicyTest.java | 22 +++++- .../server/master/FailoverStatusClient.java | 56 ++++++++++++++ .../master/events/WorkerRegistryV2Test.java | 6 +- .../master/jobcluster/JobClusterAkkaTest.java | 2 +- .../jobcluster/job/JobScaleUpDownTests.java | 18 ++--- .../jobcluster/job/JobTestLifecycle.java | 4 +- .../master/domain/DataFormatAdapterTest.java | 2 +- .../server/worker/RuntimeTaskImpl.java | 8 ++ ...WorkerExecutionOperationsNetworkStage.java | 6 +- .../jobmaster/AutoScaleMetricsConfig.java | 3 +- .../worker/jobmaster/JobAutoScaler.java | 53 ++++++++----- .../worker/jobmaster/JobMasterService.java | 26 ++++--- .../worker/jobmaster/WorkerMetricHandler.java | 75 ++++++++++--------- .../worker/jobmaster/JobAutoScalerTest.java | 29 ++++--- .../jobmaster/WorkerMetricHandlerTest.java | 9 ++- .../RpsClutchConfigurationSelectorTest.java | 8 +- 19 files changed, 245 insertions(+), 149 deletions(-) create mode 100644 mantis-control-plane/mantis-control-plane-core/src/main/java/io/mantisrx/server/master/FailoverStatusClient.java diff --git a/mantis-common/src/main/java/io/mantisrx/runtime/descriptor/SchedulingInfo.java b/mantis-common/src/main/java/io/mantisrx/runtime/descriptor/SchedulingInfo.java index 73c6950a7..7776b3940 100644 --- a/mantis-common/src/main/java/io/mantisrx/runtime/descriptor/SchedulingInfo.java +++ b/mantis-common/src/main/java/io/mantisrx/runtime/descriptor/SchedulingInfo.java @@ -132,7 +132,7 @@ public Builder multiWorkerScalableStageWithConstraints(int numberOfWorkers, Mach List hardConstraints, List softConstraints, StageScalingPolicy scalingPolicy) { StageScalingPolicy ssp = new StageScalingPolicy(currentStage, scalingPolicy.getMin(), scalingPolicy.getMax(), - scalingPolicy.getIncrement(), scalingPolicy.getDecrement(), scalingPolicy.getCoolDownSecs(), scalingPolicy.getStrategies()); + scalingPolicy.getIncrement(), scalingPolicy.getDecrement(), scalingPolicy.getCoolDownSecs(), scalingPolicy.getStrategies(), scalingPolicy.isAllowScaleDownDuringEvacuated()); return this.addStage( StageSchedulingInfo.builder() .numberOfInstances(numberOfWorkers) @@ -148,7 +148,7 @@ public Builder multiWorkerScalableStageWithConstraints(int numberOfWorkers, Mach List hardConstraints, List softConstraints, StageScalingPolicy scalingPolicy, Map containerAttributes) { StageScalingPolicy ssp = new StageScalingPolicy(currentStage, scalingPolicy.getMin(), scalingPolicy.getMax(), - scalingPolicy.getIncrement(), scalingPolicy.getDecrement(), scalingPolicy.getCoolDownSecs(), scalingPolicy.getStrategies()); + scalingPolicy.getIncrement(), scalingPolicy.getDecrement(), scalingPolicy.getCoolDownSecs(), scalingPolicy.getStrategies(), scalingPolicy.isAllowScaleDownDuringEvacuated()); return this.addStage( StageSchedulingInfo.builder() .numberOfInstances(numberOfWorkers) diff --git a/mantis-common/src/main/java/io/mantisrx/runtime/descriptor/StageScalingPolicy.java b/mantis-common/src/main/java/io/mantisrx/runtime/descriptor/StageScalingPolicy.java index 41f9518aa..6f6463086 100644 --- a/mantis-common/src/main/java/io/mantisrx/runtime/descriptor/StageScalingPolicy.java +++ b/mantis-common/src/main/java/io/mantisrx/runtime/descriptor/StageScalingPolicy.java @@ -28,6 +28,7 @@ import lombok.ToString; +@Getter public class StageScalingPolicy implements Serializable { private static final long serialVersionUID = 1L; @@ -39,13 +40,19 @@ public class StageScalingPolicy implements Serializable { private final int decrement; private final long coolDownSecs; private final Map strategies; + /** + * Controls whether scale down operations are allowed when the region is evacuated. + */ + private final boolean allowScaleDownDuringEvacuated; + @JsonCreator @JsonIgnoreProperties(ignoreUnknown = true) public StageScalingPolicy(@JsonProperty("stage") int stage, @JsonProperty("min") int min, @JsonProperty("max") int max, @JsonProperty("increment") int increment, @JsonProperty("decrement") int decrement, @JsonProperty("coolDownSecs") long coolDownSecs, - @JsonProperty("strategies") Map strategies) { + @JsonProperty("strategies") Map strategies, + @JsonProperty(value = "allowScaleDownDuringEvacuated", defaultValue = "true") Boolean allowScaleDownDuringEvacuated) { this.stage = stage; this.min = min; this.max = Math.max(max, min); @@ -54,34 +61,8 @@ public StageScalingPolicy(@JsonProperty("stage") int stage, this.decrement = Math.max(decrement, 1); this.coolDownSecs = coolDownSecs; this.strategies = strategies == null ? new HashMap() : new HashMap<>(strategies); - } - - public int getStage() { - return stage; - } - - public int getMin() { - return min; - } - - public int getMax() { - return max; - } - - public boolean isEnabled() { - return enabled; - } - - public int getIncrement() { - return increment; - } - - public int getDecrement() { - return decrement; - } - - public long getCoolDownSecs() { - return coolDownSecs; + // `defaultValue` is for documentation purpose only, use `Boolean` to determine if the field is missing on `null` + this.allowScaleDownDuringEvacuated = allowScaleDownDuringEvacuated == null || allowScaleDownDuringEvacuated; } public Map getStrategies() { @@ -99,6 +80,7 @@ public String toString() { ", decrement=" + decrement + ", coolDownSecs=" + coolDownSecs + ", strategies=" + strategies + + ", allowScaleDownDuringEvacuated=" + allowScaleDownDuringEvacuated + '}'; } @@ -114,6 +96,7 @@ public int hashCode() { result = prime * result + min; result = prime * result + stage; result = prime * result + ((strategies == null) ? 0 : strategies.hashCode()); + result = prime * result + (allowScaleDownDuringEvacuated ? 1231 : 1237); return result; } @@ -141,11 +124,10 @@ public boolean equals(Object obj) { if (stage != other.stage) return false; if (strategies == null) { - if (other.strategies != null) - return false; - } else if (!strategies.equals(other.strategies)) + return other.strategies == null; + } else if (!strategies.equals(other.strategies)) { return false; - return true; + } else return allowScaleDownDuringEvacuated == other.allowScaleDownDuringEvacuated; } public enum ScalingReason { @@ -162,7 +144,8 @@ public enum ScalingReason { ClutchRps, RPS, JVMMemory, - SourceJobDrop + SourceJobDrop, + FailoverAware } @Getter diff --git a/mantis-common/src/test/java/io/mantisrx/runtime/descriptor/SchedulingInfoTest.java b/mantis-common/src/test/java/io/mantisrx/runtime/descriptor/SchedulingInfoTest.java index 50e3b13e3..cec17d8e8 100644 --- a/mantis-common/src/test/java/io/mantisrx/runtime/descriptor/SchedulingInfoTest.java +++ b/mantis-common/src/test/java/io/mantisrx/runtime/descriptor/SchedulingInfoTest.java @@ -153,13 +153,13 @@ public void testSerialization() throws Exception { 2, new MachineDefinition(1, 1.24, 0.0, 1, 1), null, null, - new StageScalingPolicy(1, 1, 3, 1, 1, 60, smap) + new StageScalingPolicy(1, 1, 3, 1, 1, 60, smap, true) ) .multiWorkerScalableStageWithConstraints( 3, new MachineDefinition(1, 1.24, 0.0, 1, 1), null, null, - new StageScalingPolicy(1, 1, 3, 1, 1, 60, smap) + new StageScalingPolicy(1, 1, 3, 1, 1, 60, smap, true) ); JsonSerializer serializer = new JsonSerializer(); @@ -204,6 +204,7 @@ public void testSerialization() throws Exception { " }" + " }" + " }," + + " \"allowScaleDownDuringEvacuated\": true," + " \"enabled\": true" + " }," + " \"scalable\": true" + @@ -245,6 +246,7 @@ public void testSerialization() throws Exception { " }" + " }" + " }," + + " \"allowScaleDownDuringEvacuated\": true," + " \"enabled\": true" + " }," + " \"scalable\": true" + @@ -264,14 +266,14 @@ public void testSerializationWithSkuId() throws Exception { 2, new MachineDefinition(1, 1.24, 0.0, 1, 1), null, null, - new StageScalingPolicy(1, 1, 3, 1, 1, 60, smap), + new StageScalingPolicy(1, 1, 3, 1, 1, 60, smap, true), ImmutableMap.of("containerSkuID", "sku1") ) .multiWorkerScalableStageWithConstraints( 3, new MachineDefinition(1, 1.24, 0.0, 1, 1), null, null, - new StageScalingPolicy(1, 1, 3, 1, 1, 60, smap), + new StageScalingPolicy(1, 1, 3, 1, 1, 60, smap, true), ImmutableMap.of("containerSkuID", "sku2") ); @@ -317,6 +319,7 @@ public void testSerializationWithSkuId() throws Exception { " }" + " }" + " }," + + " \"allowScaleDownDuringEvacuated\": true," + " \"enabled\": true" + " }," + " \"scalable\": true," + @@ -359,6 +362,7 @@ public void testSerializationWithSkuId() throws Exception { " }" + " }" + " }," + + " \"allowScaleDownDuringEvacuated\": true," + " \"enabled\": true" + " }," + " \"scalable\": true," + diff --git a/mantis-common/src/test/java/io/mantisrx/runtime/descriptor/StageScalingPolicyTest.java b/mantis-common/src/test/java/io/mantisrx/runtime/descriptor/StageScalingPolicyTest.java index 1833479d2..941341053 100644 --- a/mantis-common/src/test/java/io/mantisrx/runtime/descriptor/StageScalingPolicyTest.java +++ b/mantis-common/src/test/java/io/mantisrx/runtime/descriptor/StageScalingPolicyTest.java @@ -35,7 +35,7 @@ public void testSerialization() throws Exception { Map smap = new HashMap<>(); smap.put(ScalingReason.CPU, new Strategy(ScalingReason.CPU, 0.5, 0.75, null)); smap.put(ScalingReason.DataDrop, new Strategy(ScalingReason.DataDrop, 0.0, 2.0, null)); - StageScalingPolicy policy = new StageScalingPolicy(1, 1, 2, 1, 1, 60, smap); + StageScalingPolicy policy = new StageScalingPolicy(1, 1, 2, 1, 1, 60, smap, true); final String expected = "{\n" + " \"stage\": 1,\n" + @@ -80,21 +80,35 @@ public void testSerialization() throws Exception { public void testDeserialization() throws Exception { String json1 = "{\"stage\":1,\"min\":1,\"max\":2,\"increment\":1,\"decrement\":1,\"strategies\":{},\"enabled\":false}"; StageScalingPolicy actual = serializer.fromJSON(json1, StageScalingPolicy.class); - StageScalingPolicy expected = new StageScalingPolicy(1, 1, 2, 1, 1, 0, null); + StageScalingPolicy expected = new StageScalingPolicy(1, 1, 2, 1, 1, 0, null, true); assertEquals(expected, actual); String json2 = "{\"stage\":1,\"min\":1,\"max\":5,\"increment\":1,\"decrement\":1,\"coolDownSecs\":600,\"strategies\":{\"CPU\":{\"reason\":\"CPU\",\"scaleDownBelowPct\":50,\"scaleUpAbovePct\":75}},\"enabled\":true}"; actual = serializer.fromJSON(json2, StageScalingPolicy.class); Map smap = new HashMap<>(); smap.put(ScalingReason.CPU, new Strategy(ScalingReason.CPU, 50, 75.0, new RollingCount(1, 1))); - expected = new StageScalingPolicy(1, 1, 5, 1, 1, 600, smap); + expected = new StageScalingPolicy(1, 1, 5, 1, 1, 600, smap, true); assertEquals(expected, actual); String json3 = "{\"stage\":1,\"min\":1,\"max\":3,\"increment\":1,\"decrement\":1,\"coolDownSecs\":0,\"strategies\":{\"Memory\":{\"reason\":\"Memory\",\"scaleDownBelowPct\":65,\"scaleUpAbovePct\":80,\"rollingCount\":{\"count\":6,\"of\":10}}},\"enabled\":true}"; actual = serializer.fromJSON(json3, StageScalingPolicy.class); smap = new HashMap<>(); smap.put(ScalingReason.Memory, new Strategy(ScalingReason.Memory, 65, 80.0, new RollingCount(6, 10))); - expected = new StageScalingPolicy(1, 1, 3, 1, 1, 0, smap); + expected = new StageScalingPolicy(1, 1, 3, 1, 1, 0, smap, true); + assertEquals(expected, actual); + + String json4 = "{\"stage\":1,\"min\":1,\"max\":3,\"increment\":1,\"decrement\":1,\"coolDownSecs\":0,\"strategies\":{\"Memory\":{\"reason\":\"Memory\",\"scaleDownBelowPct\":65,\"scaleUpAbovePct\":80,\"rollingCount\":{\"count\":6,\"of\":10}}},\"allowScaleDownDuringEvacuated\":false,\"enabled\":true}"; + actual = serializer.fromJSON(json4, StageScalingPolicy.class); + smap = new HashMap<>(); + smap.put(ScalingReason.Memory, new Strategy(ScalingReason.Memory, 65, 80.0, new RollingCount(6, 10))); + expected = new StageScalingPolicy(1, 1, 3, 1, 1, 0, smap, false); + assertEquals(expected, actual); + + String json5 = "{\"stage\":1,\"min\":1,\"max\":3,\"increment\":1,\"decrement\":1,\"coolDownSecs\":0,\"strategies\":{\"Memory\":{\"reason\":\"Memory\",\"scaleDownBelowPct\":65,\"scaleUpAbovePct\":80,\"rollingCount\":{\"count\":6,\"of\":10}}},\"allowScaleDownDuringEvacuated\":true,\"enabled\":true}"; + actual = serializer.fromJSON(json5, StageScalingPolicy.class); + smap = new HashMap<>(); + smap.put(ScalingReason.Memory, new Strategy(ScalingReason.Memory, 65, 80.0, new RollingCount(6, 10))); + expected = new StageScalingPolicy(1, 1, 3, 1, 1, 0, smap, true); assertEquals(expected, actual); } } diff --git a/mantis-control-plane/mantis-control-plane-core/src/main/java/io/mantisrx/server/master/FailoverStatusClient.java b/mantis-control-plane/mantis-control-plane-core/src/main/java/io/mantisrx/server/master/FailoverStatusClient.java new file mode 100644 index 000000000..88a15313d --- /dev/null +++ b/mantis-control-plane/mantis-control-plane-core/src/main/java/io/mantisrx/server/master/FailoverStatusClient.java @@ -0,0 +1,56 @@ +/* + * Copyright 2024 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.mantisrx.server.master; + +/** + * Failover status client interface to get the failover status of current region + */ +public interface FailoverStatusClient { + + FailoverStatusClient DEFAULT = new NoopFailoverStatusClient(); + + /** + * Get the failover status of current region + */ + FailoverStatus getFailoverStatus(); + + enum FailoverStatus { + /** + * Normal region + */ + NORMAL, + /** + * Region is evacuating (failed over) + */ + REGION_EVACUEE, + /** + * Region is savior (taking over traffic for failed over region) + */ + REGION_SAVIOR; + } + + /** + * Noop implementation of {@link FailoverStatusClient} that always returns {@link FailoverStatus#NORMAL} + */ + class NoopFailoverStatusClient implements FailoverStatusClient { + + @Override + public FailoverStatus getFailoverStatus() { + return FailoverStatus.NORMAL; + } + } +} diff --git a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/events/WorkerRegistryV2Test.java b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/events/WorkerRegistryV2Test.java index dc118a511..1ba8229dd 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/events/WorkerRegistryV2Test.java +++ b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/events/WorkerRegistryV2Test.java @@ -177,7 +177,7 @@ public void testJobScaleUp() throws Exception, InvalidJobException, io.mantisrx. new MachineDefinition(1.0,1.0,1.0,3), Lists.newArrayList(), Lists.newArrayList(), - new StageScalingPolicy(1, 0, 10, 1, 1, 0, smap)) + new StageScalingPolicy(1, 0, 10, 1, 1, 0, smap, true)) .build(); String clusterName = "testJobScaleUp"; MantisScheduler schedulerMock = mock(MantisScheduler.class); @@ -229,7 +229,7 @@ public void testJobScaleDown() throws Exception { new MachineDefinition(1.0,1.0,1.0,3), Lists.newArrayList(), Lists.newArrayList(), - new StageScalingPolicy(1, 0, 10, 1, 1, 0, smap)) + new StageScalingPolicy(1, 0, 10, 1, 1, 0, smap, true)) .build(); String clusterName = "testJobScaleDown"; MantisScheduler schedulerMock = mock(MantisScheduler.class); @@ -285,7 +285,7 @@ public void testJobShutdown() { new MachineDefinition(1.0,1.0,1.0,3), Lists.newArrayList(), Lists.newArrayList(), - new StageScalingPolicy(1, 0, 10, 1, 1, 0, smap)) + new StageScalingPolicy(1, 0, 10, 1, 1, 0, smap, true)) .build(); String clusterName = "testJobShutdown"; MantisScheduler schedulerMock = mock(MantisScheduler.class); diff --git a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/jobcluster/JobClusterAkkaTest.java b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/jobcluster/JobClusterAkkaTest.java index 0396bf89e..1ecf2008d 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 @@ -2416,7 +2416,7 @@ public void testScaleStage() { smap.put(StageScalingPolicy.ScalingReason.DataDrop, new StageScalingPolicy.Strategy(StageScalingPolicy.ScalingReason.DataDrop, 0.0, 2.0, null)); SchedulingInfo SINGLE_WORKER_SCHED_INFO = new SchedulingInfo.Builder().numberOfStages(1) - .multiWorkerScalableStageWithConstraints(1,DEFAULT_MACHINE_DEFINITION,Lists.newArrayList(),Lists.newArrayList(),new StageScalingPolicy(1,1,10,1,1,1, smap)).build(); + .multiWorkerScalableStageWithConstraints(1,DEFAULT_MACHINE_DEFINITION,Lists.newArrayList(),Lists.newArrayList(),new StageScalingPolicy(1,1,10,1,1,1, smap, true)).build(); final JobDefinition jobDefn = createJob(clusterName, 1, MantisJobDurationType.Transient, "USER_TYPE", SINGLE_WORKER_SCHED_INFO, Lists.newArrayList()); diff --git a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/jobcluster/job/JobScaleUpDownTests.java b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/jobcluster/job/JobScaleUpDownTests.java index 39d768d35..de434a776 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/jobcluster/job/JobScaleUpDownTests.java +++ b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/jobcluster/job/JobScaleUpDownTests.java @@ -118,7 +118,7 @@ public void testJobScaleUp() throws Exception, InvalidJobException, io.mantisrx. new MachineDefinition(1.0,1.0,1.0,3), Lists.newArrayList(), Lists.newArrayList(), - new StageScalingPolicy(1, 0, 10, 1, 1, 0, smap)) + new StageScalingPolicy(1, 0, 10, 1, 1, 0, smap, true)) .build(); String clusterName = "testJobScaleUp"; MantisScheduler schedulerMock = mock(MantisScheduler.class); @@ -161,7 +161,7 @@ public void testJobScaleDown() throws Exception, InvalidJobException, io.mantisr new MachineDefinition(1.0,1.0,1.0,3), Lists.newArrayList(), Lists.newArrayList(), - new StageScalingPolicy(1, 0, 10, 1, 1, 0, smap)) + new StageScalingPolicy(1, 0, 10, 1, 1, 0, smap, true)) .build(); String clusterName = "testJobScaleUp"; MantisScheduler schedulerMock = mock(MantisScheduler.class); @@ -219,7 +219,7 @@ public void testSchedulingInfo() throws Exception { new MachineDefinition(1.0,1.0,1.0,3), Lists.newArrayList(), Lists.newArrayList(), - new StageScalingPolicy(1, 0, 10, 1, 1, 0, smap)) + new StageScalingPolicy(1, 0, 10, 1, 1, 0, smap, true)) .build(); String clusterName = "testSchedulingInfo"; MantisScheduler schedulerMock = mock(MantisScheduler.class); @@ -500,7 +500,7 @@ public void testJobScaleUpFailsIfNoScaleStrategy() throws Exception { new MachineDefinition(1.0,1.0,1.0,3), Lists.newArrayList(), Lists.newArrayList(), - new StageScalingPolicy(1, 0, 10, 1, 1, 0, smap)) + new StageScalingPolicy(1, 0, 10, 1, 1, 0, smap, true)) .build(); String clusterName = "testJobScaleUpFailsIfNoScaleStrategy"; MantisScheduler schedulerMock = mock(MantisScheduler.class); @@ -543,7 +543,7 @@ public void testJobScaleUpFailsIfMinEqualsMax() throws Exception { new MachineDefinition(1.0,1.0,1.0,3), Lists.newArrayList(), Lists.newArrayList(), - new StageScalingPolicy(1, 1, 1, 1, 1, 0, smap)) + new StageScalingPolicy(1, 1, 1, 1, 1, 0, smap, true)) .build(); String clusterName = "testJobScaleUpFailsIfNoScaleStrategy"; MantisScheduler schedulerMock = mock(MantisScheduler.class); @@ -583,7 +583,7 @@ public void stageScalingPolicyTest() { long cooldownsecs = 300; Map smap = new HashMap<>(); smap.put(ScalingReason.CPU, new Strategy(ScalingReason.CPU, 0.5, 0.75, null)); - StageScalingPolicy ssp = new StageScalingPolicy(stageNo, min, max, increment, decrement, cooldownsecs, smap); + StageScalingPolicy ssp = new StageScalingPolicy(stageNo, min, max, increment, decrement, cooldownsecs, smap, true); assertTrue(ssp.isEnabled()); } @@ -598,7 +598,7 @@ public void stageScalingPolicyNoStrategyTest() { long cooldownsecs = 300; Map smap = new HashMap<>(); - StageScalingPolicy ssp = new StageScalingPolicy(stageNo, min, max, increment, decrement, cooldownsecs, smap); + StageScalingPolicy ssp = new StageScalingPolicy(stageNo, min, max, increment, decrement, cooldownsecs, smap, true); assertFalse(ssp.isEnabled()); } @@ -613,7 +613,7 @@ public void stageScalingPolicyMinEqMaxTest() { long cooldownsecs = 300; Map smap = new HashMap<>(); smap.put(ScalingReason.CPU, new Strategy(ScalingReason.CPU, 0.5, 0.75, null)); - StageScalingPolicy ssp = new StageScalingPolicy(stageNo, min, max, increment, decrement, cooldownsecs, smap); + StageScalingPolicy ssp = new StageScalingPolicy(stageNo, min, max, increment, decrement, cooldownsecs, smap, true); assertFalse(ssp.isEnabled()); } @@ -628,7 +628,7 @@ public void stageScalingPolicyMinGreaterThanMaxTest() { long cooldownsecs = 300; Map smap = new HashMap<>(); smap.put(ScalingReason.CPU, new Strategy(ScalingReason.CPU, 0.5, 0.75, null)); - StageScalingPolicy ssp = new StageScalingPolicy(stageNo, min, max, increment, decrement, cooldownsecs, smap); + StageScalingPolicy ssp = new StageScalingPolicy(stageNo, min, max, increment, decrement, cooldownsecs, smap, true); assertTrue(ssp.isEnabled()); // max will be set equal to min diff --git a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/jobcluster/job/JobTestLifecycle.java b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/jobcluster/job/JobTestLifecycle.java index 8404c4214..1aaba8e12 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/jobcluster/job/JobTestLifecycle.java +++ b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/master/jobcluster/job/JobTestLifecycle.java @@ -479,13 +479,13 @@ public void testJobSubmitWithMultipleStagesAndWorkers() { 2, new MachineDefinition(1, 1.24, 0.0, 1, 1), null, null, - new StageScalingPolicy(1, 1, 3, 1, 1, 60, smap) + new StageScalingPolicy(1, 1, 3, 1, 1, 60, smap, true) ) .multiWorkerScalableStageWithConstraints( 3, new MachineDefinition(1, 1.24, 0.0, 1, 1), null, null, - new StageScalingPolicy(1, 1, 3, 1, 1, 60, smap) + new StageScalingPolicy(1, 1, 3, 1, 1, 60, smap, true) ); SchedulingInfo sInfo = builder.build(); diff --git a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/server/master/domain/DataFormatAdapterTest.java b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/server/master/domain/DataFormatAdapterTest.java index d4f455d7f..c774f947c 100644 --- a/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/server/master/domain/DataFormatAdapterTest.java +++ b/mantis-control-plane/mantis-control-plane-server/src/test/java/io/mantisrx/server/master/domain/DataFormatAdapterTest.java @@ -779,7 +779,7 @@ public void convertMantisStageMetaTest() { int increment = 1; int decrement = 1; int coolDownSecs = 300; - StageScalingPolicy stageScalingPolicy = new StageScalingPolicy(stageNo, min, max, increment, decrement, coolDownSecs, smap); + StageScalingPolicy stageScalingPolicy = new StageScalingPolicy(stageNo, min, max, increment, decrement, coolDownSecs, smap, true); List softConstraintsList = new ArrayList<>(); softConstraintsList.add(JobConstraints.ExclusiveHost); List hardConstraintsList = new ArrayList<>(); diff --git a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/RuntimeTaskImpl.java b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/RuntimeTaskImpl.java index 4b8bb6363..4d2ed1426 100644 --- a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/RuntimeTaskImpl.java +++ b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/RuntimeTaskImpl.java @@ -28,6 +28,7 @@ import io.mantisrx.server.core.Status; import io.mantisrx.server.core.WrappedExecuteStageRequest; import io.mantisrx.server.core.metrics.MetricsFactory; +import io.mantisrx.server.master.FailoverStatusClient; import io.mantisrx.server.master.client.HighAvailabilityServices; import io.mantisrx.server.master.client.HighAvailabilityServicesUtil; import io.mantisrx.server.master.client.MantisMasterGateway; @@ -74,9 +75,15 @@ public class RuntimeTaskImpl extends AbstractIdleService implements RuntimeTask private Optional mantisJob = Optional.empty(); private ExecuteStageRequest executeStageRequest; + private FailoverStatusClient failoverStatusClient; public RuntimeTaskImpl() { + this(FailoverStatusClient.DEFAULT); + } + + public RuntimeTaskImpl(FailoverStatusClient failoverStatusClient) { this.tasksStatusSubject = PublishSubject.create(); + this.failoverStatusClient = failoverStatusClient; } public RuntimeTaskImpl(PublishSubject> tasksStatusSubject) { @@ -192,6 +199,7 @@ private void doRun() throws Exception { masterMonitor, config, workerMetricsClient, + failoverStatusClient, sinkSubscriptionStateHandlerFactory, userCodeClassLoader.asClassLoader()), getJobProviderClass(), diff --git a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/WorkerExecutionOperationsNetworkStage.java b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/WorkerExecutionOperationsNetworkStage.java index 467298e15..1c9ec1f74 100644 --- a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/WorkerExecutionOperationsNetworkStage.java +++ b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/WorkerExecutionOperationsNetworkStage.java @@ -52,6 +52,7 @@ import io.mantisrx.server.core.StatusPayloads; import io.mantisrx.server.core.WorkerAssignments; import io.mantisrx.server.core.WorkerHost; +import io.mantisrx.server.master.FailoverStatusClient; import io.mantisrx.server.master.client.MantisMasterGateway; import io.mantisrx.server.worker.client.SseWorkerConnection; import io.mantisrx.server.worker.client.WorkerMetricsClient; @@ -110,17 +111,20 @@ public class WorkerExecutionOperationsNetworkStage implements WorkerExecutionOpe private final List closeables = new ArrayList<>(); private final ScheduledExecutorService scheduledExecutorService; private final ClassLoader classLoader; + private final FailoverStatusClient failoverStatusClient; private Observer jobStatusObserver; public WorkerExecutionOperationsNetworkStage( MantisMasterGateway mantisMasterApi, WorkerConfiguration config, WorkerMetricsClient workerMetricsClient, + FailoverStatusClient failoverStatusClient, SinkSubscriptionStateHandler.Factory sinkSubscriptionStateHandlerFactory, ClassLoader classLoader) { this.mantisMasterApi = mantisMasterApi; this.config = config; this.workerMetricsClient = workerMetricsClient; + this.failoverStatusClient = failoverStatusClient; this.sinkSubscriptionStateHandlerFactory = sinkSubscriptionStateHandlerFactory; this.classLoader = classLoader; @@ -413,7 +417,7 @@ public void executeStage(final ExecutionDetails setup) throws IOException { } JobMasterService jobMasterService = new JobMasterService(rw.getJobId(), rw.getSchedulingInfo(), - workerMetricsClient, autoScaleMetricsConfig, mantisMasterApi, rw.getContext(), rw.getOnCompleteCallback(), rw.getOnErrorCallback(), rw.getOnTerminateCallback()); + workerMetricsClient, autoScaleMetricsConfig, mantisMasterApi, failoverStatusClient, rw.getContext(), rw.getOnCompleteCallback(), rw.getOnErrorCallback(), rw.getOnTerminateCallback()); jobMasterService.start(); closeables.add(jobMasterService::shutdown); diff --git a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/AutoScaleMetricsConfig.java b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/AutoScaleMetricsConfig.java index b03638a8b..cbfccbabd 100644 --- a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/AutoScaleMetricsConfig.java +++ b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/AutoScaleMetricsConfig.java @@ -142,8 +142,7 @@ public Map> getUserDefinedMetrics() { final Map> metrics = new HashMap<>(); for (Map.Entry> entry : userDefinedAutoScaleMetrics.entrySet()) { - metrics.put(entry.getKey(), - entry.getValue().keySet()); + metrics.put(entry.getKey(), entry.getValue().keySet()); } return metrics; diff --git a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoScaler.java b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoScaler.java index 06e536a6b..608931e26 100644 --- a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoScaler.java +++ b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoScaler.java @@ -26,6 +26,7 @@ import io.mantisrx.runtime.descriptor.StageScalingPolicy.ScalingReason; import io.mantisrx.runtime.descriptor.StageSchedulingInfo; import io.mantisrx.server.core.stats.UsageDataStats; +import io.mantisrx.server.master.FailoverStatusClient; import io.mantisrx.server.master.client.MantisMasterGateway; import io.mantisrx.server.worker.jobmaster.clutch.ClutchAutoScaler; import io.mantisrx.server.worker.jobmaster.clutch.ClutchConfiguration; @@ -90,14 +91,16 @@ public class JobAutoScaler { private final SchedulingInfo schedulingInfo; private final PublishSubject subject; private final Context context; + private final FailoverStatusClient failoverStatusClient; JobAutoScaler(String jobId, SchedulingInfo schedulingInfo, MantisMasterGateway masterClientApi, - Context context) { + Context context, FailoverStatusClient failoverStatusClient) { this.jobId = jobId; this.masterClientApi = masterClientApi; this.schedulingInfo = schedulingInfo; subject = PublishSubject.create(); this.context = context; + this.failoverStatusClient = failoverStatusClient; } Observer getObserver() { @@ -115,7 +118,7 @@ void start() { logger.info("onOverflow triggered, dropping old events"); }, BackpressureOverflow.ON_OVERFLOW_DROP_OLDEST) .doOnRequest(x -> logger.info("Scaler requested {} metrics.", x)) - .groupBy(event -> event.getStage()) + .groupBy(Event::getStage) .flatMap(go -> { Integer stage = Optional.ofNullable(go.getKey()).orElse(-1); @@ -140,12 +143,13 @@ void start() { boolean useClutchRps = false; boolean useClutchExperimental = false; + final StageScalingPolicy scalingPolicy = stageSchedulingInfo.getScalingPolicy(); // Determine which type of scaler to use. - if (stageSchedulingInfo.getScalingPolicy() != null) { - minSize = stageSchedulingInfo.getScalingPolicy().getMin(); - maxSize = stageSchedulingInfo.getScalingPolicy().getMax(); - if (stageSchedulingInfo.getScalingPolicy().getStrategies() != null) { - Set reasons = stageSchedulingInfo.getScalingPolicy().getStrategies() + if (scalingPolicy != null) { + minSize = scalingPolicy.getMin(); + maxSize = scalingPolicy.getMax(); + if (scalingPolicy.getStrategies() != null) { + Set reasons = scalingPolicy.getStrategies() .values() .stream() .map(StageScalingPolicy.Strategy::getReason) @@ -187,13 +191,15 @@ void start() { MantisStageActuator actuator = new MantisStageActuator(initialSize, scaler); Observable.Transformer transformToClutchEvent = - obs -> obs.map(event -> this.mantisEventToClutchEvent(event)) + obs -> obs.map(this::mantisEventToClutchEvent) .filter(event -> event.metric != null); Observable workerCounts = context.getWorkerMapObservable() .map(x -> x.getWorkersForStage(go.getKey()).size()) .distinctUntilChanged() .throttleLast(5, TimeUnit.SECONDS); + //TODO(hmitnflx): Add a failoveraware transformer here that consumes all other events as well + // to make scale up decisions // Create the scaler. if (useClutchRps) { logger.info("Using clutch rps scaler, job: {}, stage: {} ", jobId, stage); @@ -246,13 +252,11 @@ void start() { } }) .doOnCompleted(() -> logger.info("onComplete on JobAutoScaler subject")) - .doOnError(t -> logger.error("got onError in JobAutoScaler", t)) - .doOnSubscribe(() -> logger.info("onSubscribe JobAutoScaler")) - .doOnUnsubscribe(() -> { - logger.info("Unsubscribing for JobAutoScaler of job " + jobId); - }) + .doOnError(t -> logger.error("got onError in JobAutoScaler", t)) + .doOnSubscribe(() -> logger.info("onSubscribe JobAutoScaler")) + .doOnUnsubscribe(() -> logger.info("Unsubscribing for JobAutoScaler of job {}", jobId)) .retry() - .subscribe(); + .subscribe(); } /** @@ -329,7 +333,7 @@ private void setOutstandingScalingRequest(final Subscription subscription) { public int getDesiredWorkersForScaleUp(final int increment, final int numCurrentWorkers) { final int desiredWorkers; if (!stageSchedulingInfo.getScalingPolicy().isEnabled()) { - logger.warn("Job " + jobId + " stage " + stage + " is not scalable, can't increment #workers by " + increment); + logger.warn("Job {} stage {} is not scalable, can't increment #workers by {}", jobId, stage, increment); return numCurrentWorkers; } if (numCurrentWorkers < 0 || increment < 1) { @@ -357,15 +361,19 @@ public void scaleUpStage(final int numCurrentWorkers, final int desiredWorkers, public int getDesiredWorkersForScaleDown(final int decrement, final int numCurrentWorkers) { final int desiredWorkers; - if (!stageSchedulingInfo.getScalingPolicy().isEnabled()) { - logger.warn("Job " + jobId + " stage " + stage + " is not scalable, can't decrement #workers by " + decrement); + final StageScalingPolicy scalingPolicy = stageSchedulingInfo.getScalingPolicy(); + if (!scalingPolicy.isEnabled()) { + logger.warn("Job {} stage {} is not scalable, can't decrement #workers by {}", jobId, stage, decrement); return numCurrentWorkers; } if (numCurrentWorkers < 0 || decrement < 1) { logger.error("current number of workers({}) not known or decrement({}) < 1, will not scale down", numCurrentWorkers, decrement); return numCurrentWorkers; + } else if (!scalingPolicy.isAllowScaleDownDuringEvacuated() && failoverStatusClient.getFailoverStatus() == FailoverStatusClient.FailoverStatus.REGION_EVACUEE) { + logger.warn("Region is evacuated, not scaling down stage {} of job {}", stage, jobId); + return numCurrentWorkers; } else { - int min = stageSchedulingInfo.getScalingPolicy().getMin(); + int min = scalingPolicy.getMin(); desiredWorkers = Math.max(numCurrentWorkers - decrement, min); } return desiredWorkers; @@ -374,6 +382,11 @@ public int getDesiredWorkersForScaleDown(final int decrement, final int numCurre public void scaleDownStage(final int numCurrentWorkers, final int desiredWorkers, final String reason) { logger.info("scaleDownStage decrementing number of workers from {} to {}", numCurrentWorkers, desiredWorkers); cancelOutstandingScalingRequest(); + final StageScalingPolicy scalingPolicy = stageSchedulingInfo.getScalingPolicy(); + if (scalingPolicy != null && !scalingPolicy.isAllowScaleDownDuringEvacuated() && failoverStatusClient.getFailoverStatus() == FailoverStatusClient.FailoverStatus.REGION_EVACUEE) { + logger.warn("Region is evacuated, not scaling down stage {} of job {}", stage, jobId); + return; + } final Subscription subscription = masterClientApi.scaleJobStage(jobId, stage, desiredWorkers, reason) .retryWhen(retryLogic) .onErrorResumeNext(throwable -> { @@ -426,8 +439,8 @@ public void onNext(Event event) { final StageScalingPolicy scalingPolicy = stageSchedulingInfo.getScalingPolicy(); long coolDownSecs = scalingPolicy == null ? Long.MAX_VALUE : scalingPolicy.getCoolDownSecs(); boolean scalable = stageSchedulingInfo.getScalable() && scalingPolicy != null && scalingPolicy.isEnabled(); - logger.debug("Will check for autoscaling job " + jobId + " stage " + stage + " due to event: " + event); - if (scalable && scalingPolicy != null) { + logger.debug("Will check for autoscaling job {} stage {} due to event: {}", jobId, stage, event); + if (scalable) { final StageScalingPolicy.Strategy strategy = scalingPolicy.getStrategies().get(event.getType()); if (strategy != null) { double effectiveValue = event.getEffectiveValue(); diff --git a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobMasterService.java b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobMasterService.java index 61b23b08f..91195f625 100644 --- a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobMasterService.java +++ b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobMasterService.java @@ -25,6 +25,7 @@ import io.mantisrx.runtime.parameter.SourceJobParameters; import io.mantisrx.server.core.Service; import io.mantisrx.server.core.stats.MetricStringConstants; +import io.mantisrx.server.master.FailoverStatusClient; import io.mantisrx.server.master.client.MantisMasterGateway; import io.mantisrx.server.worker.client.WorkerMetricsClient; import io.mantisrx.shaded.com.fasterxml.jackson.core.JsonProcessingException; @@ -65,6 +66,7 @@ public JobMasterService(final String jobId, final WorkerMetricsClient workerMetricsClient, final AutoScaleMetricsConfig autoScaleMetricsConfig, final MantisMasterGateway masterClientApi, + final FailoverStatusClient failoverStatusClient, final Context context, final Action0 observableOnCompleteCallback, final Action1 observableOnErrorCallback, @@ -73,8 +75,8 @@ public JobMasterService(final String jobId, this.workerMetricsClient = workerMetricsClient; this.autoScaleMetricsConfig = autoScaleMetricsConfig; this.masterClientApi = masterClientApi; - this.jobAutoScaler = new JobAutoScaler(jobId, schedInfo, masterClientApi, context); - this.metricObserver = new WorkerMetricHandler(jobId, jobAutoScaler.getObserver(), masterClientApi, autoScaleMetricsConfig).initAndGetMetricDataObserver(); + this.jobAutoScaler = new JobAutoScaler(jobId, schedInfo, masterClientApi, context, failoverStatusClient); + this.metricObserver = new WorkerMetricHandler(jobId, jobAutoScaler.getObserver(), masterClientApi, autoScaleMetricsConfig, failoverStatusClient).initAndGetMetricDataObserver(); this.observableOnCompleteCallback = observableOnCompleteCallback; this.observableOnErrorCallback = observableOnErrorCallback; this.observableOnTerminateCallback = observableOnTerminateCallback; @@ -97,7 +99,7 @@ private Measurements handleMetricEvent(final String ev) { stage = 1; if (gauges.isEmpty()) { gauges = measurements.getCounters().stream().map(counter -> - new GaugeMeasurement(counter.getEvent(), counter.getCount())).collect(Collectors.toList()); + new GaugeMeasurement(counter.getEvent(), counter.getCount())).collect(Collectors.toList()); } } metricObserver.onNext(new MetricData(jobId, stage, workerIdx, workerNum, measurements.getName(), gauges)); @@ -122,33 +124,33 @@ public void start() { Observable> metrics = workerMetricSubscription.getMetricsClient().getResults(); boolean isSourceJobMetricEnabled = (boolean) context.getParameters().get( - SystemParameters.JOB_MASTER_AUTOSCALE_SOURCEJOB_METRIC_PARAM, false); + SystemParameters.JOB_MASTER_AUTOSCALE_SOURCEJOB_METRIC_PARAM, false); if (isSourceJobMetricEnabled) { metrics = metrics.mergeWith(getSourceJobMetrics()); } subscription = Observable.merge(metrics) - .map(event -> handleMetricEvent(event.getEventAsString())) - .doOnTerminate(observableOnTerminateCallback) - .doOnCompleted(observableOnCompleteCallback) - .doOnError(observableOnErrorCallback) - .subscribe(); + .map(event -> handleMetricEvent(event.getEventAsString())) + .doOnTerminate(observableOnTerminateCallback) + .doOnCompleted(observableOnCompleteCallback) + .doOnError(observableOnErrorCallback) + .subscribe(); } protected Observable> getSourceJobMetrics() { List targetInfos = SourceJobParameters.parseTargetInfo( - (String) context.getParameters().get(SystemParameters.JOB_MASTER_AUTOSCALE_SOURCEJOB_TARGET_PARAM, "{}")); + (String) context.getParameters().get(SystemParameters.JOB_MASTER_AUTOSCALE_SOURCEJOB_TARGET_PARAM, "{}")); if (targetInfos.isEmpty()) { targetInfos = SourceJobParameters.parseInputParameters(context); } targetInfos = SourceJobParameters.enforceClientIdConsistency(targetInfos, jobId); String additionalDropMetricPatterns = - (String) context.getParameters().get(SystemParameters.JOB_MASTER_AUTOSCALE_SOURCEJOB_DROP_METRIC_PATTERNS_PARAM, ""); + (String) context.getParameters().get(SystemParameters.JOB_MASTER_AUTOSCALE_SOURCEJOB_DROP_METRIC_PATTERNS_PARAM, ""); autoScaleMetricsConfig.addSourceJobDropMetrics(additionalDropMetricPatterns); SourceJobWorkerMetricsSubscription sourceSub = new SourceJobWorkerMetricsSubscription( - targetInfos, masterClientApi, workerMetricsClient, autoScaleMetricsConfig); + targetInfos, masterClientApi, workerMetricsClient, autoScaleMetricsConfig); return sourceSub.getResults(); } diff --git a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandler.java b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandler.java index b9bf9bfea..2c1b1e31f 100644 --- a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandler.java +++ b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandler.java @@ -21,6 +21,7 @@ import io.mantisrx.runtime.descriptor.StageScalingPolicy; import io.mantisrx.server.core.*; import io.mantisrx.server.core.stats.MetricStringConstants; +import io.mantisrx.server.master.FailoverStatusClient; import io.mantisrx.server.master.client.MantisMasterGateway; import io.mantisrx.shaded.com.google.common.cache.Cache; import io.mantisrx.shaded.com.google.common.cache.CacheBuilder; @@ -71,16 +72,19 @@ return -1; } }; + private final FailoverStatusClient failoverStatusClient; public WorkerMetricHandler(final String jobId, final Observer jobAutoScaleObserver, final MantisMasterGateway masterClientApi, - final AutoScaleMetricsConfig autoScaleMetricsConfig) { + final AutoScaleMetricsConfig autoScaleMetricsConfig, + final FailoverStatusClient failoverStatusClient) { this.jobId = jobId; this.jobAutoScaleObserver = jobAutoScaleObserver; this.masterClientApi = masterClientApi; this.autoScaleMetricsConfig = autoScaleMetricsConfig; this.metricAggregator = new MetricAggregator(autoScaleMetricsConfig); + this.failoverStatusClient = failoverStatusClient; } public Observer initAndGetMetricDataObserver() { @@ -94,10 +98,9 @@ private Map getAggregates(List> dataPo for (Map datapoint : dataPointsList) { for (Map.Entry gauge : datapoint.entrySet()) { - if (!transformed.containsKey(gauge.getKey())) { - transformed.put(gauge.getKey(), new ArrayList<>()); - } - transformed.get(gauge.getKey()).add(gauge.getValue()); + transformed + .computeIfAbsent(gauge.getKey(), (k) -> new ArrayList<>()) + .add(gauge.getValue()); } } @@ -276,8 +279,8 @@ public void call() { final int numWorkers = numStageWorkersFn.call(stage); // get the aggregate metric values by metric group for all workers in stage Map allWorkerAggregates = getAggregates(listofAggregates); - logger.info("Job stage " + stage + " avgResUsage from " + - workersMap.size() + " workers: " + allWorkerAggregates.toString()); + logger.info("Job stage {} avgResUsage from {} workers: {}", stage, workersMap.size(), allWorkerAggregates.toString()); + // TODO(hmitnflx): Add a failover mode based event into jobAutoScaleObserver for (Map.Entry> userDefinedMetric : autoScaleMetricsConfig.getUserDefinedMetrics().entrySet()) { final String metricGrp = userDefinedMetric.getKey(); @@ -387,33 +390,7 @@ public void call() { } } - double sourceJobDrops = 0; - boolean hasSourceJobDropsMetric = false; - Map sourceMetricsRecent = sourceJobMetricsRecent.asMap(); - for (Map.Entry worker : sourceJobWorkersMap.entrySet()) { - Map metricGroups = metricAggregator.getAggregates(worker.getValue().getGaugesByMetricGrp()); - for (Map.Entry group : metricGroups.entrySet()) { - String metricKey = worker.getKey() + ":" + group.getKey(); - for (Map.Entry gauge : group.getValue().getGauges().entrySet()) { - if (sourceMetricsRecent.containsKey(metricKey) && - autoScaleMetricsConfig.isSourceJobDropMetric(group.getKey(), gauge.getKey())) { - sourceJobDrops += gauge.getValue(); - hasSourceJobDropsMetric = true; - } - } - } - } - if (hasSourceJobDropsMetric) { - logger.info("Job stage {}, source job drop metrics: {}", stage, sourceJobDrops); - // Divide by 6 to account for 6 second reset by Atlas on counter metric. - jobAutoScaleObserver.onNext( - new JobAutoScaler.Event( - StageScalingPolicy.ScalingReason.SourceJobDrop, - stage, - sourceJobDrops / 6.0 / numWorkers, - sourceJobDrops / 6.0 / numWorkers, - numWorkers)); - } + addScalerEventForSourceJobDrops(numWorkers); } }, metricsIntervalSeconds, metricsIntervalSeconds, TimeUnit.SECONDS )); @@ -440,6 +417,36 @@ public void onNext(MetricData metricData) { } }; } + + private void addScalerEventForSourceJobDrops(int numWorkers) { + double sourceJobDrops = 0; + boolean hasSourceJobDropsMetric = false; + Map sourceMetricsRecent = sourceJobMetricsRecent.asMap(); + for (Map.Entry worker : sourceJobWorkersMap.entrySet()) { + Map metricGroups = metricAggregator.getAggregates(worker.getValue().getGaugesByMetricGrp()); + for (Map.Entry group : metricGroups.entrySet()) { + String metricKey = worker.getKey() + ":" + group.getKey(); + for (Map.Entry gauge : group.getValue().getGauges().entrySet()) { + if (sourceMetricsRecent.containsKey(metricKey) && + autoScaleMetricsConfig.isSourceJobDropMetric(group.getKey(), gauge.getKey())) { + sourceJobDrops += gauge.getValue(); + hasSourceJobDropsMetric = true; + } + } + } + } + if (hasSourceJobDropsMetric) { + logger.info("Job stage {}, source job drop metrics: {}", stage, sourceJobDrops); + // Divide by 6 to account for 6 second reset by Atlas on counter metric. + jobAutoScaleObserver.onNext( + new JobAutoScaler.Event( + StageScalingPolicy.ScalingReason.SourceJobDrop, + stage, + sourceJobDrops / 6.0 / numWorkers, + sourceJobDrops / 6.0 / numWorkers, + numWorkers)); + } + } } private void start() { diff --git a/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/JobAutoScalerTest.java b/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/JobAutoScalerTest.java index 64c9519f4..2bd9c1a85 100644 --- a/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/JobAutoScalerTest.java +++ b/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/JobAutoScalerTest.java @@ -30,6 +30,7 @@ import io.mantisrx.runtime.descriptor.SchedulingInfo; import io.mantisrx.runtime.descriptor.StageScalingPolicy; import io.mantisrx.runtime.descriptor.StageSchedulingInfo; +import io.mantisrx.server.master.FailoverStatusClient; import io.mantisrx.server.master.client.MantisMasterClientApi; import io.mantisrx.server.worker.jobmaster.clutch.ClutchConfiguration; import io.mantisrx.server.worker.jobmaster.clutch.rps.ClutchRpsPIDConfig; @@ -78,7 +79,7 @@ public void testScaleUp() throws InterruptedException { .machineDefinition(new MachineDefinition(2, workerMemoryMB, 200, 1024, 2)) .scalingPolicy(new StageScalingPolicy(scalingStageNum, min, max, increment, decrement, coolDownSec, Collections.singletonMap(StageScalingPolicy.ScalingReason.Memory, - new StageScalingPolicy.Strategy(StageScalingPolicy.ScalingReason.Memory, scaleDownBelowPct, scaleUpAbovePct, new StageScalingPolicy.RollingCount(1, 2))))) + new StageScalingPolicy.Strategy(StageScalingPolicy.ScalingReason.Memory, scaleDownBelowPct, scaleUpAbovePct, new StageScalingPolicy.RollingCount(1, 2))), true)) .scalable(true) .build(); @@ -89,7 +90,7 @@ public void testScaleUp() throws InterruptedException { Context context = mock(Context.class); when(context.getWorkerMapObservable()).thenReturn(Observable.empty()); - final JobAutoScaler jobAutoScaler = new JobAutoScaler(jobId, new SchedulingInfo(schedulingInfoMap), mockMasterClientApi, context); + final JobAutoScaler jobAutoScaler = new JobAutoScaler(jobId, new SchedulingInfo(schedulingInfoMap), mockMasterClientApi, context, FailoverStatusClient.DEFAULT); jobAutoScaler.start(); final Observer jobAutoScalerObserver = jobAutoScaler.getObserver(); @@ -143,8 +144,12 @@ public void testScalingResiliency() throws InterruptedException { .numberOfInstances(numStage1Workers) .machineDefinition(new MachineDefinition(2, workerMemoryMB, 200, 1024, 2)) .scalingPolicy(new StageScalingPolicy(scalingStageNum, min, max, increment, decrement, coolDownSec, - Collections.singletonMap(StageScalingPolicy.ScalingReason.Memory, - new StageScalingPolicy.Strategy(StageScalingPolicy.ScalingReason.Memory, scaleDownBelowPct, scaleUpAbovePct, new StageScalingPolicy.RollingCount(1, 2))))) + new HashMap() { + { + put(StageScalingPolicy.ScalingReason.Memory, new StageScalingPolicy.Strategy(StageScalingPolicy.ScalingReason.Memory, scaleDownBelowPct, scaleUpAbovePct, new StageScalingPolicy.RollingCount(1, 2))); + put(StageScalingPolicy.ScalingReason.FailoverAware, new StageScalingPolicy.Strategy(StageScalingPolicy.ScalingReason.FailoverAware, 0.0, 0.0, new StageScalingPolicy.RollingCount(1, 2))); + } + }, true)) .scalable(true) .build(); @@ -168,7 +173,7 @@ public Boolean call(Integer integer) { Context context = mock(Context.class); when(context.getWorkerMapObservable()).thenReturn(Observable.empty()); - final JobAutoScaler jobAutoScaler = new JobAutoScaler(jobId, new SchedulingInfo(schedulingInfoMap), mockMasterClientApi, context); + final JobAutoScaler jobAutoScaler = new JobAutoScaler(jobId, new SchedulingInfo(schedulingInfoMap), mockMasterClientApi, context, FailoverStatusClient.DEFAULT); jobAutoScaler.start(); final Observer jobAutoScalerObserver = jobAutoScaler.getObserver(); @@ -201,7 +206,7 @@ public void testScaleDown() throws InterruptedException { .machineDefinition(new MachineDefinition(2, workerMemoryMB, 200, 1024, 2)) .scalingPolicy(new StageScalingPolicy(scalingStageNum, min, max, increment, decrement, coolDownSec, Collections.singletonMap(StageScalingPolicy.ScalingReason.Memory, - new StageScalingPolicy.Strategy(StageScalingPolicy.ScalingReason.Memory, scaleDownBelowPct, scaleUpAbovePct, new StageScalingPolicy.RollingCount(1, 2))))) + new StageScalingPolicy.Strategy(StageScalingPolicy.ScalingReason.Memory, scaleDownBelowPct, scaleUpAbovePct, new StageScalingPolicy.RollingCount(1, 2))), true)) .scalable(true) .build(); @@ -212,7 +217,7 @@ public void testScaleDown() throws InterruptedException { Context context = mock(Context.class); when(context.getWorkerMapObservable()).thenReturn(Observable.empty()); - final JobAutoScaler jobAutoScaler = new JobAutoScaler(jobId, new SchedulingInfo(schedulingInfoMap), mockMasterClientApi, context); + final JobAutoScaler jobAutoScaler = new JobAutoScaler(jobId, new SchedulingInfo(schedulingInfoMap), mockMasterClientApi, context, FailoverStatusClient.DEFAULT); jobAutoScaler.start(); final Observer jobAutoScalerObserver = jobAutoScaler.getObserver(); @@ -255,7 +260,7 @@ public void testScaleDownNotLessThanMin() throws InterruptedException { .numberOfInstances(numStage1Workers).machineDefinition(new MachineDefinition(2, workerMemoryMB, 200, 1024, 2)) .scalingPolicy(new StageScalingPolicy(scalingStageNum, min, max, increment, decrement, coolDownSec, Collections.singletonMap(StageScalingPolicy.ScalingReason.Memory, - new StageScalingPolicy.Strategy(StageScalingPolicy.ScalingReason.Memory, scaleDownBelowPct, scaleUpAbovePct, new StageScalingPolicy.RollingCount(1, 2))))) + new StageScalingPolicy.Strategy(StageScalingPolicy.ScalingReason.Memory, scaleDownBelowPct, scaleUpAbovePct, new StageScalingPolicy.RollingCount(1, 2))), true)) .scalable(true) .build(); @@ -266,7 +271,7 @@ public void testScaleDownNotLessThanMin() throws InterruptedException { Context context = mock(Context.class); when(context.getWorkerMapObservable()).thenReturn(Observable.empty()); - final JobAutoScaler jobAutoScaler = new JobAutoScaler(jobId, new SchedulingInfo(schedulingInfoMap), mockMasterClientApi, context); + final JobAutoScaler jobAutoScaler = new JobAutoScaler(jobId, new SchedulingInfo(schedulingInfoMap), mockMasterClientApi, context, FailoverStatusClient.DEFAULT); jobAutoScaler.start(); final Observer jobAutoScalerObserver = jobAutoScaler.getObserver(); @@ -301,7 +306,7 @@ public void testScaleUpOnDifferentScalingReasons() throws InterruptedException { .machineDefinition(new MachineDefinition(2, workerMemoryMB, 200, 1024, 2)) .scalingPolicy(new StageScalingPolicy(scalingStageNum, min, max, increment, decrement, coolDownSec, Collections.singletonMap(scalingReason, - new StageScalingPolicy.Strategy(scalingReason, scaleDownBelow, scaleUpAbove, new StageScalingPolicy.RollingCount(1, 2))))) + new StageScalingPolicy.Strategy(scalingReason, scaleDownBelow, scaleUpAbove, new StageScalingPolicy.RollingCount(1, 2))), true)) .scalable(true) .build(); @@ -312,7 +317,7 @@ public void testScaleUpOnDifferentScalingReasons() throws InterruptedException { Context context = mock(Context.class); when(context.getWorkerMapObservable()).thenReturn(Observable.empty()); - final JobAutoScaler jobAutoScaler = new JobAutoScaler(jobId, new SchedulingInfo(schedulingInfoMap), mockMasterClientApi, context); + final JobAutoScaler jobAutoScaler = new JobAutoScaler(jobId, new SchedulingInfo(schedulingInfoMap), mockMasterClientApi, context, FailoverStatusClient.DEFAULT); jobAutoScaler.start(); final Observer jobAutoScalerObserver = jobAutoScaler.getObserver(); @@ -357,7 +362,7 @@ public void testGetClutchConfigurationFromJson() throws Exception { " \"scaleUpMultiplier\": 1.5" + " }" + "}"; - final JobAutoScaler jobAutoScaler = new JobAutoScaler("jobId", null, null, null); + final JobAutoScaler jobAutoScaler = new JobAutoScaler("jobId", null, null, null, FailoverStatusClient.DEFAULT); ClutchConfiguration config = jobAutoScaler.getClutchConfiguration(json).get(1); ClutchRpsPIDConfig expected = new ClutchRpsPIDConfig(0.0, Tuple.of(30.0, 0.0), 0.0, 0.0, Option.of(75.0), Option.of(30.0), Option.of(0.0), Option.of(1.5), Option.of(1.0)); diff --git a/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandlerTest.java b/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandlerTest.java index cec9ba223..e629bc353 100644 --- a/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandlerTest.java +++ b/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandlerTest.java @@ -34,6 +34,7 @@ import io.mantisrx.server.core.WorkerAssignments; import io.mantisrx.server.core.WorkerHost; import io.mantisrx.server.core.stats.MetricStringConstants; +import io.mantisrx.server.master.FailoverStatusClient; import io.mantisrx.server.master.client.MantisMasterClientApi; import io.mantisrx.shaded.com.google.common.collect.ImmutableMap; import java.util.Arrays; @@ -100,7 +101,7 @@ public void onNext(JobAutoScaler.Event event) { assertEquals(expected, event); latch.countDown(); } - }, mockMasterClientApi, aggregationConfig); + }, mockMasterClientApi, aggregationConfig, FailoverStatusClient.DEFAULT); final Observer metricDataObserver = workerMetricHandler.initAndGetMetricDataObserver(); @@ -174,7 +175,7 @@ public void onNext(JobAutoScaler.Event event) { latch.countDown(); } } - }, mockMasterClientApi, aggregationConfig); + }, mockMasterClientApi, aggregationConfig, FailoverStatusClient.DEFAULT); final Observer metricDataObserver = workerMetricHandler.initAndGetMetricDataObserver(); @@ -266,7 +267,7 @@ public void onNext(JobAutoScaler.Event event) { assertEquals(expected, event); autoScaleLatch.countDown(); } - }, mockMasterClientApi, aggregationConfig); + }, mockMasterClientApi, aggregationConfig, FailoverStatusClient.DEFAULT); final Observer metricDataObserver = workerMetricHandler.initAndGetMetricDataObserver(); @@ -322,7 +323,7 @@ public void onNext(JobAutoScaler.Event event) { latch.countDown(); } } - }, mockMasterClientApi, aggregationConfig); + }, mockMasterClientApi, aggregationConfig, FailoverStatusClient.DEFAULT); final Observer metricDataObserver = workerMetricHandler.initAndGetMetricDataObserver(); diff --git a/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/clutch/rps/RpsClutchConfigurationSelectorTest.java b/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/clutch/rps/RpsClutchConfigurationSelectorTest.java index 4bff90487..adebda650 100644 --- a/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/clutch/rps/RpsClutchConfigurationSelectorTest.java +++ b/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/clutch/rps/RpsClutchConfigurationSelectorTest.java @@ -70,7 +70,7 @@ public void testScalingPolicyFallback() { rpsSketch.update(100); Map sketches = ImmutableMap.of(Clutch.Metric.RPS, rpsSketch); - StageScalingPolicy scalingPolicy = new StageScalingPolicy(1, 2, 9, 0, 0, 400L, null); + StageScalingPolicy scalingPolicy = new StageScalingPolicy(1, 2, 9, 0, 0, 400L, null, true); StageSchedulingInfo schedulingInfo = StageSchedulingInfo.builder() .numberOfInstances(3) @@ -99,7 +99,7 @@ public void testSetPointQuantile() { } Map sketches = ImmutableMap.of(Clutch.Metric.RPS, rpsSketch); - StageScalingPolicy scalingPolicy = new StageScalingPolicy(1, 2, 9, 0, 0, 400L, null); + StageScalingPolicy scalingPolicy = new StageScalingPolicy(1, 2, 9, 0, 0, 400L, null, true); StageSchedulingInfo schedulingInfo = StageSchedulingInfo.builder() .numberOfInstances(3) @@ -123,7 +123,7 @@ public void testReturnSameConfigIfSetPointWithin5Percent() { } Map sketches = ImmutableMap.of(Clutch.Metric.RPS, rpsSketch); - StageScalingPolicy scalingPolicy = new StageScalingPolicy(1, 2, 9, 0, 0, 400L, null); + StageScalingPolicy scalingPolicy = new StageScalingPolicy(1, 2, 9, 0, 0, 400L, null, true); StageSchedulingInfo schedulingInfo = StageSchedulingInfo.builder() .numberOfInstances(3) @@ -162,7 +162,7 @@ public void testSetPointDriftAdjust() { } Map sketches = ImmutableMap.of(Clutch.Metric.RPS, rpsSketch); - StageScalingPolicy scalingPolicy = new StageScalingPolicy(1, 2, 9, 0, 0, 400L, null); + StageScalingPolicy scalingPolicy = new StageScalingPolicy(1, 2, 9, 0, 0, 400L, null, true); StageSchedulingInfo schedulingInfo = StageSchedulingInfo.builder() .numberOfInstances(3) From 570c87ef2494af9a5f9e7b64f1485379aca9a5ff Mon Sep 17 00:00:00 2001 From: Harshit Mittal Date: Mon, 20 May 2024 23:25:45 -0700 Subject: [PATCH 2/8] add docs --- docs/docs/operate/autoscalingstrategies.md | 85 +++++++++++++++------- 1 file changed, 59 insertions(+), 26 deletions(-) diff --git a/docs/docs/operate/autoscalingstrategies.md b/docs/docs/operate/autoscalingstrategies.md index ff4d9aa7b..7d98d4d37 100644 --- a/docs/docs/operate/autoscalingstrategies.md +++ b/docs/docs/operate/autoscalingstrategies.md @@ -3,25 +3,42 @@ many workers to scale up/down a stage. They fall into 2 main categories. Rule ba and scale up/down when a certain threshold is reached. PID control based strategies pick a resource utilization level and scale up/down dynamically to maintain that level. -## Rule Based Strategy +A stage's autoscaling policy is composed of following configurations - + +| Name | Type | Description | +|-------------------------|-------------------------|--------------------------------------------------------------------------------------------------------------------------| +| `stage` | `int` | Index of the stage (`1`-based since `0` is reserved for job master). | +| `min` | `int` | Minimum number of workers in the stage. | +| `max` | `int` | Maximum number of workers in the stage. | +| `increment`[1] | `int` | Number of workers to add when scaling up. | +| `decrement`[1] | `int` | Number of workers to remove when scaling down. | +| `coolDownSecs` | `int` | Number of seconds to wait after a scaling operation has been completed before beginning another scaling operation. | +| `strategies` | `Map` | List of strategies to use for autoscaling and their configurations. | +| `allowAutoScaleManager` | `boolean` | Toggles an autoscaler manager that (dis)allow scale up and scale downs when certain criteria is met. Default is `false`. | + +Note: +[1] increment/decrement are not used for Clutch strategies. Clutch strategies use PID controller to determine the number of workers to scale up/down. + +## Rule Based Strategies Rule based strategy can be defined for the following resources: -| Resource | Metric | -| --------------- | ------- | -| `CPU` | group: `ResourceUsage` name: `cpuPctUsageCurr` aggregation: `AVG` | -| `Memory` | group: `ResourceUsage` name: `totMemUsageCurr` aggregation: `AVG` | -| `Network` | group: `ResourceUsage` name: `nwBytesUsageCurr` aggregation: `AVG` | -| `JVMMemory` | group: `ResourceUsage` name: `jvmMemoryUsedBytes` aggregation: `AVG` | -| `DataDrop` | group: `DataDrop` name: `dropCount` aggregation: `AVG` | -| `KafkaLag` | group: `consumer-fetch-manager-metrics` name: `records-lag-max` aggregation: `MAX` | -| `KafkaProcessed` | group: `consumer-fetch-manager-metrics` name: `records-consumed-rate` aggregation: `AVG` | -| `UserDefined` | Metric is defined by user with job parameter `mantis.jobmaster.autoscale.metric` in this format `{group}::{name}::{aggregation}`. | +| Resource | Metric | +|------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------| +| `CPU` | group: `ResourceUsage` name: `cpuPctUsageCurr` aggregation: `AVG` | +| `Memory` | group: `ResourceUsage` name: `totMemUsageCurr` aggregation: `AVG` | +| `Network` | group: `ResourceUsage` name: `nwBytesUsageCurr` aggregation: `AVG` | +| `JVMMemory` | group: `ResourceUsage` name: `jvmMemoryUsedBytes` aggregation: `AVG` | +| `DataDrop` | group: `DataDrop` name: `dropCount` aggregation: `AVG` | +| `KafkaLag` | group: `consumer-fetch-manager-metrics` name: `records-lag-max` aggregation: `MAX` | +| `KafkaProcessed` | group: `consumer-fetch-manager-metrics` name: `records-consumed-rate` aggregation: `AVG` | +| `UserDefined` | Metric is defined by user with job parameter `mantis.jobmaster.autoscale.metric` in this format `{group}::{name}::{aggregation}`. | +| `FailoverAware` | Metric is defined by an implementation of `FailoverStatusClient`. This is used to control autoscaling during failover events. (See #failover-aware) | Each strategy has the following parameters: -| Name | Description | -| --------------- | ------------ | +| Name | Description | +|-------------------------------| ------------ | | `Scale down below percentage` | When the aggregated value for all workers falls below this value, the stage will scale down. It will scale down by the decrement value specified in the policy. For data drop, this is calculated as the number of data items dropped divided by the total number of data items, dropped+processed. For CPU, Memory, etc., it is calculated as a percentage of allocated resource when you defined the worker. | | `Scale up above percentage` | When the aggregated value for all workers rises above this value, the stage will scale up. | | `Rolling count` | This value helps to keep jitter out of the autoscaling process. Instead of scaling immediately the first time values fall outside of the scale-down and scale-up percentage thresholds you define, Mantis will wait until the thresholds are exceeded a certain number of times within a certain window. For example, a rolling count of “6 of 10” means that only if in ten consecutive observations six or more of the observations fall below the scale-down threshold will the stage be scaled down. | @@ -36,7 +53,23 @@ scaling action, the cooldown will prevent subsequent strategies from scaling for best to use the data drop strategy in conjunction with another strategy that provides the scale-down trigger. -## PID Control Based Strategy +### Failover-Aware Strategy +During a failover event in a multi-region setup, all incoming traffic is moved away from the failed over +region—`evacuee`—to other region(s)—`savior(s)` to mitigate the issue faster. + +For autoscaling mantis jobs, it's sometimes necessary to be failover aware because: + +1. Scale down in evacuee regions — `allowScaleDownDuringEvacuated`. + After an extended period of evacuated state for the region, the jobs in that region would have been + scaled down to lowest values because of no incoming traffic and low resource utilization. + When the traffic is restored in that region, the job will see a big surge in incoming requests overwhelming + the job in that region. +2. Scale up in savior regions — Use `FailoverAware` in strategies. + When the traffic is moved to savior regions, the jobs in those regions should be scaled up to handle the + additional load. Currently, failover aware strategy will scale up the configured stages to max #workers in + configured in `StageScalingPolicy`. It also doesn't work with PID Control Based Strategy(ies). + +## PID Control Based Strategies PID control system uses a continuous feedback loop to maintain a signal at a target level (set point). Mantis offers variations of this strategy that operates on different signals. Additionally, they try to learn the appropriate target over time @@ -84,18 +117,18 @@ JSON in the job parameter `mantis.jobmaster.clutch.config`. Example: } ``` -| Field | Description | -| --------------- | ------------ | -| `minSize` | Minimum number of workers in the stage. It will not scale down below this number. | -| `maxSize` | Maximum number of workers in the stage. It will not scale up above this number. | -| `cooldownSeconds` | This indicates how many seconds to wait after a scaling operation has been completed before beginning another scaling operation. | -| `maxAdjustment` | Optional. The maximum number of workers to scale up/down in a single operation. | -| `rps` | Expected RPS per worker. Must be > 0. | -| `cpu`, `memory`, `network` | Configure PID controller for each resource. | -| `setPoint` | Target set point for the resource. This is expressed as a percentage of allocated resource to the worker. For example, `60.0` on `network` means network bytes should be 60% of the network limit on machine definition. | -| `rope` | Lower and upper buffer around the set point. Metric values within this buffer are assumed to be at set point, and thus contributes an error of 0 to the PID controller. | -| `kp` | Multiplier for the proportional term of the PID controller. This will affect the size of scaling actions. | -| `kd` | Multiplier for the derivative term of the PID controller. This will affect the size of scaling actions. | +| Field | Description | +|----------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| `minSize` | Minimum number of workers in the stage. It will not scale down below this number. | +| `maxSize` | Maximum number of workers in the stage. It will not scale up above this number. | +| `cooldownSeconds` | This indicates how many seconds to wait after a scaling operation has been completed before beginning another scaling operation. | +| `maxAdjustment` | Optional. The maximum number of workers to scale up/down in a single operation. | +| `rps` | Expected RPS per worker. Must be > 0. | +| `cpu`, `memory`, `network` | Configure PID controller for each resource. | +| `setPoint` | Target set point for the resource. This is expressed as a percentage of allocated resource to the worker. For example, `60.0` on `network` means network bytes should be 60% of the network limit on machine definition. | +| `rope` | Lower and upper buffer around the set point. Metric values within this buffer are assumed to be at set point, and thus contributes an error of 0 to the PID controller. | +| `kp` | Multiplier for the proportional term of the PID controller. This will affect the size of scaling actions. | +| `kd` | Multiplier for the derivative term of the PID controller. This will affect the size of scaling actions. | ### Clutch RPS From 2e4bc2860b5572cf91d2c40d41cc1e5447d565a7 Mon Sep 17 00:00:00 2001 From: Harshit Mittal Date: Tue, 11 Jun 2024 16:42:03 -0700 Subject: [PATCH 3/8] review comments --- .../runtime/descriptor/SchedulingInfo.java | 4 +- .../descriptor/StageScalingPolicy.java | 71 ++-------------- .../descriptor/SchedulingInfoTest.java | 14 ++-- .../descriptor/StageScalingPolicyTest.java | 12 +-- .../server/master/FailoverStatusClient.java | 56 ------------- .../server/worker/RuntimeTaskImpl.java | 8 -- ...WorkerExecutionOperationsNetworkStage.java | 23 +++-- .../worker/jobmaster/JobAutoScaler.java | 28 ++++--- .../jobmaster/JobAutoscalerManager.java | 56 +++++++++++++ .../worker/jobmaster/JobMasterService.java | 9 +- .../worker/jobmaster/WorkerMetricHandler.java | 8 +- .../worker/jobmaster/JobAutoScalerTest.java | 83 ++++++++++++++++--- .../jobmaster/WorkerMetricHandlerTest.java | 9 +- .../loader/config/WorkerConfiguration.java | 4 + .../config/WorkerConfigurationUtils.java | 1 + .../config/WorkerConfigurationWritable.java | 1 + .../mantisrx/server/agent/TaskExecutor.java | 3 +- .../server/agent/TaskExecutorStarter.java | 4 +- 18 files changed, 204 insertions(+), 190 deletions(-) delete mode 100644 mantis-control-plane/mantis-control-plane-core/src/main/java/io/mantisrx/server/master/FailoverStatusClient.java create mode 100644 mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoscalerManager.java diff --git a/mantis-common/src/main/java/io/mantisrx/runtime/descriptor/SchedulingInfo.java b/mantis-common/src/main/java/io/mantisrx/runtime/descriptor/SchedulingInfo.java index 7776b3940..c0efa2c34 100644 --- a/mantis-common/src/main/java/io/mantisrx/runtime/descriptor/SchedulingInfo.java +++ b/mantis-common/src/main/java/io/mantisrx/runtime/descriptor/SchedulingInfo.java @@ -132,7 +132,7 @@ public Builder multiWorkerScalableStageWithConstraints(int numberOfWorkers, Mach List hardConstraints, List softConstraints, StageScalingPolicy scalingPolicy) { StageScalingPolicy ssp = new StageScalingPolicy(currentStage, scalingPolicy.getMin(), scalingPolicy.getMax(), - scalingPolicy.getIncrement(), scalingPolicy.getDecrement(), scalingPolicy.getCoolDownSecs(), scalingPolicy.getStrategies(), scalingPolicy.isAllowScaleDownDuringEvacuated()); + scalingPolicy.getIncrement(), scalingPolicy.getDecrement(), scalingPolicy.getCoolDownSecs(), scalingPolicy.getStrategies(), scalingPolicy.isAllowAutoScaleManager()); return this.addStage( StageSchedulingInfo.builder() .numberOfInstances(numberOfWorkers) @@ -148,7 +148,7 @@ public Builder multiWorkerScalableStageWithConstraints(int numberOfWorkers, Mach List hardConstraints, List softConstraints, StageScalingPolicy scalingPolicy, Map containerAttributes) { StageScalingPolicy ssp = new StageScalingPolicy(currentStage, scalingPolicy.getMin(), scalingPolicy.getMax(), - scalingPolicy.getIncrement(), scalingPolicy.getDecrement(), scalingPolicy.getCoolDownSecs(), scalingPolicy.getStrategies(), scalingPolicy.isAllowScaleDownDuringEvacuated()); + scalingPolicy.getIncrement(), scalingPolicy.getDecrement(), scalingPolicy.getCoolDownSecs(), scalingPolicy.getStrategies(), scalingPolicy.isAllowAutoScaleManager()); return this.addStage( StageSchedulingInfo.builder() .numberOfInstances(numberOfWorkers) diff --git a/mantis-common/src/main/java/io/mantisrx/runtime/descriptor/StageScalingPolicy.java b/mantis-common/src/main/java/io/mantisrx/runtime/descriptor/StageScalingPolicy.java index 6f6463086..517d8b347 100644 --- a/mantis-common/src/main/java/io/mantisrx/runtime/descriptor/StageScalingPolicy.java +++ b/mantis-common/src/main/java/io/mantisrx/runtime/descriptor/StageScalingPolicy.java @@ -29,6 +29,8 @@ @Getter +@ToString +@EqualsAndHashCode public class StageScalingPolicy implements Serializable { private static final long serialVersionUID = 1L; @@ -41,9 +43,9 @@ public class StageScalingPolicy implements Serializable { private final long coolDownSecs; private final Map strategies; /** - * Controls whether scale down operations are allowed when the region is evacuated. + * Controls whether AutoScaleManager is enabled or disabled */ - private final boolean allowScaleDownDuringEvacuated; + private final boolean allowAutoScaleManager; @JsonCreator @JsonIgnoreProperties(ignoreUnknown = true) @@ -52,7 +54,7 @@ public StageScalingPolicy(@JsonProperty("stage") int stage, @JsonProperty("increment") int increment, @JsonProperty("decrement") int decrement, @JsonProperty("coolDownSecs") long coolDownSecs, @JsonProperty("strategies") Map strategies, - @JsonProperty(value = "allowScaleDownDuringEvacuated", defaultValue = "true") Boolean allowScaleDownDuringEvacuated) { + @JsonProperty(value = "allowAutoScaleManager", defaultValue = "false") Boolean allowAutoScaleManager) { this.stage = stage; this.min = min; this.max = Math.max(max, min); @@ -62,74 +64,13 @@ public StageScalingPolicy(@JsonProperty("stage") int stage, this.coolDownSecs = coolDownSecs; this.strategies = strategies == null ? new HashMap() : new HashMap<>(strategies); // `defaultValue` is for documentation purpose only, use `Boolean` to determine if the field is missing on `null` - this.allowScaleDownDuringEvacuated = allowScaleDownDuringEvacuated == null || allowScaleDownDuringEvacuated; + this.allowAutoScaleManager = allowAutoScaleManager == Boolean.TRUE; } public Map getStrategies() { return Collections.unmodifiableMap(strategies); } - @Override - public String toString() { - return "StageScalingPolicy{" + - "stage=" + stage + - ", min=" + min + - ", max=" + max + - ", enabled=" + enabled + - ", increment=" + increment + - ", decrement=" + decrement + - ", coolDownSecs=" + coolDownSecs + - ", strategies=" + strategies + - ", allowScaleDownDuringEvacuated=" + allowScaleDownDuringEvacuated + - '}'; - } - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + (int) (coolDownSecs ^ (coolDownSecs >>> 32)); - result = prime * result + decrement; - result = prime * result + (enabled ? 1231 : 1237); - result = prime * result + increment; - result = prime * result + max; - result = prime * result + min; - result = prime * result + stage; - result = prime * result + ((strategies == null) ? 0 : strategies.hashCode()); - result = prime * result + (allowScaleDownDuringEvacuated ? 1231 : 1237); - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - StageScalingPolicy other = (StageScalingPolicy) obj; - if (coolDownSecs != other.coolDownSecs) - return false; - if (decrement != other.decrement) - return false; - if (enabled != other.enabled) - return false; - if (increment != other.increment) - return false; - if (max != other.max) - return false; - if (min != other.min) - return false; - if (stage != other.stage) - return false; - if (strategies == null) { - return other.strategies == null; - } else if (!strategies.equals(other.strategies)) { - return false; - } else return allowScaleDownDuringEvacuated == other.allowScaleDownDuringEvacuated; - } - public enum ScalingReason { CPU, @Deprecated diff --git a/mantis-common/src/test/java/io/mantisrx/runtime/descriptor/SchedulingInfoTest.java b/mantis-common/src/test/java/io/mantisrx/runtime/descriptor/SchedulingInfoTest.java index cec17d8e8..5f9079a9a 100644 --- a/mantis-common/src/test/java/io/mantisrx/runtime/descriptor/SchedulingInfoTest.java +++ b/mantis-common/src/test/java/io/mantisrx/runtime/descriptor/SchedulingInfoTest.java @@ -153,7 +153,7 @@ public void testSerialization() throws Exception { 2, new MachineDefinition(1, 1.24, 0.0, 1, 1), null, null, - new StageScalingPolicy(1, 1, 3, 1, 1, 60, smap, true) + new StageScalingPolicy(1, 1, 3, 1, 1, 60, smap, false) ) .multiWorkerScalableStageWithConstraints( 3, @@ -204,7 +204,7 @@ public void testSerialization() throws Exception { " }" + " }" + " }," + - " \"allowScaleDownDuringEvacuated\": true," + + " \"allowAutoScaleManager\": false," + " \"enabled\": true" + " }," + " \"scalable\": true" + @@ -246,7 +246,7 @@ public void testSerialization() throws Exception { " }" + " }" + " }," + - " \"allowScaleDownDuringEvacuated\": true," + + " \"allowAutoScaleManager\": true," + " \"enabled\": true" + " }," + " \"scalable\": true" + @@ -266,14 +266,14 @@ public void testSerializationWithSkuId() throws Exception { 2, new MachineDefinition(1, 1.24, 0.0, 1, 1), null, null, - new StageScalingPolicy(1, 1, 3, 1, 1, 60, smap, true), + new StageScalingPolicy(1, 1, 3, 1, 1, 60, smap, false), ImmutableMap.of("containerSkuID", "sku1") ) .multiWorkerScalableStageWithConstraints( 3, new MachineDefinition(1, 1.24, 0.0, 1, 1), null, null, - new StageScalingPolicy(1, 1, 3, 1, 1, 60, smap, true), + new StageScalingPolicy(1, 1, 3, 1, 1, 60, smap, false), ImmutableMap.of("containerSkuID", "sku2") ); @@ -319,7 +319,7 @@ public void testSerializationWithSkuId() throws Exception { " }" + " }" + " }," + - " \"allowScaleDownDuringEvacuated\": true," + + " \"allowAutoScaleManager\": false," + " \"enabled\": true" + " }," + " \"scalable\": true," + @@ -362,7 +362,7 @@ public void testSerializationWithSkuId() throws Exception { " }" + " }" + " }," + - " \"allowScaleDownDuringEvacuated\": true," + + " \"allowAutoScaleManager\": false," + " \"enabled\": true" + " }," + " \"scalable\": true," + diff --git a/mantis-common/src/test/java/io/mantisrx/runtime/descriptor/StageScalingPolicyTest.java b/mantis-common/src/test/java/io/mantisrx/runtime/descriptor/StageScalingPolicyTest.java index 941341053..a584f4541 100644 --- a/mantis-common/src/test/java/io/mantisrx/runtime/descriptor/StageScalingPolicyTest.java +++ b/mantis-common/src/test/java/io/mantisrx/runtime/descriptor/StageScalingPolicyTest.java @@ -35,7 +35,7 @@ public void testSerialization() throws Exception { Map smap = new HashMap<>(); smap.put(ScalingReason.CPU, new Strategy(ScalingReason.CPU, 0.5, 0.75, null)); smap.put(ScalingReason.DataDrop, new Strategy(ScalingReason.DataDrop, 0.0, 2.0, null)); - StageScalingPolicy policy = new StageScalingPolicy(1, 1, 2, 1, 1, 60, smap, true); + StageScalingPolicy policy = new StageScalingPolicy(1, 1, 2, 1, 1, 60, smap, false); final String expected = "{\n" + " \"stage\": 1,\n" + @@ -80,31 +80,31 @@ public void testSerialization() throws Exception { public void testDeserialization() throws Exception { String json1 = "{\"stage\":1,\"min\":1,\"max\":2,\"increment\":1,\"decrement\":1,\"strategies\":{},\"enabled\":false}"; StageScalingPolicy actual = serializer.fromJSON(json1, StageScalingPolicy.class); - StageScalingPolicy expected = new StageScalingPolicy(1, 1, 2, 1, 1, 0, null, true); + StageScalingPolicy expected = new StageScalingPolicy(1, 1, 2, 1, 1, 0, null, false); assertEquals(expected, actual); String json2 = "{\"stage\":1,\"min\":1,\"max\":5,\"increment\":1,\"decrement\":1,\"coolDownSecs\":600,\"strategies\":{\"CPU\":{\"reason\":\"CPU\",\"scaleDownBelowPct\":50,\"scaleUpAbovePct\":75}},\"enabled\":true}"; actual = serializer.fromJSON(json2, StageScalingPolicy.class); Map smap = new HashMap<>(); smap.put(ScalingReason.CPU, new Strategy(ScalingReason.CPU, 50, 75.0, new RollingCount(1, 1))); - expected = new StageScalingPolicy(1, 1, 5, 1, 1, 600, smap, true); + expected = new StageScalingPolicy(1, 1, 5, 1, 1, 600, smap, false); assertEquals(expected, actual); String json3 = "{\"stage\":1,\"min\":1,\"max\":3,\"increment\":1,\"decrement\":1,\"coolDownSecs\":0,\"strategies\":{\"Memory\":{\"reason\":\"Memory\",\"scaleDownBelowPct\":65,\"scaleUpAbovePct\":80,\"rollingCount\":{\"count\":6,\"of\":10}}},\"enabled\":true}"; actual = serializer.fromJSON(json3, StageScalingPolicy.class); smap = new HashMap<>(); smap.put(ScalingReason.Memory, new Strategy(ScalingReason.Memory, 65, 80.0, new RollingCount(6, 10))); - expected = new StageScalingPolicy(1, 1, 3, 1, 1, 0, smap, true); + expected = new StageScalingPolicy(1, 1, 3, 1, 1, 0, smap, false); assertEquals(expected, actual); - String json4 = "{\"stage\":1,\"min\":1,\"max\":3,\"increment\":1,\"decrement\":1,\"coolDownSecs\":0,\"strategies\":{\"Memory\":{\"reason\":\"Memory\",\"scaleDownBelowPct\":65,\"scaleUpAbovePct\":80,\"rollingCount\":{\"count\":6,\"of\":10}}},\"allowScaleDownDuringEvacuated\":false,\"enabled\":true}"; + String json4 = "{\"stage\":1,\"min\":1,\"max\":3,\"increment\":1,\"decrement\":1,\"coolDownSecs\":0,\"strategies\":{\"Memory\":{\"reason\":\"Memory\",\"scaleDownBelowPct\":65,\"scaleUpAbovePct\":80,\"rollingCount\":{\"count\":6,\"of\":10}}},\"allowAutoScaleManager\":false,\"enabled\":true}"; actual = serializer.fromJSON(json4, StageScalingPolicy.class); smap = new HashMap<>(); smap.put(ScalingReason.Memory, new Strategy(ScalingReason.Memory, 65, 80.0, new RollingCount(6, 10))); expected = new StageScalingPolicy(1, 1, 3, 1, 1, 0, smap, false); assertEquals(expected, actual); - String json5 = "{\"stage\":1,\"min\":1,\"max\":3,\"increment\":1,\"decrement\":1,\"coolDownSecs\":0,\"strategies\":{\"Memory\":{\"reason\":\"Memory\",\"scaleDownBelowPct\":65,\"scaleUpAbovePct\":80,\"rollingCount\":{\"count\":6,\"of\":10}}},\"allowScaleDownDuringEvacuated\":true,\"enabled\":true}"; + String json5 = "{\"stage\":1,\"min\":1,\"max\":3,\"increment\":1,\"decrement\":1,\"coolDownSecs\":0,\"strategies\":{\"Memory\":{\"reason\":\"Memory\",\"scaleDownBelowPct\":65,\"scaleUpAbovePct\":80,\"rollingCount\":{\"count\":6,\"of\":10}}},\"allowAutoScaleManager\":true,\"enabled\":true}"; actual = serializer.fromJSON(json5, StageScalingPolicy.class); smap = new HashMap<>(); smap.put(ScalingReason.Memory, new Strategy(ScalingReason.Memory, 65, 80.0, new RollingCount(6, 10))); diff --git a/mantis-control-plane/mantis-control-plane-core/src/main/java/io/mantisrx/server/master/FailoverStatusClient.java b/mantis-control-plane/mantis-control-plane-core/src/main/java/io/mantisrx/server/master/FailoverStatusClient.java deleted file mode 100644 index 88a15313d..000000000 --- a/mantis-control-plane/mantis-control-plane-core/src/main/java/io/mantisrx/server/master/FailoverStatusClient.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Copyright 2024 Netflix, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.mantisrx.server.master; - -/** - * Failover status client interface to get the failover status of current region - */ -public interface FailoverStatusClient { - - FailoverStatusClient DEFAULT = new NoopFailoverStatusClient(); - - /** - * Get the failover status of current region - */ - FailoverStatus getFailoverStatus(); - - enum FailoverStatus { - /** - * Normal region - */ - NORMAL, - /** - * Region is evacuating (failed over) - */ - REGION_EVACUEE, - /** - * Region is savior (taking over traffic for failed over region) - */ - REGION_SAVIOR; - } - - /** - * Noop implementation of {@link FailoverStatusClient} that always returns {@link FailoverStatus#NORMAL} - */ - class NoopFailoverStatusClient implements FailoverStatusClient { - - @Override - public FailoverStatus getFailoverStatus() { - return FailoverStatus.NORMAL; - } - } -} diff --git a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/RuntimeTaskImpl.java b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/RuntimeTaskImpl.java index 4d2ed1426..4b8bb6363 100644 --- a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/RuntimeTaskImpl.java +++ b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/RuntimeTaskImpl.java @@ -28,7 +28,6 @@ import io.mantisrx.server.core.Status; import io.mantisrx.server.core.WrappedExecuteStageRequest; import io.mantisrx.server.core.metrics.MetricsFactory; -import io.mantisrx.server.master.FailoverStatusClient; import io.mantisrx.server.master.client.HighAvailabilityServices; import io.mantisrx.server.master.client.HighAvailabilityServicesUtil; import io.mantisrx.server.master.client.MantisMasterGateway; @@ -75,15 +74,9 @@ public class RuntimeTaskImpl extends AbstractIdleService implements RuntimeTask private Optional mantisJob = Optional.empty(); private ExecuteStageRequest executeStageRequest; - private FailoverStatusClient failoverStatusClient; public RuntimeTaskImpl() { - this(FailoverStatusClient.DEFAULT); - } - - public RuntimeTaskImpl(FailoverStatusClient failoverStatusClient) { this.tasksStatusSubject = PublishSubject.create(); - this.failoverStatusClient = failoverStatusClient; } public RuntimeTaskImpl(PublishSubject> tasksStatusSubject) { @@ -199,7 +192,6 @@ private void doRun() throws Exception { masterMonitor, config, workerMetricsClient, - failoverStatusClient, sinkSubscriptionStateHandlerFactory, userCodeClassLoader.asClassLoader()), getJobProviderClass(), diff --git a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/WorkerExecutionOperationsNetworkStage.java b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/WorkerExecutionOperationsNetworkStage.java index 1c9ec1f74..a6c1ada89 100644 --- a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/WorkerExecutionOperationsNetworkStage.java +++ b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/WorkerExecutionOperationsNetworkStage.java @@ -52,11 +52,11 @@ import io.mantisrx.server.core.StatusPayloads; import io.mantisrx.server.core.WorkerAssignments; import io.mantisrx.server.core.WorkerHost; -import io.mantisrx.server.master.FailoverStatusClient; import io.mantisrx.server.master.client.MantisMasterGateway; import io.mantisrx.server.worker.client.SseWorkerConnection; import io.mantisrx.server.worker.client.WorkerMetricsClient; import io.mantisrx.server.worker.jobmaster.AutoScaleMetricsConfig; +import io.mantisrx.server.worker.jobmaster.JobAutoscalerManager; import io.mantisrx.server.worker.jobmaster.JobMasterService; import io.mantisrx.server.worker.jobmaster.JobMasterStageConfig; import io.mantisrx.shaded.com.google.common.base.Splitter; @@ -66,6 +66,8 @@ import io.reactivex.mantis.remote.observable.ToDeltaEndpointInjector; import java.io.Closeable; import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; import java.time.Duration; import java.time.temporal.ChronoUnit; import java.util.ArrayList; @@ -76,6 +78,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Properties; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; @@ -111,20 +114,17 @@ public class WorkerExecutionOperationsNetworkStage implements WorkerExecutionOpe private final List closeables = new ArrayList<>(); private final ScheduledExecutorService scheduledExecutorService; private final ClassLoader classLoader; - private final FailoverStatusClient failoverStatusClient; private Observer jobStatusObserver; public WorkerExecutionOperationsNetworkStage( MantisMasterGateway mantisMasterApi, WorkerConfiguration config, WorkerMetricsClient workerMetricsClient, - FailoverStatusClient failoverStatusClient, SinkSubscriptionStateHandler.Factory sinkSubscriptionStateHandlerFactory, ClassLoader classLoader) { this.mantisMasterApi = mantisMasterApi; this.config = config; this.workerMetricsClient = workerMetricsClient; - this.failoverStatusClient = failoverStatusClient; this.sinkSubscriptionStateHandlerFactory = sinkSubscriptionStateHandlerFactory; this.classLoader = classLoader; @@ -416,8 +416,9 @@ public void executeStage(final ExecutionDetails setup) throws IOException { logger.info("param {} is null or empty", JOB_MASTER_AUTOSCALE_METRIC_SYSTEM_PARAM); } + JobAutoscalerManager jobAutoscalerManager = getJobAutoscalerManagerInstance(config); JobMasterService jobMasterService = new JobMasterService(rw.getJobId(), rw.getSchedulingInfo(), - workerMetricsClient, autoScaleMetricsConfig, mantisMasterApi, failoverStatusClient, rw.getContext(), rw.getOnCompleteCallback(), rw.getOnErrorCallback(), rw.getOnTerminateCallback()); + workerMetricsClient, autoScaleMetricsConfig, mantisMasterApi, rw.getContext(), rw.getOnCompleteCallback(), rw.getOnErrorCallback(), rw.getOnTerminateCallback(), jobAutoscalerManager); jobMasterService.start(); closeables.add(jobMasterService::shutdown); @@ -481,6 +482,18 @@ remoteObservableName, numWorkersAtStage(selfSchedulingInfo, rw.getJobId(), rw.ge } } + private JobAutoscalerManager getJobAutoscalerManagerInstance(WorkerConfiguration config) { + try { + Class jobAutoscalerManagerClass = Class.forName(config.getJobAutoscalerManagerClassName()); + logger.info("Picking {} jobAutoscalerManager", jobAutoscalerManagerClass.getName()); + Method managerClassFactory = jobAutoscalerManagerClass.getMethod("valueOf", Properties.class); + return (JobAutoscalerManager) managerClassFactory.invoke(null, System.getProperties()); + } catch (ClassNotFoundException | NoSuchMethodException | InvocationTargetException | IllegalAccessException e) { + logger.warn("Couldnt instantiate jobAutoscalerManager from class {} because ", config.getJobAutoscalerManagerClassName(), e); + return JobAutoscalerManager.DEFAULT; + } + } + private void setupSubscriptionStateHandler(ExecuteStageRequest executeStageRequest) { final SinkSubscriptionStateHandler subscriptionStateHandler = sinkSubscriptionStateHandlerFactory.apply(executeStageRequest); diff --git a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoScaler.java b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoScaler.java index 608931e26..99e872857 100644 --- a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoScaler.java +++ b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoScaler.java @@ -26,7 +26,6 @@ import io.mantisrx.runtime.descriptor.StageScalingPolicy.ScalingReason; import io.mantisrx.runtime.descriptor.StageSchedulingInfo; import io.mantisrx.server.core.stats.UsageDataStats; -import io.mantisrx.server.master.FailoverStatusClient; import io.mantisrx.server.master.client.MantisMasterGateway; import io.mantisrx.server.worker.jobmaster.clutch.ClutchAutoScaler; import io.mantisrx.server.worker.jobmaster.clutch.ClutchConfiguration; @@ -91,16 +90,16 @@ public class JobAutoScaler { private final SchedulingInfo schedulingInfo; private final PublishSubject subject; private final Context context; - private final FailoverStatusClient failoverStatusClient; + private final JobAutoscalerManager jobAutoscalerManager; JobAutoScaler(String jobId, SchedulingInfo schedulingInfo, MantisMasterGateway masterClientApi, - Context context, FailoverStatusClient failoverStatusClient) { + Context context, JobAutoscalerManager jobAutoscalerManager) { this.jobId = jobId; this.masterClientApi = masterClientApi; this.schedulingInfo = schedulingInfo; - subject = PublishSubject.create(); + this.subject = PublishSubject.create(); this.context = context; - this.failoverStatusClient = failoverStatusClient; + this.jobAutoscalerManager = jobAutoscalerManager; } Observer getObserver() { @@ -198,9 +197,6 @@ void start() { .distinctUntilChanged() .throttleLast(5, TimeUnit.SECONDS); - //TODO(hmitnflx): Add a failoveraware transformer here that consumes all other events as well - // to make scale up decisions - // Create the scaler. if (useClutchRps) { logger.info("Using clutch rps scaler, job: {}, stage: {} ", jobId, stage); ClutchRpsPIDConfig rpsConfig = Option.of(config).flatMap(ClutchConfiguration::getRpsConfig).getOrNull(); @@ -336,9 +332,13 @@ public int getDesiredWorkersForScaleUp(final int increment, final int numCurrent logger.warn("Job {} stage {} is not scalable, can't increment #workers by {}", jobId, stage, increment); return numCurrentWorkers; } + if (numCurrentWorkers < 0 || increment < 1) { logger.error("current number of workers({}) not known or increment({}) < 1, will not scale up", numCurrentWorkers, increment); return numCurrentWorkers; + } else if (stageSchedulingInfo.getScalingPolicy().isAllowAutoScaleManager() && !jobAutoscalerManager.isScaleUpEnabled()) { + logger.warn("Scaleup is disabled for all autoscaling strategy, not scaling up stage {} of job {}", stage, jobId); + return numCurrentWorkers; } else { final int maxWorkersForStage = stageSchedulingInfo.getScalingPolicy().getMax(); desiredWorkers = Math.min(numCurrentWorkers + increment, maxWorkersForStage); @@ -349,6 +349,10 @@ public int getDesiredWorkersForScaleUp(final int increment, final int numCurrent public void scaleUpStage(final int numCurrentWorkers, final int desiredWorkers, final String reason) { logger.info("scaleUpStage incrementing number of workers from {} to {}", numCurrentWorkers, desiredWorkers); cancelOutstandingScalingRequest(); + StageScalingPolicy scalingPolicy = stageSchedulingInfo.getScalingPolicy(); + if (scalingPolicy != null && scalingPolicy.isAllowAutoScaleManager() && !jobAutoscalerManager.isScaleUpEnabled()) { + logger.warn("Scaleup is disabled for all autoscaling strategy, not scaling up stage {} of job {}", stage, jobId); + } final Subscription subscription = masterClientApi.scaleJobStage(jobId, stage, desiredWorkers, reason) .retryWhen(retryLogic) .onErrorResumeNext(throwable -> { @@ -369,8 +373,8 @@ public int getDesiredWorkersForScaleDown(final int decrement, final int numCurre if (numCurrentWorkers < 0 || decrement < 1) { logger.error("current number of workers({}) not known or decrement({}) < 1, will not scale down", numCurrentWorkers, decrement); return numCurrentWorkers; - } else if (!scalingPolicy.isAllowScaleDownDuringEvacuated() && failoverStatusClient.getFailoverStatus() == FailoverStatusClient.FailoverStatus.REGION_EVACUEE) { - logger.warn("Region is evacuated, not scaling down stage {} of job {}", stage, jobId); + } else if (scalingPolicy.isAllowAutoScaleManager() && !jobAutoscalerManager.isScaleDownEnabled()) { + logger.warn("Scaledown is disabled for all autoscaling strategy, not scaling down stage {} of job {}", stage, jobId); return numCurrentWorkers; } else { int min = scalingPolicy.getMin(); @@ -383,8 +387,8 @@ public void scaleDownStage(final int numCurrentWorkers, final int desiredWorkers logger.info("scaleDownStage decrementing number of workers from {} to {}", numCurrentWorkers, desiredWorkers); cancelOutstandingScalingRequest(); final StageScalingPolicy scalingPolicy = stageSchedulingInfo.getScalingPolicy(); - if (scalingPolicy != null && !scalingPolicy.isAllowScaleDownDuringEvacuated() && failoverStatusClient.getFailoverStatus() == FailoverStatusClient.FailoverStatus.REGION_EVACUEE) { - logger.warn("Region is evacuated, not scaling down stage {} of job {}", stage, jobId); + if (scalingPolicy != null && scalingPolicy.isAllowAutoScaleManager() && !jobAutoscalerManager.isScaleDownEnabled()) { + logger.warn("Scaledown is disabled for all autoscaling strategy. For stage {} of job {}", stage, jobId); return; } final Subscription subscription = masterClientApi.scaleJobStage(jobId, stage, desiredWorkers, reason) diff --git a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoscalerManager.java b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoscalerManager.java new file mode 100644 index 000000000..df98ce35f --- /dev/null +++ b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoscalerManager.java @@ -0,0 +1,56 @@ +/* + * Copyright 2024 Netflix, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.mantisrx.server.worker.jobmaster; + +import java.util.Properties; + +/** + * Failover status client interface to get the failover status of current region + */ +public interface JobAutoscalerManager { + + JobAutoscalerManager DEFAULT = new NoopJobAutoscalerManager(); + + /** + * Knobs to toggle autoscaling scale ups + */ + default boolean isScaleUpEnabled() { + return true; + } + + /** + * Knobs to toggle autoscaling scale downs + */ + default boolean isScaleDownEnabled() { + return true; + } + + /** + * Noop implementation of {@link JobAutoscalerManager} that always returns true + * for isScaleUpEnabled, isScaleDownEnabled + */ + class NoopJobAutoscalerManager implements JobAutoscalerManager { + + private NoopJobAutoscalerManager() { + } + + @SuppressWarnings("unused") + public static JobAutoscalerManager valueOf(Properties properties) { + return DEFAULT; + } + } +} diff --git a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobMasterService.java b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobMasterService.java index 91195f625..902b1a0a8 100644 --- a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobMasterService.java +++ b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobMasterService.java @@ -25,7 +25,6 @@ import io.mantisrx.runtime.parameter.SourceJobParameters; import io.mantisrx.server.core.Service; import io.mantisrx.server.core.stats.MetricStringConstants; -import io.mantisrx.server.master.FailoverStatusClient; import io.mantisrx.server.master.client.MantisMasterGateway; import io.mantisrx.server.worker.client.WorkerMetricsClient; import io.mantisrx.shaded.com.fasterxml.jackson.core.JsonProcessingException; @@ -66,17 +65,17 @@ public JobMasterService(final String jobId, final WorkerMetricsClient workerMetricsClient, final AutoScaleMetricsConfig autoScaleMetricsConfig, final MantisMasterGateway masterClientApi, - final FailoverStatusClient failoverStatusClient, final Context context, final Action0 observableOnCompleteCallback, final Action1 observableOnErrorCallback, - final Action0 observableOnTerminateCallback) { + final Action0 observableOnTerminateCallback, + final JobAutoscalerManager jobAutoscalerManager) { this.jobId = jobId; this.workerMetricsClient = workerMetricsClient; this.autoScaleMetricsConfig = autoScaleMetricsConfig; this.masterClientApi = masterClientApi; - this.jobAutoScaler = new JobAutoScaler(jobId, schedInfo, masterClientApi, context, failoverStatusClient); - this.metricObserver = new WorkerMetricHandler(jobId, jobAutoScaler.getObserver(), masterClientApi, autoScaleMetricsConfig, failoverStatusClient).initAndGetMetricDataObserver(); + this.jobAutoScaler = new JobAutoScaler(jobId, schedInfo, masterClientApi, context, jobAutoscalerManager); + this.metricObserver = new WorkerMetricHandler(jobId, jobAutoScaler.getObserver(), masterClientApi, autoScaleMetricsConfig, jobAutoscalerManager).initAndGetMetricDataObserver(); this.observableOnCompleteCallback = observableOnCompleteCallback; this.observableOnErrorCallback = observableOnErrorCallback; this.observableOnTerminateCallback = observableOnTerminateCallback; diff --git a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandler.java b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandler.java index 2c1b1e31f..de31172e8 100644 --- a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandler.java +++ b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandler.java @@ -21,7 +21,6 @@ import io.mantisrx.runtime.descriptor.StageScalingPolicy; import io.mantisrx.server.core.*; import io.mantisrx.server.core.stats.MetricStringConstants; -import io.mantisrx.server.master.FailoverStatusClient; import io.mantisrx.server.master.client.MantisMasterGateway; import io.mantisrx.shaded.com.google.common.cache.Cache; import io.mantisrx.shaded.com.google.common.cache.CacheBuilder; @@ -72,19 +71,19 @@ return -1; } }; - private final FailoverStatusClient failoverStatusClient; + private final JobAutoscalerManager jobAutoscalerManager; public WorkerMetricHandler(final String jobId, final Observer jobAutoScaleObserver, final MantisMasterGateway masterClientApi, final AutoScaleMetricsConfig autoScaleMetricsConfig, - final FailoverStatusClient failoverStatusClient) { + final JobAutoscalerManager jobAutoscalerManager) { this.jobId = jobId; this.jobAutoScaleObserver = jobAutoScaleObserver; this.masterClientApi = masterClientApi; this.autoScaleMetricsConfig = autoScaleMetricsConfig; this.metricAggregator = new MetricAggregator(autoScaleMetricsConfig); - this.failoverStatusClient = failoverStatusClient; + this.jobAutoscalerManager = jobAutoscalerManager; } public Observer initAndGetMetricDataObserver() { @@ -280,7 +279,6 @@ public void call() { // get the aggregate metric values by metric group for all workers in stage Map allWorkerAggregates = getAggregates(listofAggregates); logger.info("Job stage {} avgResUsage from {} workers: {}", stage, workersMap.size(), allWorkerAggregates.toString()); - // TODO(hmitnflx): Add a failover mode based event into jobAutoScaleObserver for (Map.Entry> userDefinedMetric : autoScaleMetricsConfig.getUserDefinedMetrics().entrySet()) { final String metricGrp = userDefinedMetric.getKey(); diff --git a/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/JobAutoScalerTest.java b/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/JobAutoScalerTest.java index 2bd9c1a85..0feb8d6ca 100644 --- a/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/JobAutoScalerTest.java +++ b/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/JobAutoScalerTest.java @@ -30,7 +30,6 @@ import io.mantisrx.runtime.descriptor.SchedulingInfo; import io.mantisrx.runtime.descriptor.StageScalingPolicy; import io.mantisrx.runtime.descriptor.StageSchedulingInfo; -import io.mantisrx.server.master.FailoverStatusClient; import io.mantisrx.server.master.client.MantisMasterClientApi; import io.mantisrx.server.worker.jobmaster.clutch.ClutchConfiguration; import io.mantisrx.server.worker.jobmaster.clutch.rps.ClutchRpsPIDConfig; @@ -79,7 +78,7 @@ public void testScaleUp() throws InterruptedException { .machineDefinition(new MachineDefinition(2, workerMemoryMB, 200, 1024, 2)) .scalingPolicy(new StageScalingPolicy(scalingStageNum, min, max, increment, decrement, coolDownSec, Collections.singletonMap(StageScalingPolicy.ScalingReason.Memory, - new StageScalingPolicy.Strategy(StageScalingPolicy.ScalingReason.Memory, scaleDownBelowPct, scaleUpAbovePct, new StageScalingPolicy.RollingCount(1, 2))), true)) + new StageScalingPolicy.Strategy(StageScalingPolicy.ScalingReason.Memory, scaleDownBelowPct, scaleUpAbovePct, new StageScalingPolicy.RollingCount(1, 2))), false)) .scalable(true) .build(); @@ -90,7 +89,7 @@ public void testScaleUp() throws InterruptedException { Context context = mock(Context.class); when(context.getWorkerMapObservable()).thenReturn(Observable.empty()); - final JobAutoScaler jobAutoScaler = new JobAutoScaler(jobId, new SchedulingInfo(schedulingInfoMap), mockMasterClientApi, context, FailoverStatusClient.DEFAULT); + final JobAutoScaler jobAutoScaler = new JobAutoScaler(jobId, new SchedulingInfo(schedulingInfoMap), mockMasterClientApi, context, JobAutoscalerManager.DEFAULT); jobAutoScaler.start(); final Observer jobAutoScalerObserver = jobAutoScaler.getObserver(); @@ -149,7 +148,7 @@ public void testScalingResiliency() throws InterruptedException { put(StageScalingPolicy.ScalingReason.Memory, new StageScalingPolicy.Strategy(StageScalingPolicy.ScalingReason.Memory, scaleDownBelowPct, scaleUpAbovePct, new StageScalingPolicy.RollingCount(1, 2))); put(StageScalingPolicy.ScalingReason.FailoverAware, new StageScalingPolicy.Strategy(StageScalingPolicy.ScalingReason.FailoverAware, 0.0, 0.0, new StageScalingPolicy.RollingCount(1, 2))); } - }, true)) + }, false)) .scalable(true) .build(); @@ -173,7 +172,7 @@ public Boolean call(Integer integer) { Context context = mock(Context.class); when(context.getWorkerMapObservable()).thenReturn(Observable.empty()); - final JobAutoScaler jobAutoScaler = new JobAutoScaler(jobId, new SchedulingInfo(schedulingInfoMap), mockMasterClientApi, context, FailoverStatusClient.DEFAULT); + final JobAutoScaler jobAutoScaler = new JobAutoScaler(jobId, new SchedulingInfo(schedulingInfoMap), mockMasterClientApi, context, JobAutoscalerManager.DEFAULT); jobAutoScaler.start(); final Observer jobAutoScalerObserver = jobAutoScaler.getObserver(); @@ -206,7 +205,7 @@ public void testScaleDown() throws InterruptedException { .machineDefinition(new MachineDefinition(2, workerMemoryMB, 200, 1024, 2)) .scalingPolicy(new StageScalingPolicy(scalingStageNum, min, max, increment, decrement, coolDownSec, Collections.singletonMap(StageScalingPolicy.ScalingReason.Memory, - new StageScalingPolicy.Strategy(StageScalingPolicy.ScalingReason.Memory, scaleDownBelowPct, scaleUpAbovePct, new StageScalingPolicy.RollingCount(1, 2))), true)) + new StageScalingPolicy.Strategy(StageScalingPolicy.ScalingReason.Memory, scaleDownBelowPct, scaleUpAbovePct, new StageScalingPolicy.RollingCount(1, 2))), false)) .scalable(true) .build(); @@ -217,7 +216,7 @@ public void testScaleDown() throws InterruptedException { Context context = mock(Context.class); when(context.getWorkerMapObservable()).thenReturn(Observable.empty()); - final JobAutoScaler jobAutoScaler = new JobAutoScaler(jobId, new SchedulingInfo(schedulingInfoMap), mockMasterClientApi, context, FailoverStatusClient.DEFAULT); + final JobAutoScaler jobAutoScaler = new JobAutoScaler(jobId, new SchedulingInfo(schedulingInfoMap), mockMasterClientApi, context, JobAutoscalerManager.DEFAULT); jobAutoScaler.start(); final Observer jobAutoScalerObserver = jobAutoScaler.getObserver(); @@ -239,6 +238,66 @@ public void testScaleDown() throws InterruptedException { } } + @Test + public void testScaleDownManagerDisabled() throws InterruptedException { + final String jobId = "test-job-1"; + final int coolDownSec = 2; + final int scalingStageNum = 1; + final MantisMasterClientApi mockMasterClientApi = mock(MantisMasterClientApi.class); + final Map schedulingInfoMap = new HashMap<>(); + final int numStage1Workers = 2; + final int increment = 1; + final int decrement = 1; + final int min = 1; + final int max = 5; + final double scaleUpAbovePct = 45.0; + final double scaleDownBelowPct = 15.0; + final double workerMemoryMB = 512.0; + + final StageSchedulingInfo stage1SchedInfo = StageSchedulingInfo.builder() + .numberOfInstances(numStage1Workers) + .machineDefinition(new MachineDefinition(2, workerMemoryMB, 200, 1024, 2)) + .scalingPolicy(new StageScalingPolicy(scalingStageNum, min, max, increment, decrement, coolDownSec, + Collections.singletonMap(StageScalingPolicy.ScalingReason.Memory, + new StageScalingPolicy.Strategy(StageScalingPolicy.ScalingReason.Memory, scaleDownBelowPct, scaleUpAbovePct, new StageScalingPolicy.RollingCount(1, 2))), true)) + .scalable(true) + .build(); + + schedulingInfoMap.put(scalingStageNum, stage1SchedInfo); + + when(mockMasterClientApi.scaleJobStage(eq(jobId), eq(scalingStageNum), eq(numStage1Workers - decrement), anyString())).thenReturn(Observable.just(true)); + + Context context = mock(Context.class); + when(context.getWorkerMapObservable()).thenReturn(Observable.empty()); + + JobAutoscalerManager scalarManager = new JobAutoscalerManager() { + @Override + public boolean isScaleDownEnabled() { + return false; + } + }; + final JobAutoScaler jobAutoScaler = new JobAutoScaler(jobId, new SchedulingInfo(schedulingInfoMap), mockMasterClientApi, context, scalarManager); + jobAutoScaler.start(); + final Observer jobAutoScalerObserver = jobAutoScaler.getObserver(); + + // should trigger a scale down (below 15% scaleDown threshold) + jobAutoScalerObserver.onNext(new JobAutoScaler.Event(StageScalingPolicy.ScalingReason.Memory, scalingStageNum, workerMemoryMB * (scaleDownBelowPct / 100.0 - 0.01), (scaleDownBelowPct / 100.0 - 0.01) * 100.0, numStage1Workers)); + + verify(mockMasterClientApi, timeout(1000).times(0)).scaleJobStage(jobId, scalingStageNum, numStage1Workers - decrement, String.format("Memory with value %1$,.2f is below scaleDown threshold of %2$,.1f", (scaleDownBelowPct / 100.0 - 0.01) * 100.0, scaleDownBelowPct)); + + // should *not* trigger a scale down before cooldown period (below 15% scaleDown threshold) + jobAutoScalerObserver.onNext(new JobAutoScaler.Event(StageScalingPolicy.ScalingReason.Memory, scalingStageNum, workerMemoryMB * (scaleDownBelowPct / 100.0 - 0.01), scaleDownBelowPct - 0.01, numStage1Workers - decrement)); + jobAutoScalerObserver.onNext(new JobAutoScaler.Event(StageScalingPolicy.ScalingReason.Memory, scalingStageNum, workerMemoryMB * (scaleDownBelowPct / 100.0 - 0.01), scaleDownBelowPct - 0.01, numStage1Workers - decrement)); + + Thread.sleep(coolDownSec * 1000); + + if (numStage1Workers - decrement == min) { + // should not trigger a scale down after cooldown period if numWorkers=min (below 15% scaleDown threshold) + jobAutoScalerObserver.onNext(new JobAutoScaler.Event(StageScalingPolicy.ScalingReason.Memory, scalingStageNum, workerMemoryMB * (scaleDownBelowPct / 100.0 - 0.01), scaleDownBelowPct - 0.01, numStage1Workers - decrement)); + verifyNoMoreInteractions(mockMasterClientApi); + } + } + @Test public void testScaleDownNotLessThanMin() throws InterruptedException { final String jobId = "test-job-1"; @@ -260,7 +319,7 @@ public void testScaleDownNotLessThanMin() throws InterruptedException { .numberOfInstances(numStage1Workers).machineDefinition(new MachineDefinition(2, workerMemoryMB, 200, 1024, 2)) .scalingPolicy(new StageScalingPolicy(scalingStageNum, min, max, increment, decrement, coolDownSec, Collections.singletonMap(StageScalingPolicy.ScalingReason.Memory, - new StageScalingPolicy.Strategy(StageScalingPolicy.ScalingReason.Memory, scaleDownBelowPct, scaleUpAbovePct, new StageScalingPolicy.RollingCount(1, 2))), true)) + new StageScalingPolicy.Strategy(StageScalingPolicy.ScalingReason.Memory, scaleDownBelowPct, scaleUpAbovePct, new StageScalingPolicy.RollingCount(1, 2))), false)) .scalable(true) .build(); @@ -271,7 +330,7 @@ public void testScaleDownNotLessThanMin() throws InterruptedException { Context context = mock(Context.class); when(context.getWorkerMapObservable()).thenReturn(Observable.empty()); - final JobAutoScaler jobAutoScaler = new JobAutoScaler(jobId, new SchedulingInfo(schedulingInfoMap), mockMasterClientApi, context, FailoverStatusClient.DEFAULT); + final JobAutoScaler jobAutoScaler = new JobAutoScaler(jobId, new SchedulingInfo(schedulingInfoMap), mockMasterClientApi, context, JobAutoscalerManager.DEFAULT); jobAutoScaler.start(); final Observer jobAutoScalerObserver = jobAutoScaler.getObserver(); @@ -306,7 +365,7 @@ public void testScaleUpOnDifferentScalingReasons() throws InterruptedException { .machineDefinition(new MachineDefinition(2, workerMemoryMB, 200, 1024, 2)) .scalingPolicy(new StageScalingPolicy(scalingStageNum, min, max, increment, decrement, coolDownSec, Collections.singletonMap(scalingReason, - new StageScalingPolicy.Strategy(scalingReason, scaleDownBelow, scaleUpAbove, new StageScalingPolicy.RollingCount(1, 2))), true)) + new StageScalingPolicy.Strategy(scalingReason, scaleDownBelow, scaleUpAbove, new StageScalingPolicy.RollingCount(1, 2))), false)) .scalable(true) .build(); @@ -317,7 +376,7 @@ public void testScaleUpOnDifferentScalingReasons() throws InterruptedException { Context context = mock(Context.class); when(context.getWorkerMapObservable()).thenReturn(Observable.empty()); - final JobAutoScaler jobAutoScaler = new JobAutoScaler(jobId, new SchedulingInfo(schedulingInfoMap), mockMasterClientApi, context, FailoverStatusClient.DEFAULT); + final JobAutoScaler jobAutoScaler = new JobAutoScaler(jobId, new SchedulingInfo(schedulingInfoMap), mockMasterClientApi, context, JobAutoscalerManager.DEFAULT); jobAutoScaler.start(); final Observer jobAutoScalerObserver = jobAutoScaler.getObserver(); @@ -362,7 +421,7 @@ public void testGetClutchConfigurationFromJson() throws Exception { " \"scaleUpMultiplier\": 1.5" + " }" + "}"; - final JobAutoScaler jobAutoScaler = new JobAutoScaler("jobId", null, null, null, FailoverStatusClient.DEFAULT); + final JobAutoScaler jobAutoScaler = new JobAutoScaler("jobId", null, null, null, JobAutoscalerManager.DEFAULT); ClutchConfiguration config = jobAutoScaler.getClutchConfiguration(json).get(1); ClutchRpsPIDConfig expected = new ClutchRpsPIDConfig(0.0, Tuple.of(30.0, 0.0), 0.0, 0.0, Option.of(75.0), Option.of(30.0), Option.of(0.0), Option.of(1.5), Option.of(1.0)); diff --git a/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandlerTest.java b/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandlerTest.java index e629bc353..c784db21d 100644 --- a/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandlerTest.java +++ b/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandlerTest.java @@ -34,7 +34,6 @@ import io.mantisrx.server.core.WorkerAssignments; import io.mantisrx.server.core.WorkerHost; import io.mantisrx.server.core.stats.MetricStringConstants; -import io.mantisrx.server.master.FailoverStatusClient; import io.mantisrx.server.master.client.MantisMasterClientApi; import io.mantisrx.shaded.com.google.common.collect.ImmutableMap; import java.util.Arrays; @@ -101,7 +100,7 @@ public void onNext(JobAutoScaler.Event event) { assertEquals(expected, event); latch.countDown(); } - }, mockMasterClientApi, aggregationConfig, FailoverStatusClient.DEFAULT); + }, mockMasterClientApi, aggregationConfig, JobAutoscalerManager.DEFAULT); final Observer metricDataObserver = workerMetricHandler.initAndGetMetricDataObserver(); @@ -175,7 +174,7 @@ public void onNext(JobAutoScaler.Event event) { latch.countDown(); } } - }, mockMasterClientApi, aggregationConfig, FailoverStatusClient.DEFAULT); + }, mockMasterClientApi, aggregationConfig, JobAutoscalerManager.DEFAULT); final Observer metricDataObserver = workerMetricHandler.initAndGetMetricDataObserver(); @@ -267,7 +266,7 @@ public void onNext(JobAutoScaler.Event event) { assertEquals(expected, event); autoScaleLatch.countDown(); } - }, mockMasterClientApi, aggregationConfig, FailoverStatusClient.DEFAULT); + }, mockMasterClientApi, aggregationConfig, JobAutoscalerManager.DEFAULT); final Observer metricDataObserver = workerMetricHandler.initAndGetMetricDataObserver(); @@ -323,7 +322,7 @@ public void onNext(JobAutoScaler.Event event) { latch.countDown(); } } - }, mockMasterClientApi, aggregationConfig, FailoverStatusClient.DEFAULT); + }, mockMasterClientApi, aggregationConfig, JobAutoscalerManager.DEFAULT); final Observer metricDataObserver = workerMetricHandler.initAndGetMetricDataObserver(); diff --git a/mantis-runtime-loader/src/main/java/io/mantisrx/runtime/loader/config/WorkerConfiguration.java b/mantis-runtime-loader/src/main/java/io/mantisrx/runtime/loader/config/WorkerConfiguration.java index 065e4ca11..d69092ec8 100644 --- a/mantis-runtime-loader/src/main/java/io/mantisrx/runtime/loader/config/WorkerConfiguration.java +++ b/mantis-runtime-loader/src/main/java/io/mantisrx/runtime/loader/config/WorkerConfiguration.java @@ -69,6 +69,10 @@ default String getTaskExecutorHostName() { @Default("io.mantisrx.runtime.loader.cgroups.CgroupsMetricsCollector") String getMetricsCollectorClassName(); + @Config("mantis.taskexecutor.runtime.jobautoscalermanager") + @Default("io.mantisrx.server.worker.jobmaster.NoopJobAutoscalerManager") + String getJobAutoscalerManagerClassName(); + // ------------------------------------------------------------------------ // heartbeat connection related configurations // ------------------------------------------------------------------------ diff --git a/mantis-runtime-loader/src/main/java/io/mantisrx/runtime/loader/config/WorkerConfigurationUtils.java b/mantis-runtime-loader/src/main/java/io/mantisrx/runtime/loader/config/WorkerConfigurationUtils.java index 9cd86f231..7e20dae68 100644 --- a/mantis-runtime-loader/src/main/java/io/mantisrx/runtime/loader/config/WorkerConfigurationUtils.java +++ b/mantis-runtime-loader/src/main/java/io/mantisrx/runtime/loader/config/WorkerConfigurationUtils.java @@ -50,6 +50,7 @@ public static WorkerConfigurationWritable toWrit .localStorageDir(configSource.getLocalStorageDir()) .metricsCollector(configSource.getUsageSupplier()) .metricsCollectorClass(configSource.getMetricsCollectorClassName()) + .jobAutoscalerManagerClassName(configSource.getJobAutoscalerManagerClassName()) .metricsPort(configSource.getMetricsPort()) .metricsPublisher(configSource.getMetricsPublisher()) .metricsPublisherFrequencyInSeconds(configSource.getMetricsPublisherFrequencyInSeconds()) diff --git a/mantis-runtime-loader/src/main/java/io/mantisrx/runtime/loader/config/WorkerConfigurationWritable.java b/mantis-runtime-loader/src/main/java/io/mantisrx/runtime/loader/config/WorkerConfigurationWritable.java index fca252252..a5cbfac10 100644 --- a/mantis-runtime-loader/src/main/java/io/mantisrx/runtime/loader/config/WorkerConfigurationWritable.java +++ b/mantis-runtime-loader/src/main/java/io/mantisrx/runtime/loader/config/WorkerConfigurationWritable.java @@ -74,6 +74,7 @@ public class WorkerConfigurationWritable implements WorkerConfiguration { int asyncHttpClientRequestTimeoutMs; int asyncHttpClientReadTimeoutMs; String metricsCollectorClass; + String jobAutoscalerManagerClassName; @JsonIgnore MetricsPublisher metricsPublisher; diff --git a/mantis-server/mantis-server-agent/src/main/java/io/mantisrx/server/agent/TaskExecutor.java b/mantis-server/mantis-server-agent/src/main/java/io/mantisrx/server/agent/TaskExecutor.java index 386f14f20..b655c21a6 100644 --- a/mantis-server/mantis-server-agent/src/main/java/io/mantisrx/server/agent/TaskExecutor.java +++ b/mantis-server/mantis-server-agent/src/main/java/io/mantisrx/server/agent/TaskExecutor.java @@ -132,7 +132,8 @@ public TaskExecutor( new DefaultMantisPropertiesLoader(System.getProperties()), highAvailabilityServices, classLoaderHandle, - null); + null + ); } public TaskExecutor( diff --git a/mantis-server/mantis-server-agent/src/main/java/io/mantisrx/server/agent/TaskExecutorStarter.java b/mantis-server/mantis-server-agent/src/main/java/io/mantisrx/server/agent/TaskExecutorStarter.java index 26a44d25c..9d863dac4 100644 --- a/mantis-server/mantis-server-agent/src/main/java/io/mantisrx/server/agent/TaskExecutorStarter.java +++ b/mantis-server/mantis-server-agent/src/main/java/io/mantisrx/server/agent/TaskExecutorStarter.java @@ -110,6 +110,7 @@ public static class TaskExecutorStarterBuilder { private final List> listeners = new ArrayList<>(); + private TaskExecutorStarterBuilder(WorkerConfiguration workerConfiguration) { this.workerConfiguration = workerConfiguration; this.configuration = new Configuration(); @@ -194,7 +195,8 @@ public TaskExecutorStarter build() throws Exception { Preconditions.checkNotNull(propertiesLoader, "propertiesLoader for TaskExecutor is null"), highAvailabilityServices, getClassLoaderHandle(), - this.taskFactory); + this.taskFactory + ); for (Tuple2 listener : listeners) { taskExecutor.addListener(listener._1(), listener._2()); From 473be12372e4e94ffb2dfaa5565b4fd09e757c86 Mon Sep 17 00:00:00 2001 From: Harshit Mittal Date: Sat, 8 Jun 2024 13:54:42 -0700 Subject: [PATCH 4/8] support scaling up to maxWorkers for stage on failover event --- .../descriptor/StageScalingPolicy.java | 2 +- .../worker/jobmaster/JobAutoScaler.java | 35 ++++++++++-------- .../jobmaster/JobAutoscalerManager.java | 4 ++ .../worker/jobmaster/WorkerMetricHandler.java | 7 ++++ .../worker/jobmaster/JobAutoScalerTest.java | 2 +- .../jobmaster/WorkerMetricHandlerTest.java | 37 +++++++++++++++---- 6 files changed, 62 insertions(+), 25 deletions(-) diff --git a/mantis-common/src/main/java/io/mantisrx/runtime/descriptor/StageScalingPolicy.java b/mantis-common/src/main/java/io/mantisrx/runtime/descriptor/StageScalingPolicy.java index 517d8b347..1450d43fe 100644 --- a/mantis-common/src/main/java/io/mantisrx/runtime/descriptor/StageScalingPolicy.java +++ b/mantis-common/src/main/java/io/mantisrx/runtime/descriptor/StageScalingPolicy.java @@ -86,7 +86,7 @@ public enum ScalingReason { RPS, JVMMemory, SourceJobDrop, - FailoverAware + AutoscalerManager } @Getter diff --git a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoScaler.java b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoScaler.java index 99e872857..c0dcf9b6a 100644 --- a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoScaler.java +++ b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoScaler.java @@ -326,7 +326,7 @@ private void setOutstandingScalingRequest(final Subscription subscription) { inProgressScalingSubscription.compareAndSet(null, subscription); } - public int getDesiredWorkersForScaleUp(final int increment, final int numCurrentWorkers) { + public int getDesiredWorkersForScaleUp(final int increment, final int numCurrentWorkers, ScalingReason reason) { final int desiredWorkers; if (!stageSchedulingInfo.getScalingPolicy().isEnabled()) { logger.warn("Job {} stage {} is not scalable, can't increment #workers by {}", jobId, stage, increment); @@ -341,7 +341,12 @@ public int getDesiredWorkersForScaleUp(final int increment, final int numCurrent return numCurrentWorkers; } else { final int maxWorkersForStage = stageSchedulingInfo.getScalingPolicy().getMax(); - desiredWorkers = Math.min(numCurrentWorkers + increment, maxWorkersForStage); + if (reason == ScalingReason.AutoscalerManager) { + logger.info("FailoverAware scaling up stage {} of job {} to maxWorkersForStage {}", stage, jobId, maxWorkersForStage); + desiredWorkers = maxWorkersForStage; + } else { + desiredWorkers = Math.min(numCurrentWorkers + increment, maxWorkersForStage); + } return desiredWorkers; } } @@ -363,7 +368,7 @@ public void scaleUpStage(final int numCurrentWorkers, final int desiredWorkers, setOutstandingScalingRequest(subscription); } - public int getDesiredWorkersForScaleDown(final int decrement, final int numCurrentWorkers) { + public int getDesiredWorkersForScaleDown(final int decrement, final int numCurrentWorkers, ScalingReason reason) { final int desiredWorkers; final StageScalingPolicy scalingPolicy = stageSchedulingInfo.getScalingPolicy(); if (!scalingPolicy.isEnabled()) { @@ -456,16 +461,15 @@ public void onNext(Event event) { } stats.add(effectiveValue); if (lastScaledAt < (System.currentTimeMillis() - coolDownSecs * 1000)) { - logger.info(jobId + ", stage " + stage + ": eff=" + - String.format(PercentNumberFormat, effectiveValue) + ", thresh=" + strategy.getScaleUpAbovePct()); + logger.info("{}, stage {}, eventType {}: eff={}, thresh={}", jobId, stage, event.getType(), + String.format(PercentNumberFormat, effectiveValue), strategy.getScaleUpAbovePct()); if (stats.getHighThreshTriggered()) { - logger.info("Attempting to scale up stage " + stage + " of job " + jobId + " by " + - scalingPolicy.getIncrement() + " workers, because " + - event.type + " exceeded scaleUpThreshold of " + - String.format(PercentNumberFormat, strategy.getScaleUpAbovePct()) + " " + - stats.getCurrentHighCount() + " times"); + logger.info("Attempting to scale up stage {} of job {} by {} workers, because {} exceeded scaleUpThreshold of {} {} times", + stage, jobId, scalingPolicy.getIncrement(), event.getType(), + String.format(PercentNumberFormat, strategy.getScaleUpAbovePct()), + stats.getCurrentHighCount()); final int numCurrWorkers = event.getNumWorkers(); - final int desiredWorkers = scaler.getDesiredWorkersForScaleUp(scalingPolicy.getIncrement(), numCurrWorkers); + final int desiredWorkers = scaler.getDesiredWorkersForScaleUp(scalingPolicy.getIncrement(), numCurrWorkers, strategy.getReason()); if (desiredWorkers > numCurrWorkers) { scaler.scaleUpStage(numCurrWorkers, desiredWorkers, event.getType() + " with value " + String.format(PercentNumberFormat, effectiveValue) + @@ -476,12 +480,11 @@ public void onNext(Event event) { logger.debug("scale up NOOP: desiredWorkers same as current workers"); } } else if (stats.getLowThreshTriggered()) { - logger.info("Attempting to scale down stage " + stage + " of job " + jobId + " by " + - scalingPolicy.getDecrement() + " workers because " + event.getType() + - " is below scaleDownThreshold of " + strategy.getScaleDownBelowPct() + - " " + stats.getCurrentLowCount() + " times"); + logger.info("Attempting to scale down stage {} of job {} by {} workers, because {} exceeded scaleUpThreshold of {} {} times", + stage, jobId, scalingPolicy.getDecrement(), event.getType(), + strategy.getScaleDownBelowPct(), stats.getCurrentLowCount()); final int numCurrentWorkers = event.getNumWorkers(); - final int desiredWorkers = scaler.getDesiredWorkersForScaleDown(scalingPolicy.getDecrement(), numCurrentWorkers); + final int desiredWorkers = scaler.getDesiredWorkersForScaleDown(scalingPolicy.getDecrement(), numCurrentWorkers, strategy.getReason()); if (desiredWorkers < numCurrentWorkers) { scaler.scaleDownStage(numCurrentWorkers, desiredWorkers, event.getType() + " with value " + String.format(PercentNumberFormat, effectiveValue) + diff --git a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoscalerManager.java b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoscalerManager.java index df98ce35f..c7db0387f 100644 --- a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoscalerManager.java +++ b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoscalerManager.java @@ -39,6 +39,10 @@ default boolean isScaleDownEnabled() { return true; } + default double getCurrentValue() { + return -1.0; + } + /** * Noop implementation of {@link JobAutoscalerManager} that always returns true * for isScaleUpEnabled, isScaleDownEnabled diff --git a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandler.java b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandler.java index de31172e8..ace682843 100644 --- a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandler.java +++ b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandler.java @@ -279,6 +279,13 @@ public void call() { // get the aggregate metric values by metric group for all workers in stage Map allWorkerAggregates = getAggregates(listofAggregates); logger.info("Job stage {} avgResUsage from {} workers: {}", stage, workersMap.size(), allWorkerAggregates.toString()); + jobAutoScaleObserver.onNext( + new JobAutoScaler.Event( + StageScalingPolicy.ScalingReason.AutoscalerManager, stage, + jobAutoscalerManager.getCurrentValue(), + jobAutoscalerManager.getCurrentValue(), + numWorkers) + ); for (Map.Entry> userDefinedMetric : autoScaleMetricsConfig.getUserDefinedMetrics().entrySet()) { final String metricGrp = userDefinedMetric.getKey(); diff --git a/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/JobAutoScalerTest.java b/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/JobAutoScalerTest.java index 0feb8d6ca..68ee7ab66 100644 --- a/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/JobAutoScalerTest.java +++ b/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/JobAutoScalerTest.java @@ -146,7 +146,7 @@ public void testScalingResiliency() throws InterruptedException { new HashMap() { { put(StageScalingPolicy.ScalingReason.Memory, new StageScalingPolicy.Strategy(StageScalingPolicy.ScalingReason.Memory, scaleDownBelowPct, scaleUpAbovePct, new StageScalingPolicy.RollingCount(1, 2))); - put(StageScalingPolicy.ScalingReason.FailoverAware, new StageScalingPolicy.Strategy(StageScalingPolicy.ScalingReason.FailoverAware, 0.0, 0.0, new StageScalingPolicy.RollingCount(1, 2))); + put(StageScalingPolicy.ScalingReason.AutoscalerManager, new StageScalingPolicy.Strategy(StageScalingPolicy.ScalingReason.AutoscalerManager, 0.0, 0.0, new StageScalingPolicy.RollingCount(1, 2))); } }, false)) .scalable(true) diff --git a/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandlerTest.java b/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandlerTest.java index c784db21d..5bf419441 100644 --- a/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandlerTest.java +++ b/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandlerTest.java @@ -16,6 +16,7 @@ package io.mantisrx.server.worker.jobmaster; +import static io.mantisrx.runtime.descriptor.StageScalingPolicy.ScalingReason.AutoscalerManager; import static io.mantisrx.server.core.stats.MetricStringConstants.DATA_DROP_METRIC_GROUP; import static io.mantisrx.server.core.stats.MetricStringConstants.KAFKA_CONSUMER_FETCH_MGR_METRIC_GROUP; import static io.reactivex.mantis.network.push.PushServerSse.DROPPED_COUNTER_METRIC_NAME; @@ -36,6 +37,8 @@ import io.mantisrx.server.core.stats.MetricStringConstants; import io.mantisrx.server.master.client.MantisMasterClientApi; import io.mantisrx.shaded.com.google.common.collect.ImmutableMap; + +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -78,10 +81,11 @@ public void testDropDataMetricTriggersAutoScale() throws InterruptedException { Collections.singletonMap(1, new WorkerHost("localhost", workerIdx, Arrays.asList(31300), MantisJobState.Started, workerNum, 31301, -1)))); when(mockMasterClientApi.schedulingChanges(jobId)).thenReturn(Observable.just(new JobSchedulingInfo(jobId, assignmentsMap))); - final CountDownLatch latch = new CountDownLatch(1); + final CountDownLatch latch = new CountDownLatch(2); final AutoScaleMetricsConfig aggregationConfig = new AutoScaleMetricsConfig(); + final List events = new ArrayList<>(); final WorkerMetricHandler workerMetricHandler = new WorkerMetricHandler(jobId, new Observer() { @Override public void onCompleted() { @@ -96,8 +100,15 @@ public void onError(Throwable e) { @Override public void onNext(JobAutoScaler.Event event) { logger.info("got auto scale event {}", event); - JobAutoScaler.Event expected = new JobAutoScaler.Event(StageScalingPolicy.ScalingReason.DataDrop, stage, dropPercent, dropPercent, 1); - assertEquals(expected, event); + long count = latch.getCount(); + if (count == 2) { + JobAutoScaler.Event expected = new JobAutoScaler.Event(AutoscalerManager, 1, -1.0, -1.0, 1); + assertEquals(expected, event); + } + if (count == 1) { + JobAutoScaler.Event expected = new JobAutoScaler.Event(StageScalingPolicy.ScalingReason.DataDrop, stage, dropPercent, dropPercent, 1); + assertEquals(expected, event); + } latch.countDown(); } }, mockMasterClientApi, aggregationConfig, JobAutoscalerManager.DEFAULT); @@ -108,6 +119,7 @@ public void onNext(JobAutoScaler.Event event) { metricDataObserver.onNext(new MetricData(new String(jobId), stage, workerIdx, workerNum, DATA_DROP_METRIC_GROUP, gauges)); assertTrue(latch.await(30 + 5/* leeway */, TimeUnit.SECONDS)); + } @Test @@ -144,7 +156,7 @@ public void testKafkaLagAndUserDefinedTriggersAutoScale() throws InterruptedExce assignmentsMap.put(stage, new WorkerAssignments(stage, numWorkers, hosts)); when(mockMasterClientApi.schedulingChanges(jobId)).thenReturn(Observable.just(new JobSchedulingInfo(jobId, assignmentsMap))); - final CountDownLatch latch = new CountDownLatch(2); + final CountDownLatch latch = new CountDownLatch(3); final AutoScaleMetricsConfig aggregationConfig = new AutoScaleMetricsConfig(Collections.singletonMap(testMetricGroup, Collections.singletonMap(testMetricName, AutoScaleMetricsConfig.AggregationAlgo.AVERAGE))); @@ -163,6 +175,10 @@ public void onError(Throwable e) { public void onNext(JobAutoScaler.Event event) { logger.info("got auto scale event {}", event); final long count = latch.getCount(); + if (count == 3) { + JobAutoScaler.Event expected = new JobAutoScaler.Event(AutoscalerManager, 1, -1.0, -1.0, numWorkers); + assertEquals(expected, event); + } if (count == 2) { JobAutoScaler.Event expected1 = new JobAutoScaler.Event(StageScalingPolicy.ScalingReason.UserDefined, stage, metricValue * 3 / 4, metricValue * 3 / 4, numWorkers); assertEquals(expected1, event); @@ -217,7 +233,7 @@ public void testOutlierResubmitWorks() throws InterruptedException { assignmentsMap.put(stage, new WorkerAssignments(stage, numWorkers, hosts)); final CountDownLatch resubmitLatch = new CountDownLatch(1); - final CountDownLatch autoScaleLatch = new CountDownLatch(1); + final CountDownLatch autoScaleLatch = new CountDownLatch(2); when(mockMasterClientApi.schedulingChanges(jobId)).thenReturn(Observable.just(new JobSchedulingInfo(jobId, assignmentsMap))); when(mockMasterClientApi.resubmitJobWorker(anyString(), anyString(), anyInt(), anyString())).thenAnswer(new Answer>() { @@ -262,8 +278,15 @@ public void onError(Throwable e) { @Override public void onNext(JobAutoScaler.Event event) { logger.info("got auto scale event {}", event); - JobAutoScaler.Event expected = new JobAutoScaler.Event(StageScalingPolicy.ScalingReason.DataDrop, stage, dropPercent / numWorkers, dropPercent / numWorkers, numWorkers); - assertEquals(expected, event); + long count = autoScaleLatch.getCount(); + if (count == 2) { + JobAutoScaler.Event expected = new JobAutoScaler.Event(AutoscalerManager, 1, -1.0, -1.0, numWorkers); + assertEquals(expected, event); + } + if (count == 1) { + JobAutoScaler.Event expected = new JobAutoScaler.Event(StageScalingPolicy.ScalingReason.DataDrop, stage, dropPercent / numWorkers, dropPercent / numWorkers, numWorkers); + assertEquals(expected, event); + } autoScaleLatch.countDown(); } }, mockMasterClientApi, aggregationConfig, JobAutoscalerManager.DEFAULT); From 4b2e251bfbf10bbd4b0afa8b0cdf4a3a51fab838 Mon Sep 17 00:00:00 2001 From: Harshit Mittal Date: Thu, 4 Jul 2024 00:17:52 -0700 Subject: [PATCH 5/8] fix test --- .../server/worker/jobmaster/WorkerMetricHandlerTest.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandlerTest.java b/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandlerTest.java index 5bf419441..929ea570a 100644 --- a/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandlerTest.java +++ b/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandlerTest.java @@ -182,13 +182,12 @@ public void onNext(JobAutoScaler.Event event) { if (count == 2) { JobAutoScaler.Event expected1 = new JobAutoScaler.Event(StageScalingPolicy.ScalingReason.UserDefined, stage, metricValue * 3 / 4, metricValue * 3 / 4, numWorkers); assertEquals(expected1, event); - latch.countDown(); } if (count == 1) { JobAutoScaler.Event expected2 = new JobAutoScaler.Event(StageScalingPolicy.ScalingReason.KafkaLag, stage, kafkaLag, kafkaLag, numWorkers); assertEquals(expected2, event); - latch.countDown(); } + latch.countDown(); } }, mockMasterClientApi, aggregationConfig, JobAutoscalerManager.DEFAULT); From f099c4719c7c85faf561dc8a7d79c2782c12ec87 Mon Sep 17 00:00:00 2001 From: Harshit Mittal Date: Mon, 8 Jul 2024 21:25:01 -0700 Subject: [PATCH 6/8] update --- docs/docs/operate/autoscalingstrategies.md | 2 +- .../worker/jobmaster/JobAutoScaler.java | 41 +++++++++++-------- .../jobmaster/JobAutoscalerManager.java | 2 +- .../worker/jobmaster/WorkerMetricHandler.java | 21 ++++++---- .../jobmaster/WorkerMetricHandlerTest.java | 1 - 5 files changed, 41 insertions(+), 26 deletions(-) diff --git a/docs/docs/operate/autoscalingstrategies.md b/docs/docs/operate/autoscalingstrategies.md index b7f6bc4e8..d19696269 100644 --- a/docs/docs/operate/autoscalingstrategies.md +++ b/docs/docs/operate/autoscalingstrategies.md @@ -56,7 +56,7 @@ scaling action, the cooldown will prevent subsequent strategies from scaling for ### AutoscalerManagerEvent Strategy This is a custom strategy to set a target worker size at runtime. The strategy uses `getCurrentValue` from `JobAutoScalerManager` to determine the target worker size. -For a non-negative value `[0.0, 1.0]`, the autoscaler will scale the stage from min to max. +For a non-negative value `[0.0, 100.0]`, the autoscaler will scale the stage from min to max. All other values are ignored. Default implementation returns a -1.0 for `currentValue` meaning a no-op for event based scaling. Example #1: Use-case during region failover in a multi-region setup diff --git a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoScaler.java b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoScaler.java index 7b483d0d8..ee1555e45 100644 --- a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoScaler.java +++ b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoScaler.java @@ -326,9 +326,16 @@ private void setOutstandingScalingRequest(final Subscription subscription) { inProgressScalingSubscription.compareAndSet(null, subscription); } - public int getDesiredWorkersForScaleUp(final int increment, final int numCurrentWorkers, ScalingReason reason) { + private int getDesiredWorkers(StageScalingPolicy scalingPolicy, Event event) { + final int maxWorkersForStage = scalingPolicy.getMax(); + final int minWorkersForStage = scalingPolicy.getMin(); + return minWorkersForStage + (int) Math.round((maxWorkersForStage - minWorkersForStage) * event.getEffectiveValue() / 100.0); + } + + public int getDesiredWorkersForScaleUp(final int increment, final int numCurrentWorkers, Event event) { final int desiredWorkers; - if (!stageSchedulingInfo.getScalingPolicy().isEnabled()) { + final StageScalingPolicy scalingPolicy = stageSchedulingInfo.getScalingPolicy(); + if (!scalingPolicy.isEnabled()) { logger.warn("Job {} stage {} is not scalable, can't increment #workers by {}", jobId, stage, increment); return numCurrentWorkers; } @@ -336,19 +343,18 @@ public int getDesiredWorkersForScaleUp(final int increment, final int numCurrent if (numCurrentWorkers < 0 || increment < 1) { logger.error("current number of workers({}) not known or increment({}) < 1, will not scale up", numCurrentWorkers, increment); return numCurrentWorkers; - } else if (stageSchedulingInfo.getScalingPolicy().isAllowAutoScaleManager() && !jobAutoscalerManager.isScaleUpEnabled()) { + } else if (scalingPolicy.isAllowAutoScaleManager() && !jobAutoscalerManager.isScaleUpEnabled()) { logger.warn("Scaleup is disabled for all autoscaling strategy, not scaling up stage {} of job {}", stage, jobId); return numCurrentWorkers; + } else if (event.getType() == ScalingReason.AutoscalerManagerEvent) { + desiredWorkers = getDesiredWorkers(scalingPolicy, event); + logger.info("AutoscalerManagerEvent scaling up stage {} of job {} to desiredWorkers {}", stage, jobId, desiredWorkers); } else { - final int maxWorkersForStage = stageSchedulingInfo.getScalingPolicy().getMax(); - if (reason == ScalingReason.AutoscalerManagerEvent) { - logger.info("AutoscalerManagerEvent scaling up stage {} of job {} to maxWorkersForStage {}", stage, jobId, maxWorkersForStage); - desiredWorkers = maxWorkersForStage; - } else { - desiredWorkers = Math.min(numCurrentWorkers + increment, maxWorkersForStage); - } - return desiredWorkers; + final int maxWorkersForStage = scalingPolicy.getMax(); + desiredWorkers = Math.min(numCurrentWorkers + increment, maxWorkersForStage); } + return desiredWorkers; + } public void scaleUpStage(final int numCurrentWorkers, final int desiredWorkers, final String reason) { @@ -369,7 +375,7 @@ public void scaleUpStage(final int numCurrentWorkers, final int desiredWorkers, setOutstandingScalingRequest(subscription); } - public int getDesiredWorkersForScaleDown(final int decrement, final int numCurrentWorkers, ScalingReason reason) { + public int getDesiredWorkersForScaleDown(final int decrement, final int numCurrentWorkers, Event event) { final int desiredWorkers; final StageScalingPolicy scalingPolicy = stageSchedulingInfo.getScalingPolicy(); if (!scalingPolicy.isEnabled()) { @@ -382,9 +388,12 @@ public int getDesiredWorkersForScaleDown(final int decrement, final int numCurre } else if (scalingPolicy.isAllowAutoScaleManager() && !jobAutoscalerManager.isScaleDownEnabled()) { logger.warn("Scaledown is disabled for all autoscaling strategy, not scaling down stage {} of job {}", stage, jobId); return numCurrentWorkers; + } else if (event.getType() == ScalingReason.AutoscalerManagerEvent) { + desiredWorkers = getDesiredWorkers(scalingPolicy, event); + logger.info("AutoscalerManagerEvent scaling up stage {} of job {} to desiredWorkers {}", stage, jobId, desiredWorkers); } else { - int min = scalingPolicy.getMin(); - desiredWorkers = Math.max(numCurrentWorkers - decrement, min); + int min = scalingPolicy.getMin(); + desiredWorkers = Math.max(numCurrentWorkers - decrement, min); } return desiredWorkers; } @@ -470,7 +479,7 @@ public void onNext(Event event) { String.format(PercentNumberFormat, strategy.getScaleUpAbovePct()), stats.getCurrentHighCount()); final int numCurrWorkers = event.getNumWorkers(); - final int desiredWorkers = scaler.getDesiredWorkersForScaleUp(scalingPolicy.getIncrement(), numCurrWorkers, strategy.getReason()); + final int desiredWorkers = scaler.getDesiredWorkersForScaleUp(scalingPolicy.getIncrement(), numCurrWorkers, event); if (desiredWorkers > numCurrWorkers) { scaler.scaleUpStage(numCurrWorkers, desiredWorkers, event.getType() + " with value " + String.format(PercentNumberFormat, effectiveValue) + @@ -485,7 +494,7 @@ public void onNext(Event event) { stage, jobId, scalingPolicy.getDecrement(), event.getType(), strategy.getScaleDownBelowPct(), stats.getCurrentLowCount()); final int numCurrentWorkers = event.getNumWorkers(); - final int desiredWorkers = scaler.getDesiredWorkersForScaleDown(scalingPolicy.getDecrement(), numCurrentWorkers, strategy.getReason()); + final int desiredWorkers = scaler.getDesiredWorkersForScaleDown(scalingPolicy.getDecrement(), numCurrentWorkers, event); if (desiredWorkers < numCurrentWorkers) { scaler.scaleDownStage(numCurrentWorkers, desiredWorkers, event.getType() + " with value " + String.format(PercentNumberFormat, effectiveValue) + diff --git a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoscalerManager.java b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoscalerManager.java index 225406534..b99aabc4e 100644 --- a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoscalerManager.java +++ b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/JobAutoscalerManager.java @@ -44,7 +44,7 @@ default boolean isScaleDownEnabled() { /** * Get the current fractional value to set size for stage numWorkers. - * Valid values are [0.0, 1.0] which set numWorkers from [min, max]. + * Valid values are [0.0, 100.0] which set numWorkers from [min, max]. * All other values are ignored for scaling decisions. */ default double getCurrentValue() { diff --git a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandler.java b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandler.java index b48cc8606..64416e630 100644 --- a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandler.java +++ b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandler.java @@ -279,13 +279,7 @@ public void call() { // get the aggregate metric values by metric group for all workers in stage Map allWorkerAggregates = getAggregates(listofAggregates); logger.info("Job stage {} avgResUsage from {} workers: {}", stage, workersMap.size(), allWorkerAggregates.toString()); - jobAutoScaleObserver.onNext( - new JobAutoScaler.Event( - StageScalingPolicy.ScalingReason.AutoscalerManagerEvent, stage, - jobAutoscalerManager.getCurrentValue(), - jobAutoscalerManager.getCurrentValue(), - numWorkers) - ); + maybeEmitAutoscalerManagerEvent(numWorkers); for (Map.Entry> userDefinedMetric : autoScaleMetricsConfig.getUserDefinedMetrics().entrySet()) { final String metricGrp = userDefinedMetric.getKey(); @@ -423,6 +417,19 @@ public void onNext(MetricData metricData) { }; } + private void maybeEmitAutoscalerManagerEvent(int numWorkers) { + final double currentValue = jobAutoscalerManager.getCurrentValue(); + if (currentValue >= 0.0 && currentValue <= 100.0) { + jobAutoScaleObserver.onNext( + new JobAutoScaler.Event( + StageScalingPolicy.ScalingReason.AutoscalerManagerEvent, stage, + currentValue, + currentValue, + numWorkers) + ); + } + } + private void addScalerEventForSourceJobDrops(int numWorkers) { double sourceJobDrops = 0; boolean hasSourceJobDropsMetric = false; diff --git a/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandlerTest.java b/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandlerTest.java index 494381f83..7a53f2471 100644 --- a/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandlerTest.java +++ b/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandlerTest.java @@ -37,7 +37,6 @@ import io.mantisrx.server.core.stats.MetricStringConstants; import io.mantisrx.server.master.client.MantisMasterClientApi; import io.mantisrx.shaded.com.google.common.collect.ImmutableMap; - import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; From 9c0c51b425be7d0fe44e4935be7e5df5ab8b27ad Mon Sep 17 00:00:00 2001 From: Harshit Mittal Date: Mon, 8 Jul 2024 23:48:47 -0700 Subject: [PATCH 7/8] update tests --- .../jobmaster/WorkerMetricHandlerTest.java | 19 +++---------------- 1 file changed, 3 insertions(+), 16 deletions(-) diff --git a/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandlerTest.java b/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandlerTest.java index 7a53f2471..cfa81ce71 100644 --- a/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandlerTest.java +++ b/mantis-runtime-executor/src/test/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandlerTest.java @@ -16,7 +16,6 @@ package io.mantisrx.server.worker.jobmaster; -import static io.mantisrx.runtime.descriptor.StageScalingPolicy.ScalingReason.AutoscalerManagerEvent; import static io.mantisrx.server.core.stats.MetricStringConstants.DATA_DROP_METRIC_GROUP; import static io.mantisrx.server.core.stats.MetricStringConstants.KAFKA_CONSUMER_FETCH_MGR_METRIC_GROUP; import static io.reactivex.mantis.network.push.PushServerSse.DROPPED_COUNTER_METRIC_NAME; @@ -80,7 +79,7 @@ public void testDropDataMetricTriggersAutoScale() throws InterruptedException { Collections.singletonMap(1, new WorkerHost("localhost", workerIdx, Arrays.asList(31300), MantisJobState.Started, workerNum, 31301, -1)))); when(mockMasterClientApi.schedulingChanges(jobId)).thenReturn(Observable.just(new JobSchedulingInfo(jobId, assignmentsMap))); - final CountDownLatch latch = new CountDownLatch(2); + final CountDownLatch latch = new CountDownLatch(1); final AutoScaleMetricsConfig aggregationConfig = new AutoScaleMetricsConfig(); @@ -100,10 +99,6 @@ public void onError(Throwable e) { public void onNext(JobAutoScaler.Event event) { logger.info("got auto scale event {}", event); long count = latch.getCount(); - if (count == 2) { - JobAutoScaler.Event expected = new JobAutoScaler.Event(AutoscalerManagerEvent, 1, -1.0, -1.0, 1); - assertEquals(expected, event); - } if (count == 1) { JobAutoScaler.Event expected = new JobAutoScaler.Event(StageScalingPolicy.ScalingReason.DataDrop, stage, dropPercent, dropPercent, 1); assertEquals(expected, event); @@ -155,7 +150,7 @@ public void testKafkaLagAndUserDefinedTriggersAutoScale() throws InterruptedExce assignmentsMap.put(stage, new WorkerAssignments(stage, numWorkers, hosts)); when(mockMasterClientApi.schedulingChanges(jobId)).thenReturn(Observable.just(new JobSchedulingInfo(jobId, assignmentsMap))); - final CountDownLatch latch = new CountDownLatch(3); + final CountDownLatch latch = new CountDownLatch(2); final AutoScaleMetricsConfig aggregationConfig = new AutoScaleMetricsConfig(Collections.singletonMap(testMetricGroup, Collections.singletonMap(testMetricName, AutoScaleMetricsConfig.AggregationAlgo.AVERAGE))); @@ -174,10 +169,6 @@ public void onError(Throwable e) { public void onNext(JobAutoScaler.Event event) { logger.info("got auto scale event {}", event); final long count = latch.getCount(); - if (count == 3) { - JobAutoScaler.Event expected = new JobAutoScaler.Event(AutoscalerManagerEvent, 1, -1.0, -1.0, numWorkers); - assertEquals(expected, event); - } if (count == 2) { JobAutoScaler.Event expected1 = new JobAutoScaler.Event(StageScalingPolicy.ScalingReason.UserDefined, stage, metricValue * 3 / 4, metricValue * 3 / 4, numWorkers); assertEquals(expected1, event); @@ -231,7 +222,7 @@ public void testOutlierResubmitWorks() throws InterruptedException { assignmentsMap.put(stage, new WorkerAssignments(stage, numWorkers, hosts)); final CountDownLatch resubmitLatch = new CountDownLatch(1); - final CountDownLatch autoScaleLatch = new CountDownLatch(2); + final CountDownLatch autoScaleLatch = new CountDownLatch(1); when(mockMasterClientApi.schedulingChanges(jobId)).thenReturn(Observable.just(new JobSchedulingInfo(jobId, assignmentsMap))); when(mockMasterClientApi.resubmitJobWorker(anyString(), anyString(), anyInt(), anyString())).thenAnswer(new Answer>() { @@ -277,10 +268,6 @@ public void onError(Throwable e) { public void onNext(JobAutoScaler.Event event) { logger.info("got auto scale event {}", event); long count = autoScaleLatch.getCount(); - if (count == 2) { - JobAutoScaler.Event expected = new JobAutoScaler.Event(AutoscalerManagerEvent, 1, -1.0, -1.0, numWorkers); - assertEquals(expected, event); - } if (count == 1) { JobAutoScaler.Event expected = new JobAutoScaler.Event(StageScalingPolicy.ScalingReason.DataDrop, stage, dropPercent / numWorkers, dropPercent / numWorkers, numWorkers); assertEquals(expected, event); From 4b8346ba3b3746fdd1ab4a183ef008839b8da1a0 Mon Sep 17 00:00:00 2001 From: Harshit Mittal Date: Wed, 10 Jul 2024 15:59:53 -0700 Subject: [PATCH 8/8] use service locator instead for initializing JobAutoScalerManager, add inline comment --- .../WorkerExecutionOperationsNetworkStage.java | 18 ++++-------------- .../worker/jobmaster/WorkerMetricHandler.java | 2 ++ 2 files changed, 6 insertions(+), 14 deletions(-) diff --git a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/WorkerExecutionOperationsNetworkStage.java b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/WorkerExecutionOperationsNetworkStage.java index a6c1ada89..6056dfa12 100644 --- a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/WorkerExecutionOperationsNetworkStage.java +++ b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/WorkerExecutionOperationsNetworkStage.java @@ -66,8 +66,6 @@ import io.reactivex.mantis.remote.observable.ToDeltaEndpointInjector; import java.io.Closeable; import java.io.IOException; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; import java.time.Duration; import java.time.temporal.ChronoUnit; import java.util.ArrayList; @@ -78,7 +76,6 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.Properties; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; @@ -416,7 +413,7 @@ public void executeStage(final ExecutionDetails setup) throws IOException { logger.info("param {} is null or empty", JOB_MASTER_AUTOSCALE_METRIC_SYSTEM_PARAM); } - JobAutoscalerManager jobAutoscalerManager = getJobAutoscalerManagerInstance(config); + JobAutoscalerManager jobAutoscalerManager = getJobAutoscalerManagerInstance(serviceLocator); JobMasterService jobMasterService = new JobMasterService(rw.getJobId(), rw.getSchedulingInfo(), workerMetricsClient, autoScaleMetricsConfig, mantisMasterApi, rw.getContext(), rw.getOnCompleteCallback(), rw.getOnErrorCallback(), rw.getOnTerminateCallback(), jobAutoscalerManager); jobMasterService.start(); @@ -482,16 +479,9 @@ remoteObservableName, numWorkersAtStage(selfSchedulingInfo, rw.getJobId(), rw.ge } } - private JobAutoscalerManager getJobAutoscalerManagerInstance(WorkerConfiguration config) { - try { - Class jobAutoscalerManagerClass = Class.forName(config.getJobAutoscalerManagerClassName()); - logger.info("Picking {} jobAutoscalerManager", jobAutoscalerManagerClass.getName()); - Method managerClassFactory = jobAutoscalerManagerClass.getMethod("valueOf", Properties.class); - return (JobAutoscalerManager) managerClassFactory.invoke(null, System.getProperties()); - } catch (ClassNotFoundException | NoSuchMethodException | InvocationTargetException | IllegalAccessException e) { - logger.warn("Couldnt instantiate jobAutoscalerManager from class {} because ", config.getJobAutoscalerManagerClassName(), e); - return JobAutoscalerManager.DEFAULT; - } + private JobAutoscalerManager getJobAutoscalerManagerInstance(ServiceLocator serviceLocator) { + final JobAutoscalerManager autoscalerManager = serviceLocator.service(JobAutoscalerManager.class); + return Optional.ofNullable(autoscalerManager).orElse(JobAutoscalerManager.DEFAULT); } private void setupSubscriptionStateHandler(ExecuteStageRequest executeStageRequest) { diff --git a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandler.java b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandler.java index 64416e630..658bc8378 100644 --- a/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandler.java +++ b/mantis-runtime-executor/src/main/java/io/mantisrx/server/worker/jobmaster/WorkerMetricHandler.java @@ -419,6 +419,8 @@ public void onNext(MetricData metricData) { private void maybeEmitAutoscalerManagerEvent(int numWorkers) { final double currentValue = jobAutoscalerManager.getCurrentValue(); + // The effective value is a pct value and hence ranges from [0, 100]. + // Ignore all other values to disable autoscaling for custom events. if (currentValue >= 0.0 && currentValue <= 100.0) { jobAutoScaleObserver.onNext( new JobAutoScaler.Event(