-
Notifications
You must be signed in to change notification settings - Fork 433
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-37126] Add Validator for Autoscaler
- Loading branch information
huyuanfeng
committed
Feb 6, 2025
1 parent
4c2c90c
commit 81de787
Showing
5 changed files
with
237 additions
and
60 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
87 changes: 87 additions & 0 deletions
87
...c/test/java/org/apache/flink/autoscaler/standalone/StandaloneAutoscalerValidatorTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,87 @@ | ||
package org.apache.flink.autoscaler.standalone; | ||
|
||
import org.apache.flink.api.common.JobID; | ||
import org.apache.flink.api.common.JobStatus; | ||
import org.apache.flink.autoscaler.JobAutoScaler; | ||
import org.apache.flink.autoscaler.JobAutoScalerContext; | ||
import org.apache.flink.autoscaler.config.AutoScalerOptions; | ||
import org.apache.flink.autoscaler.event.TestingEventCollector; | ||
import org.apache.flink.configuration.Configuration; | ||
|
||
import org.junit.jupiter.api.BeforeEach; | ||
import org.junit.jupiter.api.Test; | ||
|
||
import java.util.ArrayList; | ||
import java.util.List; | ||
import java.util.concurrent.CompletableFuture; | ||
import java.util.concurrent.ConcurrentHashMap; | ||
|
||
import static org.assertj.core.api.Assertions.assertThat; | ||
|
||
class StandaloneAutoscalerValidatorTest { | ||
private List<JobAutoScalerContext<JobID>> jobList; | ||
private TestingEventCollector<JobID, JobAutoScalerContext<JobID>> eventCollector; | ||
private ConcurrentHashMap<JobID, Integer> scaleCounter; | ||
private Configuration correctConfiguration; | ||
private Configuration invalidConfiguration; | ||
|
||
@BeforeEach | ||
void setUp() { | ||
jobList = new ArrayList<>(); | ||
eventCollector = new TestingEventCollector<>(); | ||
scaleCounter = new ConcurrentHashMap<>(); | ||
|
||
correctConfiguration = new Configuration(); | ||
correctConfiguration.set(AutoScalerOptions.AUTOSCALER_ENABLED, true); | ||
|
||
invalidConfiguration = new Configuration(); | ||
invalidConfiguration.set(AutoScalerOptions.AUTOSCALER_ENABLED, true); | ||
invalidConfiguration.set(AutoScalerOptions.MAX_SCALE_DOWN_FACTOR, -1.0); | ||
} | ||
|
||
@Test | ||
void testAutoScalerWithInvalidConfig() { | ||
JobAutoScalerContext<JobID> validJob = createJobAutoScalerContext(correctConfiguration); | ||
JobAutoScalerContext<JobID> invalidJob = createJobAutoScalerContext(invalidConfiguration); | ||
|
||
jobList.add(validJob); | ||
jobList.add(invalidJob); | ||
|
||
try (StandaloneAutoscalerExecutor<JobID, JobAutoScalerContext<JobID>> autoscalerExecutor = | ||
new StandaloneAutoscalerExecutor<>( | ||
new Configuration(), | ||
baseConf -> jobList, | ||
eventCollector, | ||
new JobAutoScaler<>() { | ||
@Override | ||
public void scale(JobAutoScalerContext<JobID> context) { | ||
scaleCounter.merge(context.getJobKey(), 1, Integer::sum); | ||
} | ||
|
||
@Override | ||
public void cleanup(JobAutoScalerContext<JobID> context) { | ||
// No cleanup required for the test | ||
} | ||
})) { | ||
|
||
List<CompletableFuture<Void>> scaledFutures = autoscalerExecutor.scaling(); | ||
|
||
// Verification triggers two scaling tasks | ||
assertThat(scaledFutures).hasSize(2); | ||
|
||
// Only legally configured tasks are scaled | ||
assertThat(scaleCounter).hasSize(1).containsKey(validJob.getJobKey()); | ||
|
||
// Verification Event Collector captures an event | ||
assertThat(eventCollector.events).hasSize(1); | ||
assertThat(eventCollector.events) | ||
.allMatch(event -> event.getContext().equals(invalidJob)); | ||
} | ||
} | ||
|
||
private JobAutoScalerContext<JobID> createJobAutoScalerContext(Configuration configuration) { | ||
JobID jobID = new JobID(); | ||
return new JobAutoScalerContext<>( | ||
jobID, jobID, JobStatus.RUNNING, configuration, null, null); | ||
} | ||
} |
35 changes: 35 additions & 0 deletions
35
...-autoscaler/src/main/java/org/apache/flink/autoscaler/validation/AutoscalerValidator.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,35 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.flink.autoscaler.validation; | ||
|
||
import org.apache.flink.configuration.Configuration; | ||
|
||
import java.util.Optional; | ||
|
||
/** Validator for Autoscaler. */ | ||
public interface AutoscalerValidator { | ||
|
||
/** | ||
* Validate autoscaler config and return optional error. | ||
* | ||
* @param flinkConf autoscaler config | ||
* @return Optional error string, should be present iff validation resulted in an error | ||
*/ | ||
Optional<String> validateAutoscalerOptions(Configuration flinkConf); | ||
} |
90 changes: 90 additions & 0 deletions
90
...aler/src/main/java/org/apache/flink/autoscaler/validation/DefaultAutoscalerValidator.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,90 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.flink.autoscaler.validation; | ||
|
||
import org.apache.flink.autoscaler.config.AutoScalerOptions; | ||
import org.apache.flink.autoscaler.utils.CalendarUtils; | ||
import org.apache.flink.configuration.ConfigOption; | ||
import org.apache.flink.configuration.Configuration; | ||
|
||
import java.util.Optional; | ||
|
||
import static org.apache.flink.autoscaler.config.AutoScalerOptions.UTILIZATION_MAX; | ||
import static org.apache.flink.autoscaler.config.AutoScalerOptions.UTILIZATION_MIN; | ||
import static org.apache.flink.autoscaler.config.AutoScalerOptions.UTILIZATION_TARGET; | ||
|
||
/** Default implementation of {@link AutoscalerValidator}. */ | ||
public class DefaultAutoscalerValidator implements AutoscalerValidator { | ||
|
||
public Optional<String> validateAutoscalerOptions(Configuration flinkConf) { | ||
|
||
if (!flinkConf.getBoolean(AutoScalerOptions.AUTOSCALER_ENABLED)) { | ||
return Optional.empty(); | ||
} | ||
return firstPresent( | ||
validateNumber(flinkConf, AutoScalerOptions.MAX_SCALE_DOWN_FACTOR, 0.0d), | ||
validateNumber(flinkConf, AutoScalerOptions.MAX_SCALE_UP_FACTOR, 0.0d), | ||
validateNumber(flinkConf, UTILIZATION_TARGET, 0.0d, 1.0d), | ||
validateNumber(flinkConf, AutoScalerOptions.TARGET_UTILIZATION_BOUNDARY, 0.0d), | ||
validateNumber(flinkConf, UTILIZATION_MAX, flinkConf.get(UTILIZATION_TARGET), 1.0d), | ||
validateNumber(flinkConf, UTILIZATION_MIN, 0.0d, flinkConf.get(UTILIZATION_TARGET)), | ||
CalendarUtils.validateExcludedPeriods(flinkConf)); | ||
} | ||
|
||
@SafeVarargs | ||
private static Optional<String> firstPresent(Optional<String>... errOpts) { | ||
for (Optional<String> opt : errOpts) { | ||
if (opt.isPresent()) { | ||
return opt; | ||
} | ||
} | ||
return Optional.empty(); | ||
} | ||
|
||
private static <T extends Number> Optional<String> validateNumber( | ||
Configuration flinkConfiguration, | ||
ConfigOption<T> autoScalerConfig, | ||
Double min, | ||
Double max) { | ||
try { | ||
var configValue = flinkConfiguration.get(autoScalerConfig); | ||
if (configValue != null) { | ||
double value = configValue.doubleValue(); | ||
if ((min != null && value < min) || (max != null && value > max)) { | ||
return Optional.of( | ||
String.format( | ||
"The AutoScalerOption %s is invalid, it should be a value within the range [%s, %s]", | ||
autoScalerConfig.key(), | ||
min != null ? min.toString() : "-Infinity", | ||
max != null ? max.toString() : "+Infinity")); | ||
} | ||
} | ||
return Optional.empty(); | ||
} catch (IllegalArgumentException e) { | ||
return Optional.of( | ||
String.format( | ||
"Invalid value in the autoscaler config %s", autoScalerConfig.key())); | ||
} | ||
} | ||
|
||
private static <T extends Number> Optional<String> validateNumber( | ||
Configuration flinkConfiguration, ConfigOption<T> autoScalerConfig, Double min) { | ||
return validateNumber(flinkConfiguration, autoScalerConfig, min, null); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters