gyfora commented on code in PR #682:
URL: 
https://github.com/apache/flink-kubernetes-operator/pull/682#discussion_r1345402764


##########
flink-kubernetes-operator-autoscaler/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/validation/AutoScalerValidator.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.kubernetes.operator.autoscaler.validation;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.kubernetes.operator.api.FlinkDeployment;
+import org.apache.flink.kubernetes.operator.api.FlinkSessionJob;
+import org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec;
+import 
org.apache.flink.kubernetes.operator.autoscaler.config.AutoScalerOptions;
+import org.apache.flink.kubernetes.operator.utils.FlinkUtils;
+import org.apache.flink.kubernetes.operator.validation.FlinkResourceValidator;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Optional;
+
+/** Validator implementation for the AutoScaler from {@link Configuration}. */
+public class AutoScalerValidator implements FlinkResourceValidator {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(FlinkUtils.class);
+
+    @Override
+    public Optional<String> validateSessionJob(
+            FlinkSessionJob sessionJob, Optional<FlinkDeployment> session) {
+        LOG.info("AutoScaler Configurations validator {}", sessionJob);
+        var spec = sessionJob.getSpec();
+        if (spec.getFlinkConfiguration() != null) {
+            var flinkConfiguration = 
Configuration.fromMap(spec.getFlinkConfiguration());
+            return validateAutoScalerFlinkConfiguration(flinkConfiguration);
+        }
+        return Optional.empty();
+    }
+
+    @Override
+    public Optional<String> validateDeployment(FlinkDeployment deployment) {
+        LOG.info("AutoScaler Configurations validator {}", deployment);
+        FlinkDeploymentSpec spec = deployment.getSpec();
+        if (spec.getFlinkConfiguration() != null) {
+            var flinkConfiguration = 
Configuration.fromMap(spec.getFlinkConfiguration());
+            return validateAutoScalerFlinkConfiguration(flinkConfiguration);
+        }
+        return Optional.empty();
+    }
+
+    private Optional<String> validateAutoScalerFlinkConfiguration(
+            Configuration flinkConfiguration) {
+        return firstPresent(
+                validateNumber(flinkConfiguration, 
AutoScalerOptions.MAX_SCALE_DOWN_FACTOR, 0.0d),
+                validateNumber(flinkConfiguration, 
AutoScalerOptions.MAX_SCALE_UP_FACTOR, 1.0d),
+                validateNumber(flinkConfiguration, 
AutoScalerOptions.TARGET_UTILIZATION, 0.0d),
+                validateNumber(
+                        flinkConfiguration,
+                        AutoScalerOptions.TARGET_UTILIZATION_BOUNDARY,
+                        0.0d,
+                        1.0d));

Review Comment:
   I think we should not validate the max here.



##########
flink-kubernetes-operator-autoscaler/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/validation/AutoScalerValidator.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.kubernetes.operator.autoscaler.validation;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.kubernetes.operator.api.FlinkDeployment;
+import org.apache.flink.kubernetes.operator.api.FlinkSessionJob;
+import org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec;
+import 
org.apache.flink.kubernetes.operator.autoscaler.config.AutoScalerOptions;
+import org.apache.flink.kubernetes.operator.utils.FlinkUtils;
+import org.apache.flink.kubernetes.operator.validation.FlinkResourceValidator;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Optional;
+
+/** Validator implementation for the AutoScaler from {@link Configuration}. */
+public class AutoScalerValidator implements FlinkResourceValidator {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(FlinkUtils.class);
+
+    @Override
+    public Optional<String> validateSessionJob(
+            FlinkSessionJob sessionJob, Optional<FlinkDeployment> session) {
+        LOG.info("AutoScaler Configurations validator {}", sessionJob);
+        var spec = sessionJob.getSpec();
+        if (spec.getFlinkConfiguration() != null) {
+            var flinkConfiguration = 
Configuration.fromMap(spec.getFlinkConfiguration());
+            return validateAutoScalerFlinkConfiguration(flinkConfiguration);
+        }
+        return Optional.empty();
+    }
+
+    @Override
+    public Optional<String> validateDeployment(FlinkDeployment deployment) {
+        LOG.info("AutoScaler Configurations validator {}", deployment);
+        FlinkDeploymentSpec spec = deployment.getSpec();
+        if (spec.getFlinkConfiguration() != null) {
+            var flinkConfiguration = 
Configuration.fromMap(spec.getFlinkConfiguration());
+            return validateAutoScalerFlinkConfiguration(flinkConfiguration);
+        }
+        return Optional.empty();
+    }
+
+    private Optional<String> validateAutoScalerFlinkConfiguration(
+            Configuration flinkConfiguration) {
+        return firstPresent(
+                validateNumber(flinkConfiguration, 
AutoScalerOptions.MAX_SCALE_DOWN_FACTOR, 0.0d),
+                validateNumber(flinkConfiguration, 
AutoScalerOptions.MAX_SCALE_UP_FACTOR, 1.0d),
+                validateNumber(flinkConfiguration, 
AutoScalerOptions.TARGET_UTILIZATION, 0.0d),
+                validateNumber(
+                        flinkConfiguration,
+                        AutoScalerOptions.TARGET_UTILIZATION_BOUNDARY,
+                        0.0d,
+                        1.0d));
+    }
+
+    private <T extends Number> Optional<String> validateNumber(
+            Configuration flinkConfiguration,
+            ConfigOption<T> autoScalerConfig,
+            Double min,
+            Double max) {
+        var configValue = flinkConfiguration.get(autoScalerConfig);

Review Comment:
   This line may throw an exception if the config value is not actually a 
number, we should catch and turn it into a validation error probably.



##########
e2e-tests/data/autoscaler.yaml:
##########
@@ -41,6 +41,13 @@ spec:
     kubernetes.operator.job.autoscaler.stabilization.interval: "5s"
     kubernetes.operator.job.autoscaler.metrics.window: "1m"
 
+#    Invalid Validations for testing autoscaler configurations
+  #    kubernetes.operator.job.autoscaler.scale-down.max-factor: "-0.6"
+  #    kubernetes.operator.job.autoscaler.scale-up.max-factor: "100001.0"
+  #    pipeline.max-parallelism: "2"
+  #    kubernetes.operator.job.autoscaler.target.utilization: "10.7"

Review Comment:
   we should update this as the large values are not invalid anymore right?



##########
flink-kubernetes-operator-autoscaler/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/validation/AutoScalerValidator.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.kubernetes.operator.autoscaler.validation;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.kubernetes.operator.api.FlinkDeployment;
+import org.apache.flink.kubernetes.operator.api.FlinkSessionJob;
+import org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec;
+import 
org.apache.flink.kubernetes.operator.autoscaler.config.AutoScalerOptions;
+import org.apache.flink.kubernetes.operator.utils.FlinkUtils;
+import org.apache.flink.kubernetes.operator.validation.FlinkResourceValidator;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Optional;
+
+/** Validator implementation for the AutoScaler from {@link Configuration}. */
+public class AutoScalerValidator implements FlinkResourceValidator {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(FlinkUtils.class);
+
+    @Override
+    public Optional<String> validateSessionJob(
+            FlinkSessionJob sessionJob, Optional<FlinkDeployment> session) {
+        LOG.info("AutoScaler Configurations validator {}", sessionJob);
+        var spec = sessionJob.getSpec();
+        if (spec.getFlinkConfiguration() != null) {
+            var flinkConfiguration = 
Configuration.fromMap(spec.getFlinkConfiguration());
+            return validateAutoScalerFlinkConfiguration(flinkConfiguration);
+        }
+        return Optional.empty();
+    }
+
+    @Override
+    public Optional<String> validateDeployment(FlinkDeployment deployment) {
+        LOG.info("AutoScaler Configurations validator {}", deployment);
+        FlinkDeploymentSpec spec = deployment.getSpec();
+        if (spec.getFlinkConfiguration() != null) {
+            var flinkConfiguration = 
Configuration.fromMap(spec.getFlinkConfiguration());
+            return validateAutoScalerFlinkConfiguration(flinkConfiguration);
+        }
+        return Optional.empty();
+    }
+
+    private Optional<String> validateAutoScalerFlinkConfiguration(
+            Configuration flinkConfiguration) {
+        return firstPresent(
+                validateNumber(flinkConfiguration, 
AutoScalerOptions.MAX_SCALE_DOWN_FACTOR, 0.0d),
+                validateNumber(flinkConfiguration, 
AutoScalerOptions.MAX_SCALE_UP_FACTOR, 1.0d),

Review Comment:
   I think this may be incorrect, MAX_SCALE_UP factor can be anything positive. 
Probably best to simplify all validations as "non-negative" 



##########
flink-kubernetes-operator-autoscaler/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/validation/AutoScalerValidator.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.kubernetes.operator.autoscaler.validation;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.kubernetes.operator.api.FlinkDeployment;
+import org.apache.flink.kubernetes.operator.api.FlinkSessionJob;
+import org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec;
+import 
org.apache.flink.kubernetes.operator.autoscaler.config.AutoScalerOptions;
+import org.apache.flink.kubernetes.operator.utils.FlinkUtils;
+import org.apache.flink.kubernetes.operator.validation.FlinkResourceValidator;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Optional;
+
+/** Validator implementation for the AutoScaler from {@link Configuration}. */
+public class AutoScalerValidator implements FlinkResourceValidator {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(FlinkUtils.class);
+
+    @Override
+    public Optional<String> validateSessionJob(
+            FlinkSessionJob sessionJob, Optional<FlinkDeployment> session) {
+        LOG.info("AutoScaler Configurations validator {}", sessionJob);
+        var spec = sessionJob.getSpec();
+        if (spec.getFlinkConfiguration() != null) {
+            var flinkConfiguration = 
Configuration.fromMap(spec.getFlinkConfiguration());
+            return validateAutoScalerFlinkConfiguration(flinkConfiguration);
+        }
+        return Optional.empty();
+    }
+
+    @Override
+    public Optional<String> validateDeployment(FlinkDeployment deployment) {
+        LOG.info("AutoScaler Configurations validator {}", deployment);

Review Comment:
   For this you may also need the default config, you can take a look at how to 
get that based on the `DefaultValidator`



##########
flink-kubernetes-operator-autoscaler/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/validation/AutoScalerValidator.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.kubernetes.operator.autoscaler.validation;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.kubernetes.operator.api.FlinkDeployment;
+import org.apache.flink.kubernetes.operator.api.FlinkSessionJob;
+import org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec;
+import 
org.apache.flink.kubernetes.operator.autoscaler.config.AutoScalerOptions;
+import org.apache.flink.kubernetes.operator.utils.FlinkUtils;
+import org.apache.flink.kubernetes.operator.validation.FlinkResourceValidator;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Optional;
+
+/** Validator implementation for the AutoScaler from {@link Configuration}. */
+public class AutoScalerValidator implements FlinkResourceValidator {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(FlinkUtils.class);
+
+    @Override
+    public Optional<String> validateSessionJob(
+            FlinkSessionJob sessionJob, Optional<FlinkDeployment> session) {
+        LOG.info("AutoScaler Configurations validator {}", sessionJob);
+        var spec = sessionJob.getSpec();
+        if (spec.getFlinkConfiguration() != null) {
+            var flinkConfiguration = 
Configuration.fromMap(spec.getFlinkConfiguration());
+            return validateAutoScalerFlinkConfiguration(flinkConfiguration);
+        }
+        return Optional.empty();
+    }
+
+    @Override
+    public Optional<String> validateDeployment(FlinkDeployment deployment) {
+        LOG.info("AutoScaler Configurations validator {}", deployment);

Review Comment:
   I think we should only validate these configs if the autoscaler itself is 
enabled. It would be a bit strange to get a validation error if you are not 
even using the autoscaler :) 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to