17
17
18
18
package org .apache .flink .kubernetes .operator .validation ;
19
19
20
- import org .apache .flink .autoscaler .config . AutoScalerOptions ;
21
- import org .apache .flink .autoscaler .utils . CalendarUtils ;
20
+ import org .apache .flink .autoscaler .validation . AutoscalerValidator ;
21
+ import org .apache .flink .autoscaler .validation . DefaultAutoscalerValidator ;
22
22
import org .apache .flink .configuration .CheckpointingOptions ;
23
- import org .apache .flink .configuration .ConfigOption ;
24
23
import org .apache .flink .configuration .Configuration ;
25
24
import org .apache .flink .configuration .HighAvailabilityOptions ;
26
25
import org .apache .flink .configuration .JobManagerOptions ;
65
64
import java .util .regex .Matcher ;
66
65
import java .util .regex .Pattern ;
67
66
68
- import static org .apache .flink .autoscaler .config .AutoScalerOptions .UTILIZATION_MAX ;
69
- import static org .apache .flink .autoscaler .config .AutoScalerOptions .UTILIZATION_MIN ;
70
- import static org .apache .flink .autoscaler .config .AutoScalerOptions .UTILIZATION_TARGET ;
71
-
72
67
/** Default validator implementation for {@link FlinkDeployment}. */
73
68
public class DefaultValidator implements FlinkResourceValidator {
74
69
@@ -87,9 +82,11 @@ public class DefaultValidator implements FlinkResourceValidator {
87
82
Set .of (Constants .CONFIG_FILE_LOG4J_NAME , Constants .CONFIG_FILE_LOGBACK_NAME );
88
83
89
84
private final FlinkConfigManager configManager ;
85
+ private final AutoscalerValidator autoscalerValidator ;
90
86
91
87
public DefaultValidator (FlinkConfigManager configManager ) {
92
88
this .configManager = configManager ;
89
+ this .autoscalerValidator = new DefaultAutoscalerValidator ();
93
90
}
94
91
95
92
@ Override
@@ -597,62 +594,12 @@ private Optional<String> validateServiceAccount(String serviceAccount) {
597
594
return Optional .empty ();
598
595
}
599
596
600
- public static Optional <String > validateAutoScalerFlinkConfiguration (
597
+ public Optional <String > validateAutoScalerFlinkConfiguration (
601
598
Map <String , String > effectiveConfig ) {
602
599
if (effectiveConfig == null ) {
603
600
return Optional .empty ();
604
601
}
605
602
Configuration flinkConfiguration = Configuration .fromMap (effectiveConfig );
606
- if (!flinkConfiguration .getBoolean (AutoScalerOptions .AUTOSCALER_ENABLED )) {
607
- return Optional .empty ();
608
- }
609
- return firstPresent (
610
- validateNumber (flinkConfiguration , AutoScalerOptions .MAX_SCALE_DOWN_FACTOR , 0.0d ),
611
- validateNumber (flinkConfiguration , AutoScalerOptions .MAX_SCALE_UP_FACTOR , 0.0d ),
612
- validateNumber (flinkConfiguration , UTILIZATION_TARGET , 0.0d , 1.0d ),
613
- validateNumber (
614
- flinkConfiguration , AutoScalerOptions .TARGET_UTILIZATION_BOUNDARY , 0.0d ),
615
- validateNumber (
616
- flinkConfiguration ,
617
- UTILIZATION_MAX ,
618
- flinkConfiguration .get (UTILIZATION_TARGET ),
619
- 1.0d ),
620
- validateNumber (
621
- flinkConfiguration ,
622
- UTILIZATION_MIN ,
623
- 0.0d ,
624
- flinkConfiguration .get (UTILIZATION_TARGET )),
625
- CalendarUtils .validateExcludedPeriods (flinkConfiguration ));
626
- }
627
-
628
- private static <T extends Number > Optional <String > validateNumber (
629
- Configuration flinkConfiguration ,
630
- ConfigOption <T > autoScalerConfig ,
631
- Double min ,
632
- Double max ) {
633
- try {
634
- var configValue = flinkConfiguration .get (autoScalerConfig );
635
- if (configValue != null ) {
636
- double value = configValue .doubleValue ();
637
- if ((min != null && value < min ) || (max != null && value > max )) {
638
- return Optional .of (
639
- String .format (
640
- "The AutoScalerOption %s is invalid, it should be a value within the range [%s, %s]" ,
641
- autoScalerConfig .key (),
642
- min != null ? min .toString () : "-Infinity" ,
643
- max != null ? max .toString () : "+Infinity" ));
644
- }
645
- }
646
- return Optional .empty ();
647
- } catch (IllegalArgumentException e ) {
648
- return Optional .of (
649
- String .format (
650
- "Invalid value in the autoscaler config %s" , autoScalerConfig .key ()));
651
- }
652
- }
653
-
654
- private static <T extends Number > Optional <String > validateNumber (
655
- Configuration flinkConfiguration , ConfigOption <T > autoScalerConfig , Double min ) {
656
- return validateNumber (flinkConfiguration , autoScalerConfig , min , null );
603
+ return autoscalerValidator .validateAutoscalerOptions (flinkConfiguration );
657
604
}
658
605
}
0 commit comments