From b90676a814aa91e970c64f07e112a54caae43f08 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Attila=20M=C3=A9sz=C3=A1ros?= Date: Mon, 18 Aug 2025 09:09:07 +0200 Subject: [PATCH 01/19] [FLINK-37406] Add support for structured YAML config in FlinkDeployment CRD MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Attila Mészáros --- .gitignore | 1 + flink-kubernetes-operator-api/pom.xml | 7 ++++ .../operator/api/spec/AbstractFlinkSpec.java | 5 ++- .../operator/api/utils/SpecUtils.java | 34 +++++++++++++++++++ 4 files changed, 44 insertions(+), 3 deletions(-) diff --git a/.gitignore b/.gitignore index 0fd15a8f2d..5b3bf5c799 100644 --- a/.gitignore +++ b/.gitignore @@ -38,6 +38,7 @@ buildNumber.properties *.DS_Store .kube +.aider* # VSCode settings .vscode/ \ No newline at end of file diff --git a/flink-kubernetes-operator-api/pom.xml b/flink-kubernetes-operator-api/pom.xml index 21b635e5ee..d9cfa8570a 100644 --- a/flink-kubernetes-operator-api/pom.xml +++ b/flink-kubernetes-operator-api/pom.xml @@ -126,6 +126,13 @@ under the License. test + + org.assertj + assertj-core + ${assertj.version} + test + + org.apache.flink flink-runtime diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/AbstractFlinkSpec.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/AbstractFlinkSpec.java index 8cecc5ebcf..14af5e8f5d 100644 --- a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/AbstractFlinkSpec.java +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/AbstractFlinkSpec.java @@ -22,13 +22,12 @@ import org.apache.flink.kubernetes.operator.api.diff.Diffable; import org.apache.flink.kubernetes.operator.api.diff.SpecDiff; +import com.fasterxml.jackson.databind.JsonNode; import lombok.AllArgsConstructor; import lombok.Data; import lombok.NoArgsConstructor; import lombok.experimental.SuperBuilder; -import java.util.Map; - /** The common spec. */ @Experimental @Data @@ -57,5 +56,5 @@ public abstract class AbstractFlinkSpec implements Diffable { type = DiffType.SCALE, mode = KubernetesDeploymentMode.NATIVE) }) - private Map flinkConfiguration; + private JsonNode flinkConfiguration; } diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java index 458dd69173..68f1a029fb 100644 --- a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java @@ -23,15 +23,22 @@ import org.apache.flink.kubernetes.operator.api.spec.AbstractFlinkSpec; import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; +import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; import javax.annotation.Nullable; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.Map; + /** Spec utilities. */ public class SpecUtils { public static final String INTERNAL_METADATA_JSON_KEY = "resource_metadata"; private static final ObjectMapper objectMapper = new ObjectMapper(); + private static final ObjectMapper yamlObjectMapper = new ObjectMapper(new YAMLFactory()); /** * Deserializes the spec and custom metadata object from JSON. @@ -120,4 +127,31 @@ public static T clone(T object) { throw new IllegalStateException(e); } } + + public static Map parseConfigToStringMap(JsonNode node) { + Map flatMap = new LinkedHashMap<>(); + flattenHelper(node, "", flatMap); + return flatMap; + } + + private static void flattenHelper( + JsonNode node, String parentKey, Map flatMap) { + if (node.isObject()) { + Iterator> fields = node.fields(); + while (fields.hasNext()) { + Map.Entry field = fields.next(); + String newKey = + parentKey.isEmpty() ? field.getKey() : parentKey + "." + field.getKey(); + flattenHelper(field.getValue(), newKey, flatMap); + } + } else if (node.isArray()) { + for (int i = 0; i < node.size(); i++) { + String newKey = parentKey + "[" + i + "]"; + flattenHelper(node.get(i), newKey, flatMap); + } + } else { + // Store values as strings + flatMap.put(parentKey, node.asText()); + } + } } From d9da3fb8b7955f216dfe49b285a25fd95ddbdbe1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Attila=20M=C3=A9sz=C3=A1ros?= Date: Mon, 18 Aug 2025 13:48:35 +0200 Subject: [PATCH 02/19] diff, test fixes MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Attila Mészáros --- .../java/org/apache/flink/examples/Basic.java | 3 +- .../operator/api/spec/AbstractFlinkSpec.java | 2 + .../api/spec/JsonNodeNullDeserializer.java | 31 +++ .../operator/api/utils/SpecUtils.java | 61 +++++- .../operator/api/utils/BaseTestUtils.java | 6 +- .../autoscaler/KubernetesScalingRealizer.java | 25 +-- .../operator/config/FlinkConfigBuilder.java | 3 +- .../operator/config/FlinkConfigManager.java | 9 +- .../diff/ReflectiveDiffBuilder.java | 11 +- .../operator/validation/DefaultValidator.java | 9 +- .../flink/kubernetes/operator/TestUtils.java | 5 +- .../KubernetesScalingRealizerTest.java | 22 +-- .../config/FlinkConfigBuilderTest.java | 38 ++-- .../config/FlinkConfigManagerTest.java | 26 ++- .../controller/DeploymentRecoveryTest.java | 7 +- .../FlinkDeploymentControllerTest.java | 35 ++-- .../FlinkSessionJobControllerTest.java | 19 +- .../FlinkStateSnapshotControllerTest.java | 7 +- .../operator/controller/RollbackTest.java | 62 +++--- .../observer/JobStatusObserverTest.java | 13 +- .../deployment/ApplicationObserverTest.java | 35 ++-- .../deployment/SessionObserverTest.java | 7 +- .../deployment/ApplicationReconcilerTest.java | 147 +++++++------- .../ApplicationReconcilerUpgradeModeTest.java | 90 ++++----- .../deployment/SessionReconcilerTest.java | 14 +- .../reconciler/diff/SpecDiffTest.java | 30 ++- .../sessionjob/SessionJobReconcilerTest.java | 28 +-- .../service/AbstractFlinkServiceTest.java | 60 +++--- .../service/NativeFlinkServiceTest.java | 84 ++++---- .../service/StandaloneFlinkServiceTest.java | 28 ++- .../operator/utils/SnapshotUtilsTest.java | 11 +- .../validation/DefaultValidatorTest.java | 183 ++++++++++-------- 32 files changed, 613 insertions(+), 498 deletions(-) create mode 100644 flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/JsonNodeNullDeserializer.java diff --git a/examples/kubernetes-client-examples/src/main/java/org/apache/flink/examples/Basic.java b/examples/kubernetes-client-examples/src/main/java/org/apache/flink/examples/Basic.java index 17a7358693..407d4c7aeb 100644 --- a/examples/kubernetes-client-examples/src/main/java/org/apache/flink/examples/Basic.java +++ b/examples/kubernetes-client-examples/src/main/java/org/apache/flink/examples/Basic.java @@ -34,6 +34,7 @@ import java.util.Map; import static java.util.Map.entry; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.toJsonNode; /** client code for ../basic.yaml. */ public class Basic { @@ -49,7 +50,7 @@ public static void main(String[] args) { flinkDeploymentSpec.setImage("flink:1.19"); Map flinkConfiguration = Map.ofEntries(entry("taskmanager.numberOfTaskSlots", "2")); - flinkDeploymentSpec.setFlinkConfiguration(flinkConfiguration); + flinkDeploymentSpec.setFlinkConfiguration(toJsonNode(flinkConfiguration)); flinkDeployment.setSpec(flinkDeploymentSpec); flinkDeploymentSpec.setServiceAccount("flink"); JobManagerSpec jobManagerSpec = new JobManagerSpec(); diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/AbstractFlinkSpec.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/AbstractFlinkSpec.java index 14af5e8f5d..9d60649d0c 100644 --- a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/AbstractFlinkSpec.java +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/AbstractFlinkSpec.java @@ -23,6 +23,7 @@ import org.apache.flink.kubernetes.operator.api.diff.SpecDiff; import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; import lombok.AllArgsConstructor; import lombok.Data; import lombok.NoArgsConstructor; @@ -56,5 +57,6 @@ public abstract class AbstractFlinkSpec implements Diffable { type = DiffType.SCALE, mode = KubernetesDeploymentMode.NATIVE) }) + @JsonDeserialize(using = JsonNodeNullDeserializer.class) private JsonNode flinkConfiguration; } diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/JsonNodeNullDeserializer.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/JsonNodeNullDeserializer.java new file mode 100644 index 0000000000..335175c534 --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/JsonNodeNullDeserializer.java @@ -0,0 +1,31 @@ +/* + * 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.api.spec; + +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.deser.std.JsonNodeDeserializer; + +/** Makes sure JsonNode is properly deserialized to null not NullNode. */ +public class JsonNodeNullDeserializer extends JsonNodeDeserializer { + + @Override + public JsonNode getNullValue(DeserializationContext ctxt) { + return null; + } +} diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java index 68f1a029fb..9f863795a9 100644 --- a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java @@ -18,6 +18,7 @@ package org.apache.flink.kubernetes.operator.api.utils; +import org.apache.flink.configuration.Configuration; import org.apache.flink.kubernetes.operator.api.AbstractFlinkResource; import org.apache.flink.kubernetes.operator.api.reconciler.ReconciliationMetadata; import org.apache.flink.kubernetes.operator.api.spec.AbstractFlinkSpec; @@ -33,6 +34,7 @@ import java.util.Iterator; import java.util.LinkedHashMap; import java.util.Map; +import java.util.Set; /** Spec utilities. */ public class SpecUtils { @@ -128,7 +130,16 @@ public static T clone(T object) { } } - public static Map parseConfigToStringMap(JsonNode node) { + // todo unit test + public static JsonNode toJsonNode(Map properties) { + ObjectNode jsonNode = yamlObjectMapper.createObjectNode(); + for (Map.Entry entry : properties.entrySet()) { + jsonNode.put(entry.getKey(), entry.getValue()); + } + return jsonNode; + } + + public static Map toStringMap(JsonNode node) { Map flatMap = new LinkedHashMap<>(); flattenHelper(node, "", flatMap); return flatMap; @@ -154,4 +165,52 @@ private static void flattenHelper( flatMap.put(parentKey, node.asText()); } } + + public static void addConfigProperties(AbstractFlinkSpec spec, Map properties) { + spec.setFlinkConfiguration(addProperties(spec.getFlinkConfiguration(), properties)); + } + + public static void addConfigProperty(AbstractFlinkSpec spec, String key, String value) { + spec.setFlinkConfiguration(addProperties(spec.getFlinkConfiguration(), Map.of(key, value))); + } + + public static void removeConfigProperties(AbstractFlinkSpec spec, Set keys) { + spec.setFlinkConfiguration(removeProperties(spec.getFlinkConfiguration(), keys)); + } + + public static void removeConfigProperties(AbstractFlinkSpec spec, String... keys) { + spec.setFlinkConfiguration(removeProperties(spec.getFlinkConfiguration(), Set.of(keys))); + } + + public static JsonNode addProperties(JsonNode node, Map properties) { + var map = toStringMap(node); + map.putAll(properties); + return mapToJsonNode(map); + } + + public static JsonNode addProperty(JsonNode node, String key, String value) { + var map = toStringMap(node); + map.put(key, value); + return mapToJsonNode(map); + } + + public static JsonNode removeProperty(JsonNode node, String key) { + var map = toStringMap(node); + map.remove(key); + return mapToJsonNode(map); + } + + public static JsonNode removeProperties(JsonNode node, Set keys) { + var map = toStringMap(node); + map.keySet().removeAll(keys); + return mapToJsonNode(map); + } + + public static JsonNode mapToJsonNode(Map config) { + return yamlObjectMapper.valueToTree(config); + } + + public static JsonNode configurationToJsonNode(Configuration configuration) { + return mapToJsonNode(configuration.toMap()); + } } diff --git a/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/utils/BaseTestUtils.java b/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/utils/BaseTestUtils.java index 58c5cfe233..858915d541 100644 --- a/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/utils/BaseTestUtils.java +++ b/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/utils/BaseTestUtils.java @@ -53,6 +53,8 @@ import java.util.Map; import java.util.UUID; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.toJsonNode; + /** Base Testing utilities. */ public class BaseTestUtils { @@ -156,7 +158,7 @@ public static FlinkSessionJob buildSessionJob( .upgradeMode(UpgradeMode.STATELESS) .state(jobState) .build()) - .flinkConfiguration(conf) + .flinkConfiguration(toJsonNode(conf)) .build()); return sessionJob; } @@ -186,7 +188,7 @@ public static FlinkDeploymentSpec getTestFlinkDeploymentSpec(FlinkVersion versio .imagePullPolicy(IMAGE_POLICY) .serviceAccount(SERVICE_ACCOUNT) .flinkVersion(version) - .flinkConfiguration(conf) + .flinkConfiguration(toJsonNode(conf)) .jobManager(new JobManagerSpec(new Resource(1.0, "2048m", "2G"), 1, null)) .taskManager(new TaskManagerSpec(new Resource(1.0, "2048m", "2G"), null, null)) .build(); diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/KubernetesScalingRealizer.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/KubernetesScalingRealizer.java index 16455bf185..7e653c585a 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/KubernetesScalingRealizer.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/KubernetesScalingRealizer.java @@ -26,6 +26,7 @@ import org.apache.flink.configuration.PipelineOptions; import org.apache.flink.kubernetes.operator.api.FlinkDeployment; import org.apache.flink.kubernetes.operator.api.spec.Resource; +import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.apache.flink.kubernetes.operator.config.FlinkConfigBuilder; import io.javaoperatorsdk.operator.processing.event.ResourceID; @@ -45,13 +46,11 @@ public class KubernetesScalingRealizer @Override public void realizeParallelismOverrides( KubernetesJobAutoScalerContext context, Map parallelismOverrides) { - - context.getResource() - .getSpec() - .getFlinkConfiguration() - .put( + SpecUtils.addConfigProperties( + context.getResource().getSpec(), + Map.of( PipelineOptions.PARALLELISM_OVERRIDES.key(), - getOverrideString(context, parallelismOverrides)); + getOverrideString(context, parallelismOverrides))); } @Override @@ -63,15 +62,17 @@ public void realizeConfigOverrides( } FlinkDeployment flinkDeployment = ((FlinkDeployment) context.getResource()); // Apply config overrides - Map flinkConf = flinkDeployment.getSpec().getFlinkConfiguration(); - for (String keyToRemove : configChanges.getRemovals()) { - flinkConf.remove(keyToRemove); - } - flinkConf.putAll(configChanges.getOverrides()); + + SpecUtils.removeConfigProperties(flinkDeployment.getSpec(), configChanges.getRemovals()); + SpecUtils.addConfigProperties(flinkDeployment.getSpec(), configChanges.getOverrides()); // Update total memory in spec var totalMemoryOverride = - MemoryTuning.getTotalMemory(Configuration.fromMap(flinkConf), context); + MemoryTuning.getTotalMemory( + Configuration.fromMap( + SpecUtils.toStringMap( + flinkDeployment.getSpec().getFlinkConfiguration())), + context); if (totalMemoryOverride.compareTo(MemorySize.ZERO) <= 0) { LOG.warn("Total memory override {} is not valid", totalMemoryOverride); return; diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilder.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilder.java index ba16068e74..53534396cd 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilder.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilder.java @@ -40,6 +40,7 @@ import org.apache.flink.kubernetes.operator.api.spec.KubernetesDeploymentMode; import org.apache.flink.kubernetes.operator.api.spec.Resource; import org.apache.flink.kubernetes.operator.api.spec.UpgradeMode; +import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; import org.apache.flink.kubernetes.operator.standalone.StandaloneKubernetesConfigOptionsInternal; import org.apache.flink.kubernetes.operator.utils.FlinkUtils; @@ -140,7 +141,7 @@ protected FlinkConfigBuilder applyImagePullPolicy() { protected FlinkConfigBuilder applyFlinkConfiguration() { // Parse config from spec's flinkConfiguration if (spec.getFlinkConfiguration() != null && !spec.getFlinkConfiguration().isEmpty()) { - spec.getFlinkConfiguration().forEach(effectiveConfig::setString); + SpecUtils.toStringMap(spec.getFlinkConfiguration()).forEach(effectiveConfig::setString); } // Adapt default rest service type from 1.15+ diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManager.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManager.java index 8a2fd2651d..6f0e2b70d4 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManager.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManager.java @@ -29,6 +29,7 @@ import org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec; import org.apache.flink.kubernetes.operator.api.spec.FlinkSessionJobSpec; import org.apache.flink.kubernetes.operator.api.spec.FlinkVersion; +import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; import org.apache.flink.kubernetes.operator.utils.EnvUtils; import org.apache.flink.kubernetes.operator.utils.FlinkUtils; @@ -355,7 +356,7 @@ public Configuration getObserveConfig(FlinkDeployment deployment) { private void addOperatorConfigsFromSpec(AbstractFlinkSpec spec, Configuration conf) { // Observe config should include the latest operator related settings if (spec.getFlinkConfiguration() != null) { - spec.getFlinkConfiguration() + SpecUtils.toStringMap(spec.getFlinkConfiguration()) .forEach( (k, v) -> { if (k.startsWith(K8S_OP_CONF_PREFIX) @@ -371,7 +372,8 @@ private void applyConfigsFromCurrentSpec( AbstractFlinkSpec spec, Configuration conf, ConfigOption... configOptions) { addOperatorConfigsFromSpec(spec, conf); if (spec.getFlinkConfiguration() != null) { - var deployConfig = Configuration.fromMap(spec.getFlinkConfiguration()); + var deployConfig = + Configuration.fromMap(SpecUtils.toStringMap(spec.getFlinkConfiguration())); for (ConfigOption configOption : configOptions) { deployConfig.getOptional(configOption).ifPresent(v -> conf.set(configOption, v)); } @@ -394,7 +396,8 @@ public Configuration getSessionJobConfig( // merge session job specific config var sessionJobFlinkConfiguration = sessionJobSpec.getFlinkConfiguration(); if (sessionJobFlinkConfiguration != null) { - sessionJobFlinkConfiguration.forEach(sessionJobConfig::setString); + SpecUtils.toStringMap(sessionJobFlinkConfiguration) + .forEach(sessionJobConfig::setString); } applyJobConfig(name, sessionJobConfig, sessionJobSpec.getJob()); return sessionJobConfig; diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/diff/ReflectiveDiffBuilder.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/diff/ReflectiveDiffBuilder.java index 5ab52d7f94..cdfc2cb467 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/diff/ReflectiveDiffBuilder.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/diff/ReflectiveDiffBuilder.java @@ -23,7 +23,9 @@ import org.apache.flink.kubernetes.operator.api.diff.SpecDiff; import org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec; import org.apache.flink.kubernetes.operator.api.spec.KubernetesDeploymentMode; +import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; +import com.fasterxml.jackson.databind.JsonNode; import io.fabric8.kubernetes.api.model.PodTemplateSpec; import lombok.NonNull; import org.apache.commons.lang3.ClassUtils; @@ -50,6 +52,7 @@ @Experimental public class ReflectiveDiffBuilder implements Builder> { + public static final String FLINK_CONFIGURATION_PROPERTY_NAME = "flinkConfiguration"; private final KubernetesDeploymentMode deploymentMode; private final Object before; private final Object after; @@ -83,9 +86,15 @@ private void appendFields(final Class clazz) { try { var leftField = readField(field, before, true); var rightField = readField(field, after, true); + if (field.getName().equals(FLINK_CONFIGURATION_PROPERTY_NAME)) { + leftField = SpecUtils.toStringMap((JsonNode) leftField); + rightField = SpecUtils.toStringMap((JsonNode) rightField); + } if (field.isAnnotationPresent(SpecDiff.Config.class) - && Map.class.isAssignableFrom(field.getType())) { + && (Map.class.isAssignableFrom(field.getType()) + || (field.getType().equals(JsonNode.class) + && field.getName().equals("flinkConfiguration")))) { diffBuilder.append( field.getName(), configDiff( diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/validation/DefaultValidator.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/validation/DefaultValidator.java index 42fd56e76c..57d3887665 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/validation/DefaultValidator.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/validation/DefaultValidator.java @@ -41,6 +41,7 @@ import org.apache.flink.kubernetes.operator.api.spec.TaskManagerSpec; import org.apache.flink.kubernetes.operator.api.spec.UpgradeMode; import org.apache.flink.kubernetes.operator.api.status.FlinkStateSnapshotStatus; +import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.apache.flink.kubernetes.operator.config.FlinkConfigBuilder; import org.apache.flink.kubernetes.operator.config.FlinkConfigManager; import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; @@ -102,7 +103,7 @@ public Optional validateDeployment(FlinkDeployment deployment) { deployment.getMetadata().getNamespace(), spec.getFlinkVersion()) .toMap(); if (spec.getFlinkConfiguration() != null) { - effectiveConfig.putAll(spec.getFlinkConfiguration()); + effectiveConfig.putAll(SpecUtils.toStringMap(spec.getFlinkConfiguration())); } return firstPresent( validateDeploymentName(deployment.getMetadata().getName()), @@ -531,11 +532,13 @@ private Optional validateSessionJobWithCluster( sessionCluster.getSpec().getFlinkVersion()) .toMap(); if (sessionCluster.getSpec().getFlinkConfiguration() != null) { - effectiveConfig.putAll(sessionCluster.getSpec().getFlinkConfiguration()); + effectiveConfig.putAll( + SpecUtils.toStringMap(sessionCluster.getSpec().getFlinkConfiguration())); } if (sessionJob.getSpec().getFlinkConfiguration() != null) { - effectiveConfig.putAll(sessionJob.getSpec().getFlinkConfiguration()); + effectiveConfig.putAll( + SpecUtils.toStringMap(sessionJob.getSpec().getFlinkConfiguration())); } return firstPresent( diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/TestUtils.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/TestUtils.java index b961953a53..2af251dc2b 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/TestUtils.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/TestUtils.java @@ -83,6 +83,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperties; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.params.provider.Arguments.arguments; @@ -221,7 +222,7 @@ public static Context createContextWithReadyFlinkDepl public Optional getSecondaryResource(Class expectedType, String eventSourceName) { var session = buildSessionCluster(version); session.getStatus().setJobManagerDeploymentStatus(JobManagerDeploymentStatus.READY); - session.getSpec().getFlinkConfiguration().putAll(flinkDepConfig); + addConfigProperties(session.getSpec(), flinkDepConfig); session.getStatus() .getReconciliationStatus() .serializeAndSetLastReconciledSpec(session.getSpec(), session); @@ -440,7 +441,7 @@ public static Instant setupCronTrigger(SnapshotType snapshotType, FlinkDeploymen throw new IllegalArgumentException("Unsupported snapshot type: " + snapshotType); } - deployment.getSpec().getFlinkConfiguration().put(cronOptionKey, "0 0 12 5 6 ? 2022"); + addConfigProperties(deployment.getSpec(), Map.of(cronOptionKey, "0 0 12 5 6 ? 2022")); reconcileSpec(deployment); return calendar.toInstant(); } diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/autoscaler/KubernetesScalingRealizerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/autoscaler/KubernetesScalingRealizerTest.java index 4a60cf7ed6..58459b2115 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/autoscaler/KubernetesScalingRealizerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/autoscaler/KubernetesScalingRealizerTest.java @@ -23,12 +23,15 @@ import org.apache.flink.configuration.PipelineOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.kubernetes.operator.api.FlinkDeployment; +import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.junit.jupiter.api.Test; import java.util.LinkedHashMap; import java.util.Map; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.removeProperty; import static org.assertj.core.api.Assertions.assertThat; /** Tests for KubernetesScalingRealizer. */ @@ -46,9 +49,7 @@ public void testApplyOverrides() { .realizeParallelismOverrides(ctx, Map.of("a", "1", "b", "2")); assertThat( - ctx.getResource() - .getSpec() - .getFlinkConfiguration() + SpecUtils.toStringMap(ctx.getResource().getSpec().getFlinkConfiguration()) .get(PipelineOptions.PARALLELISM_OVERRIDES.key())) .satisfiesAnyOf( // Currently no enforced order inside the overrides string @@ -95,22 +96,19 @@ private void assertOverridesDoNotChange( FlinkDeployment resource = (FlinkDeployment) ctx.getResource(); // Create resource with existing parallelism overrides - resource.getSpec() - .getFlinkConfiguration() - .put(PipelineOptions.PARALLELISM_OVERRIDES.key(), currentOverrides); + addConfigProperty( + resource.getSpec(), PipelineOptions.PARALLELISM_OVERRIDES.key(), currentOverrides); resource.getStatus() .getReconciliationStatus() .serializeAndSetLastReconciledSpec(resource.getSpec(), resource); - resource.getSpec() - .getFlinkConfiguration() - .remove(PipelineOptions.PARALLELISM_OVERRIDES.key()); + removeProperty( + resource.getSpec().getFlinkConfiguration(), + PipelineOptions.PARALLELISM_OVERRIDES.key()); new KubernetesScalingRealizer().realizeParallelismOverrides(ctx, newOverrides); assertThat( - ctx.getResource() - .getSpec() - .getFlinkConfiguration() + SpecUtils.toStringMap(ctx.getResource().getSpec().getFlinkConfiguration()) .get(PipelineOptions.PARALLELISM_OVERRIDES.key())) .isEqualTo(currentOverrides); } diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilderTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilderTest.java index 7673060890..db8f71874a 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilderTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilderTest.java @@ -40,6 +40,7 @@ import org.apache.flink.kubernetes.operator.api.spec.Resource; import org.apache.flink.kubernetes.operator.api.spec.TaskManagerSpec; import org.apache.flink.kubernetes.operator.api.spec.UpgradeMode; +import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; import org.apache.flink.kubernetes.operator.standalone.StandaloneKubernetesConfigOptionsInternal; import org.apache.flink.kubernetes.utils.Constants; @@ -127,9 +128,11 @@ public void testApplyFlinkConfiguration() { deployment .getSpec() .setFlinkConfiguration( - Map.of( - KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE.key(), - KubernetesConfigOptions.ServiceExposedType.LoadBalancer.name())); + SpecUtils.mapToJsonNode( + Map.of( + KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE.key(), + KubernetesConfigOptions.ServiceExposedType.LoadBalancer + .name()))); configuration = new FlinkConfigBuilder(deployment, new Configuration()) @@ -715,7 +718,7 @@ public void testApplyJobOrSessionSpec() throws Exception { var dep = ReconciliationUtils.clone(deploymentClone); dep.getSpec().setTaskManager(new TaskManagerSpec()); dep.getSpec().getTaskManager().setReplicas(3); - dep.getSpec().getFlinkConfiguration().put(TaskManagerOptions.NUM_TASK_SLOTS.key(), "4"); + SpecUtils.addConfigProperty(dep.getSpec(), TaskManagerOptions.NUM_TASK_SLOTS.key(), "4"); configuration = new FlinkConfigBuilder(dep, new Configuration()) .applyFlinkConfiguration() @@ -757,10 +760,10 @@ public void testApplyJobOrSessionSpecWithNoJar() throws Exception { @Test public void testAllowNonRestoredStateInSpecOverrideInFlinkConf() throws URISyntaxException { flinkDeployment.getSpec().getJob().setAllowNonRestoredState(false); - flinkDeployment - .getSpec() - .getFlinkConfiguration() - .put(SavepointConfigOptions.SAVEPOINT_IGNORE_UNCLAIMED_STATE.key(), "true"); + SpecUtils.addConfigProperty( + flinkDeployment.getSpec(), + SavepointConfigOptions.SAVEPOINT_IGNORE_UNCLAIMED_STATE.key(), + "true"); Configuration configuration = new FlinkConfigBuilder(flinkDeployment, new Configuration()) .applyJobOrSessionSpec() @@ -769,10 +772,10 @@ public void testAllowNonRestoredStateInSpecOverrideInFlinkConf() throws URISynta configuration.getBoolean(SavepointConfigOptions.SAVEPOINT_IGNORE_UNCLAIMED_STATE)); flinkDeployment.getSpec().getJob().setAllowNonRestoredState(true); - flinkDeployment - .getSpec() - .getFlinkConfiguration() - .put(SavepointConfigOptions.SAVEPOINT_IGNORE_UNCLAIMED_STATE.key(), "false"); + SpecUtils.addConfigProperty( + flinkDeployment.getSpec(), + SavepointConfigOptions.SAVEPOINT_IGNORE_UNCLAIMED_STATE.key(), + "false"); configuration = new FlinkConfigBuilder(flinkDeployment, new Configuration()) .applyJobOrSessionSpec() @@ -792,7 +795,7 @@ public void testApplyStandaloneApplicationSpec() throws URISyntaxException, IOEx dep.getSpec().getJob().setJarURI(jarUri); dep.getSpec().setTaskManager(new TaskManagerSpec()); dep.getSpec().getTaskManager().setReplicas(3); - dep.getSpec().getFlinkConfiguration().put(TaskManagerOptions.NUM_TASK_SLOTS.key(), "2"); + SpecUtils.addConfigProperty(dep.getSpec(), TaskManagerOptions.NUM_TASK_SLOTS.key(), "2"); Configuration configuration = new FlinkConfigBuilder(dep, new Configuration()) @@ -832,9 +835,10 @@ public void testApplyStandaloneApplicationSpec() throws URISyntaxException, IOEx configuration.get( StandaloneKubernetesConfigOptionsInternal.KUBERNETES_TASKMANAGER_REPLICAS)); - dep.getSpec() - .getFlinkConfiguration() - .put(PipelineOptions.PARALLELISM_OVERRIDES.key(), "vertex1:10,vertex2:20"); + SpecUtils.addConfigProperty( + dep.getSpec(), + PipelineOptions.PARALLELISM_OVERRIDES.key(), + "vertex1:10,vertex2:20"); configuration = new FlinkConfigBuilder(dep, new Configuration()) .applyFlinkConfiguration() @@ -854,7 +858,7 @@ public void testApplyStandaloneSessionSpec() throws URISyntaxException, IOExcept dep.getSpec().setJob(null); dep.getSpec().setTaskManager(new TaskManagerSpec()); dep.getSpec().getTaskManager().setReplicas(5); - dep.getSpec().getFlinkConfiguration().put(TaskManagerOptions.NUM_TASK_SLOTS.key(), "2"); + SpecUtils.addConfigProperty(dep.getSpec(), TaskManagerOptions.NUM_TASK_SLOTS.key(), "2"); Configuration configuration = new FlinkConfigBuilder(dep, new Configuration()) diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManagerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManagerTest.java index b6bca056d3..d00c0100a1 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManagerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManagerTest.java @@ -52,6 +52,8 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Matcher; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperties; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.OPERATOR_WATCHED_NAMESPACES; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -78,17 +80,21 @@ public void testConfigGeneration() { FlinkDeployment deployment = TestUtils.buildApplicationCluster(); var reconciliationStatus = deployment.getStatus().getReconciliationStatus(); - deployment.getSpec().getFlinkConfiguration().put(testConf.key(), "reconciled"); - deployment.getSpec().getFlinkConfiguration().put(opTestConf.key(), "reconciled"); + addConfigProperties( + deployment.getSpec(), + Map.of(testConf.key(), "reconciled", opTestConf.key(), "reconciled")); reconciliationStatus.serializeAndSetLastReconciledSpec(deployment.getSpec(), deployment); reconciliationStatus.markReconciledSpecAsStable(); - deployment.getSpec().getFlinkConfiguration().put(testConf.key(), "latest"); - deployment.getSpec().getFlinkConfiguration().put(opTestConf.key(), "latest"); - deployment - .getSpec() - .getFlinkConfiguration() - .put(AutoScalerOptions.METRICS_WINDOW.key(), "1234m"); + addConfigProperties( + deployment.getSpec(), + Map.of( + testConf.key(), + "latest", + opTestConf.key(), + "latest", + AutoScalerOptions.METRICS_WINDOW.key(), + "1234m")); assertEquals( "latest", @@ -106,11 +112,11 @@ public void testConfigGeneration() { Duration.ofMinutes(1234), configManager.getObserveConfig(deployment).get(AutoScalerOptions.METRICS_WINDOW)); - deployment.getSpec().getFlinkConfiguration().put(testConf.key(), "stable"); + addConfigProperty(deployment.getSpec(), testConf.key(), "stable"); reconciliationStatus.serializeAndSetLastReconciledSpec(deployment.getSpec(), deployment); reconciliationStatus.markReconciledSpecAsStable(); - deployment.getSpec().getFlinkConfiguration().put(testConf.key(), "rolled-back"); + addConfigProperty(deployment.getSpec(), testConf.key(), "rolled-back"); reconciliationStatus.serializeAndSetLastReconciledSpec(deployment.getSpec(), deployment); reconciliationStatus.setState(ReconciliationState.ROLLED_BACK); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/DeploymentRecoveryTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/DeploymentRecoveryTest.java index 07d23a5d5c..e72d22cf3c 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/DeploymentRecoveryTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/DeploymentRecoveryTest.java @@ -25,6 +25,7 @@ import org.apache.flink.kubernetes.operator.api.spec.FlinkVersion; import org.apache.flink.kubernetes.operator.api.spec.UpgradeMode; import org.apache.flink.kubernetes.operator.api.status.JobManagerDeploymentStatus; +import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.apache.flink.kubernetes.operator.config.FlinkConfigManager; import io.fabric8.kubernetes.client.KubernetesClient; @@ -177,10 +178,8 @@ public void verifyRecoveryWithoutHaData(FlinkVersion flinkVersion, UpgradeMode u // We disable HA for stateless to test recovery without HA metadata if (upgradeMode == UpgradeMode.STATELESS) { - appCluster - .getSpec() - .getFlinkConfiguration() - .put(HighAvailabilityOptions.HA_MODE.key(), "none"); + SpecUtils.addConfigProperty( + appCluster.getSpec(), HighAvailabilityOptions.HA_MODE.key(), "none"); } testController.reconcile(appCluster, context); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentControllerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentControllerTest.java index 4d379139b6..dcaa452fcc 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentControllerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentControllerTest.java @@ -65,6 +65,7 @@ import java.util.stream.Collectors; import static org.apache.flink.kubernetes.operator.TestUtils.MAX_RECONCILE_TIMES; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.OPERATOR_JOB_UPGRADE_LAST_STATE_FALLBACK_ENABLED; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.SNAPSHOT_RESOURCE_ENABLED; import static org.apache.flink.kubernetes.operator.utils.EventRecorder.Reason.ValidationError; @@ -197,12 +198,10 @@ public void verifyReconcileLoopForInitialSuspendedDeploymentWithSavepoint( TestUtils.buildApplicationCluster(flinkVersion, JobState.SUSPENDED); appCluster.getSpec().getJob().setUpgradeMode(UpgradeMode.SAVEPOINT); appCluster.getSpec().getJob().setInitialSavepointPath("s0"); - appCluster - .getSpec() - .getFlinkConfiguration() - .put( - CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), - "file:///flink-data/savepoints"); + addConfigProperty( + appCluster.getSpec(), + CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), + "file:///flink-data/savepoints"); int reconcileTimes = 0; while (reconcileTimes < MAX_RECONCILE_TIMES) { @@ -304,13 +303,11 @@ public void verifyUpgradeFromSavepointLegacyMode(FlinkVersion flinkVersion) thro FlinkDeployment appCluster = TestUtils.buildApplicationCluster(flinkVersion); appCluster.getSpec().getJob().setUpgradeMode(UpgradeMode.SAVEPOINT); appCluster.getSpec().getJob().setInitialSavepointPath("s0"); - appCluster - .getSpec() - .getFlinkConfiguration() - .put( - CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), - "file:///flink-data/savepoints"); - appCluster.getSpec().getFlinkConfiguration().put(SNAPSHOT_RESOURCE_ENABLED.key(), "false"); + addConfigProperty( + appCluster.getSpec(), + CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), + "file:///flink-data/savepoints"); + addConfigProperty(appCluster.getSpec(), SNAPSHOT_RESOURCE_ENABLED.key(), "false"); testController.reconcile(appCluster, context); var jobs = flinkService.listJobs(); assertEquals(1, jobs.size()); @@ -562,7 +559,7 @@ public void verifyReconcileWithBadConfig() throws Exception { UpdateControl updateControl; // Override rest port, and it should be saved in lastReconciledSpec once a successful // reconcile() finishes. - appCluster.getSpec().getFlinkConfiguration().put(RestOptions.PORT.key(), "8088"); + addConfigProperty(appCluster.getSpec(), RestOptions.PORT.key(), "8088"); updateControl = testController.reconcile(appCluster, context); assertFalse(updateControl.isPatchStatus()); assertEquals( @@ -588,7 +585,7 @@ public void verifyReconcileWithBadConfig() throws Exception { appCluster.getSpec().getJob().setParallelism(0); // Verify the saved rest port in lastReconciledSpec is actually used in observe() by // utilizing listJobConsumer - appCluster.getSpec().getFlinkConfiguration().put(RestOptions.PORT.key(), "12345"); + addConfigProperty(appCluster.getSpec(), RestOptions.PORT.key(), "12345"); flinkService.setListJobConsumer( (configuration) -> assertEquals(8088, configuration.get(RestOptions.PORT))); testController.reconcile(appCluster, context); @@ -796,10 +793,10 @@ private void testUpgradeNotReadyCluster(FlinkDeployment appCluster) throws Excep // triggering upgrade with no last-state fallback on non-healthy app flinkService.setPortReady(false); - appCluster - .getSpec() - .getFlinkConfiguration() - .put(OPERATOR_JOB_UPGRADE_LAST_STATE_FALLBACK_ENABLED.key(), "false"); + addConfigProperty( + appCluster.getSpec(), + OPERATOR_JOB_UPGRADE_LAST_STATE_FALLBACK_ENABLED.key(), + "false"); appCluster.getSpec().setServiceAccount(appCluster.getSpec().getServiceAccount() + "-5"); // not upgrading the cluster with no last-state fallback testController.reconcile(appCluster, context); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobControllerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobControllerTest.java index 09ff806e8c..c00c43e484 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobControllerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobControllerTest.java @@ -56,6 +56,7 @@ import static org.apache.flink.api.common.JobStatus.RECONCILING; import static org.apache.flink.api.common.JobStatus.RUNNING; import static org.apache.flink.kubernetes.operator.TestUtils.MAX_RECONCILE_TIMES; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.SNAPSHOT_RESOURCE_ENABLED; import static org.apache.flink.kubernetes.operator.utils.EventRecorder.Reason.ValidationError; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -194,7 +195,7 @@ public void verifyUpgradeFromSavepointLegacy() throws Exception { sessionJob.getSpec().getJob().setUpgradeMode(UpgradeMode.SAVEPOINT); sessionJob.getSpec().getJob().setInitialSavepointPath("s0"); - sessionJob.getSpec().getFlinkConfiguration().put(SNAPSHOT_RESOURCE_ENABLED.key(), "false"); + addConfigProperty(sessionJob.getSpec(), SNAPSHOT_RESOURCE_ENABLED.key(), "false"); testController.reconcile(sessionJob, context); var jobs = flinkService.listJobs(); assertEquals(1, jobs.size()); @@ -516,10 +517,10 @@ public void verifyReconcileWithBadConfig() throws Exception { UpdateControl updateControl; // Override headers, and it should be saved in lastReconciledSpec once a successful // reconcile() finishes. - sessionJob - .getSpec() - .getFlinkConfiguration() - .put(KubernetesOperatorConfigOptions.JAR_ARTIFACT_HTTP_HEADER.key(), "changed"); + addConfigProperty( + sessionJob.getSpec(), + KubernetesOperatorConfigOptions.JAR_ARTIFACT_HTTP_HEADER.key(), + "changed"); updateControl = testController.reconcile(sessionJob, context); assertFalse(updateControl.isPatchStatus()); assertEquals(RECONCILING, sessionJob.getStatus().getJobStatus().getState()); @@ -539,10 +540,10 @@ public void verifyReconcileWithBadConfig() throws Exception { // Make sure we do validation before getting effective config in reconcile(). // Verify the saved headers in lastReconciledSpec is actually used in observe() by // utilizing listJobConsumer - sessionJob - .getSpec() - .getFlinkConfiguration() - .put(KubernetesOperatorConfigOptions.JAR_ARTIFACT_HTTP_HEADER.key(), "again"); + addConfigProperty( + sessionJob.getSpec(), + KubernetesOperatorConfigOptions.JAR_ARTIFACT_HTTP_HEADER.key(), + "again"); flinkService.setListJobConsumer( (configuration) -> assertEquals( diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkStateSnapshotControllerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkStateSnapshotControllerTest.java index cfe6947343..5fb321a47a 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkStateSnapshotControllerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkStateSnapshotControllerTest.java @@ -77,6 +77,7 @@ import static org.apache.flink.kubernetes.operator.api.status.FlinkStateSnapshotStatus.State.FAILED; import static org.apache.flink.kubernetes.operator.api.status.FlinkStateSnapshotStatus.State.IN_PROGRESS; import static org.apache.flink.kubernetes.operator.api.status.FlinkStateSnapshotStatus.State.TRIGGER_PENDING; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.removeConfigProperties; import static org.apache.flink.kubernetes.operator.metrics.FlinkStateSnapshotMetricsUtils.assertSnapshotMetrics; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -750,10 +751,8 @@ private FlinkDeployment createDeployment(FlinkVersion flinkVersion) { .getStatus() .setJobStatus(JobStatus.builder().state(RUNNING).jobId(JOB_ID).build()); deployment.getSpec().setFlinkVersion(flinkVersion); - deployment - .getSpec() - .getFlinkConfiguration() - .remove(CheckpointingOptions.SAVEPOINT_DIRECTORY.key()); + removeConfigProperties( + deployment.getSpec(), CheckpointingOptions.SAVEPOINT_DIRECTORY.key()); ReconciliationUtils.updateStatusForDeployedSpec(deployment, new Configuration()); client.resource(deployment).create(); return deployment; diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/RollbackTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/RollbackTest.java index 83fb762440..92a4622f4e 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/RollbackTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/RollbackTest.java @@ -27,6 +27,7 @@ import org.apache.flink.kubernetes.operator.api.status.ReconciliationState; import org.apache.flink.kubernetes.operator.api.status.Savepoint; import org.apache.flink.kubernetes.operator.api.status.SnapshotTriggerType; +import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.apache.flink.kubernetes.operator.config.FlinkConfigManager; import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; import org.apache.flink.kubernetes.operator.reconciler.deployment.AbstractFlinkResourceReconciler; @@ -46,6 +47,8 @@ import java.util.Map; import static org.apache.flink.api.common.JobStatus.RUNNING; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperties; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotEquals; @@ -90,7 +93,7 @@ public void testStatefulRollback(UpgradeMode upgradeMode) throws Exception { dep, () -> { dep.getSpec().getJob().setParallelism(9999); - dep.getSpec().getFlinkConfiguration().put("test.deploy.config", "roll_back"); + addConfigProperty(dep.getSpec(), "test.deploy.config", "roll_back"); testController.reconcile(dep, context); assertEquals( JobState.SUSPENDED, @@ -130,7 +133,7 @@ public void testSavepointRollbackWithoutHaMetadata() throws Exception { dep, () -> { dep.getSpec().getJob().setParallelism(9999); - dep.getSpec().getFlinkConfiguration().put("test.deploy.config", "roll_back"); + addConfigProperty(dep.getSpec(), "test.deploy.config", "roll_back"); testController.reconcile(dep, context); assertEquals( JobState.SUSPENDED, @@ -171,10 +174,11 @@ public void testSavepointNoRollbackWithoutHaMetadataAndJMWasReady() throws Excep offsetReconcilerClock(deployment, Duration.ZERO); var flinkConfiguration = deployment.getSpec().getFlinkConfiguration(); - flinkConfiguration.put( - KubernetesOperatorConfigOptions.DEPLOYMENT_ROLLBACK_ENABLED.key(), "true"); - flinkConfiguration.put( - KubernetesOperatorConfigOptions.DEPLOYMENT_READINESS_TIMEOUT.key(), "10s"); + addConfigProperties( + deployment.getSpec(), + Map.of( + KubernetesOperatorConfigOptions.DEPLOYMENT_ROLLBACK_ENABLED.key(), "true", + KubernetesOperatorConfigOptions.DEPLOYMENT_READINESS_TIMEOUT.key(), "10s")); testController.reconcile(deployment, context); @@ -215,15 +219,16 @@ public void testSavepointNoRollbackWithoutHaMetadataAndJMWasReady() throws Excep public void testRollbackFailureWithLastState() throws Exception { var dep = TestUtils.buildApplicationCluster(); dep.getSpec().getJob().setUpgradeMode(UpgradeMode.LAST_STATE); - dep.getSpec().getFlinkConfiguration().put("t", "1"); + + addConfigProperty(dep.getSpec(), "t", "1"); offsetReconcilerClock(dep, Duration.ZERO); testRollback( dep, () -> { dep.getSpec().getJob().setParallelism(9999); - dep.getSpec().getFlinkConfiguration().put("test.deploy.config", "roll_back"); - dep.getSpec().getFlinkConfiguration().remove("t"); + addConfigProperty(dep.getSpec(), "test.deploy.config", "roll_back"); + SpecUtils.removeConfigProperties(dep.getSpec(), "t"); testController.reconcile(dep, context); assertEquals( JobState.SUSPENDED, @@ -278,14 +283,12 @@ public void testRollbackStateless() throws Exception { testRollback( dep, () -> { - dep.getSpec() - .getFlinkConfiguration() - .put( - KubernetesOperatorConfigOptions.DEPLOYMENT_ROLLBACK_ENABLED - .key(), - "false"); + addConfigProperty( + dep.getSpec(), + KubernetesOperatorConfigOptions.DEPLOYMENT_ROLLBACK_ENABLED.key(), + "false"); dep.getSpec().getJob().setParallelism(9999); - dep.getSpec().getFlinkConfiguration().put("test.deploy.config", "roll_back"); + addConfigProperty(dep.getSpec(), "test.deploy.config", "roll_back"); testController.reconcile(dep, context); assertEquals( JobState.SUSPENDED, @@ -301,12 +304,10 @@ public void testRollbackStateless() throws Exception { .getSubmittedConf() .getString("test.deploy.config", "unknown")); // Validate that rollback config is picked up from latest deploy conf - dep.getSpec() - .getFlinkConfiguration() - .put( - KubernetesOperatorConfigOptions.DEPLOYMENT_ROLLBACK_ENABLED - .key(), - "true"); + addConfigProperty( + dep.getSpec(), + KubernetesOperatorConfigOptions.DEPLOYMENT_ROLLBACK_ENABLED.key(), + "true"); // Trigger rollback by delaying the recovery offsetReconcilerClock(dep, Duration.ofSeconds(15)); @@ -334,7 +335,7 @@ public void testRollbackSession() throws Exception { testRollback( dep, () -> { - dep.getSpec().getFlinkConfiguration().put("random", "config"); + addConfigProperty(dep.getSpec(), "random", "config"); testController.reconcile(dep, context); // Trigger rollback by delaying the recovery offsetReconcilerClock(dep, Duration.ofSeconds(15)); @@ -356,12 +357,15 @@ public void testRollback( boolean expectTwoStepRollback) throws Exception { - var flinkConfiguration = deployment.getSpec().getFlinkConfiguration(); - flinkConfiguration.put( - KubernetesOperatorConfigOptions.DEPLOYMENT_ROLLBACK_ENABLED.key(), "true"); - flinkConfiguration.put( - KubernetesOperatorConfigOptions.DEPLOYMENT_READINESS_TIMEOUT.key(), "10s"); - flinkConfiguration.put("test.deploy.config", "stable"); + addConfigProperties( + deployment.getSpec(), + Map.of( + KubernetesOperatorConfigOptions.DEPLOYMENT_ROLLBACK_ENABLED.key(), + "true", + KubernetesOperatorConfigOptions.DEPLOYMENT_READINESS_TIMEOUT.key(), + "10s", + "test.deploy.config", + "stable")); testController.reconcile(deployment, context); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserverTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserverTest.java index f3ff7a8b73..e4f939d061 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserverTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserverTest.java @@ -51,6 +51,7 @@ import java.util.Map; import java.util.stream.Stream; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -745,10 +746,8 @@ private static Stream cancellingArgs() { private static FlinkDeployment initDeployment() { FlinkDeployment deployment = TestUtils.buildApplicationCluster(); var jobId = new JobID().toHexString(); - deployment - .getSpec() - .getFlinkConfiguration() - .put(PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID.key(), jobId); + addConfigProperty( + deployment.getSpec(), PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID.key(), jobId); deployment.getStatus().getJobStatus().setJobId(jobId); deployment .getStatus() @@ -760,9 +759,9 @@ private static FlinkDeployment initDeployment() { private static FlinkSessionJob initSessionJob() { var job = TestUtils.buildSessionJob(); var jobId = new JobID().toHexString(); - job.getSpec() - .getFlinkConfiguration() - .put(PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID.key(), jobId); + addConfigProperty( + job.getSpec(), PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID.key(), jobId); + job.getStatus().getJobStatus().setJobId(jobId); job.getStatus() .getReconciliationStatus() diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/ApplicationObserverTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/ApplicationObserverTest.java index 68233d397e..e3afa78724 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/ApplicationObserverTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/ApplicationObserverTest.java @@ -31,6 +31,7 @@ import org.apache.flink.kubernetes.operator.api.status.ReconciliationState; import org.apache.flink.kubernetes.operator.api.status.SavepointFormatType; import org.apache.flink.kubernetes.operator.api.status.SnapshotTriggerType; +import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.apache.flink.kubernetes.operator.config.FlinkConfigManager; import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; import org.apache.flink.kubernetes.operator.exception.DeploymentFailedException; @@ -61,6 +62,7 @@ import static org.apache.flink.kubernetes.operator.api.utils.FlinkResourceUtils.getCheckpointInfo; import static org.apache.flink.kubernetes.operator.api.utils.FlinkResourceUtils.getJobStatus; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.OPERATOR_SAVEPOINT_FORMAT_TYPE; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -83,10 +85,9 @@ public void setup() { readyContext = TestUtils.createContextWithReadyJobManagerDeployment(kubernetesClient); deployment = TestUtils.buildApplicationCluster(); var jobId = new JobID().toHexString(); - deployment - .getSpec() - .getFlinkConfiguration() - .put(PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID.key(), jobId); + addConfigProperty( + deployment.getSpec(), PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID.key(), jobId); + deployment.getStatus().getJobStatus().setJobId(jobId); } @@ -246,10 +247,10 @@ public void observeSavepoint() throws Exception { deployment.getSpec().getJob().setSavepointTriggerNonce(timedOutNonce); Configuration conf = configManager.getDeployConfig(deployment.getMetadata(), deployment.getSpec()); - deployment - .getSpec() - .getFlinkConfiguration() - .put(KubernetesOperatorConfigOptions.SNAPSHOT_RESOURCE_ENABLED.key(), "false"); + addConfigProperty( + deployment.getSpec(), + KubernetesOperatorConfigOptions.SNAPSHOT_RESOURCE_ENABLED.key(), + "false"); flinkService.submitApplicationCluster(deployment.getSpec().getJob(), conf, false); bringToReadyStatus(deployment); assertTrue(ReconciliationUtils.isJobRunning(deployment.getStatus())); @@ -511,12 +512,10 @@ public void observeSavepoint() throws Exception { .getSavepointHistory() .size()); - deployment - .getSpec() - .getFlinkConfiguration() - .put( - KubernetesOperatorConfigOptions.OPERATOR_SAVEPOINT_HISTORY_MAX_COUNT.key(), - "1"); + addConfigProperty( + deployment.getSpec(), + KubernetesOperatorConfigOptions.OPERATOR_SAVEPOINT_HISTORY_MAX_COUNT.key(), + "1"); observer.observe(deployment, readyContext); assertEquals( 1, @@ -717,9 +716,11 @@ public void testSavepointFormat() throws Exception { deployment .getSpec() .setFlinkConfiguration( - Map.of( - OPERATOR_SAVEPOINT_FORMAT_TYPE.key(), - org.apache.flink.core.execution.SavepointFormatType.NATIVE.name())); + SpecUtils.mapToJsonNode( + Map.of( + OPERATOR_SAVEPOINT_FORMAT_TYPE.key(), + org.apache.flink.core.execution.SavepointFormatType.NATIVE + .name()))); conf = configManager.getDeployConfig(deployment.getMetadata(), deployment.getSpec()); flinkService.triggerSavepointLegacy( deployment.getStatus().getJobStatus().getJobId(), diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/SessionObserverTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/SessionObserverTest.java index 1b748ec755..66cd6874ae 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/SessionObserverTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/SessionObserverTest.java @@ -23,6 +23,7 @@ import org.apache.flink.kubernetes.operator.api.FlinkDeployment; import org.apache.flink.kubernetes.operator.api.status.JobManagerDeploymentStatus; import org.apache.flink.kubernetes.operator.api.status.ReconciliationState; +import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.apache.flink.kubernetes.operator.config.FlinkConfigManager; import org.apache.flink.kubernetes.operator.observer.TestObserverAdapter; import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; @@ -118,7 +119,7 @@ public void observeAlreadyUpgraded() { ReconciliationUtils.updateStatusForDeployedSpec(deployment, new Configuration()); // Test regular upgrades - deployment.getSpec().getFlinkConfiguration().put("k", "1"); + SpecUtils.addConfigProperty(deployment.getSpec(), "k", "1"); deployment.getMetadata().setGeneration(321L); ReconciliationUtils.updateStatusBeforeDeploymentAttempt( deployment, @@ -142,7 +143,7 @@ public void observeAlreadyUpgraded() { .put(FlinkUtils.CR_GENERATION_LABEL, "321"); deployment.getMetadata().setGeneration(322L); - deployment.getSpec().getFlinkConfiguration().put("k", "2"); + SpecUtils.addConfigProperty(deployment.getSpec(), "k", "2"); observer.observe(deployment, context); @@ -153,6 +154,6 @@ public void observeAlreadyUpgraded() { var specWithMeta = status.getReconciliationStatus().deserializeLastReconciledSpecWithMeta(); assertEquals(321L, status.getObservedGeneration()); - assertEquals("1", specWithMeta.getSpec().getFlinkConfiguration().get("k")); + assertEquals(1, specWithMeta.getSpec().getFlinkConfiguration().get("k").asInt()); } } diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerTest.java index 3d8ff29086..108802e400 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerTest.java @@ -124,6 +124,8 @@ import static org.apache.flink.kubernetes.operator.api.utils.FlinkResourceUtils.getReconciledJobSpec; import static org.apache.flink.kubernetes.operator.api.utils.FlinkResourceUtils.getReconciledJobState; import static org.apache.flink.kubernetes.operator.api.utils.FlinkResourceUtils.getSavepointInfo; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperties; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.OPERATOR_CLUSTER_HEALTH_CHECK_ENABLED; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.OPERATOR_JOB_SAVEPOINT_DISPOSE_ON_DELETE; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.SNAPSHOT_RESOURCE_ENABLED; @@ -207,10 +209,10 @@ public void testSubmitAndCleanUpWithSavepoint(FlinkVersion flinkVersion) throws public void testSubmitAndCleanUpWithSavepointOnResource(FlinkVersion flinkVersion) throws Exception { FlinkDeployment deployment = TestUtils.buildApplicationCluster(flinkVersion); - deployment - .getSpec() - .getFlinkConfiguration() - .put(KubernetesOperatorConfigOptions.SAVEPOINT_ON_DELETION.key(), "true"); + addConfigProperty( + deployment.getSpec(), + KubernetesOperatorConfigOptions.SAVEPOINT_ON_DELETION.key(), + "true"); // session ready reconciler.reconcile( @@ -270,7 +272,7 @@ public void testUpgrade(FlinkVersion flinkVersion, boolean snapshotResource) thr // Test stateless upgrade FlinkDeployment statelessUpgrade = ReconciliationUtils.clone(deployment); getJobSpec(statelessUpgrade).setUpgradeMode(UpgradeMode.STATELESS); - statelessUpgrade.getSpec().getFlinkConfiguration().put("new", "conf"); + addConfigProperty(statelessUpgrade.getSpec(), "new", "conf"); reconciler.reconcile(statelessUpgrade, context); assertFalse( statelessUpgrade @@ -304,7 +306,7 @@ public void testUpgrade(FlinkVersion flinkVersion, boolean snapshotResource) thr // Test stateful upgrade FlinkDeployment statefulUpgrade = ReconciliationUtils.clone(deployment); getJobSpec(statefulUpgrade).setUpgradeMode(UpgradeMode.SAVEPOINT); - statefulUpgrade.getSpec().getFlinkConfiguration().put("new", "conf2"); + addConfigProperty(statefulUpgrade.getSpec(), "new", "conf2"); reconciler.reconcile(statefulUpgrade, context); @@ -426,14 +428,14 @@ private static Savepoint savepointFromSavepointInfo( @Test public void triggerCheckpointLegacy() throws Exception { FlinkDeployment deployment = TestUtils.buildApplicationCluster(); - deployment.getSpec().getFlinkConfiguration().put(SNAPSHOT_RESOURCE_ENABLED.key(), "false"); + addConfigProperty(deployment.getSpec(), SNAPSHOT_RESOURCE_ENABLED.key(), "false"); testSnapshotLegacy(deployment, CHECKPOINT); } @Test public void triggerSavepointLegacy() throws Exception { FlinkDeployment deployment = TestUtils.buildApplicationCluster(); - deployment.getSpec().getFlinkConfiguration().put(SNAPSHOT_RESOURCE_ENABLED.key(), "false"); + addConfigProperty(deployment.getSpec(), SNAPSHOT_RESOURCE_ENABLED.key(), "false"); testSnapshotLegacy(deployment, SAVEPOINT); } @@ -442,10 +444,8 @@ public void triggerSavepointLegacy() throws Exception { public void triggerSavepointWithSnapshotResource(boolean disposeOnDelete) throws Exception { var deployment = TestUtils.buildApplicationCluster(); if (disposeOnDelete) { - deployment - .getSpec() - .getFlinkConfiguration() - .put(OPERATOR_JOB_SAVEPOINT_DISPOSE_ON_DELETE.key(), "true"); + addConfigProperty( + deployment.getSpec(), OPERATOR_JOB_SAVEPOINT_DISPOSE_ON_DELETE.key(), "true"); } reconciler.reconcile(deployment, context); @@ -666,11 +666,11 @@ private void testSnapshotLegacy(FlinkDeployment deployment, SnapshotType snapsho assertFalse(isSnapshotInProgress.test(getJobStatus(snDeployment))); // trigger by periodic interval settings - snDeployment.getSpec().getFlinkConfiguration().put(triggerSnapshotExpression.key(), "1"); + addConfigProperty(snDeployment.getSpec(), triggerSnapshotExpression.key(), "1"); reconciler.reconcile(snDeployment, context); assertTrue(isSnapshotInProgress.test(getJobStatus(snDeployment))); assertEquals(SnapshotStatus.PENDING, getLastSnapshotStatus(snDeployment, snapshotType)); - snDeployment.getSpec().getFlinkConfiguration().put(triggerSnapshotExpression.key(), "0"); + addConfigProperty(snDeployment.getSpec(), triggerSnapshotExpression.key(), "0"); } @NotNull @@ -740,7 +740,7 @@ private void verifyAndSetRunningJobsToStatus( @Test public void testJobUpgradeIgnorePendingSavepointLegacy() throws Exception { FlinkDeployment deployment = TestUtils.buildApplicationCluster(); - deployment.getSpec().getFlinkConfiguration().put(SNAPSHOT_RESOURCE_ENABLED.key(), "false"); + addConfigProperty(deployment.getSpec(), SNAPSHOT_RESOURCE_ENABLED.key(), "false"); reconciler.reconcile(deployment, context); var runningJobs = flinkService.listJobs(); @@ -755,12 +755,10 @@ public void testJobUpgradeIgnorePendingSavepointLegacy() throws Exception { getJobStatus(spDeployment).getState()); // Force upgrade when savepoint is in progress. - spDeployment - .getSpec() - .getFlinkConfiguration() - .put( - KubernetesOperatorConfigOptions.JOB_UPGRADE_IGNORE_PENDING_SAVEPOINT.key(), - "true"); + addConfigProperty( + spDeployment.getSpec(), + KubernetesOperatorConfigOptions.JOB_UPGRADE_IGNORE_PENDING_SAVEPOINT.key(), + "true"); spDeployment.getSpec().setImage("flink:greatest"); reconciler.reconcile(spDeployment, context); assertEquals("savepoint_trigger_0", getSavepointInfo(spDeployment).getTriggerId()); @@ -773,9 +771,8 @@ public void testJobUpgradeIgnorePendingSavepointLegacy() throws Exception { public void testRandomJobResultStorePath() throws Exception { FlinkDeployment flinkApp = TestUtils.buildApplicationCluster(); final String haStoragePath = "file:///flink-data/ha"; - flinkApp.getSpec() - .getFlinkConfiguration() - .put(HighAvailabilityOptions.HA_STORAGE_PATH.key(), haStoragePath); + addConfigProperty( + flinkApp.getSpec(), HighAvailabilityOptions.HA_STORAGE_PATH.key(), haStoragePath); ObjectMeta deployMeta = flinkApp.getMetadata(); FlinkDeploymentStatus status = flinkApp.getStatus(); @@ -843,21 +840,16 @@ public void testScaleWithReactiveModeEnabled() throws Exception { FlinkDeployment deployment = TestUtils.buildApplicationCluster(); deployment.getSpec().setMode(KubernetesDeploymentMode.STANDALONE); - deployment - .getSpec() - .getFlinkConfiguration() - .put( - JobManagerOptions.SCHEDULER_MODE.key(), - SchedulerExecutionMode.REACTIVE.name()); + addConfigProperty( + deployment.getSpec(), + JobManagerOptions.SCHEDULER_MODE.key(), + SchedulerExecutionMode.REACTIVE.name()); reconciler.reconcile(deployment, context); verifyAndSetRunningJobsToStatus(deployment, flinkService.listJobs()); // the default.parallelism is always ignored - deployment - .getSpec() - .getFlinkConfiguration() - .put(CoreOptions.DEFAULT_PARALLELISM.key(), "100"); + addConfigProperty(deployment.getSpec(), CoreOptions.DEFAULT_PARALLELISM.key(), "100"); reconciler.reconcile(deployment, context); assertEquals(JobState.RUNNING, getReconciledJobState(deployment)); @@ -922,12 +914,10 @@ public CancelResult cancelJob( // Set all the properties required by the rescale api deployment.getSpec().setFlinkVersion(FlinkVersion.v1_18); deployment.getSpec().setMode(KubernetesDeploymentMode.NATIVE); - deployment - .getSpec() - .getFlinkConfiguration() - .put( - JobManagerOptions.SCHEDULER.key(), - JobManagerOptions.SchedulerType.Adaptive.name()); + addConfigProperty( + deployment.getSpec(), + JobManagerOptions.SCHEDULER.key(), + JobManagerOptions.SchedulerType.Adaptive.name()); deployment.getMetadata().setGeneration(1L); // Deploy the job and update the status accordingly so we can proceed to rescaling it @@ -935,10 +925,10 @@ public CancelResult cancelJob( verifyAndSetRunningJobsToStatus(deployment, flinkService.listJobs()); // Override parallelism for a vertex and trigger rescaling - deployment - .getSpec() - .getFlinkConfiguration() - .put(PipelineOptions.PARALLELISM_OVERRIDES.key(), v1.toHexString() + ":2"); + addConfigProperty( + deployment.getSpec(), + PipelineOptions.PARALLELISM_OVERRIDES.key(), + v1.toHexString() + ":2"); deployment.getMetadata().setGeneration(2L); appReconciler.reconcile(ctxFactory.getResourceContext(deployment, context)); assertEquals(1, rescaleCounter.get()); @@ -955,7 +945,8 @@ public CancelResult cancelJob( reconStatus .deserializeLastReconciledSpec() .getFlinkConfiguration() - .get(PipelineOptions.PARALLELISM_OVERRIDES.key())); + .get(PipelineOptions.PARALLELISM_OVERRIDES.key()) + .asText()); assertEquals(ReconciliationState.DEPLOYED, reconStatus.getState()); assertFalse(reconStatus.isLastReconciledSpecStable()); @@ -990,9 +981,13 @@ public void cleanup(KubernetesJobAutoScalerContext ctx) { appReconciler = new ApplicationReconciler(eventRecorder, statusRecorder, autoscaler); var deployment = TestUtils.buildApplicationCluster(); - var config = deployment.getSpec().getFlinkConfiguration(); - config.put(AutoScalerOptions.AUTOSCALER_ENABLED.key(), "true"); - config.put(PipelineOptions.PARALLELISM_OVERRIDES.key(), v1 + ":1"); + addConfigProperties( + deployment.getSpec(), + Map.of( + AutoScalerOptions.AUTOSCALER_ENABLED.key(), + "true", + PipelineOptions.PARALLELISM_OVERRIDES.key(), + v1 + ":1")); var specCopy = SpecUtils.clone(deployment.getSpec()); @@ -1010,9 +1005,7 @@ public void cleanup(KubernetesJobAutoScalerContext ctx) { // Test overrides are applied correctly overrideFunction.set( - s -> - s.getFlinkConfiguration() - .put(PipelineOptions.PARALLELISM_OVERRIDES.key(), v1 + ":2")); + s -> addConfigProperty(s, PipelineOptions.PARALLELISM_OVERRIDES.key(), v1 + ":2")); appReconciler.reconcile(ctxFactory.getResourceContext(deployment, context)); deployment.setSpec(SpecUtils.clone(specCopy)); @@ -1152,12 +1145,10 @@ public void testTerminalJmTtl(ThrowingConsumer deploymentSetup) var status = deployment.getStatus(); assertEquals(JobManagerDeploymentStatus.READY, status.getJobManagerDeploymentStatus()); - deployment - .getSpec() - .getFlinkConfiguration() - .put( - KubernetesOperatorConfigOptions.OPERATOR_JM_SHUTDOWN_TTL.key(), - String.valueOf(Duration.ofMinutes(5).toMillis())); + addConfigProperty( + deployment.getSpec(), + KubernetesOperatorConfigOptions.OPERATOR_JM_SHUTDOWN_TTL.key(), + String.valueOf(Duration.ofMinutes(5).toMillis())); var now = Instant.now(); status.getJobStatus().setUpdateTime(String.valueOf(now.toEpochMilli())); @@ -1233,10 +1224,9 @@ public void testDeploymentRecoveryEvent() throws Exception { @Test public void testRestartUnhealthyEvent() throws Exception { FlinkDeployment deployment = TestUtils.buildApplicationCluster(); - deployment - .getSpec() - .getFlinkConfiguration() - .put(OPERATOR_CLUSTER_HEALTH_CHECK_ENABLED.key(), "true"); + addConfigProperty( + deployment.getSpec(), OPERATOR_CLUSTER_HEALTH_CHECK_ENABLED.key(), "true"); + reconciler.reconcile(deployment, context); Assertions.assertEquals( MSG_SUBMIT, flinkResourceEventCollector.events.remove().getMessage()); @@ -1259,14 +1249,11 @@ public void testReconcileIfUpgradeModeNotAvailable() throws Exception { // We disable last state fallback as we want to test that the deployment is properly // recovered before upgrade - deployment - .getSpec() - .getFlinkConfiguration() - .put( - KubernetesOperatorConfigOptions - .OPERATOR_JOB_UPGRADE_LAST_STATE_FALLBACK_ENABLED - .key(), - "false"); + addConfigProperty( + deployment.getSpec(), + KubernetesOperatorConfigOptions.OPERATOR_JOB_UPGRADE_LAST_STATE_FALLBACK_ENABLED + .key(), + "false"); // Initial deployment reconciler.reconcile(deployment, context); @@ -1327,7 +1314,7 @@ public void testUpgradeReconciledGeneration() throws Exception { assertEquals(1L, deployment.getStatus().getObservedGeneration()); // Submit no-op upgrade - deployment.getSpec().getFlinkConfiguration().put("kubernetes.operator.test", "value"); + addConfigProperty(deployment.getSpec(), "kubernetes.operator.test", "value"); deployment.getMetadata().setGeneration(2L); reconciler.reconcile(deployment, context); @@ -1341,15 +1328,17 @@ public void testRollbackUpgradeModeHandling(boolean jmStarted) throws Exception deployment.getSpec().getJob().setUpgradeMode(UpgradeMode.SAVEPOINT); offsetReconcilerClock(deployment, Duration.ZERO); - var flinkConfiguration = deployment.getSpec().getFlinkConfiguration(); - flinkConfiguration.put( - KubernetesOperatorConfigOptions.DEPLOYMENT_ROLLBACK_ENABLED.key(), "true"); - flinkConfiguration.put( - KubernetesOperatorConfigOptions.DEPLOYMENT_READINESS_TIMEOUT.key(), "10s"); - flinkConfiguration.put( - KubernetesOperatorConfigOptions.OPERATOR_JOB_UPGRADE_LAST_STATE_FALLBACK_ENABLED - .key(), - "false"); + addConfigProperties( + deployment.getSpec(), + Map.of( + KubernetesOperatorConfigOptions.DEPLOYMENT_ROLLBACK_ENABLED.key(), + "true", + KubernetesOperatorConfigOptions.DEPLOYMENT_READINESS_TIMEOUT.key(), + "10s", + KubernetesOperatorConfigOptions + .OPERATOR_JOB_UPGRADE_LAST_STATE_FALLBACK_ENABLED + .key(), + "false")); // Initial deployment, mark as stable reconciler.reconcile(deployment, context); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java index a208a00f80..858d19ef61 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java @@ -39,6 +39,7 @@ import org.apache.flink.kubernetes.operator.api.status.Savepoint; import org.apache.flink.kubernetes.operator.api.status.SavepointFormatType; import org.apache.flink.kubernetes.operator.api.status.SnapshotTriggerType; +import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; import org.apache.flink.kubernetes.operator.exception.UpgradeFailureException; import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; @@ -69,6 +70,9 @@ import static org.apache.flink.api.common.JobStatus.RECONCILING; import static org.apache.flink.api.common.JobStatus.RESTARTING; import static org.apache.flink.api.common.JobStatus.RUNNING; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.removeConfigProperties; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.toStringMap; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.SNAPSHOT_RESOURCE_ENABLED; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -161,10 +165,10 @@ private void testUpgradeToSavepoint(FlinkVersion flinkVersion, UpgradeMode fromU FlinkDeployment modifiedDeployment = cloneDeploymentWithUpgradeMode(deployment, UpgradeMode.SAVEPOINT); - modifiedDeployment - .getSpec() - .getFlinkConfiguration() - .put(CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), "test-savepoint-dir"); + addConfigProperty( + modifiedDeployment.getSpec(), + CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), + "test-savepoint-dir"); reconciler.reconcile(modifiedDeployment, context); assertEquals(0, flinkService.getRunningCount()); @@ -254,10 +258,10 @@ private void testUpgradeToLastState(FlinkVersion flinkVersion, UpgradeMode fromU .getJobStatus() .setState(org.apache.flink.api.common.JobStatus.FINISHED); deployment.getStatus().setJobManagerDeploymentStatus(JobManagerDeploymentStatus.READY); - deployment - .getSpec() - .getFlinkConfiguration() - .put(CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), "test-savepoint-dir"); + addConfigProperty( + deployment.getSpec(), + CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), + "test-savepoint-dir"); reconciler.reconcile(deployment, context); reconciler.reconcile(deployment, context); @@ -267,7 +271,7 @@ private void testUpgradeToLastState(FlinkVersion flinkVersion, UpgradeMode fromU @ParameterizedTest @ValueSource(booleans = {true, false}) - public void testUpgradeUsesLatestSnapshot(boolean useLegacyFields) throws Exception { + void testUpgradeUsesLatestSnapshot(boolean useLegacyFields) throws Exception { var savepointPath = "finished_sp"; var deployment = buildApplicationCluster(FlinkVersion.v1_19, UpgradeMode.SAVEPOINT); @@ -316,7 +320,7 @@ private FlinkDeployment cloneDeploymentWithUpgradeMode( FlinkDeployment result = ReconciliationUtils.clone(deployment); result.getSpec().getJob().setUpgradeMode(upgradeMode); - result.getSpec().getFlinkConfiguration().put("new", "conf"); + addConfigProperty(result.getSpec(), "new", "conf"); return result; } @@ -433,7 +437,7 @@ public void testInitialJmDeployCannotStartLegacy(UpgradeMode upgradeMode, boolea flinkService.setJobManagerReady(false); var deployment = TestUtils.buildApplicationCluster(); - deployment.getSpec().getFlinkConfiguration().put(SNAPSHOT_RESOURCE_ENABLED.key(), "false"); + addConfigProperty(deployment.getSpec(), SNAPSHOT_RESOURCE_ENABLED.key(), "false"); if (initSavepoint) { deployment.getSpec().getJob().setInitialSavepointPath("init-sp"); } @@ -500,13 +504,10 @@ public void testInitialJmDeployCannotStartLegacy(UpgradeMode upgradeMode, boolea @ValueSource(booleans = {true, false}) public void testLastStateMaxCheckpointAge(boolean cancellable) throws Exception { var deployment = TestUtils.buildApplicationCluster(); - deployment - .getSpec() - .getFlinkConfiguration() - .put( - KubernetesOperatorConfigOptions.OPERATOR_JOB_UPGRADE_LAST_STATE_CANCEL_JOB - .key(), - Boolean.toString(cancellable)); + addConfigProperty( + deployment.getSpec(), + KubernetesOperatorConfigOptions.OPERATOR_JOB_UPGRADE_LAST_STATE_CANCEL_JOB.key(), + Boolean.toString(cancellable)); deployment.getSpec().getJob().setUpgradeMode(UpgradeMode.LAST_STATE); ReconciliationUtils.updateStatusForDeployedSpec(deployment, new Configuration()); @@ -617,19 +618,14 @@ public void testFlinkVersionSwitching( var jobReconciler = (ApplicationReconciler) this.reconciler.getReconciler(); var deployment = TestUtils.buildApplicationCluster(FlinkVersion.v1_18); if (!savepointsEnabled) { - deployment - .getSpec() - .getFlinkConfiguration() - .remove(CheckpointingOptions.SAVEPOINT_DIRECTORY.key()); + SpecUtils.removeConfigProperties( + deployment.getSpec(), CheckpointingOptions.SAVEPOINT_DIRECTORY.key()); } - deployment - .getSpec() - .getFlinkConfiguration() - .put( - KubernetesOperatorConfigOptions - .OPERATOR_JOB_UPGRADE_LAST_STATE_FALLBACK_ENABLED - .key(), - Boolean.toString(allowFallback)); + addConfigProperty( + deployment.getSpec(), + KubernetesOperatorConfigOptions.OPERATOR_JOB_UPGRADE_LAST_STATE_FALLBACK_ENABLED + .key(), + Boolean.toString(allowFallback)); deployment.getSpec().getJob().setUpgradeMode(upgradeMode); ReconciliationUtils.updateStatusForDeployedSpec(deployment, new Configuration()); deployment.getSpec().setFlinkVersion(FlinkVersion.v1_19); @@ -679,22 +675,16 @@ public void testLastStateNoHaMeta(UpgradeMode upgradeMode, boolean allowFallback throws Exception { var jobReconciler = (ApplicationReconciler) this.reconciler.getReconciler(); var deployment = TestUtils.buildApplicationCluster(); - deployment - .getSpec() - .getFlinkConfiguration() - .put( - KubernetesOperatorConfigOptions - .OPERATOR_JOB_UPGRADE_LAST_STATE_FALLBACK_ENABLED - .key(), - Boolean.toString(allowFallback)); - deployment.getSpec().getFlinkConfiguration().remove(HighAvailabilityOptions.HA_MODE.key()); - deployment - .getSpec() - .getFlinkConfiguration() - .put( - KubernetesOperatorConfigOptions.OPERATOR_JOB_UPGRADE_LAST_STATE_CANCEL_JOB - .key(), - Boolean.toString(false)); + addConfigProperty( + deployment.getSpec(), + KubernetesOperatorConfigOptions.OPERATOR_JOB_UPGRADE_LAST_STATE_FALLBACK_ENABLED + .key(), + Boolean.toString(allowFallback)); + removeConfigProperties(deployment.getSpec(), HighAvailabilityOptions.HA_MODE.key()); + addConfigProperty( + deployment.getSpec(), + KubernetesOperatorConfigOptions.OPERATOR_JOB_UPGRADE_LAST_STATE_CANCEL_JOB.key(), + Boolean.toString(false)); deployment.getSpec().getJob().setUpgradeMode(upgradeMode); ReconciliationUtils.updateStatusForDeployedSpec(deployment, new Configuration()); @@ -743,7 +733,9 @@ public void testLastStateOnDeletedDeployment() throws Exception { verifyAndSetRunningJobsToStatus(deployment, flinkService.listJobs()); // Delete cluster and keep HA metadata - var conf = Configuration.fromMap(deployment.getSpec().getFlinkConfiguration()); + var conf = + Configuration.fromMap( + SpecUtils.toStringMap(deployment.getSpec().getFlinkConfiguration())); flinkService.deleteClusterDeployment( deployment.getMetadata(), deployment.getStatus(), conf, false); flinkService.setHaDataAvailable(true); @@ -836,7 +828,7 @@ public void testUpgradeModeChangedToLastStateShouldCancelWhileHADisabled() throw flinkService.setHaDataAvailable(false); var deployment = TestUtils.buildApplicationCluster(); - deployment.getSpec().getFlinkConfiguration().remove(HighAvailabilityOptions.HA_MODE.key()); + removeConfigProperties(deployment.getSpec(), HighAvailabilityOptions.HA_MODE.key()); reconciler.reconcile(deployment, context); assertEquals( @@ -924,7 +916,7 @@ public static FlinkDeployment buildApplicationCluster( FlinkVersion flinkVersion, UpgradeMode upgradeMode) { FlinkDeployment deployment = TestUtils.buildApplicationCluster(flinkVersion); deployment.getSpec().getJob().setUpgradeMode(upgradeMode); - Map conf = deployment.getSpec().getFlinkConfiguration(); + Map conf = toStringMap(deployment.getSpec().getFlinkConfiguration()); switch (upgradeMode) { case STATELESS: diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/SessionReconcilerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/SessionReconcilerTest.java index 284af2213b..429169dab3 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/SessionReconcilerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/SessionReconcilerTest.java @@ -49,6 +49,7 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -151,17 +152,18 @@ public void testSetOwnerReference() throws Exception { @Test public void testGetNonTerminalJobs() throws Exception { FlinkDeployment deployment = TestUtils.buildSessionCluster(); - deployment - .getSpec() - .getFlinkConfiguration() - .put(KubernetesOperatorConfigOptions.BLOCK_ON_UNMANAGED_JOBS.key(), "true"); + addConfigProperty( + deployment.getSpec(), + KubernetesOperatorConfigOptions.BLOCK_ON_UNMANAGED_JOBS.key(), + "true"); assertEquals( - "true", + true, deployment .getSpec() .getFlinkConfiguration() - .get(KubernetesOperatorConfigOptions.BLOCK_ON_UNMANAGED_JOBS.key())); + .get(KubernetesOperatorConfigOptions.BLOCK_ON_UNMANAGED_JOBS.key()) + .asBoolean()); reconciler.reconcile(deployment, flinkService.getContext()); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/diff/SpecDiffTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/diff/SpecDiffTest.java index 07bc421b7e..f1dfcb7c89 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/diff/SpecDiffTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/diff/SpecDiffTest.java @@ -46,6 +46,7 @@ import java.util.Map; import java.util.stream.Collectors; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.OPERATOR_RECONCILE_INTERVAL; import static org.apache.flink.kubernetes.operator.metrics.KubernetesOperatorMetricOptions.SCOPE_NAMING_KUBERNETES_OPERATOR; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -83,17 +84,26 @@ public void testFlinkDeploymentSpecChanges() { right.getJob().setAllowNonRestoredState(true); right.getJob().setInitialSavepointPath("local:///tmp"); right.getJob().setSavepointTriggerNonce(123L); - right.getFlinkConfiguration().put(OPERATOR_RECONCILE_INTERVAL.key(), "100 SECONDS"); - right.getFlinkConfiguration().put(SCOPE_NAMING_KUBERNETES_OPERATOR.key(), "foo.bar"); - right.getFlinkConfiguration().put(CoreOptions.DEFAULT_PARALLELISM.key(), "100"); - right.getFlinkConfiguration().put(AutoScalerOptions.METRICS_WINDOW.key(), "1234m"); + SpecUtils.addConfigProperties( + right, + Map.of( + OPERATOR_RECONCILE_INTERVAL.key(), + "100 SECONDS", + SCOPE_NAMING_KUBERNETES_OPERATOR.key(), + "foo.bar", + CoreOptions.DEFAULT_PARALLELISM.key(), + "100", + AutoScalerOptions.METRICS_WINDOW.key(), + "1234m")); diff = new ReflectiveDiffBuilder<>(KubernetesDeploymentMode.NATIVE, left, right).build(); assertEquals(DiffType.IGNORE, diff.getType()); assertEquals(8, diff.getNumDiffs()); - right.getFlinkConfiguration().remove(SCOPE_NAMING_KUBERNETES_OPERATOR.key()); - right.getFlinkConfiguration().remove(AutoScalerOptions.METRICS_WINDOW.key()); + SpecUtils.removeConfigProperties( + right, + SCOPE_NAMING_KUBERNETES_OPERATOR.key(), + AutoScalerOptions.METRICS_WINDOW.key()); diff = new ReflectiveDiffBuilder<>(KubernetesDeploymentMode.NATIVE, left, right).build(); assertEquals(DiffType.IGNORE, diff.getType()); @@ -140,7 +150,7 @@ public void testFlinkDeploymentSpecChanges() { assertEquals(DiffType.UPGRADE, diff.getType()); assertEquals(21, diff.getNumDiffs()); - right.getFlinkConfiguration().put(CoreOptions.FLINK_TM_JVM_OPTIONS.key(), "-Dfoo=bar"); + addConfigProperty(right, CoreOptions.FLINK_TM_JVM_OPTIONS.key(), "-Dfoo=bar"); diff = new ReflectiveDiffBuilder<>(KubernetesDeploymentMode.NATIVE, left, right).build(); assertEquals(DiffType.UPGRADE, diff.getType()); @@ -166,7 +176,7 @@ public void testFlinkDeploymentSpecChanges() { // verify parallelism override handling for native/standalone left = TestUtils.buildApplicationCluster().getSpec(); right = TestUtils.buildApplicationCluster().getSpec(); - left.getFlinkConfiguration().put(PipelineOptions.PARALLELISM_OVERRIDES.key(), "new"); + addConfigProperty(left, PipelineOptions.PARALLELISM_OVERRIDES.key(), "new"); diff = new ReflectiveDiffBuilder<>(KubernetesDeploymentMode.NATIVE, left, right).build(); assertEquals(DiffType.SCALE, diff.getType()); @@ -199,8 +209,8 @@ public void testFlinkSessionJobSpecChanges() { right.getJob().setAllowNonRestoredState(true); right.getJob().setInitialSavepointPath("local:///tmp"); right.getJob().setSavepointTriggerNonce(123L); - right.getFlinkConfiguration() - .put(KubernetesOperatorConfigOptions.JAR_ARTIFACT_HTTP_HEADER.key(), "changed"); + addConfigProperty( + right, KubernetesOperatorConfigOptions.JAR_ARTIFACT_HTTP_HEADER.key(), "changed"); diff = new ReflectiveDiffBuilder<>(KubernetesDeploymentMode.NATIVE, left, right).build(); assertEquals(DiffType.IGNORE, diff.getType()); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/sessionjob/SessionJobReconcilerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/sessionjob/SessionJobReconcilerTest.java index b001ed7739..b00928930e 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/sessionjob/SessionJobReconcilerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/sessionjob/SessionJobReconcilerTest.java @@ -72,6 +72,7 @@ import static org.apache.flink.kubernetes.operator.api.utils.FlinkResourceUtils.getJobSpec; import static org.apache.flink.kubernetes.operator.api.utils.FlinkResourceUtils.getJobStatus; import static org.apache.flink.kubernetes.operator.api.utils.FlinkResourceUtils.getReconciledJobSpec; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.OPERATOR_JOB_RESTART_FAILED; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.SNAPSHOT_RESOURCE_ENABLED; import static org.apache.flink.kubernetes.operator.reconciler.SnapshotType.CHECKPOINT; @@ -148,16 +149,13 @@ public void testSubmitAndCleanUpWithSavepoint(boolean legacySnapshots) throws Ex public void testSubmitAndCleanUpWithSavepointOnResource(boolean legacySnapshots) throws Exception { FlinkSessionJob sessionJob = TestUtils.buildSessionJob(); - sessionJob - .getSpec() - .getFlinkConfiguration() - .put(KubernetesOperatorConfigOptions.SAVEPOINT_ON_DELETION.key(), "true"); + addConfigProperty( + sessionJob.getSpec(), + KubernetesOperatorConfigOptions.SAVEPOINT_ON_DELETION.key(), + "true"); if (legacySnapshots) { - sessionJob - .getSpec() - .getFlinkConfiguration() - .put(SNAPSHOT_RESOURCE_ENABLED.key(), "false"); + addConfigProperty(sessionJob.getSpec(), SNAPSHOT_RESOURCE_ENABLED.key(), "false"); } // session ready @@ -364,10 +362,7 @@ public void testStatelessUpgrade() throws Exception { public void testSavepointUpgrade(boolean legacySnapshots) throws Exception { FlinkSessionJob sessionJob = TestUtils.buildSessionJob(); if (legacySnapshots) { - sessionJob - .getSpec() - .getFlinkConfiguration() - .put(SNAPSHOT_RESOURCE_ENABLED.key(), "false"); + addConfigProperty(sessionJob.getSpec(), SNAPSHOT_RESOURCE_ENABLED.key(), "false"); } var readyContext = TestUtils.createContextWithReadyFlinkDeployment(kubernetesClient); @@ -431,7 +426,7 @@ public void testSavepointUpgrade(boolean legacySnapshots) throws Exception { @Test public void testTriggerSavepointLegacyLegacy() throws Exception { FlinkSessionJob sessionJob = TestUtils.buildSessionJob(); - sessionJob.getSpec().getFlinkConfiguration().put(SNAPSHOT_RESOURCE_ENABLED.key(), "false"); + addConfigProperty(sessionJob.getSpec(), SNAPSHOT_RESOURCE_ENABLED.key(), "false"); assertFalse(SnapshotUtils.savepointInProgress(sessionJob.getStatus().getJobStatus())); @@ -558,7 +553,7 @@ public void testTriggerSavepointLegacyLegacy() throws Exception { @Test public void testTriggerCheckpoint() throws Exception { FlinkSessionJob sessionJob = TestUtils.buildSessionJob(); - sessionJob.getSpec().getFlinkConfiguration().put(SNAPSHOT_RESOURCE_ENABLED.key(), "false"); + addConfigProperty(sessionJob.getSpec(), SNAPSHOT_RESOURCE_ENABLED.key(), "false"); assertFalse(SnapshotUtils.checkpointInProgress(getJobStatus(sessionJob))); @@ -791,10 +786,7 @@ public void testJobUpgradeIgnorePendingSavepoint() throws Exception { sessionJob, JobState.RUNNING, RECONCILING, null, flinkService.listJobs()); FlinkSessionJob spSessionJob = ReconciliationUtils.clone(sessionJob); - spSessionJob - .getSpec() - .getFlinkConfiguration() - .put(SNAPSHOT_RESOURCE_ENABLED.key(), "false"); + addConfigProperty(spSessionJob.getSpec(), SNAPSHOT_RESOURCE_ENABLED.key(), "false"); spSessionJob .getSpec() .getJob() diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkServiceTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkServiceTest.java index a57d60a8a3..e07f1a2c3f 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkServiceTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkServiceTest.java @@ -142,6 +142,7 @@ import static org.apache.flink.api.common.JobStatus.FAILING; import static org.apache.flink.api.common.JobStatus.FINISHED; import static org.apache.flink.api.common.JobStatus.RUNNING; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; import static org.apache.flink.kubernetes.operator.config.FlinkConfigBuilder.FLINK_VERSION; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.OPERATOR_SAVEPOINT_FORMAT_TYPE; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; @@ -382,10 +383,10 @@ public void cancelJobWithSavepointUpgradeModeTest(boolean deleteAfterSavepoint) JobID jobID = JobID.generate(); FlinkDeployment deployment = TestUtils.buildApplicationCluster(); - deployment - .getSpec() - .getFlinkConfiguration() - .put(CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), savepointPath); + addConfigProperty( + deployment.getSpec(), + CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), + savepointPath); deployment.getStatus().setJobManagerDeploymentStatus(JobManagerDeploymentStatus.READY); JobStatus jobStatus = deployment.getStatus().getJobStatus(); jobStatus.setJobId(jobID.toHexString()); @@ -491,10 +492,10 @@ public void cancelJobWithDrainOnSavepointUpgradeModeTest(boolean drainOnSavepoin JobID jobID = JobID.generate(); FlinkDeployment deployment = TestUtils.buildApplicationCluster(); - deployment - .getSpec() - .getFlinkConfiguration() - .put(CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), savepointPath); + addConfigProperty( + deployment.getSpec(), + CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), + savepointPath); deployment.getStatus().setJobManagerDeploymentStatus(JobManagerDeploymentStatus.READY); JobStatus jobStatus = deployment.getStatus().getJobStatus(); jobStatus.setJobId(jobID.toHexString()); @@ -502,14 +503,14 @@ public void cancelJobWithDrainOnSavepointUpgradeModeTest(boolean drainOnSavepoin ReconciliationUtils.updateStatusForDeployedSpec(deployment, new Configuration()); if (drainOnSavepoint) { - deployment - .getSpec() - .getFlinkConfiguration() - .put(KubernetesOperatorConfigOptions.SAVEPOINT_ON_DELETION.key(), "true"); - deployment - .getSpec() - .getFlinkConfiguration() - .put(KubernetesOperatorConfigOptions.DRAIN_ON_SAVEPOINT_DELETION.key(), "true"); + addConfigProperty( + deployment.getSpec(), + KubernetesOperatorConfigOptions.SAVEPOINT_ON_DELETION.key(), + "true"); + addConfigProperty( + deployment.getSpec(), + KubernetesOperatorConfigOptions.DRAIN_ON_SAVEPOINT_DELETION.key(), + "true"); } var result = @@ -558,9 +559,8 @@ public void cancelSessionJobWithDrainOnSavepointUpgradeModeTest(boolean drainOnS .serializeAndSetLastReconciledSpec(session.getSpec(), session); var job = TestUtils.buildSessionJob(); - job.getSpec() - .getFlinkConfiguration() - .put(CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), savepointPath); + addConfigProperty( + job.getSpec(), CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), savepointPath); JobStatus jobStatus = job.getStatus().getJobStatus(); jobStatus.setJobId(jobID.toHexString()); @@ -568,12 +568,14 @@ public void cancelSessionJobWithDrainOnSavepointUpgradeModeTest(boolean drainOnS ReconciliationUtils.updateStatusForDeployedSpec(job, new Configuration()); if (drainOnSavepoint) { - job.getSpec() - .getFlinkConfiguration() - .put(KubernetesOperatorConfigOptions.SAVEPOINT_ON_DELETION.key(), "true"); - job.getSpec() - .getFlinkConfiguration() - .put(KubernetesOperatorConfigOptions.DRAIN_ON_SAVEPOINT_DELETION.key(), "true"); + addConfigProperty( + job.getSpec(), + KubernetesOperatorConfigOptions.SAVEPOINT_ON_DELETION.key(), + "true"); + addConfigProperty( + job.getSpec(), + KubernetesOperatorConfigOptions.DRAIN_ON_SAVEPOINT_DELETION.key(), + "true"); } var deployConf = configManager.getSessionJobConfig( @@ -801,10 +803,10 @@ private void runNativeSavepointFormatTest(boolean failAfterSavepointCompletes) var flinkService = new TestingService(testingClusterClient); final FlinkDeployment deployment = TestUtils.buildApplicationCluster(); - deployment - .getSpec() - .getFlinkConfiguration() - .put(CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), savepointPath); + addConfigProperty( + deployment.getSpec(), + CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), + savepointPath); deployment.getStatus().setJobManagerDeploymentStatus(JobManagerDeploymentStatus.READY); JobStatus jobStatus = deployment.getStatus().getJobStatus(); jobStatus.setJobId(jobID.toHexString()); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/NativeFlinkServiceTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/NativeFlinkServiceTest.java index 64b216324c..c8f13998ca 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/NativeFlinkServiceTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/NativeFlinkServiceTest.java @@ -31,6 +31,7 @@ import org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec; import org.apache.flink.kubernetes.operator.api.spec.FlinkVersion; import org.apache.flink.kubernetes.operator.api.spec.JobSpec; +import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.apache.flink.kubernetes.operator.config.FlinkConfigManager; import org.apache.flink.kubernetes.operator.config.FlinkOperatorConfiguration; import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; @@ -70,6 +71,9 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.configurationToJsonNode; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.removeConfigProperties; import static org.apache.flink.kubernetes.operator.config.FlinkConfigBuilder.FLINK_VERSION; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.OPERATOR_HEALTH_PROBE_PORT; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -277,17 +281,17 @@ protected void updateVertexResources( var spec = flinkDep.getSpec(); spec.setFlinkVersion(FlinkVersion.v1_18); - var appConfig = Configuration.fromMap(spec.getFlinkConfiguration()); + var appConfig = Configuration.fromMap(SpecUtils.toStringMap(spec.getFlinkConfiguration())); appConfig.set(JobManagerOptions.SCHEDULER, JobManagerOptions.SchedulerType.Adaptive); - spec.setFlinkConfiguration(appConfig.toMap()); + spec.setFlinkConfiguration(SpecUtils.mapToJsonNode(appConfig.toMap())); var reconStatus = flinkDep.getStatus().getReconciliationStatus(); reconStatus.serializeAndSetLastReconciledSpec(spec, flinkDep); appConfig.set( PipelineOptions.PARALLELISM_OVERRIDES, Map.of(v1.toHexString(), "4", v2.toHexString(), "1")); - spec.setFlinkConfiguration(appConfig.toMap()); + spec.setFlinkConfiguration(SpecUtils.mapToJsonNode(appConfig.toMap())); flinkDep.getStatus().getJobStatus().setState(JobStatus.RUNNING); @@ -321,7 +325,7 @@ protected void updateVertexResources( // Baseline appConfig.set(PipelineOptions.PARALLELISM_OVERRIDES, Map.of(v1.toHexString(), "4")); - spec.setFlinkConfiguration(appConfig.toMap()); + spec.setFlinkConfiguration(configurationToJsonNode(appConfig)); testScaleConditionDep(flinkDep, service, d -> {}, true); testScaleConditionLastSpec(flinkDep, service, d -> {}, true); @@ -330,13 +334,11 @@ protected void updateVertexResources( flinkDep, service, d -> - d.getSpec() - .getFlinkConfiguration() - .put( - KubernetesOperatorConfigOptions - .JOB_UPGRADE_INPLACE_SCALING_ENABLED - .key(), - "false"), + addConfigProperty( + d.getSpec(), + KubernetesOperatorConfigOptions.JOB_UPGRADE_INPLACE_SCALING_ENABLED + .key(), + "false"), false); // Do not scale without adaptive scheduler deployed @@ -344,10 +346,10 @@ protected void updateVertexResources( flinkDep, service, ls -> - ls.getFlinkConfiguration() - .put( - JobManagerOptions.SCHEDULER.key(), - JobManagerOptions.SchedulerType.Default.name()), + addConfigProperty( + ls, + JobManagerOptions.SCHEDULER.key(), + JobManagerOptions.SchedulerType.Default.name()), false); // Do not scale without adaptive scheduler deployed @@ -382,9 +384,7 @@ protected void updateVertexResources( testScaleConditionLastSpec( flinkDep, service, - s -> - s.getFlinkConfiguration() - .put(PipelineOptions.PARALLELISM_OVERRIDES.key(), v2 + ":3"), + s -> addConfigProperty(s, PipelineOptions.PARALLELISM_OVERRIDES.key(), v2 + ":3"), false); // Scale if parallelism overrides were removed only from a non-active vertex @@ -392,24 +392,20 @@ protected void updateVertexResources( flinkDep, service, s -> - s.getFlinkConfiguration() - .put( - PipelineOptions.PARALLELISM_OVERRIDES.key(), - v1 + ":1," + new JobVertexID() + ":5"), + addConfigProperty( + s, + PipelineOptions.PARALLELISM_OVERRIDES.key(), + v1 + ":1," + new JobVertexID() + ":5"), true); // Do not scale if parallelism overrides were completely removed var flinkDep2 = ReconciliationUtils.clone(flinkDep); - flinkDep2 - .getSpec() - .getFlinkConfiguration() - .remove(PipelineOptions.PARALLELISM_OVERRIDES.key()); + SpecUtils.removeConfigProperties( + flinkDep2.getSpec(), PipelineOptions.PARALLELISM_OVERRIDES.key()); testScaleConditionLastSpec( flinkDep2, service, - s -> - s.getFlinkConfiguration() - .put(PipelineOptions.PARALLELISM_OVERRIDES.key(), v2 + ":3"), + s -> addConfigProperty(s, PipelineOptions.PARALLELISM_OVERRIDES.key(), v2 + ":3"), false); // Do not scale if overrides never set @@ -417,9 +413,8 @@ protected void updateVertexResources( flinkDep2, service, d -> - d.getSpec() - .getFlinkConfiguration() - .remove(PipelineOptions.PARALLELISM_OVERRIDES.key()), + removeConfigProperties( + d.getSpec(), PipelineOptions.PARALLELISM_OVERRIDES.key()), false); // Do not scale if non active vertices are overridden only @@ -435,9 +430,10 @@ protected void updateVertexResources( flinkDep, service, d -> - d.getSpec() - .getFlinkConfiguration() - .put(PipelineOptions.PARALLELISM_OVERRIDES.key(), v2 + ":5"), + addConfigProperty( + d.getSpec(), + PipelineOptions.PARALLELISM_OVERRIDES.key(), + v2 + ":5"), true); assertNull(updated.get()); @@ -446,11 +442,10 @@ protected void updateVertexResources( flinkDep, service, d -> - d.getSpec() - .getFlinkConfiguration() - .put( - PipelineOptions.PARALLELISM_OVERRIDES.key(), - v2 + ":5," + v1 + ":1"), + addConfigProperty( + d.getSpec(), + PipelineOptions.PARALLELISM_OVERRIDES.key(), + v2 + ":5," + v1 + ":1"), true); assertNull(updated.get()); @@ -464,11 +459,10 @@ protected void updateVertexResources( flinkDep, service, d -> - d.getSpec() - .getFlinkConfiguration() - .put( - PipelineOptions.PARALLELISM_OVERRIDES.key(), - v2 + ":5," + v1 + ":1"), + addConfigProperty( + d.getSpec(), + PipelineOptions.PARALLELISM_OVERRIDES.key(), + v2 + ":5," + v1 + ":1"), true); assertEquals( new JobVertexResourceRequirements.Parallelism(1, 1), diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/StandaloneFlinkServiceTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/StandaloneFlinkServiceTest.java index fde2b12421..7517057ce6 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/StandaloneFlinkServiceTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/StandaloneFlinkServiceTest.java @@ -26,6 +26,7 @@ import org.apache.flink.kubernetes.operator.api.FlinkDeployment; import org.apache.flink.kubernetes.operator.api.spec.JobSpec; import org.apache.flink.kubernetes.operator.api.spec.KubernetesDeploymentMode; +import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.apache.flink.kubernetes.operator.artifact.ArtifactManager; import org.apache.flink.kubernetes.operator.config.FlinkConfigManager; import org.apache.flink.kubernetes.operator.config.Mode; @@ -43,6 +44,7 @@ import java.util.List; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.OPERATOR_HEALTH_PROBE_PORT; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -133,12 +135,10 @@ public void testTMReplicaScaleApplication() { flinkDeployment.getSpec().setMode(KubernetesDeploymentMode.STANDALONE); // Add parallelism change, verify it is honoured in reactive mode - flinkDeployment - .getSpec() - .getFlinkConfiguration() - .put( - JobManagerOptions.SCHEDULER_MODE.key(), - SchedulerExecutionMode.REACTIVE.name()); + addConfigProperty( + flinkDeployment.getSpec(), + JobManagerOptions.SCHEDULER_MODE.key(), + SchedulerExecutionMode.REACTIVE.name()); flinkDeployment .getStatus() .getReconciliationStatus() @@ -163,10 +163,8 @@ public void testTMReplicaScaleApplication() { .getSpec() .getReplicas()); - flinkDeployment - .getSpec() - .getFlinkConfiguration() - .remove(JobManagerOptions.SCHEDULER_MODE.key()); + SpecUtils.removeConfigProperties( + flinkDeployment.getSpec(), JobManagerOptions.SCHEDULER_MODE.key()); flinkDeployment .getStatus() .getReconciliationStatus() @@ -191,12 +189,10 @@ public void testTMReplicaScaleSession() { flinkDeployment.getSpec().setMode(KubernetesDeploymentMode.STANDALONE); // Add replicas flinkDeployment.getSpec().getTaskManager().setReplicas(3); - flinkDeployment - .getSpec() - .getFlinkConfiguration() - .put( - JobManagerOptions.SCHEDULER_MODE.key(), - SchedulerExecutionMode.REACTIVE.name()); + addConfigProperty( + flinkDeployment.getSpec(), + JobManagerOptions.SCHEDULER_MODE.key(), + SchedulerExecutionMode.REACTIVE.name()); flinkDeployment .getStatus() .getReconciliationStatus() diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/SnapshotUtilsTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/SnapshotUtilsTest.java index 5b3cb541cc..3ee0a82310 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/SnapshotUtilsTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/SnapshotUtilsTest.java @@ -41,6 +41,7 @@ import static org.apache.flink.kubernetes.operator.TestUtils.reconcileSpec; import static org.apache.flink.kubernetes.operator.TestUtils.setupCronTrigger; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.PERIODIC_CHECKPOINT_INTERVAL; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.PERIODIC_SAVEPOINT_INTERVAL; import static org.apache.flink.kubernetes.operator.reconciler.SnapshotType.CHECKPOINT; @@ -82,7 +83,7 @@ public void testCheckpointTriggeringPre1_17() { snapshotType, Instant.MIN)); - deployment.getSpec().getFlinkConfiguration().put(PERIODIC_CHECKPOINT_INTERVAL.key(), "10m"); + addConfigProperty(deployment.getSpec(), PERIODIC_CHECKPOINT_INTERVAL.key(), "10m"); reconcileSpec(deployment); assertEquals( @@ -128,10 +129,8 @@ private void testSnapshotTriggering( snapshotType, Instant.MIN)); - deployment - .getSpec() - .getFlinkConfiguration() - .put(periodicSnapshotIntervalOption.key(), "10m"); + addConfigProperty(deployment.getSpec(), periodicSnapshotIntervalOption.key(), "10m"); + reconcileSpec(deployment); assertEquals( @@ -142,7 +141,7 @@ private void testSnapshotTriggering( snapshotType, Instant.MIN)); resetTrigger(deployment, snapshotType); - deployment.getSpec().getFlinkConfiguration().put(periodicSnapshotIntervalOption.key(), "0"); + addConfigProperty(deployment.getSpec(), periodicSnapshotIntervalOption.key(), "0"); reconcileSpec(deployment); setTriggerNonce(deployment, snapshotType, 123L); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/validation/DefaultValidatorTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/validation/DefaultValidatorTest.java index 08388b79ea..d17ec2b9be 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/validation/DefaultValidatorTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/validation/DefaultValidatorTest.java @@ -46,6 +46,7 @@ import org.apache.flink.kubernetes.operator.api.status.JobStatus; import org.apache.flink.kubernetes.operator.api.status.Savepoint; import org.apache.flink.kubernetes.operator.api.status.SnapshotTriggerType; +import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.apache.flink.kubernetes.operator.config.FlinkConfigManager; import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; @@ -70,6 +71,9 @@ import static org.apache.flink.configuration.TaskManagerOptions.MANAGED_MEMORY_SIZE; import static org.apache.flink.configuration.TaskManagerOptions.TASK_HEAP_MEMORY; import static org.apache.flink.configuration.TaskManagerOptions.TOTAL_FLINK_MEMORY; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.removeConfigProperties; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.toJsonNode; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; @@ -113,9 +117,8 @@ public void testValidationWithoutDefaultConfig() { testError( dep -> { - dep.getSpec() - .getFlinkConfiguration() - .remove(CheckpointingOptions.SAVEPOINT_DIRECTORY.key()); + removeConfigProperties( + dep.getSpec(), CheckpointingOptions.SAVEPOINT_DIRECTORY.key()); dep.getSpec().getJob().setUpgradeMode(UpgradeMode.SAVEPOINT); }, String.format( @@ -124,33 +127,31 @@ public void testValidationWithoutDefaultConfig() { testError( dep -> { - dep.getSpec() - .getFlinkConfiguration() - .remove(CheckpointingOptions.CHECKPOINTS_DIRECTORY.key()); + removeConfigProperties( + dep.getSpec(), CheckpointingOptions.CHECKPOINTS_DIRECTORY.key()); dep.getSpec().getJob().setUpgradeMode(UpgradeMode.SAVEPOINT); }, "Checkpoint directory"); testError( dep -> { - dep.getSpec() - .getFlinkConfiguration() - .remove(CheckpointingOptions.CHECKPOINTS_DIRECTORY.key()); + removeConfigProperties( + dep.getSpec(), CheckpointingOptions.CHECKPOINTS_DIRECTORY.key()); + dep.getSpec().getJob().setUpgradeMode(UpgradeMode.LAST_STATE); }, "Checkpoint directory"); testSuccess( dep -> { - dep.getSpec() - .getFlinkConfiguration() - .remove(CheckpointingOptions.CHECKPOINTS_DIRECTORY.key()); + removeConfigProperties( + dep.getSpec(), CheckpointingOptions.CHECKPOINTS_DIRECTORY.key()); dep.getSpec().getJob().setUpgradeMode(UpgradeMode.STATELESS); }); testError( dep -> { - dep.getSpec().setFlinkConfiguration(new HashMap<>()); + dep.getSpec().setFlinkConfiguration(toJsonNode(new HashMap<>())); dep.getSpec() .getJob() .setSavepointTriggerNonce(ThreadLocalRandom.current().nextLong()); @@ -163,11 +164,12 @@ public void testValidationWithoutDefaultConfig() { dep -> dep.getSpec() .setFlinkConfiguration( - Map.of( - KubernetesOperatorConfigOptions - .PERIODIC_SAVEPOINT_INTERVAL - .key(), - "1m")), + toJsonNode( + Map.of( + KubernetesOperatorConfigOptions + .PERIODIC_SAVEPOINT_INTERVAL + .key(), + "1m"))), String.format( "Periodic savepoints cannot be enabled when config key[%s] is not set", CheckpointingOptions.SAVEPOINT_DIRECTORY.key())); @@ -176,11 +178,12 @@ public void testValidationWithoutDefaultConfig() { dep -> dep.getSpec() .setFlinkConfiguration( - Map.of( - KubernetesOperatorConfigOptions - .OPERATOR_JOB_UPGRADE_LAST_STATE_CHECKPOINT_MAX_AGE - .key(), - "1m")), + toJsonNode( + Map.of( + KubernetesOperatorConfigOptions + .OPERATOR_JOB_UPGRADE_LAST_STATE_CHECKPOINT_MAX_AGE + .key(), + "1m"))), String.format( "In order to use max-checkpoint age functionality config key[%s] must be set to allow triggering savepoint upgrades.", CheckpointingOptions.SAVEPOINT_DIRECTORY.key())); @@ -190,37 +193,41 @@ public void testValidationWithoutDefaultConfig() { dep -> dep.getSpec() .setFlinkConfiguration( - Collections.singletonMap("random", "config"))); + toJsonNode(Collections.singletonMap("random", "config")))); testError( dep -> dep.getSpec() .setFlinkConfiguration( - Collections.singletonMap( - KubernetesConfigOptions.NAMESPACE.key(), "myns")), + toJsonNode( + Collections.singletonMap( + KubernetesConfigOptions.NAMESPACE.key(), + "myns"))), "Forbidden Flink config key"); testError( dep -> dep.getSpec() .setFlinkConfiguration( - Collections.singletonMap( - HighAvailabilityOptions.HA_CLUSTER_ID.key(), - "my-cluster-id")), + toJsonNode( + Collections.singletonMap( + HighAvailabilityOptions.HA_CLUSTER_ID.key(), + "my-cluster-id"))), "Forbidden Flink config key"); testError( dep -> dep.getSpec() .setFlinkConfiguration( - Map.of( - KubernetesOperatorConfigOptions - .OPERATOR_CLUSTER_HEALTH_CHECK_ENABLED - .key(), - "true", - KubernetesOperatorConfigOptions - .OPERATOR_JM_DEPLOYMENT_RECOVERY_ENABLED - .key(), - "false")), + toJsonNode( + Map.of( + KubernetesOperatorConfigOptions + .OPERATOR_CLUSTER_HEALTH_CHECK_ENABLED + .key(), + "true", + KubernetesOperatorConfigOptions + .OPERATOR_JM_DEPLOYMENT_RECOVERY_ENABLED + .key(), + "false"))), "Deployment recovery (" + KubernetesOperatorConfigOptions.OPERATOR_JM_DEPLOYMENT_RECOVERY_ENABLED .key() @@ -259,7 +266,7 @@ public void testValidationWithoutDefaultConfig() { testError( dep -> { - dep.getSpec().setFlinkConfiguration(new HashMap<>()); + dep.getSpec().setFlinkConfiguration(toJsonNode(new HashMap<>())); dep.getSpec().getJobManager().setReplicas(2); }, "High availability should be enabled when starting standby JobManagers."); @@ -268,11 +275,12 @@ public void testValidationWithoutDefaultConfig() { dep -> dep.getSpec() .setFlinkConfiguration( - Map.of( - KubernetesOperatorConfigOptions - .DEPLOYMENT_ROLLBACK_ENABLED - .key(), - "true")), + toJsonNode( + Map.of( + KubernetesOperatorConfigOptions + .DEPLOYMENT_ROLLBACK_ENABLED + .key(), + "true"))), "HA must be enabled for rollback support."); testError( @@ -302,30 +310,37 @@ public void testValidationWithoutDefaultConfig() { testError( dep -> { dep.getSpec().getTaskManager().getResource().setMemory(null); - dep.getSpec().setFlinkConfiguration(Map.of(TASK_HEAP_MEMORY.key(), "1024m")); + dep.getSpec() + .setFlinkConfiguration( + toJsonNode(Map.of(TASK_HEAP_MEMORY.key(), "1024m"))); }, "TaskManager resource memory must be defined using `spec.taskManager.resource.memory`"); testSuccess( dep -> { dep.getSpec().getJobManager().getResource().setMemory(null); - dep.getSpec().setFlinkConfiguration(Map.of(JVM_HEAP_MEMORY.key(), "2048m")); + dep.getSpec() + .setFlinkConfiguration( + toJsonNode(Map.of(JVM_HEAP_MEMORY.key(), "2048m"))); }); testSuccess( dep -> { dep.getSpec().getTaskManager().getResource().setMemory(null); - dep.getSpec().setFlinkConfiguration(Map.of(TOTAL_FLINK_MEMORY.key(), "2048m")); + dep.getSpec() + .setFlinkConfiguration( + toJsonNode(Map.of(TOTAL_FLINK_MEMORY.key(), "2048m"))); }); testSuccess( dep -> { dep.getSpec().getTaskManager().getResource().setMemory(null); dep.getSpec() .setFlinkConfiguration( - Map.of( - TASK_HEAP_MEMORY.key(), - "1024m", - MANAGED_MEMORY_SIZE.key(), - "1024m")); + toJsonNode( + Map.of( + TASK_HEAP_MEMORY.key(), + "1024m", + MANAGED_MEMORY_SIZE.key(), + "1024m"))); }); // Test savepoint restore validation @@ -347,11 +362,10 @@ public void testValidationWithoutDefaultConfig() { .getReconciliationStatus() .serializeAndSetLastReconciledSpec(spec, dep); - dep.getSpec() - .getFlinkConfiguration() - .put( - CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), - "file:///flink-data/savepoints"); + SpecUtils.addConfigProperty( + dep.getSpec(), + CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), + "file:///flink-data/savepoints"); dep.getSpec().getJob().setUpgradeMode(UpgradeMode.SAVEPOINT); }); @@ -473,13 +487,12 @@ public void testValidationWithoutDefaultConfig() { testSuccess( dep -> { dep.getSpec().getJob().setUpgradeMode(UpgradeMode.LAST_STATE); - dep.getSpec() - .getFlinkConfiguration() - .put( - HighAvailabilityOptions.HA_MODE.key(), - // Hardcoded config value should be removed when upgrading Flink - // dependency to 1.16 - "kubernetes"); + addConfigProperty( + dep.getSpec(), + HighAvailabilityOptions.HA_MODE.key(), + // Hardcoded config value should be removed when upgrading Flink + // dependency to 1.16 + "kubernetes"); }); testError( @@ -506,7 +519,7 @@ public void testValidationWithDefaultConfig() { new DefaultValidator(new FlinkConfigManager(defaultFlinkConf)); testSuccess( dep -> { - dep.getSpec().setFlinkConfiguration(new HashMap<>()); + dep.getSpec().setFlinkConfiguration(toJsonNode(new HashMap<>())); dep.getSpec().getJob().setUpgradeMode(UpgradeMode.LAST_STATE); }, validatorWithDefaultConfig); @@ -693,11 +706,12 @@ public void testSessionJobWithSession() { sessionJob .getSpec() .setFlinkConfiguration( - Map.of( - KubernetesOperatorConfigOptions - .JAR_ARTIFACT_HTTP_HEADER - .key(), - "headerKey1:headerValue1,headerKey2:headerValue2")), + toJsonNode( + Map.of( + KubernetesOperatorConfigOptions + .JAR_ARTIFACT_HTTP_HEADER + .key(), + "headerKey1:headerValue1,headerKey2:headerValue2"))), flinkDeployment -> {}, null); @@ -706,11 +720,12 @@ public void testSessionJobWithSession() { sessionJob .getSpec() .setFlinkConfiguration( - Map.of( - KubernetesOperatorConfigOptions - .PERIODIC_SAVEPOINT_INTERVAL - .key(), - "1m")), + toJsonNode( + Map.of( + KubernetesOperatorConfigOptions + .PERIODIC_SAVEPOINT_INTERVAL + .key(), + "1m"))), flinkDeployment -> {}, null); @@ -731,14 +746,16 @@ public void testSessionJobWithSession() { sessionJob .getSpec() .setFlinkConfiguration( - Map.of( - CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), - "test-savepoint-dir", - CheckpointingOptions.CHECKPOINTS_DIRECTORY.key(), - "test-checkpoint-dir")); + toJsonNode( + Map.of( + CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), + "test-savepoint-dir", + CheckpointingOptions.CHECKPOINTS_DIRECTORY + .key(), + "test-checkpoint-dir"))); }, flinkDeployment -> { - flinkDeployment.getSpec().setFlinkConfiguration(Map.of()); + flinkDeployment.getSpec().setFlinkConfiguration(toJsonNode(Map.of())); }, null); } @@ -1091,7 +1108,7 @@ private Optional testSessionJobAutoScalerConfiguration( var sessionJob = TestUtils.buildSessionJob(); var flinkConfiguration = getDefaultTestAutoScalerFlinkConfigurationMap(); flinkConfigurationModifier.accept(flinkConfiguration); - sessionCluster.getSpec().setFlinkConfiguration(flinkConfiguration); + sessionCluster.getSpec().setFlinkConfiguration(toJsonNode(flinkConfiguration)); return validator.validateSessionJob(sessionJob, Optional.of(sessionCluster)); } @@ -1100,7 +1117,7 @@ public Optional testAutoScalerConfiguration( FlinkDeployment deployment = TestUtils.buildApplicationCluster(); var flinkConfiguration = getDefaultTestAutoScalerFlinkConfigurationMap(); flinkConfigurationModifier.accept(flinkConfiguration); - deployment.getSpec().setFlinkConfiguration(flinkConfiguration); + deployment.getSpec().setFlinkConfiguration(toJsonNode(flinkConfiguration)); return validator.validateDeployment(deployment); } From ddf83c0729a3d2da57a6a5444d0b50bae88871fe Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Attila=20M=C3=A9sz=C3=A1ros?= Date: Mon, 18 Aug 2025 14:26:38 +0200 Subject: [PATCH 03/19] wip MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Attila Mészáros --- .../flink/kubernetes/operator/api/utils/SpecUtils.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java index 9f863795a9..69e9539efb 100644 --- a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java @@ -188,12 +188,6 @@ public static JsonNode addProperties(JsonNode node, Map properti return mapToJsonNode(map); } - public static JsonNode addProperty(JsonNode node, String key, String value) { - var map = toStringMap(node); - map.put(key, value); - return mapToJsonNode(map); - } - public static JsonNode removeProperty(JsonNode node, String key) { var map = toStringMap(node); map.remove(key); From 03614b3cf5cd751cc2579662b8f4677967943669 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Attila=20M=C3=A9sz=C3=A1ros?= Date: Mon, 18 Aug 2025 16:10:06 +0200 Subject: [PATCH 04/19] fix MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Attila Mészáros --- .../deployment/ApplicationReconcilerUpgradeModeTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java index 858d19ef61..c091189527 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java @@ -72,6 +72,7 @@ import static org.apache.flink.api.common.JobStatus.RUNNING; import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.removeConfigProperties; +import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.toJsonNode; import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.toStringMap; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.SNAPSHOT_RESOURCE_ENABLED; import static org.assertj.core.api.Assertions.assertThat; @@ -938,7 +939,7 @@ public static FlinkDeployment buildApplicationCluster( default: throw new RuntimeException("Unsupported upgrade mode " + upgradeMode); } - + deployment.getSpec().setFlinkConfiguration(toJsonNode(conf)); return deployment; } From dfbbb33407b625f6e8591fcfe1c9700a3e7d8b7b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Attila=20M=C3=A9sz=C3=A1ros?= Date: Mon, 18 Aug 2025 16:11:36 +0200 Subject: [PATCH 05/19] wip MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Attila Mészáros --- .gitignore | 1 - 1 file changed, 1 deletion(-) diff --git a/.gitignore b/.gitignore index 5b3bf5c799..0fd15a8f2d 100644 --- a/.gitignore +++ b/.gitignore @@ -38,7 +38,6 @@ buildNumber.properties *.DS_Store .kube -.aider* # VSCode settings .vscode/ \ No newline at end of file From 38eda4c7bbd6865debd155ec975c0fdba8cf6ccc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Attila=20M=C3=A9sz=C3=A1ros?= Date: Mon, 18 Aug 2025 16:29:50 +0200 Subject: [PATCH 06/19] wip MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Attila Mészáros --- .../flink/kubernetes/operator/api/utils/SpecUtils.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java index 69e9539efb..f5073dffd8 100644 --- a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java @@ -26,11 +26,13 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.NullNode; import com.fasterxml.jackson.databind.node.ObjectNode; import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; import javax.annotation.Nullable; +import java.util.Collections; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.Map; @@ -140,6 +142,12 @@ public static JsonNode toJsonNode(Map properties) { } public static Map toStringMap(JsonNode node) { + if (node == null) { + return Collections.emptyMap(); + } + if (node instanceof NullNode) { + return Collections.emptyMap(); + } Map flatMap = new LinkedHashMap<>(); flattenHelper(node, "", flatMap); return flatMap; From a9ee837048f26e23f4c27fde549a702018e964c3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Attila=20M=C3=A9sz=C3=A1ros?= Date: Tue, 19 Aug 2025 10:08:41 +0200 Subject: [PATCH 07/19] unit tests MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Attila Mészáros --- flink-kubernetes-operator-api/pom.xml | 7 ++ .../operator/api/utils/SpecUtils.java | 10 ++- .../operator/api/utils/SpecUtilsTest.java | 65 ++++++++++++++++++- 3 files changed, 73 insertions(+), 9 deletions(-) diff --git a/flink-kubernetes-operator-api/pom.xml b/flink-kubernetes-operator-api/pom.xml index d9cfa8570a..ba4aa4618c 100644 --- a/flink-kubernetes-operator-api/pom.xml +++ b/flink-kubernetes-operator-api/pom.xml @@ -145,6 +145,13 @@ under the License. junit-jupiter-params test + + + org.assertj + assertj-core + ${assertj.version} + test + diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java index f5073dffd8..5455b7045a 100644 --- a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java @@ -32,9 +32,8 @@ import javax.annotation.Nullable; -import java.util.Collections; +import java.util.HashMap; import java.util.Iterator; -import java.util.LinkedHashMap; import java.util.Map; import java.util.Set; @@ -132,7 +131,6 @@ public static T clone(T object) { } } - // todo unit test public static JsonNode toJsonNode(Map properties) { ObjectNode jsonNode = yamlObjectMapper.createObjectNode(); for (Map.Entry entry : properties.entrySet()) { @@ -143,12 +141,12 @@ public static JsonNode toJsonNode(Map properties) { public static Map toStringMap(JsonNode node) { if (node == null) { - return Collections.emptyMap(); + return new HashMap<>(); } if (node instanceof NullNode) { - return Collections.emptyMap(); + return new HashMap<>(); } - Map flatMap = new LinkedHashMap<>(); + Map flatMap = new HashMap<>(); flattenHelper(node, "", flatMap); return flatMap; } diff --git a/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtilsTest.java b/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtilsTest.java index dfb48e4917..05f11af685 100644 --- a/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtilsTest.java +++ b/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtilsTest.java @@ -24,16 +24,22 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; +import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; import org.junit.jupiter.api.Test; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNull; /** Test for {@link SpecUtils}. */ -public class SpecUtilsTest { +class SpecUtilsTest { + + private static final ObjectMapper yamlObjectMapper = new ObjectMapper(new YAMLFactory()); @Test - public void testSpecSerializationWithVersion() throws JsonProcessingException { + void testSpecSerializationWithVersion() throws JsonProcessingException { FlinkDeployment app = BaseTestUtils.buildApplicationCluster(); String serialized = SpecUtils.writeSpecWithMeta(app.getSpec(), app); ObjectNode node = (ObjectNode) new ObjectMapper().readTree(serialized); @@ -56,7 +62,7 @@ public void testSpecSerializationWithVersion() throws JsonProcessingException { } @Test - public void testSpecSerializationWithoutGeneration() throws JsonProcessingException { + void testSpecSerializationWithoutGeneration() throws JsonProcessingException { // with regards to ReconcialiationMetadata & SpecWithMeta FlinkDeployment app = BaseTestUtils.buildApplicationCluster(); app.getMetadata().setGeneration(12L); @@ -76,4 +82,57 @@ public void testSpecSerializationWithoutGeneration() throws JsonProcessingExcept var migrated = SpecUtils.deserializeSpecWithMeta(oldSerialized, FlinkDeploymentSpec.class); assertNull(migrated.getMeta()); } + + @Test + void convertsStringMapToJsonNode() { + var map = Map.of("k1", "v1", "k2", "v2", "k3.nested", "v3"); + var node = SpecUtils.mapToJsonNode(map); + + assertThat(node).hasSize(3); + assertThat(node.get("k1").asText()).isEqualTo("v1"); + assertThat(node.get("k2").asText()).isEqualTo("v2"); + assertThat(node.get("k3.nested").asText()).isEqualTo("v3"); + } + + @Test + void convertsJsonNodeToMap() throws JsonProcessingException { + var node = + yamlObjectMapper.readTree("k1: v1 \n" + "k2: v2 \n" + "k3:\n" + " nested: v3\n"); + + var map = SpecUtils.toStringMap(node); + assertThat(map).hasSize(3); + assertThat(map.get("k1")).isEqualTo("v1"); + assertThat(map.get("k2")).isEqualTo("v2"); + assertThat(map.get("k3.nested")).isEqualTo("v3"); + } + + @Test + void addConfigPropertyToSpec() { + var spec = new FlinkDeploymentSpec(); + + SpecUtils.addConfigProperty(spec, "k1", "v1"); + + assertThat(spec.getFlinkConfiguration().get("k1").asText()).isEqualTo("v1"); + } + + @Test + void addConfigPropertiesToSpec() { + var spec = new FlinkDeploymentSpec(); + + SpecUtils.addConfigProperties(spec, Map.of("k1", "v1", "k2", "v2")); + + assertThat(spec.getFlinkConfiguration().get("k1").asText()).isEqualTo("v1"); + assertThat(spec.getFlinkConfiguration().get("k2").asText()).isEqualTo("v2"); + } + + @Test + void removeConfigPropertiesFromSpec() { + var spec = new FlinkDeploymentSpec(); + SpecUtils.addConfigProperties(spec, Map.of("k1", "v1", "k2", "v2")); + + SpecUtils.removeConfigProperties(spec, "k1"); + + assertThat(spec.getFlinkConfiguration().get("k1")).isNull(); + assertThat(spec.getFlinkConfiguration().get("k2").asText()).isEqualTo("v2"); + } } From 6a89134f84350aeee2e0c55df94f093fa95fcf74 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Attila=20M=C3=A9sz=C3=A1ros?= Date: Tue, 19 Aug 2025 11:34:20 +0200 Subject: [PATCH 08/19] CRD update in helm chart, add migration test MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Attila Mészáros --- flink-kubernetes-operator-api/pom.xml | 20 + .../FlinkConfigurationYamlSupportTest.java | 95 + .../pre-flink-configuration-yaml-crd.yml | 10899 ++++++++++++++++ ...st-deployment-key-value-configuration.yaml | 44 + .../test-deployment-yaml-configuration.yaml | 49 + .../flinkdeployments.flink.apache.org-v1.yml | 4 +- .../flinksessionjobs.flink.apache.org-v1.yml | 4 +- 7 files changed, 11109 insertions(+), 6 deletions(-) create mode 100644 flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/FlinkConfigurationYamlSupportTest.java create mode 100644 flink-kubernetes-operator-api/src/test/resources/pre-flink-configuration-yaml-crd.yml create mode 100644 flink-kubernetes-operator-api/src/test/resources/test-deployment-key-value-configuration.yaml create mode 100644 flink-kubernetes-operator-api/src/test/resources/test-deployment-yaml-configuration.yaml diff --git a/flink-kubernetes-operator-api/pom.xml b/flink-kubernetes-operator-api/pom.xml index ba4aa4618c..2e83d291d2 100644 --- a/flink-kubernetes-operator-api/pom.xml +++ b/flink-kubernetes-operator-api/pom.xml @@ -152,6 +152,26 @@ under the License. ${assertj.version} test + + + io.fabric8 + kube-api-test-client-inject + ${fabric8.version} + test + + + + io.fabric8 + kubernetes-httpclient-${fabric8.httpclient.impl} + ${fabric8.version} + + + com.squareup.okhttp3 + okhttp + + + test + diff --git a/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/FlinkConfigurationYamlSupportTest.java b/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/FlinkConfigurationYamlSupportTest.java new file mode 100644 index 0000000000..6f68b6debd --- /dev/null +++ b/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/FlinkConfigurationYamlSupportTest.java @@ -0,0 +1,95 @@ +/* + * 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.api; + +import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; +import io.fabric8.kubeapitest.junit.EnableKubeAPIServer; +import io.fabric8.kubernetes.api.model.GenericKubernetesResource; +import io.fabric8.kubernetes.api.model.apiextensions.v1.CustomResourceDefinition; +import io.fabric8.kubernetes.client.KubernetesClient; +import io.fabric8.kubernetes.client.dsl.NonDeletingOperation; +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.io.IOException; + +import static org.assertj.core.api.Assertions.assertThat; + +@EnableKubeAPIServer +class FlinkConfigurationYamlSupportTest { + + private static final ObjectMapper objectMapper = new ObjectMapper(new YAMLFactory()); + public static final String OLD_CRD_PATH = + "src/test/resources/pre-flink-configuration-yaml-crd.yml"; + public static final String NEW_CRD_PATH = + "../helm/flink-kubernetes-operator/crds/flinkdeployments.flink.apache.org-v1.yml"; + + static KubernetesClient client; + + @Test + void upgradeCRDToYamlFlinkConfiguration() { + applyCRD(OLD_CRD_PATH); + + applyResource("src/test/resources/test-deployment-key-value-configuration.yaml"); + + var deployment = + client.resources(FlinkDeployment.class) + .inNamespace("default") + .withName("basic-example") + .get(); + assertThat(deployment.getSpec().getFlinkConfiguration()).hasSize(5); + + applyCRD(NEW_CRD_PATH); + + applyResource("src/test/resources/test-deployment-yaml-configuration.yaml"); + + deployment = + client.resources(FlinkDeployment.class) + .inNamespace("default") + .withName("basic-example") + .get(); + assertThat(deployment.getSpec().getFlinkConfiguration()).hasSize(3); + assertThat(SpecUtils.toStringMap(deployment.getSpec().getFlinkConfiguration())).hasSize(5); + } + + private GenericKubernetesResource applyResource(String path) { + try { + GenericKubernetesResource d = + objectMapper.readValue(new File(path), GenericKubernetesResource.class); + + return client.resource(d).createOr(NonDeletingOperation::update); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private CustomResourceDefinition applyCRD(String path) { + try { + var crd = objectMapper.readValue(new File(path), CustomResourceDefinition.class); + + var res = client.resource(crd).createOr(NonDeletingOperation::update); + Thread.sleep(1000); + return res; + } catch (IOException | InterruptedException e) { + throw new RuntimeException(e); + } + } +} diff --git a/flink-kubernetes-operator-api/src/test/resources/pre-flink-configuration-yaml-crd.yml b/flink-kubernetes-operator-api/src/test/resources/pre-flink-configuration-yaml-crd.yml new file mode 100644 index 0000000000..1fc40d1820 --- /dev/null +++ b/flink-kubernetes-operator-api/src/test/resources/pre-flink-configuration-yaml-crd.yml @@ -0,0 +1,10899 @@ +################################################################################ +# 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. +################################################################################ + +apiVersion: apiextensions.k8s.io/v1 +kind: CustomResourceDefinition +metadata: + name: flinkdeployments.flink.apache.org +spec: + group: flink.apache.org + names: + kind: FlinkDeployment + plural: flinkdeployments + shortNames: + - flinkdep + singular: flinkdeployment + scope: Namespaced + versions: + - additionalPrinterColumns: + - description: Last observed state of the job. + jsonPath: .status.jobStatus.state + name: Job Status + priority: 0 + type: string + - description: "Lifecycle state of the Flink resource (including being rolled\ + \ back, failed etc.)." + jsonPath: .status.lifecycleState + name: Lifecycle State + priority: 0 + type: string + name: v1beta1 + schema: + openAPIV3Schema: + properties: + spec: + properties: + flinkConfiguration: + additionalProperties: + type: string + type: object + flinkVersion: + enum: + - v1_13 + - v1_14 + - v1_15 + - v1_16 + - v1_17 + - v1_18 + - v1_19 + - v1_20 + - v2_0 + type: string + image: + type: string + imagePullPolicy: + type: string + ingress: + properties: + annotations: + additionalProperties: + type: string + type: object + className: + type: string + labels: + additionalProperties: + type: string + type: object + template: + type: string + tls: + items: + properties: + hosts: + items: + type: string + type: array + secretName: + type: string + type: object + type: array + type: object + job: + properties: + allowNonRestoredState: + type: boolean + args: + items: + type: string + type: array + autoscalerResetNonce: + type: integer + checkpointTriggerNonce: + type: integer + entryClass: + type: string + initialSavepointPath: + type: string + jarURI: + type: string + parallelism: + type: integer + savepointRedeployNonce: + type: integer + savepointTriggerNonce: + type: integer + state: + enum: + - running + - suspended + type: string + upgradeMode: + enum: + - last-state + - savepoint + - stateless + type: string + type: object + jobManager: + properties: + podTemplate: + properties: + apiVersion: + type: string + kind: + type: string + metadata: + properties: + annotations: + additionalProperties: + type: string + type: object + creationTimestamp: + type: string + deletionGracePeriodSeconds: + type: integer + deletionTimestamp: + type: string + finalizers: + items: + type: string + type: array + generateName: + type: string + generation: + type: integer + labels: + additionalProperties: + type: string + type: object + managedFields: + items: + properties: + apiVersion: + type: string + fieldsType: + type: string + fieldsV1: + type: object + manager: + type: string + operation: + type: string + subresource: + type: string + time: + type: string + type: object + type: array + name: + type: string + namespace: + type: string + ownerReferences: + items: + properties: + apiVersion: + type: string + blockOwnerDeletion: + type: boolean + controller: + type: boolean + kind: + type: string + name: + type: string + uid: + type: string + type: object + type: array + resourceVersion: + type: string + selfLink: + type: string + uid: + type: string + type: object + spec: + properties: + activeDeadlineSeconds: + type: integer + affinity: + properties: + nodeAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + preference: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + properties: + nodeSelectorTerms: + items: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + type: array + type: object + type: object + podAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + podAntiAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + type: object + automountServiceAccountToken: + type: boolean + containers: + items: + properties: + args: + items: + type: string + type: array + command: + items: + type: string + type: array + env: + items: + properties: + name: + type: string + value: + type: string + valueFrom: + properties: + configMapKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + secretKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + envFrom: + items: + properties: + configMapRef: + properties: + name: + type: string + optional: + type: boolean + type: object + prefix: + type: string + secretRef: + properties: + name: + type: string + optional: + type: boolean + type: object + type: object + type: array + image: + type: string + imagePullPolicy: + type: string + lifecycle: + properties: + postStart: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + preStop: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + stopSignal: + type: string + type: object + livenessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + name: + type: string + ports: + items: + properties: + containerPort: + type: integer + hostIP: + type: string + hostPort: + type: integer + name: + type: string + protocol: + type: string + type: object + type: array + readinessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + resizePolicy: + items: + properties: + resourceName: + type: string + restartPolicy: + type: string + type: object + type: array + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartPolicy: + type: string + securityContext: + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + startupProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + stdin: + type: boolean + stdinOnce: + type: boolean + terminationMessagePath: + type: string + terminationMessagePolicy: + type: string + tty: + type: boolean + volumeDevices: + items: + properties: + devicePath: + type: string + name: + type: string + type: object + type: array + volumeMounts: + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + workingDir: + type: string + type: object + type: array + dnsConfig: + properties: + nameservers: + items: + type: string + type: array + options: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + searches: + items: + type: string + type: array + type: object + dnsPolicy: + type: string + enableServiceLinks: + type: boolean + ephemeralContainers: + items: + properties: + args: + items: + type: string + type: array + command: + items: + type: string + type: array + env: + items: + properties: + name: + type: string + value: + type: string + valueFrom: + properties: + configMapKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + secretKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + envFrom: + items: + properties: + configMapRef: + properties: + name: + type: string + optional: + type: boolean + type: object + prefix: + type: string + secretRef: + properties: + name: + type: string + optional: + type: boolean + type: object + type: object + type: array + image: + type: string + imagePullPolicy: + type: string + lifecycle: + properties: + postStart: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + preStop: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + stopSignal: + type: string + type: object + livenessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + name: + type: string + ports: + items: + properties: + containerPort: + type: integer + hostIP: + type: string + hostPort: + type: integer + name: + type: string + protocol: + type: string + type: object + type: array + readinessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + resizePolicy: + items: + properties: + resourceName: + type: string + restartPolicy: + type: string + type: object + type: array + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartPolicy: + type: string + securityContext: + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + startupProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + stdin: + type: boolean + stdinOnce: + type: boolean + targetContainerName: + type: string + terminationMessagePath: + type: string + terminationMessagePolicy: + type: string + tty: + type: boolean + volumeDevices: + items: + properties: + devicePath: + type: string + name: + type: string + type: object + type: array + volumeMounts: + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + workingDir: + type: string + type: object + type: array + hostAliases: + items: + properties: + hostnames: + items: + type: string + type: array + ip: + type: string + type: object + type: array + hostIPC: + type: boolean + hostNetwork: + type: boolean + hostPID: + type: boolean + hostUsers: + type: boolean + hostname: + type: string + imagePullSecrets: + items: + properties: + name: + type: string + type: object + type: array + initContainers: + items: + properties: + args: + items: + type: string + type: array + command: + items: + type: string + type: array + env: + items: + properties: + name: + type: string + value: + type: string + valueFrom: + properties: + configMapKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + secretKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + envFrom: + items: + properties: + configMapRef: + properties: + name: + type: string + optional: + type: boolean + type: object + prefix: + type: string + secretRef: + properties: + name: + type: string + optional: + type: boolean + type: object + type: object + type: array + image: + type: string + imagePullPolicy: + type: string + lifecycle: + properties: + postStart: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + preStop: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + stopSignal: + type: string + type: object + livenessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + name: + type: string + ports: + items: + properties: + containerPort: + type: integer + hostIP: + type: string + hostPort: + type: integer + name: + type: string + protocol: + type: string + type: object + type: array + readinessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + resizePolicy: + items: + properties: + resourceName: + type: string + restartPolicy: + type: string + type: object + type: array + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartPolicy: + type: string + securityContext: + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + startupProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + stdin: + type: boolean + stdinOnce: + type: boolean + terminationMessagePath: + type: string + terminationMessagePolicy: + type: string + tty: + type: boolean + volumeDevices: + items: + properties: + devicePath: + type: string + name: + type: string + type: object + type: array + volumeMounts: + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + workingDir: + type: string + type: object + type: array + nodeName: + type: string + nodeSelector: + additionalProperties: + type: string + type: object + os: + properties: + name: + type: string + type: object + overhead: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + preemptionPolicy: + type: string + priority: + type: integer + priorityClassName: + type: string + readinessGates: + items: + properties: + conditionType: + type: string + type: object + type: array + resourceClaims: + items: + properties: + name: + type: string + resourceClaimName: + type: string + resourceClaimTemplateName: + type: string + type: object + type: array + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartPolicy: + type: string + runtimeClassName: + type: string + schedulerName: + type: string + schedulingGates: + items: + properties: + name: + type: string + type: object + type: array + securityContext: + properties: + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + fsGroup: + type: integer + fsGroupChangePolicy: + type: string + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxChangePolicy: + type: string + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + supplementalGroups: + items: + type: integer + type: array + supplementalGroupsPolicy: + type: string + sysctls: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + serviceAccount: + type: string + serviceAccountName: + type: string + setHostnameAsFQDN: + type: boolean + shareProcessNamespace: + type: boolean + subdomain: + type: string + terminationGracePeriodSeconds: + type: integer + tolerations: + items: + properties: + effect: + type: string + key: + type: string + operator: + type: string + tolerationSeconds: + type: integer + value: + type: string + type: object + type: array + topologySpreadConstraints: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + maxSkew: + type: integer + minDomains: + type: integer + nodeAffinityPolicy: + type: string + nodeTaintsPolicy: + type: string + topologyKey: + type: string + whenUnsatisfiable: + type: string + type: object + type: array + volumes: + items: + properties: + awsElasticBlockStore: + properties: + fsType: + type: string + partition: + type: integer + readOnly: + type: boolean + volumeID: + type: string + type: object + azureDisk: + properties: + cachingMode: + type: string + diskName: + type: string + diskURI: + type: string + fsType: + type: string + kind: + type: string + readOnly: + type: boolean + type: object + azureFile: + properties: + readOnly: + type: boolean + secretName: + type: string + shareName: + type: string + type: object + cephfs: + properties: + monitors: + items: + type: string + type: array + path: + type: string + readOnly: + type: boolean + secretFile: + type: string + secretRef: + properties: + name: + type: string + type: object + user: + type: string + type: object + cinder: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + volumeID: + type: string + type: object + configMap: + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + csi: + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: + properties: + name: + type: string + type: object + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string + type: object + type: object + downwardAPI: + properties: + defaultMode: + type: integer + items: + items: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + mode: + type: integer + path: + type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + type: object + type: array + type: object + emptyDir: + properties: + medium: + type: string + sizeLimit: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + ephemeral: + properties: + volumeClaimTemplate: + properties: + metadata: + properties: + annotations: + additionalProperties: + type: string + type: object + creationTimestamp: + type: string + deletionGracePeriodSeconds: + type: integer + deletionTimestamp: + type: string + finalizers: + items: + type: string + type: array + generateName: + type: string + generation: + type: integer + labels: + additionalProperties: + type: string + type: object + managedFields: + items: + properties: + apiVersion: + type: string + fieldsType: + type: string + fieldsV1: + type: object + manager: + type: string + operation: + type: string + subresource: + type: string + time: + type: string + type: object + type: array + name: + type: string + namespace: + type: string + ownerReferences: + items: + properties: + apiVersion: + type: string + blockOwnerDeletion: + type: boolean + controller: + type: boolean + kind: + type: string + name: + type: string + uid: + type: string + type: object + type: array + resourceVersion: + type: string + selfLink: + type: string + uid: + type: string + type: object + spec: + properties: + accessModes: + items: + type: string + type: array + dataSource: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + type: object + dataSourceRef: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + namespace: + type: string + type: object + resources: + properties: + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + selector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + storageClassName: + type: string + volumeAttributesClassName: + type: string + volumeMode: + type: string + volumeName: + type: string + type: object + type: object + type: object + fc: + properties: + fsType: + type: string + lun: + type: integer + readOnly: + type: boolean + targetWWNs: + items: + type: string + type: array + wwids: + items: + type: string + type: array + type: object + flexVolume: + properties: + driver: + type: string + fsType: + type: string + options: + additionalProperties: + type: string + type: object + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + type: object + flocker: + properties: + datasetName: + type: string + datasetUUID: + type: string + type: object + gcePersistentDisk: + properties: + fsType: + type: string + partition: + type: integer + pdName: + type: string + readOnly: + type: boolean + type: object + gitRepo: + properties: + directory: + type: string + repository: + type: string + revision: + type: string + type: object + glusterfs: + properties: + endpoints: + type: string + path: + type: string + readOnly: + type: boolean + type: object + hostPath: + properties: + path: + type: string + type: + type: string + type: object + image: + properties: + pullPolicy: + type: string + reference: + type: string + type: object + iscsi: + properties: + chapAuthDiscovery: + type: boolean + chapAuthSession: + type: boolean + fsType: + type: string + initiatorName: + type: string + iqn: + type: string + iscsiInterface: + type: string + lun: + type: integer + portals: + items: + type: string + type: array + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + targetPortal: + type: string + type: object + name: + type: string + nfs: + properties: + path: + type: string + readOnly: + type: boolean + server: + type: string + type: object + persistentVolumeClaim: + properties: + claimName: + type: string + readOnly: + type: boolean + type: object + photonPersistentDisk: + properties: + fsType: + type: string + pdID: + type: string + type: object + portworxVolume: + properties: + fsType: + type: string + readOnly: + type: boolean + volumeID: + type: string + type: object + projected: + properties: + defaultMode: + type: integer + sources: + items: + properties: + clusterTrustBundle: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + name: + type: string + optional: + type: boolean + path: + type: string + signerName: + type: string + type: object + configMap: + properties: + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + downwardAPI: + properties: + items: + items: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + mode: + type: integer + path: + type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + type: object + type: array + type: object + secret: + properties: + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + serviceAccountToken: + properties: + audience: + type: string + expirationSeconds: + type: integer + path: + type: string + type: object + type: object + type: array + type: object + quobyte: + properties: + group: + type: string + readOnly: + type: boolean + registry: + type: string + tenant: + type: string + user: + type: string + volume: + type: string + type: object + rbd: + properties: + fsType: + type: string + image: + type: string + keyring: + type: string + monitors: + items: + type: string + type: array + pool: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + user: + type: string + type: object + scaleIO: + properties: + fsType: + type: string + gateway: + type: string + protectionDomain: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + sslEnabled: + type: boolean + storageMode: + type: string + storagePool: + type: string + system: + type: string + volumeName: + type: string + type: object + secret: + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + optional: + type: boolean + secretName: + type: string + type: object + storageos: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + volumeName: + type: string + volumeNamespace: + type: string + type: object + vsphereVolume: + properties: + fsType: + type: string + storagePolicyID: + type: string + storagePolicyName: + type: string + volumePath: + type: string + type: object + type: object + type: array + type: object + status: + properties: + conditions: + items: + properties: + lastProbeTime: + type: string + lastTransitionTime: + type: string + message: + type: string + observedGeneration: + type: integer + reason: + type: string + status: + type: string + type: + type: string + type: object + type: array + containerStatuses: + items: + properties: + allocatedResources: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + allocatedResourcesStatus: + items: + properties: + name: + type: string + resources: + items: + properties: + health: + type: string + resourceID: + type: string + type: object + type: array + type: object + type: array + containerID: + type: string + image: + type: string + imageID: + type: string + lastState: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + name: + type: string + ready: + type: boolean + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartCount: + type: integer + started: + type: boolean + state: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + stopSignal: + type: string + user: + properties: + linux: + properties: + gid: + type: integer + supplementalGroups: + items: + type: integer + type: array + uid: + type: integer + type: object + type: object + volumeMounts: + items: + properties: + mountPath: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + type: object + type: array + type: object + type: array + ephemeralContainerStatuses: + items: + properties: + allocatedResources: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + allocatedResourcesStatus: + items: + properties: + name: + type: string + resources: + items: + properties: + health: + type: string + resourceID: + type: string + type: object + type: array + type: object + type: array + containerID: + type: string + image: + type: string + imageID: + type: string + lastState: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + name: + type: string + ready: + type: boolean + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartCount: + type: integer + started: + type: boolean + state: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + stopSignal: + type: string + user: + properties: + linux: + properties: + gid: + type: integer + supplementalGroups: + items: + type: integer + type: array + uid: + type: integer + type: object + type: object + volumeMounts: + items: + properties: + mountPath: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + type: object + type: array + type: object + type: array + hostIP: + type: string + hostIPs: + items: + properties: + ip: + type: string + type: object + type: array + initContainerStatuses: + items: + properties: + allocatedResources: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + allocatedResourcesStatus: + items: + properties: + name: + type: string + resources: + items: + properties: + health: + type: string + resourceID: + type: string + type: object + type: array + type: object + type: array + containerID: + type: string + image: + type: string + imageID: + type: string + lastState: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + name: + type: string + ready: + type: boolean + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartCount: + type: integer + started: + type: boolean + state: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + stopSignal: + type: string + user: + properties: + linux: + properties: + gid: + type: integer + supplementalGroups: + items: + type: integer + type: array + uid: + type: integer + type: object + type: object + volumeMounts: + items: + properties: + mountPath: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + type: object + type: array + type: object + type: array + message: + type: string + nominatedNodeName: + type: string + observedGeneration: + type: integer + phase: + type: string + podIP: + type: string + podIPs: + items: + properties: + ip: + type: string + type: object + type: array + qosClass: + type: string + reason: + type: string + resize: + type: string + resourceClaimStatuses: + items: + properties: + name: + type: string + resourceClaimName: + type: string + type: object + type: array + startTime: + type: string + type: object + type: object + replicas: + type: integer + resource: + properties: + cpu: + type: number + ephemeralStorage: + type: string + memory: + type: string + type: object + type: object + logConfiguration: + additionalProperties: + type: string + type: object + mode: + enum: + - native + - standalone + type: string + podTemplate: + properties: + apiVersion: + type: string + kind: + type: string + metadata: + properties: + annotations: + additionalProperties: + type: string + type: object + creationTimestamp: + type: string + deletionGracePeriodSeconds: + type: integer + deletionTimestamp: + type: string + finalizers: + items: + type: string + type: array + generateName: + type: string + generation: + type: integer + labels: + additionalProperties: + type: string + type: object + managedFields: + items: + properties: + apiVersion: + type: string + fieldsType: + type: string + fieldsV1: + type: object + manager: + type: string + operation: + type: string + subresource: + type: string + time: + type: string + type: object + type: array + name: + type: string + namespace: + type: string + ownerReferences: + items: + properties: + apiVersion: + type: string + blockOwnerDeletion: + type: boolean + controller: + type: boolean + kind: + type: string + name: + type: string + uid: + type: string + type: object + type: array + resourceVersion: + type: string + selfLink: + type: string + uid: + type: string + type: object + spec: + properties: + activeDeadlineSeconds: + type: integer + affinity: + properties: + nodeAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + preference: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + properties: + nodeSelectorTerms: + items: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + type: array + type: object + type: object + podAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + podAntiAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + type: object + automountServiceAccountToken: + type: boolean + containers: + items: + properties: + args: + items: + type: string + type: array + command: + items: + type: string + type: array + env: + items: + properties: + name: + type: string + value: + type: string + valueFrom: + properties: + configMapKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + secretKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + envFrom: + items: + properties: + configMapRef: + properties: + name: + type: string + optional: + type: boolean + type: object + prefix: + type: string + secretRef: + properties: + name: + type: string + optional: + type: boolean + type: object + type: object + type: array + image: + type: string + imagePullPolicy: + type: string + lifecycle: + properties: + postStart: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + preStop: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + stopSignal: + type: string + type: object + livenessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + name: + type: string + ports: + items: + properties: + containerPort: + type: integer + hostIP: + type: string + hostPort: + type: integer + name: + type: string + protocol: + type: string + type: object + type: array + readinessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + resizePolicy: + items: + properties: + resourceName: + type: string + restartPolicy: + type: string + type: object + type: array + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartPolicy: + type: string + securityContext: + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + startupProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + stdin: + type: boolean + stdinOnce: + type: boolean + terminationMessagePath: + type: string + terminationMessagePolicy: + type: string + tty: + type: boolean + volumeDevices: + items: + properties: + devicePath: + type: string + name: + type: string + type: object + type: array + volumeMounts: + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + workingDir: + type: string + type: object + type: array + dnsConfig: + properties: + nameservers: + items: + type: string + type: array + options: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + searches: + items: + type: string + type: array + type: object + dnsPolicy: + type: string + enableServiceLinks: + type: boolean + ephemeralContainers: + items: + properties: + args: + items: + type: string + type: array + command: + items: + type: string + type: array + env: + items: + properties: + name: + type: string + value: + type: string + valueFrom: + properties: + configMapKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + secretKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + envFrom: + items: + properties: + configMapRef: + properties: + name: + type: string + optional: + type: boolean + type: object + prefix: + type: string + secretRef: + properties: + name: + type: string + optional: + type: boolean + type: object + type: object + type: array + image: + type: string + imagePullPolicy: + type: string + lifecycle: + properties: + postStart: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + preStop: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + stopSignal: + type: string + type: object + livenessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + name: + type: string + ports: + items: + properties: + containerPort: + type: integer + hostIP: + type: string + hostPort: + type: integer + name: + type: string + protocol: + type: string + type: object + type: array + readinessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + resizePolicy: + items: + properties: + resourceName: + type: string + restartPolicy: + type: string + type: object + type: array + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartPolicy: + type: string + securityContext: + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + startupProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + stdin: + type: boolean + stdinOnce: + type: boolean + targetContainerName: + type: string + terminationMessagePath: + type: string + terminationMessagePolicy: + type: string + tty: + type: boolean + volumeDevices: + items: + properties: + devicePath: + type: string + name: + type: string + type: object + type: array + volumeMounts: + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + workingDir: + type: string + type: object + type: array + hostAliases: + items: + properties: + hostnames: + items: + type: string + type: array + ip: + type: string + type: object + type: array + hostIPC: + type: boolean + hostNetwork: + type: boolean + hostPID: + type: boolean + hostUsers: + type: boolean + hostname: + type: string + imagePullSecrets: + items: + properties: + name: + type: string + type: object + type: array + initContainers: + items: + properties: + args: + items: + type: string + type: array + command: + items: + type: string + type: array + env: + items: + properties: + name: + type: string + value: + type: string + valueFrom: + properties: + configMapKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + secretKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + envFrom: + items: + properties: + configMapRef: + properties: + name: + type: string + optional: + type: boolean + type: object + prefix: + type: string + secretRef: + properties: + name: + type: string + optional: + type: boolean + type: object + type: object + type: array + image: + type: string + imagePullPolicy: + type: string + lifecycle: + properties: + postStart: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + preStop: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + stopSignal: + type: string + type: object + livenessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + name: + type: string + ports: + items: + properties: + containerPort: + type: integer + hostIP: + type: string + hostPort: + type: integer + name: + type: string + protocol: + type: string + type: object + type: array + readinessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + resizePolicy: + items: + properties: + resourceName: + type: string + restartPolicy: + type: string + type: object + type: array + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartPolicy: + type: string + securityContext: + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + startupProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + stdin: + type: boolean + stdinOnce: + type: boolean + terminationMessagePath: + type: string + terminationMessagePolicy: + type: string + tty: + type: boolean + volumeDevices: + items: + properties: + devicePath: + type: string + name: + type: string + type: object + type: array + volumeMounts: + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + workingDir: + type: string + type: object + type: array + nodeName: + type: string + nodeSelector: + additionalProperties: + type: string + type: object + os: + properties: + name: + type: string + type: object + overhead: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + preemptionPolicy: + type: string + priority: + type: integer + priorityClassName: + type: string + readinessGates: + items: + properties: + conditionType: + type: string + type: object + type: array + resourceClaims: + items: + properties: + name: + type: string + resourceClaimName: + type: string + resourceClaimTemplateName: + type: string + type: object + type: array + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartPolicy: + type: string + runtimeClassName: + type: string + schedulerName: + type: string + schedulingGates: + items: + properties: + name: + type: string + type: object + type: array + securityContext: + properties: + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + fsGroup: + type: integer + fsGroupChangePolicy: + type: string + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxChangePolicy: + type: string + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + supplementalGroups: + items: + type: integer + type: array + supplementalGroupsPolicy: + type: string + sysctls: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + serviceAccount: + type: string + serviceAccountName: + type: string + setHostnameAsFQDN: + type: boolean + shareProcessNamespace: + type: boolean + subdomain: + type: string + terminationGracePeriodSeconds: + type: integer + tolerations: + items: + properties: + effect: + type: string + key: + type: string + operator: + type: string + tolerationSeconds: + type: integer + value: + type: string + type: object + type: array + topologySpreadConstraints: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + maxSkew: + type: integer + minDomains: + type: integer + nodeAffinityPolicy: + type: string + nodeTaintsPolicy: + type: string + topologyKey: + type: string + whenUnsatisfiable: + type: string + type: object + type: array + volumes: + items: + properties: + awsElasticBlockStore: + properties: + fsType: + type: string + partition: + type: integer + readOnly: + type: boolean + volumeID: + type: string + type: object + azureDisk: + properties: + cachingMode: + type: string + diskName: + type: string + diskURI: + type: string + fsType: + type: string + kind: + type: string + readOnly: + type: boolean + type: object + azureFile: + properties: + readOnly: + type: boolean + secretName: + type: string + shareName: + type: string + type: object + cephfs: + properties: + monitors: + items: + type: string + type: array + path: + type: string + readOnly: + type: boolean + secretFile: + type: string + secretRef: + properties: + name: + type: string + type: object + user: + type: string + type: object + cinder: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + volumeID: + type: string + type: object + configMap: + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + csi: + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: + properties: + name: + type: string + type: object + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string + type: object + type: object + downwardAPI: + properties: + defaultMode: + type: integer + items: + items: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + mode: + type: integer + path: + type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + type: object + type: array + type: object + emptyDir: + properties: + medium: + type: string + sizeLimit: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + ephemeral: + properties: + volumeClaimTemplate: + properties: + metadata: + properties: + annotations: + additionalProperties: + type: string + type: object + creationTimestamp: + type: string + deletionGracePeriodSeconds: + type: integer + deletionTimestamp: + type: string + finalizers: + items: + type: string + type: array + generateName: + type: string + generation: + type: integer + labels: + additionalProperties: + type: string + type: object + managedFields: + items: + properties: + apiVersion: + type: string + fieldsType: + type: string + fieldsV1: + type: object + manager: + type: string + operation: + type: string + subresource: + type: string + time: + type: string + type: object + type: array + name: + type: string + namespace: + type: string + ownerReferences: + items: + properties: + apiVersion: + type: string + blockOwnerDeletion: + type: boolean + controller: + type: boolean + kind: + type: string + name: + type: string + uid: + type: string + type: object + type: array + resourceVersion: + type: string + selfLink: + type: string + uid: + type: string + type: object + spec: + properties: + accessModes: + items: + type: string + type: array + dataSource: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + type: object + dataSourceRef: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + namespace: + type: string + type: object + resources: + properties: + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + selector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + storageClassName: + type: string + volumeAttributesClassName: + type: string + volumeMode: + type: string + volumeName: + type: string + type: object + type: object + type: object + fc: + properties: + fsType: + type: string + lun: + type: integer + readOnly: + type: boolean + targetWWNs: + items: + type: string + type: array + wwids: + items: + type: string + type: array + type: object + flexVolume: + properties: + driver: + type: string + fsType: + type: string + options: + additionalProperties: + type: string + type: object + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + type: object + flocker: + properties: + datasetName: + type: string + datasetUUID: + type: string + type: object + gcePersistentDisk: + properties: + fsType: + type: string + partition: + type: integer + pdName: + type: string + readOnly: + type: boolean + type: object + gitRepo: + properties: + directory: + type: string + repository: + type: string + revision: + type: string + type: object + glusterfs: + properties: + endpoints: + type: string + path: + type: string + readOnly: + type: boolean + type: object + hostPath: + properties: + path: + type: string + type: + type: string + type: object + image: + properties: + pullPolicy: + type: string + reference: + type: string + type: object + iscsi: + properties: + chapAuthDiscovery: + type: boolean + chapAuthSession: + type: boolean + fsType: + type: string + initiatorName: + type: string + iqn: + type: string + iscsiInterface: + type: string + lun: + type: integer + portals: + items: + type: string + type: array + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + targetPortal: + type: string + type: object + name: + type: string + nfs: + properties: + path: + type: string + readOnly: + type: boolean + server: + type: string + type: object + persistentVolumeClaim: + properties: + claimName: + type: string + readOnly: + type: boolean + type: object + photonPersistentDisk: + properties: + fsType: + type: string + pdID: + type: string + type: object + portworxVolume: + properties: + fsType: + type: string + readOnly: + type: boolean + volumeID: + type: string + type: object + projected: + properties: + defaultMode: + type: integer + sources: + items: + properties: + clusterTrustBundle: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + name: + type: string + optional: + type: boolean + path: + type: string + signerName: + type: string + type: object + configMap: + properties: + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + downwardAPI: + properties: + items: + items: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + mode: + type: integer + path: + type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + type: object + type: array + type: object + secret: + properties: + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + serviceAccountToken: + properties: + audience: + type: string + expirationSeconds: + type: integer + path: + type: string + type: object + type: object + type: array + type: object + quobyte: + properties: + group: + type: string + readOnly: + type: boolean + registry: + type: string + tenant: + type: string + user: + type: string + volume: + type: string + type: object + rbd: + properties: + fsType: + type: string + image: + type: string + keyring: + type: string + monitors: + items: + type: string + type: array + pool: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + user: + type: string + type: object + scaleIO: + properties: + fsType: + type: string + gateway: + type: string + protectionDomain: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + sslEnabled: + type: boolean + storageMode: + type: string + storagePool: + type: string + system: + type: string + volumeName: + type: string + type: object + secret: + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + optional: + type: boolean + secretName: + type: string + type: object + storageos: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + volumeName: + type: string + volumeNamespace: + type: string + type: object + vsphereVolume: + properties: + fsType: + type: string + storagePolicyID: + type: string + storagePolicyName: + type: string + volumePath: + type: string + type: object + type: object + type: array + type: object + status: + properties: + conditions: + items: + properties: + lastProbeTime: + type: string + lastTransitionTime: + type: string + message: + type: string + observedGeneration: + type: integer + reason: + type: string + status: + type: string + type: + type: string + type: object + type: array + containerStatuses: + items: + properties: + allocatedResources: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + allocatedResourcesStatus: + items: + properties: + name: + type: string + resources: + items: + properties: + health: + type: string + resourceID: + type: string + type: object + type: array + type: object + type: array + containerID: + type: string + image: + type: string + imageID: + type: string + lastState: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + name: + type: string + ready: + type: boolean + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartCount: + type: integer + started: + type: boolean + state: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + stopSignal: + type: string + user: + properties: + linux: + properties: + gid: + type: integer + supplementalGroups: + items: + type: integer + type: array + uid: + type: integer + type: object + type: object + volumeMounts: + items: + properties: + mountPath: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + type: object + type: array + type: object + type: array + ephemeralContainerStatuses: + items: + properties: + allocatedResources: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + allocatedResourcesStatus: + items: + properties: + name: + type: string + resources: + items: + properties: + health: + type: string + resourceID: + type: string + type: object + type: array + type: object + type: array + containerID: + type: string + image: + type: string + imageID: + type: string + lastState: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + name: + type: string + ready: + type: boolean + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartCount: + type: integer + started: + type: boolean + state: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + stopSignal: + type: string + user: + properties: + linux: + properties: + gid: + type: integer + supplementalGroups: + items: + type: integer + type: array + uid: + type: integer + type: object + type: object + volumeMounts: + items: + properties: + mountPath: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + type: object + type: array + type: object + type: array + hostIP: + type: string + hostIPs: + items: + properties: + ip: + type: string + type: object + type: array + initContainerStatuses: + items: + properties: + allocatedResources: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + allocatedResourcesStatus: + items: + properties: + name: + type: string + resources: + items: + properties: + health: + type: string + resourceID: + type: string + type: object + type: array + type: object + type: array + containerID: + type: string + image: + type: string + imageID: + type: string + lastState: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + name: + type: string + ready: + type: boolean + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartCount: + type: integer + started: + type: boolean + state: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + stopSignal: + type: string + user: + properties: + linux: + properties: + gid: + type: integer + supplementalGroups: + items: + type: integer + type: array + uid: + type: integer + type: object + type: object + volumeMounts: + items: + properties: + mountPath: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + type: object + type: array + type: object + type: array + message: + type: string + nominatedNodeName: + type: string + observedGeneration: + type: integer + phase: + type: string + podIP: + type: string + podIPs: + items: + properties: + ip: + type: string + type: object + type: array + qosClass: + type: string + reason: + type: string + resize: + type: string + resourceClaimStatuses: + items: + properties: + name: + type: string + resourceClaimName: + type: string + type: object + type: array + startTime: + type: string + type: object + type: object + restartNonce: + type: integer + serviceAccount: + type: string + taskManager: + properties: + podTemplate: + properties: + apiVersion: + type: string + kind: + type: string + metadata: + properties: + annotations: + additionalProperties: + type: string + type: object + creationTimestamp: + type: string + deletionGracePeriodSeconds: + type: integer + deletionTimestamp: + type: string + finalizers: + items: + type: string + type: array + generateName: + type: string + generation: + type: integer + labels: + additionalProperties: + type: string + type: object + managedFields: + items: + properties: + apiVersion: + type: string + fieldsType: + type: string + fieldsV1: + type: object + manager: + type: string + operation: + type: string + subresource: + type: string + time: + type: string + type: object + type: array + name: + type: string + namespace: + type: string + ownerReferences: + items: + properties: + apiVersion: + type: string + blockOwnerDeletion: + type: boolean + controller: + type: boolean + kind: + type: string + name: + type: string + uid: + type: string + type: object + type: array + resourceVersion: + type: string + selfLink: + type: string + uid: + type: string + type: object + spec: + properties: + activeDeadlineSeconds: + type: integer + affinity: + properties: + nodeAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + preference: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + properties: + nodeSelectorTerms: + items: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + type: array + type: object + type: object + podAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + podAntiAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + type: object + automountServiceAccountToken: + type: boolean + containers: + items: + properties: + args: + items: + type: string + type: array + command: + items: + type: string + type: array + env: + items: + properties: + name: + type: string + value: + type: string + valueFrom: + properties: + configMapKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + secretKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + envFrom: + items: + properties: + configMapRef: + properties: + name: + type: string + optional: + type: boolean + type: object + prefix: + type: string + secretRef: + properties: + name: + type: string + optional: + type: boolean + type: object + type: object + type: array + image: + type: string + imagePullPolicy: + type: string + lifecycle: + properties: + postStart: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + preStop: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + stopSignal: + type: string + type: object + livenessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + name: + type: string + ports: + items: + properties: + containerPort: + type: integer + hostIP: + type: string + hostPort: + type: integer + name: + type: string + protocol: + type: string + type: object + type: array + readinessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + resizePolicy: + items: + properties: + resourceName: + type: string + restartPolicy: + type: string + type: object + type: array + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartPolicy: + type: string + securityContext: + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + startupProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + stdin: + type: boolean + stdinOnce: + type: boolean + terminationMessagePath: + type: string + terminationMessagePolicy: + type: string + tty: + type: boolean + volumeDevices: + items: + properties: + devicePath: + type: string + name: + type: string + type: object + type: array + volumeMounts: + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + workingDir: + type: string + type: object + type: array + dnsConfig: + properties: + nameservers: + items: + type: string + type: array + options: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + searches: + items: + type: string + type: array + type: object + dnsPolicy: + type: string + enableServiceLinks: + type: boolean + ephemeralContainers: + items: + properties: + args: + items: + type: string + type: array + command: + items: + type: string + type: array + env: + items: + properties: + name: + type: string + value: + type: string + valueFrom: + properties: + configMapKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + secretKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + envFrom: + items: + properties: + configMapRef: + properties: + name: + type: string + optional: + type: boolean + type: object + prefix: + type: string + secretRef: + properties: + name: + type: string + optional: + type: boolean + type: object + type: object + type: array + image: + type: string + imagePullPolicy: + type: string + lifecycle: + properties: + postStart: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + preStop: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + stopSignal: + type: string + type: object + livenessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + name: + type: string + ports: + items: + properties: + containerPort: + type: integer + hostIP: + type: string + hostPort: + type: integer + name: + type: string + protocol: + type: string + type: object + type: array + readinessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + resizePolicy: + items: + properties: + resourceName: + type: string + restartPolicy: + type: string + type: object + type: array + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartPolicy: + type: string + securityContext: + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + startupProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + stdin: + type: boolean + stdinOnce: + type: boolean + targetContainerName: + type: string + terminationMessagePath: + type: string + terminationMessagePolicy: + type: string + tty: + type: boolean + volumeDevices: + items: + properties: + devicePath: + type: string + name: + type: string + type: object + type: array + volumeMounts: + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + workingDir: + type: string + type: object + type: array + hostAliases: + items: + properties: + hostnames: + items: + type: string + type: array + ip: + type: string + type: object + type: array + hostIPC: + type: boolean + hostNetwork: + type: boolean + hostPID: + type: boolean + hostUsers: + type: boolean + hostname: + type: string + imagePullSecrets: + items: + properties: + name: + type: string + type: object + type: array + initContainers: + items: + properties: + args: + items: + type: string + type: array + command: + items: + type: string + type: array + env: + items: + properties: + name: + type: string + value: + type: string + valueFrom: + properties: + configMapKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + secretKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + envFrom: + items: + properties: + configMapRef: + properties: + name: + type: string + optional: + type: boolean + type: object + prefix: + type: string + secretRef: + properties: + name: + type: string + optional: + type: boolean + type: object + type: object + type: array + image: + type: string + imagePullPolicy: + type: string + lifecycle: + properties: + postStart: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + preStop: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + stopSignal: + type: string + type: object + livenessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + name: + type: string + ports: + items: + properties: + containerPort: + type: integer + hostIP: + type: string + hostPort: + type: integer + name: + type: string + protocol: + type: string + type: object + type: array + readinessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + resizePolicy: + items: + properties: + resourceName: + type: string + restartPolicy: + type: string + type: object + type: array + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartPolicy: + type: string + securityContext: + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + startupProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + stdin: + type: boolean + stdinOnce: + type: boolean + terminationMessagePath: + type: string + terminationMessagePolicy: + type: string + tty: + type: boolean + volumeDevices: + items: + properties: + devicePath: + type: string + name: + type: string + type: object + type: array + volumeMounts: + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + workingDir: + type: string + type: object + type: array + nodeName: + type: string + nodeSelector: + additionalProperties: + type: string + type: object + os: + properties: + name: + type: string + type: object + overhead: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + preemptionPolicy: + type: string + priority: + type: integer + priorityClassName: + type: string + readinessGates: + items: + properties: + conditionType: + type: string + type: object + type: array + resourceClaims: + items: + properties: + name: + type: string + resourceClaimName: + type: string + resourceClaimTemplateName: + type: string + type: object + type: array + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartPolicy: + type: string + runtimeClassName: + type: string + schedulerName: + type: string + schedulingGates: + items: + properties: + name: + type: string + type: object + type: array + securityContext: + properties: + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + fsGroup: + type: integer + fsGroupChangePolicy: + type: string + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxChangePolicy: + type: string + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + supplementalGroups: + items: + type: integer + type: array + supplementalGroupsPolicy: + type: string + sysctls: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + serviceAccount: + type: string + serviceAccountName: + type: string + setHostnameAsFQDN: + type: boolean + shareProcessNamespace: + type: boolean + subdomain: + type: string + terminationGracePeriodSeconds: + type: integer + tolerations: + items: + properties: + effect: + type: string + key: + type: string + operator: + type: string + tolerationSeconds: + type: integer + value: + type: string + type: object + type: array + topologySpreadConstraints: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + maxSkew: + type: integer + minDomains: + type: integer + nodeAffinityPolicy: + type: string + nodeTaintsPolicy: + type: string + topologyKey: + type: string + whenUnsatisfiable: + type: string + type: object + type: array + volumes: + items: + properties: + awsElasticBlockStore: + properties: + fsType: + type: string + partition: + type: integer + readOnly: + type: boolean + volumeID: + type: string + type: object + azureDisk: + properties: + cachingMode: + type: string + diskName: + type: string + diskURI: + type: string + fsType: + type: string + kind: + type: string + readOnly: + type: boolean + type: object + azureFile: + properties: + readOnly: + type: boolean + secretName: + type: string + shareName: + type: string + type: object + cephfs: + properties: + monitors: + items: + type: string + type: array + path: + type: string + readOnly: + type: boolean + secretFile: + type: string + secretRef: + properties: + name: + type: string + type: object + user: + type: string + type: object + cinder: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + volumeID: + type: string + type: object + configMap: + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + csi: + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: + properties: + name: + type: string + type: object + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string + type: object + type: object + downwardAPI: + properties: + defaultMode: + type: integer + items: + items: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + mode: + type: integer + path: + type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + type: object + type: array + type: object + emptyDir: + properties: + medium: + type: string + sizeLimit: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + ephemeral: + properties: + volumeClaimTemplate: + properties: + metadata: + properties: + annotations: + additionalProperties: + type: string + type: object + creationTimestamp: + type: string + deletionGracePeriodSeconds: + type: integer + deletionTimestamp: + type: string + finalizers: + items: + type: string + type: array + generateName: + type: string + generation: + type: integer + labels: + additionalProperties: + type: string + type: object + managedFields: + items: + properties: + apiVersion: + type: string + fieldsType: + type: string + fieldsV1: + type: object + manager: + type: string + operation: + type: string + subresource: + type: string + time: + type: string + type: object + type: array + name: + type: string + namespace: + type: string + ownerReferences: + items: + properties: + apiVersion: + type: string + blockOwnerDeletion: + type: boolean + controller: + type: boolean + kind: + type: string + name: + type: string + uid: + type: string + type: object + type: array + resourceVersion: + type: string + selfLink: + type: string + uid: + type: string + type: object + spec: + properties: + accessModes: + items: + type: string + type: array + dataSource: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + type: object + dataSourceRef: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + namespace: + type: string + type: object + resources: + properties: + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + selector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + storageClassName: + type: string + volumeAttributesClassName: + type: string + volumeMode: + type: string + volumeName: + type: string + type: object + type: object + type: object + fc: + properties: + fsType: + type: string + lun: + type: integer + readOnly: + type: boolean + targetWWNs: + items: + type: string + type: array + wwids: + items: + type: string + type: array + type: object + flexVolume: + properties: + driver: + type: string + fsType: + type: string + options: + additionalProperties: + type: string + type: object + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + type: object + flocker: + properties: + datasetName: + type: string + datasetUUID: + type: string + type: object + gcePersistentDisk: + properties: + fsType: + type: string + partition: + type: integer + pdName: + type: string + readOnly: + type: boolean + type: object + gitRepo: + properties: + directory: + type: string + repository: + type: string + revision: + type: string + type: object + glusterfs: + properties: + endpoints: + type: string + path: + type: string + readOnly: + type: boolean + type: object + hostPath: + properties: + path: + type: string + type: + type: string + type: object + image: + properties: + pullPolicy: + type: string + reference: + type: string + type: object + iscsi: + properties: + chapAuthDiscovery: + type: boolean + chapAuthSession: + type: boolean + fsType: + type: string + initiatorName: + type: string + iqn: + type: string + iscsiInterface: + type: string + lun: + type: integer + portals: + items: + type: string + type: array + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + targetPortal: + type: string + type: object + name: + type: string + nfs: + properties: + path: + type: string + readOnly: + type: boolean + server: + type: string + type: object + persistentVolumeClaim: + properties: + claimName: + type: string + readOnly: + type: boolean + type: object + photonPersistentDisk: + properties: + fsType: + type: string + pdID: + type: string + type: object + portworxVolume: + properties: + fsType: + type: string + readOnly: + type: boolean + volumeID: + type: string + type: object + projected: + properties: + defaultMode: + type: integer + sources: + items: + properties: + clusterTrustBundle: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + name: + type: string + optional: + type: boolean + path: + type: string + signerName: + type: string + type: object + configMap: + properties: + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + downwardAPI: + properties: + items: + items: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + mode: + type: integer + path: + type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + type: object + type: array + type: object + secret: + properties: + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + serviceAccountToken: + properties: + audience: + type: string + expirationSeconds: + type: integer + path: + type: string + type: object + type: object + type: array + type: object + quobyte: + properties: + group: + type: string + readOnly: + type: boolean + registry: + type: string + tenant: + type: string + user: + type: string + volume: + type: string + type: object + rbd: + properties: + fsType: + type: string + image: + type: string + keyring: + type: string + monitors: + items: + type: string + type: array + pool: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + user: + type: string + type: object + scaleIO: + properties: + fsType: + type: string + gateway: + type: string + protectionDomain: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + sslEnabled: + type: boolean + storageMode: + type: string + storagePool: + type: string + system: + type: string + volumeName: + type: string + type: object + secret: + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + optional: + type: boolean + secretName: + type: string + type: object + storageos: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + volumeName: + type: string + volumeNamespace: + type: string + type: object + vsphereVolume: + properties: + fsType: + type: string + storagePolicyID: + type: string + storagePolicyName: + type: string + volumePath: + type: string + type: object + type: object + type: array + type: object + status: + properties: + conditions: + items: + properties: + lastProbeTime: + type: string + lastTransitionTime: + type: string + message: + type: string + observedGeneration: + type: integer + reason: + type: string + status: + type: string + type: + type: string + type: object + type: array + containerStatuses: + items: + properties: + allocatedResources: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + allocatedResourcesStatus: + items: + properties: + name: + type: string + resources: + items: + properties: + health: + type: string + resourceID: + type: string + type: object + type: array + type: object + type: array + containerID: + type: string + image: + type: string + imageID: + type: string + lastState: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + name: + type: string + ready: + type: boolean + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartCount: + type: integer + started: + type: boolean + state: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + stopSignal: + type: string + user: + properties: + linux: + properties: + gid: + type: integer + supplementalGroups: + items: + type: integer + type: array + uid: + type: integer + type: object + type: object + volumeMounts: + items: + properties: + mountPath: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + type: object + type: array + type: object + type: array + ephemeralContainerStatuses: + items: + properties: + allocatedResources: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + allocatedResourcesStatus: + items: + properties: + name: + type: string + resources: + items: + properties: + health: + type: string + resourceID: + type: string + type: object + type: array + type: object + type: array + containerID: + type: string + image: + type: string + imageID: + type: string + lastState: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + name: + type: string + ready: + type: boolean + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartCount: + type: integer + started: + type: boolean + state: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + stopSignal: + type: string + user: + properties: + linux: + properties: + gid: + type: integer + supplementalGroups: + items: + type: integer + type: array + uid: + type: integer + type: object + type: object + volumeMounts: + items: + properties: + mountPath: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + type: object + type: array + type: object + type: array + hostIP: + type: string + hostIPs: + items: + properties: + ip: + type: string + type: object + type: array + initContainerStatuses: + items: + properties: + allocatedResources: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + allocatedResourcesStatus: + items: + properties: + name: + type: string + resources: + items: + properties: + health: + type: string + resourceID: + type: string + type: object + type: array + type: object + type: array + containerID: + type: string + image: + type: string + imageID: + type: string + lastState: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + name: + type: string + ready: + type: boolean + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartCount: + type: integer + started: + type: boolean + state: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + stopSignal: + type: string + user: + properties: + linux: + properties: + gid: + type: integer + supplementalGroups: + items: + type: integer + type: array + uid: + type: integer + type: object + type: object + volumeMounts: + items: + properties: + mountPath: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + type: object + type: array + type: object + type: array + message: + type: string + nominatedNodeName: + type: string + observedGeneration: + type: integer + phase: + type: string + podIP: + type: string + podIPs: + items: + properties: + ip: + type: string + type: object + type: array + qosClass: + type: string + reason: + type: string + resize: + type: string + resourceClaimStatuses: + items: + properties: + name: + type: string + resourceClaimName: + type: string + type: object + type: array + startTime: + type: string + type: object + type: object + replicas: + type: integer + resource: + properties: + cpu: + type: number + ephemeralStorage: + type: string + memory: + type: string + type: object + type: object + type: object + status: + properties: + clusterInfo: + additionalProperties: + type: string + type: object + error: + type: string + jobManagerDeploymentStatus: + enum: + - DEPLOYED_NOT_READY + - DEPLOYING + - ERROR + - MISSING + - READY + type: string + jobStatus: + properties: + checkpointInfo: + properties: + formatType: + enum: + - FULL + - INCREMENTAL + - UNKNOWN + type: string + lastCheckpoint: + properties: + formatType: + enum: + - FULL + - INCREMENTAL + - UNKNOWN + type: string + timeStamp: + type: integer + triggerNonce: + type: integer + triggerType: + enum: + - MANUAL + - PERIODIC + - UNKNOWN + - UPGRADE + type: string + type: object + lastPeriodicCheckpointTimestamp: + type: integer + triggerId: + type: string + triggerTimestamp: + type: integer + triggerType: + enum: + - MANUAL + - PERIODIC + - UNKNOWN + - UPGRADE + type: string + type: object + jobId: + type: string + jobName: + type: string + savepointInfo: + properties: + formatType: + enum: + - CANONICAL + - NATIVE + - UNKNOWN + type: string + lastPeriodicSavepointTimestamp: + type: integer + lastSavepoint: + properties: + formatType: + enum: + - CANONICAL + - NATIVE + - UNKNOWN + type: string + location: + type: string + timeStamp: + type: integer + triggerNonce: + type: integer + triggerType: + enum: + - MANUAL + - PERIODIC + - UNKNOWN + - UPGRADE + type: string + type: object + savepointHistory: + items: + properties: + formatType: + enum: + - CANONICAL + - NATIVE + - UNKNOWN + type: string + location: + type: string + timeStamp: + type: integer + triggerNonce: + type: integer + triggerType: + enum: + - MANUAL + - PERIODIC + - UNKNOWN + - UPGRADE + type: string + type: object + type: array + triggerId: + type: string + triggerTimestamp: + type: integer + triggerType: + enum: + - MANUAL + - PERIODIC + - UNKNOWN + - UPGRADE + type: string + type: object + startTime: + type: string + state: + enum: + - CANCELED + - CANCELLING + - CREATED + - FAILED + - FAILING + - FINISHED + - INITIALIZING + - RECONCILING + - RESTARTING + - RUNNING + - SUSPENDED + type: string + updateTime: + type: string + upgradeSavepointPath: + type: string + type: object + lifecycleState: + enum: + - CREATED + - DELETED + - DELETING + - DEPLOYED + - FAILED + - ROLLED_BACK + - ROLLING_BACK + - STABLE + - SUSPENDED + - UPGRADING + type: string + observedGeneration: + type: integer + reconciliationStatus: + properties: + lastReconciledSpec: + type: string + lastStableSpec: + type: string + reconciliationTimestamp: + type: integer + state: + enum: + - DEPLOYED + - ROLLED_BACK + - ROLLING_BACK + - UPGRADING + type: string + type: object + taskManager: + properties: + labelSelector: + type: string + replicas: + type: integer + type: object + type: object + type: object + served: true + storage: true + subresources: + scale: + labelSelectorPath: .status.taskManager.labelSelector + specReplicasPath: .spec.taskManager.replicas + statusReplicasPath: .status.taskManager.replicas + status: {} diff --git a/flink-kubernetes-operator-api/src/test/resources/test-deployment-key-value-configuration.yaml b/flink-kubernetes-operator-api/src/test/resources/test-deployment-key-value-configuration.yaml new file mode 100644 index 0000000000..6a02f1237d --- /dev/null +++ b/flink-kubernetes-operator-api/src/test/resources/test-deployment-key-value-configuration.yaml @@ -0,0 +1,44 @@ +################################################################################ +# 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. +################################################################################ + +apiVersion: flink.apache.org/v1beta1 +kind: FlinkDeployment +metadata: + name: basic-example +spec: + image: flink:1.20 + flinkVersion: v1_20 + flinkConfiguration: + taskmanager.numberOfTaskSlots: "2" + state.savepoints.dir: file:///flink-data/savepoints + state.checkpoints.dir: file:///flink-data/checkpoints + high-availability.type: kubernetes + high-availability.storageDir: file:///flink-data/ha + serviceAccount: flink + jobManager: + resource: + memory: "2048m" + cpu: 1 + taskManager: + resource: + memory: "2048m" + cpu: 1 + job: + jarURI: local:///opt/flink/examples/streaming/StateMachineExample.jar + parallelism: 2 + upgradeMode: stateless diff --git a/flink-kubernetes-operator-api/src/test/resources/test-deployment-yaml-configuration.yaml b/flink-kubernetes-operator-api/src/test/resources/test-deployment-yaml-configuration.yaml new file mode 100644 index 0000000000..2d8bf92865 --- /dev/null +++ b/flink-kubernetes-operator-api/src/test/resources/test-deployment-yaml-configuration.yaml @@ -0,0 +1,49 @@ +################################################################################ +# 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. +################################################################################ + +apiVersion: flink.apache.org/v1beta1 +kind: FlinkDeployment +metadata: + name: basic-example +spec: + image: flink:1.20 + flinkVersion: v1_20 + flinkConfiguration: + taskmanager: + numberOfTaskSlots: "2" + state: + savepoints: + dir: file:///flink-data/savepoints + checkpoints: + dir: file:///flink-data/checkpoints + high-availability: + type: kubernetes + storageDir: file:///flink-data/ha + serviceAccount: flink + jobManager: + resource: + memory: "2048m" + cpu: 1 + taskManager: + resource: + memory: "2048m" + cpu: 1 + job: + jarURI: local:///opt/flink/examples/streaming/StateMachineExample.jar + parallelism: 2 + upgradeMode: stateless diff --git a/helm/flink-kubernetes-operator/crds/flinkdeployments.flink.apache.org-v1.yml b/helm/flink-kubernetes-operator/crds/flinkdeployments.flink.apache.org-v1.yml index 292ecf3d4c..05b65c4ab1 100644 --- a/helm/flink-kubernetes-operator/crds/flinkdeployments.flink.apache.org-v1.yml +++ b/helm/flink-kubernetes-operator/crds/flinkdeployments.flink.apache.org-v1.yml @@ -32,9 +32,7 @@ spec: spec: properties: flinkConfiguration: - additionalProperties: - type: string - type: object + x-kubernetes-preserve-unknown-fields: true flinkVersion: enum: - v1_13 diff --git a/helm/flink-kubernetes-operator/crds/flinksessionjobs.flink.apache.org-v1.yml b/helm/flink-kubernetes-operator/crds/flinksessionjobs.flink.apache.org-v1.yml index f66d67ea72..668a84b6c7 100644 --- a/helm/flink-kubernetes-operator/crds/flinksessionjobs.flink.apache.org-v1.yml +++ b/helm/flink-kubernetes-operator/crds/flinksessionjobs.flink.apache.org-v1.yml @@ -34,9 +34,7 @@ spec: deploymentName: type: string flinkConfiguration: - additionalProperties: - type: string - type: object + x-kubernetes-preserve-unknown-fields: true job: properties: allowNonRestoredState: From e4a4d24cd1fd0d31f71d150c71e318fcbc86bac5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Attila=20M=C3=A9sz=C3=A1ros?= Date: Tue, 19 Aug 2025 18:10:17 +0200 Subject: [PATCH 09/19] Config MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Attila Mészáros --- .../java/org/apache/flink/examples/Basic.java | 3 +- flink-kubernetes-operator-api/pom.xml | 112 ++++----- .../operator/api/spec/AbstractFlinkSpec.java | 5 +- .../operator/api/spec/ConfigJsonNode.java | 84 +++++++ .../api/spec/ConfigJsonNodeDeserializer.java | 39 ++++ .../operator/api/utils/SpecUtils.java | 95 +------- .../FlinkConfigurationYamlSupportTest.java | 4 +- .../operator/api/utils/BaseTestUtils.java | 13 +- .../operator/api/utils/SpecUtilsTest.java | 59 ----- .../autoscaler/KubernetesScalingRealizer.java | 17 +- .../operator/config/FlinkConfigBuilder.java | 3 +- .../operator/config/FlinkConfigManager.java | 10 +- .../diff/ReflectiveDiffBuilder.java | 9 +- .../operator/validation/DefaultValidator.java | 9 +- .../flink/kubernetes/operator/TestUtils.java | 5 +- .../KubernetesScalingRealizerTest.java | 26 ++- .../config/FlinkConfigBuilderTest.java | 43 ++-- .../config/FlinkConfigManagerTest.java | 33 +-- .../controller/DeploymentRecoveryTest.java | 8 +- .../FlinkDeploymentControllerTest.java | 38 ++-- .../FlinkSessionJobControllerTest.java | 21 +- .../FlinkStateSnapshotControllerTest.java | 8 +- .../operator/controller/RollbackTest.java | 73 +++--- .../observer/JobStatusObserverTest.java | 14 +- .../deployment/ApplicationObserverTest.java | 39 ++-- .../deployment/SessionObserverTest.java | 5 +- .../deployment/ApplicationReconcilerTest.java | 163 +++++++------ .../ApplicationReconcilerUpgradeModeTest.java | 100 ++++---- .../deployment/SessionReconcilerTest.java | 10 +- .../reconciler/diff/SpecDiffTest.java | 41 ++-- .../sessionjob/SessionJobReconcilerTest.java | 29 ++- .../service/AbstractFlinkServiceTest.java | 67 +++--- .../service/NativeFlinkServiceTest.java | 85 +++---- .../service/StandaloneFlinkServiceTest.java | 30 ++- .../operator/utils/SnapshotUtilsTest.java | 10 +- .../validation/DefaultValidatorTest.java | 215 +++++++++--------- 36 files changed, 795 insertions(+), 730 deletions(-) create mode 100644 flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigJsonNode.java create mode 100644 flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigJsonNodeDeserializer.java diff --git a/examples/kubernetes-client-examples/src/main/java/org/apache/flink/examples/Basic.java b/examples/kubernetes-client-examples/src/main/java/org/apache/flink/examples/Basic.java index 407d4c7aeb..a762d99eb8 100644 --- a/examples/kubernetes-client-examples/src/main/java/org/apache/flink/examples/Basic.java +++ b/examples/kubernetes-client-examples/src/main/java/org/apache/flink/examples/Basic.java @@ -34,7 +34,6 @@ import java.util.Map; import static java.util.Map.entry; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.toJsonNode; /** client code for ../basic.yaml. */ public class Basic { @@ -50,7 +49,7 @@ public static void main(String[] args) { flinkDeploymentSpec.setImage("flink:1.19"); Map flinkConfiguration = Map.ofEntries(entry("taskmanager.numberOfTaskSlots", "2")); - flinkDeploymentSpec.setFlinkConfiguration(toJsonNode(flinkConfiguration)); + flinkDeploymentSpec.getFlinkConfiguration().set(flinkConfiguration); flinkDeployment.setSpec(flinkDeploymentSpec); flinkDeploymentSpec.setServiceAccount("flink"); JobManagerSpec jobManagerSpec = new JobManagerSpec(); diff --git a/flink-kubernetes-operator-api/pom.xml b/flink-kubernetes-operator-api/pom.xml index 2e83d291d2..0befbb0534 100644 --- a/flink-kubernetes-operator-api/pom.xml +++ b/flink-kubernetes-operator-api/pom.xml @@ -254,62 +254,62 @@ under the License. - - maven-antrun-plugin - - - deployment-crd-compatibility-check - package - - run - - - - - - - - - - - - - sessionjob-crd-compatibility-check - package - - run - - - - - - - - - - - - - statesnapshot-crd-compatibility-check - package - - run - - - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + org.apache.maven.plugins maven-jar-plugin diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/AbstractFlinkSpec.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/AbstractFlinkSpec.java index 9d60649d0c..17b3591b98 100644 --- a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/AbstractFlinkSpec.java +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/AbstractFlinkSpec.java @@ -22,7 +22,6 @@ import org.apache.flink.kubernetes.operator.api.diff.Diffable; import org.apache.flink.kubernetes.operator.api.diff.SpecDiff; -import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.annotation.JsonDeserialize; import lombok.AllArgsConstructor; import lombok.Data; @@ -57,6 +56,6 @@ public abstract class AbstractFlinkSpec implements Diffable { type = DiffType.SCALE, mode = KubernetesDeploymentMode.NATIVE) }) - @JsonDeserialize(using = JsonNodeNullDeserializer.class) - private JsonNode flinkConfiguration; + @JsonDeserialize(using = ConfigJsonNodeDeserializer.class) + private ConfigJsonNode flinkConfiguration = new ConfigJsonNode(); } diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigJsonNode.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigJsonNode.java new file mode 100644 index 0000000000..77c0e46984 --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigJsonNode.java @@ -0,0 +1,84 @@ +/* + * 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.api.spec; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.JsonNodeFactory; +import com.fasterxml.jackson.databind.node.ObjectNode; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; + +/** */ +public class ConfigJsonNode extends ObjectNode { + + public ConfigJsonNode() { + this(JsonNodeFactory.instance); + } + + public ConfigJsonNode(JsonNodeFactory nc, Map kids) { + super(nc, kids); + } + + public ConfigJsonNode(JsonNodeFactory nc) { + super(nc); + } + + public void removeAll(String... names) { + for (String name : names) { + remove(name); + } + } + + public void putAllFrom(Map value) { + value.forEach(this::put); + } + + public void set(Map value) { + removeAll(); + putAllFrom(value); + } + + public Map asFlatMap() { + Map flatMap = new HashMap<>(); + flattenHelper(this, "", flatMap); + return flatMap; + } + + private static void flattenHelper( + JsonNode node, String parentKey, Map flatMap) { + if (node.isObject()) { + Iterator> fields = node.fields(); + while (fields.hasNext()) { + Map.Entry field = fields.next(); + String newKey = + parentKey.isEmpty() ? field.getKey() : parentKey + "." + field.getKey(); + flattenHelper(field.getValue(), newKey, flatMap); + } + } else if (node.isArray()) { + for (int i = 0; i < node.size(); i++) { + String newKey = parentKey + "[" + i + "]"; + flattenHelper(node.get(i), newKey, flatMap); + } + } else { + // Store values as strings + flatMap.put(parentKey, node.asText()); + } + } +} diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigJsonNodeDeserializer.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigJsonNodeDeserializer.java new file mode 100644 index 0000000000..bdb56c57c4 --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigJsonNodeDeserializer.java @@ -0,0 +1,39 @@ +/* + * 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.api.spec; + +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonDeserializer; +import com.fasterxml.jackson.databind.node.ObjectNode; + +import java.io.IOException; + +/** ConfigJsonNode deserializer. */ +public class ConfigJsonNodeDeserializer extends JsonDeserializer { + + @Override + public ConfigJsonNode deserialize( + JsonParser jsonParser, DeserializationContext deserializationContext) + throws IOException { + ObjectNode tree = jsonParser.readValueAsTree(); + var res = new ConfigJsonNode(); + tree.fields().forEachRemaining(entry -> res.set(entry.getKey(), entry.getValue())); + return res; + } +} diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java index 5455b7045a..4f93d69463 100644 --- a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java @@ -18,30 +18,20 @@ package org.apache.flink.kubernetes.operator.api.utils; -import org.apache.flink.configuration.Configuration; import org.apache.flink.kubernetes.operator.api.AbstractFlinkResource; import org.apache.flink.kubernetes.operator.api.reconciler.ReconciliationMetadata; import org.apache.flink.kubernetes.operator.api.spec.AbstractFlinkSpec; import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.node.NullNode; import com.fasterxml.jackson.databind.node.ObjectNode; -import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; import javax.annotation.Nullable; -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; -import java.util.Set; - /** Spec utilities. */ public class SpecUtils { public static final String INTERNAL_METADATA_JSON_KEY = "resource_metadata"; private static final ObjectMapper objectMapper = new ObjectMapper(); - private static final ObjectMapper yamlObjectMapper = new ObjectMapper(new YAMLFactory()); /** * Deserializes the spec and custom metadata object from JSON. @@ -109,7 +99,7 @@ public static String writeSpecWithMeta( } } - // We do not have access to Flink's Preconditions from here + // We do not have access to Flink's Preconditions from here private static T checkNotNull(T object) { if (object == null) { throw new NullPointerException(); @@ -130,87 +120,4 @@ public static T clone(T object) { throw new IllegalStateException(e); } } - - public static JsonNode toJsonNode(Map properties) { - ObjectNode jsonNode = yamlObjectMapper.createObjectNode(); - for (Map.Entry entry : properties.entrySet()) { - jsonNode.put(entry.getKey(), entry.getValue()); - } - return jsonNode; - } - - public static Map toStringMap(JsonNode node) { - if (node == null) { - return new HashMap<>(); - } - if (node instanceof NullNode) { - return new HashMap<>(); - } - Map flatMap = new HashMap<>(); - flattenHelper(node, "", flatMap); - return flatMap; - } - - private static void flattenHelper( - JsonNode node, String parentKey, Map flatMap) { - if (node.isObject()) { - Iterator> fields = node.fields(); - while (fields.hasNext()) { - Map.Entry field = fields.next(); - String newKey = - parentKey.isEmpty() ? field.getKey() : parentKey + "." + field.getKey(); - flattenHelper(field.getValue(), newKey, flatMap); - } - } else if (node.isArray()) { - for (int i = 0; i < node.size(); i++) { - String newKey = parentKey + "[" + i + "]"; - flattenHelper(node.get(i), newKey, flatMap); - } - } else { - // Store values as strings - flatMap.put(parentKey, node.asText()); - } - } - - public static void addConfigProperties(AbstractFlinkSpec spec, Map properties) { - spec.setFlinkConfiguration(addProperties(spec.getFlinkConfiguration(), properties)); - } - - public static void addConfigProperty(AbstractFlinkSpec spec, String key, String value) { - spec.setFlinkConfiguration(addProperties(spec.getFlinkConfiguration(), Map.of(key, value))); - } - - public static void removeConfigProperties(AbstractFlinkSpec spec, Set keys) { - spec.setFlinkConfiguration(removeProperties(spec.getFlinkConfiguration(), keys)); - } - - public static void removeConfigProperties(AbstractFlinkSpec spec, String... keys) { - spec.setFlinkConfiguration(removeProperties(spec.getFlinkConfiguration(), Set.of(keys))); - } - - public static JsonNode addProperties(JsonNode node, Map properties) { - var map = toStringMap(node); - map.putAll(properties); - return mapToJsonNode(map); - } - - public static JsonNode removeProperty(JsonNode node, String key) { - var map = toStringMap(node); - map.remove(key); - return mapToJsonNode(map); - } - - public static JsonNode removeProperties(JsonNode node, Set keys) { - var map = toStringMap(node); - map.keySet().removeAll(keys); - return mapToJsonNode(map); - } - - public static JsonNode mapToJsonNode(Map config) { - return yamlObjectMapper.valueToTree(config); - } - - public static JsonNode configurationToJsonNode(Configuration configuration) { - return mapToJsonNode(configuration.toMap()); - } } diff --git a/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/FlinkConfigurationYamlSupportTest.java b/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/FlinkConfigurationYamlSupportTest.java index 6f68b6debd..2302938d59 100644 --- a/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/FlinkConfigurationYamlSupportTest.java +++ b/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/FlinkConfigurationYamlSupportTest.java @@ -17,8 +17,6 @@ package org.apache.flink.kubernetes.operator.api; -import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; - import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; import io.fabric8.kubeapitest.junit.EnableKubeAPIServer; @@ -67,7 +65,7 @@ void upgradeCRDToYamlFlinkConfiguration() { .withName("basic-example") .get(); assertThat(deployment.getSpec().getFlinkConfiguration()).hasSize(3); - assertThat(SpecUtils.toStringMap(deployment.getSpec().getFlinkConfiguration())).hasSize(5); + assertThat(deployment.getSpec().getFlinkConfiguration().asFlatMap()).hasSize(5); } private GenericKubernetesResource applyResource(String path) { diff --git a/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/utils/BaseTestUtils.java b/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/utils/BaseTestUtils.java index 858915d541..8088d7708a 100644 --- a/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/utils/BaseTestUtils.java +++ b/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/utils/BaseTestUtils.java @@ -25,6 +25,7 @@ import org.apache.flink.kubernetes.operator.api.FlinkSessionJob; import org.apache.flink.kubernetes.operator.api.FlinkStateSnapshot; import org.apache.flink.kubernetes.operator.api.spec.CheckpointSpec; +import org.apache.flink.kubernetes.operator.api.spec.ConfigJsonNode; import org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec; import org.apache.flink.kubernetes.operator.api.spec.FlinkSessionJobSpec; import org.apache.flink.kubernetes.operator.api.spec.FlinkStateSnapshotSpec; @@ -48,13 +49,9 @@ import io.fabric8.kubernetes.api.model.PodTemplateSpec; import java.time.Instant; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.UUID; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.toJsonNode; - /** Base Testing utilities. */ public class BaseTestUtils { @@ -146,7 +143,7 @@ public static FlinkSessionJob buildSessionJob( .withResourceVersion("1") .build()); - Map conf = new HashMap<>(); + ConfigJsonNode conf = new ConfigJsonNode(); conf.put("kubernetes.operator.user.artifacts.http.header", "header"); sessionJob.setSpec( FlinkSessionJobSpec.builder() @@ -158,7 +155,7 @@ public static FlinkSessionJob buildSessionJob( .upgradeMode(UpgradeMode.STATELESS) .state(jobState) .build()) - .flinkConfiguration(toJsonNode(conf)) + .flinkConfiguration(conf) .build()); return sessionJob; } @@ -172,7 +169,7 @@ public static FlinkSessionJob buildSessionJob(JobState state) { } public static FlinkDeploymentSpec getTestFlinkDeploymentSpec(FlinkVersion version) { - Map conf = new HashMap<>(); + ConfigJsonNode conf = new ConfigJsonNode(); conf.put(TaskManagerOptions.NUM_TASK_SLOTS.key(), "2"); conf.put( HighAvailabilityOptions.HA_MODE.key(), @@ -188,7 +185,7 @@ public static FlinkDeploymentSpec getTestFlinkDeploymentSpec(FlinkVersion versio .imagePullPolicy(IMAGE_POLICY) .serviceAccount(SERVICE_ACCOUNT) .flinkVersion(version) - .flinkConfiguration(toJsonNode(conf)) + .flinkConfiguration(conf) .jobManager(new JobManagerSpec(new Resource(1.0, "2048m", "2G"), 1, null)) .taskManager(new TaskManagerSpec(new Resource(1.0, "2048m", "2G"), null, null)) .build(); diff --git a/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtilsTest.java b/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtilsTest.java index 05f11af685..bcb22e864d 100644 --- a/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtilsTest.java +++ b/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtilsTest.java @@ -24,20 +24,14 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; -import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; import org.junit.jupiter.api.Test; -import java.util.Map; - -import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNull; /** Test for {@link SpecUtils}. */ class SpecUtilsTest { - private static final ObjectMapper yamlObjectMapper = new ObjectMapper(new YAMLFactory()); - @Test void testSpecSerializationWithVersion() throws JsonProcessingException { FlinkDeployment app = BaseTestUtils.buildApplicationCluster(); @@ -82,57 +76,4 @@ void testSpecSerializationWithoutGeneration() throws JsonProcessingException { var migrated = SpecUtils.deserializeSpecWithMeta(oldSerialized, FlinkDeploymentSpec.class); assertNull(migrated.getMeta()); } - - @Test - void convertsStringMapToJsonNode() { - var map = Map.of("k1", "v1", "k2", "v2", "k3.nested", "v3"); - var node = SpecUtils.mapToJsonNode(map); - - assertThat(node).hasSize(3); - assertThat(node.get("k1").asText()).isEqualTo("v1"); - assertThat(node.get("k2").asText()).isEqualTo("v2"); - assertThat(node.get("k3.nested").asText()).isEqualTo("v3"); - } - - @Test - void convertsJsonNodeToMap() throws JsonProcessingException { - var node = - yamlObjectMapper.readTree("k1: v1 \n" + "k2: v2 \n" + "k3:\n" + " nested: v3\n"); - - var map = SpecUtils.toStringMap(node); - assertThat(map).hasSize(3); - assertThat(map.get("k1")).isEqualTo("v1"); - assertThat(map.get("k2")).isEqualTo("v2"); - assertThat(map.get("k3.nested")).isEqualTo("v3"); - } - - @Test - void addConfigPropertyToSpec() { - var spec = new FlinkDeploymentSpec(); - - SpecUtils.addConfigProperty(spec, "k1", "v1"); - - assertThat(spec.getFlinkConfiguration().get("k1").asText()).isEqualTo("v1"); - } - - @Test - void addConfigPropertiesToSpec() { - var spec = new FlinkDeploymentSpec(); - - SpecUtils.addConfigProperties(spec, Map.of("k1", "v1", "k2", "v2")); - - assertThat(spec.getFlinkConfiguration().get("k1").asText()).isEqualTo("v1"); - assertThat(spec.getFlinkConfiguration().get("k2").asText()).isEqualTo("v2"); - } - - @Test - void removeConfigPropertiesFromSpec() { - var spec = new FlinkDeploymentSpec(); - SpecUtils.addConfigProperties(spec, Map.of("k1", "v1", "k2", "v2")); - - SpecUtils.removeConfigProperties(spec, "k1"); - - assertThat(spec.getFlinkConfiguration().get("k1")).isNull(); - assertThat(spec.getFlinkConfiguration().get("k2").asText()).isEqualTo("v2"); - } } diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/KubernetesScalingRealizer.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/KubernetesScalingRealizer.java index 7e653c585a..4d34bfa622 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/KubernetesScalingRealizer.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/KubernetesScalingRealizer.java @@ -26,7 +26,6 @@ import org.apache.flink.configuration.PipelineOptions; import org.apache.flink.kubernetes.operator.api.FlinkDeployment; import org.apache.flink.kubernetes.operator.api.spec.Resource; -import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.apache.flink.kubernetes.operator.config.FlinkConfigBuilder; import io.javaoperatorsdk.operator.processing.event.ResourceID; @@ -46,11 +45,12 @@ public class KubernetesScalingRealizer @Override public void realizeParallelismOverrides( KubernetesJobAutoScalerContext context, Map parallelismOverrides) { - SpecUtils.addConfigProperties( - context.getResource().getSpec(), - Map.of( + context.getResource() + .getSpec() + .getFlinkConfiguration() + .put( PipelineOptions.PARALLELISM_OVERRIDES.key(), - getOverrideString(context, parallelismOverrides))); + getOverrideString(context, parallelismOverrides)); } @Override @@ -63,15 +63,14 @@ public void realizeConfigOverrides( FlinkDeployment flinkDeployment = ((FlinkDeployment) context.getResource()); // Apply config overrides - SpecUtils.removeConfigProperties(flinkDeployment.getSpec(), configChanges.getRemovals()); - SpecUtils.addConfigProperties(flinkDeployment.getSpec(), configChanges.getOverrides()); + flinkDeployment.getSpec().getFlinkConfiguration().remove(configChanges.getRemovals()); + flinkDeployment.getSpec().getFlinkConfiguration().putAllFrom(configChanges.getOverrides()); // Update total memory in spec var totalMemoryOverride = MemoryTuning.getTotalMemory( Configuration.fromMap( - SpecUtils.toStringMap( - flinkDeployment.getSpec().getFlinkConfiguration())), + flinkDeployment.getSpec().getFlinkConfiguration().asFlatMap()), context); if (totalMemoryOverride.compareTo(MemorySize.ZERO) <= 0) { LOG.warn("Total memory override {} is not valid", totalMemoryOverride); diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilder.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilder.java index 53534396cd..e50e1268cc 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilder.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilder.java @@ -40,7 +40,6 @@ import org.apache.flink.kubernetes.operator.api.spec.KubernetesDeploymentMode; import org.apache.flink.kubernetes.operator.api.spec.Resource; import org.apache.flink.kubernetes.operator.api.spec.UpgradeMode; -import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; import org.apache.flink.kubernetes.operator.standalone.StandaloneKubernetesConfigOptionsInternal; import org.apache.flink.kubernetes.operator.utils.FlinkUtils; @@ -141,7 +140,7 @@ protected FlinkConfigBuilder applyImagePullPolicy() { protected FlinkConfigBuilder applyFlinkConfiguration() { // Parse config from spec's flinkConfiguration if (spec.getFlinkConfiguration() != null && !spec.getFlinkConfiguration().isEmpty()) { - SpecUtils.toStringMap(spec.getFlinkConfiguration()).forEach(effectiveConfig::setString); + spec.getFlinkConfiguration().asFlatMap().forEach(effectiveConfig::setString); } // Adapt default rest service type from 1.15+ diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManager.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManager.java index 6f0e2b70d4..4c024e2382 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManager.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManager.java @@ -29,7 +29,6 @@ import org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec; import org.apache.flink.kubernetes.operator.api.spec.FlinkSessionJobSpec; import org.apache.flink.kubernetes.operator.api.spec.FlinkVersion; -import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; import org.apache.flink.kubernetes.operator.utils.EnvUtils; import org.apache.flink.kubernetes.operator.utils.FlinkUtils; @@ -356,7 +355,8 @@ public Configuration getObserveConfig(FlinkDeployment deployment) { private void addOperatorConfigsFromSpec(AbstractFlinkSpec spec, Configuration conf) { // Observe config should include the latest operator related settings if (spec.getFlinkConfiguration() != null) { - SpecUtils.toStringMap(spec.getFlinkConfiguration()) + spec.getFlinkConfiguration() + .asFlatMap() .forEach( (k, v) -> { if (k.startsWith(K8S_OP_CONF_PREFIX) @@ -372,8 +372,7 @@ private void applyConfigsFromCurrentSpec( AbstractFlinkSpec spec, Configuration conf, ConfigOption... configOptions) { addOperatorConfigsFromSpec(spec, conf); if (spec.getFlinkConfiguration() != null) { - var deployConfig = - Configuration.fromMap(SpecUtils.toStringMap(spec.getFlinkConfiguration())); + var deployConfig = Configuration.fromMap(spec.getFlinkConfiguration().asFlatMap()); for (ConfigOption configOption : configOptions) { deployConfig.getOptional(configOption).ifPresent(v -> conf.set(configOption, v)); } @@ -396,8 +395,7 @@ public Configuration getSessionJobConfig( // merge session job specific config var sessionJobFlinkConfiguration = sessionJobSpec.getFlinkConfiguration(); if (sessionJobFlinkConfiguration != null) { - SpecUtils.toStringMap(sessionJobFlinkConfiguration) - .forEach(sessionJobConfig::setString); + sessionJobFlinkConfiguration.asFlatMap().forEach(sessionJobConfig::setString); } applyJobConfig(name, sessionJobConfig, sessionJobSpec.getJob()); return sessionJobConfig; diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/diff/ReflectiveDiffBuilder.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/diff/ReflectiveDiffBuilder.java index cdfc2cb467..77339adc0a 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/diff/ReflectiveDiffBuilder.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/diff/ReflectiveDiffBuilder.java @@ -21,11 +21,10 @@ import org.apache.flink.kubernetes.operator.api.diff.DiffType; import org.apache.flink.kubernetes.operator.api.diff.Diffable; import org.apache.flink.kubernetes.operator.api.diff.SpecDiff; +import org.apache.flink.kubernetes.operator.api.spec.ConfigJsonNode; import org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec; import org.apache.flink.kubernetes.operator.api.spec.KubernetesDeploymentMode; -import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; -import com.fasterxml.jackson.databind.JsonNode; import io.fabric8.kubernetes.api.model.PodTemplateSpec; import lombok.NonNull; import org.apache.commons.lang3.ClassUtils; @@ -87,13 +86,13 @@ private void appendFields(final Class clazz) { var leftField = readField(field, before, true); var rightField = readField(field, after, true); if (field.getName().equals(FLINK_CONFIGURATION_PROPERTY_NAME)) { - leftField = SpecUtils.toStringMap((JsonNode) leftField); - rightField = SpecUtils.toStringMap((JsonNode) rightField); + leftField = ((ConfigJsonNode) leftField).asFlatMap(); + rightField = ((ConfigJsonNode) rightField).asFlatMap(); } if (field.isAnnotationPresent(SpecDiff.Config.class) && (Map.class.isAssignableFrom(field.getType()) - || (field.getType().equals(JsonNode.class) + || (field.getType().equals(ConfigJsonNode.class) && field.getName().equals("flinkConfiguration")))) { diffBuilder.append( field.getName(), diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/validation/DefaultValidator.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/validation/DefaultValidator.java index 57d3887665..f05f680359 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/validation/DefaultValidator.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/validation/DefaultValidator.java @@ -41,7 +41,6 @@ import org.apache.flink.kubernetes.operator.api.spec.TaskManagerSpec; import org.apache.flink.kubernetes.operator.api.spec.UpgradeMode; import org.apache.flink.kubernetes.operator.api.status.FlinkStateSnapshotStatus; -import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.apache.flink.kubernetes.operator.config.FlinkConfigBuilder; import org.apache.flink.kubernetes.operator.config.FlinkConfigManager; import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; @@ -103,7 +102,7 @@ public Optional validateDeployment(FlinkDeployment deployment) { deployment.getMetadata().getNamespace(), spec.getFlinkVersion()) .toMap(); if (spec.getFlinkConfiguration() != null) { - effectiveConfig.putAll(SpecUtils.toStringMap(spec.getFlinkConfiguration())); + effectiveConfig.putAll(spec.getFlinkConfiguration().asFlatMap()); } return firstPresent( validateDeploymentName(deployment.getMetadata().getName()), @@ -532,13 +531,11 @@ private Optional validateSessionJobWithCluster( sessionCluster.getSpec().getFlinkVersion()) .toMap(); if (sessionCluster.getSpec().getFlinkConfiguration() != null) { - effectiveConfig.putAll( - SpecUtils.toStringMap(sessionCluster.getSpec().getFlinkConfiguration())); + effectiveConfig.putAll(sessionCluster.getSpec().getFlinkConfiguration().asFlatMap()); } if (sessionJob.getSpec().getFlinkConfiguration() != null) { - effectiveConfig.putAll( - SpecUtils.toStringMap(sessionJob.getSpec().getFlinkConfiguration())); + effectiveConfig.putAll(sessionJob.getSpec().getFlinkConfiguration().asFlatMap()); } return firstPresent( diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/TestUtils.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/TestUtils.java index 2af251dc2b..aa06e303ee 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/TestUtils.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/TestUtils.java @@ -83,7 +83,6 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperties; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.params.provider.Arguments.arguments; @@ -222,7 +221,7 @@ public static Context createContextWithReadyFlinkDepl public Optional getSecondaryResource(Class expectedType, String eventSourceName) { var session = buildSessionCluster(version); session.getStatus().setJobManagerDeploymentStatus(JobManagerDeploymentStatus.READY); - addConfigProperties(session.getSpec(), flinkDepConfig); + session.getSpec().getFlinkConfiguration().putAllFrom(flinkDepConfig); session.getStatus() .getReconciliationStatus() .serializeAndSetLastReconciledSpec(session.getSpec(), session); @@ -441,7 +440,7 @@ public static Instant setupCronTrigger(SnapshotType snapshotType, FlinkDeploymen throw new IllegalArgumentException("Unsupported snapshot type: " + snapshotType); } - addConfigProperties(deployment.getSpec(), Map.of(cronOptionKey, "0 0 12 5 6 ? 2022")); + deployment.getSpec().getFlinkConfiguration().put(cronOptionKey, "0 0 12 5 6 ? 2022"); reconcileSpec(deployment); return calendar.toInstant(); } diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/autoscaler/KubernetesScalingRealizerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/autoscaler/KubernetesScalingRealizerTest.java index 58459b2115..f0ecd9bea2 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/autoscaler/KubernetesScalingRealizerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/autoscaler/KubernetesScalingRealizerTest.java @@ -23,15 +23,12 @@ import org.apache.flink.configuration.PipelineOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.kubernetes.operator.api.FlinkDeployment; -import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.junit.jupiter.api.Test; import java.util.LinkedHashMap; import java.util.Map; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.removeProperty; import static org.assertj.core.api.Assertions.assertThat; /** Tests for KubernetesScalingRealizer. */ @@ -49,7 +46,10 @@ public void testApplyOverrides() { .realizeParallelismOverrides(ctx, Map.of("a", "1", "b", "2")); assertThat( - SpecUtils.toStringMap(ctx.getResource().getSpec().getFlinkConfiguration()) + ctx.getResource() + .getSpec() + .getFlinkConfiguration() + .asFlatMap() .get(PipelineOptions.PARALLELISM_OVERRIDES.key())) .satisfiesAnyOf( // Currently no enforced order inside the overrides string @@ -96,19 +96,25 @@ private void assertOverridesDoNotChange( FlinkDeployment resource = (FlinkDeployment) ctx.getResource(); // Create resource with existing parallelism overrides - addConfigProperty( - resource.getSpec(), PipelineOptions.PARALLELISM_OVERRIDES.key(), currentOverrides); + + resource.getSpec() + .getFlinkConfiguration() + .put(PipelineOptions.PARALLELISM_OVERRIDES.key(), currentOverrides); resource.getStatus() .getReconciliationStatus() .serializeAndSetLastReconciledSpec(resource.getSpec(), resource); - removeProperty( - resource.getSpec().getFlinkConfiguration(), - PipelineOptions.PARALLELISM_OVERRIDES.key()); + + resource.getSpec() + .getFlinkConfiguration() + .remove(PipelineOptions.PARALLELISM_OVERRIDES.key()); new KubernetesScalingRealizer().realizeParallelismOverrides(ctx, newOverrides); assertThat( - SpecUtils.toStringMap(ctx.getResource().getSpec().getFlinkConfiguration()) + ctx.getResource() + .getSpec() + .getFlinkConfiguration() + .asFlatMap() .get(PipelineOptions.PARALLELISM_OVERRIDES.key())) .isEqualTo(currentOverrides); } diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilderTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilderTest.java index db8f71874a..e55934136f 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilderTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilderTest.java @@ -40,7 +40,6 @@ import org.apache.flink.kubernetes.operator.api.spec.Resource; import org.apache.flink.kubernetes.operator.api.spec.TaskManagerSpec; import org.apache.flink.kubernetes.operator.api.spec.UpgradeMode; -import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; import org.apache.flink.kubernetes.operator.standalone.StandaloneKubernetesConfigOptionsInternal; import org.apache.flink.kubernetes.utils.Constants; @@ -127,12 +126,11 @@ public void testApplyFlinkConfiguration() { FlinkDeployment deployment = ReconciliationUtils.clone(flinkDeployment); deployment .getSpec() - .setFlinkConfiguration( - SpecUtils.mapToJsonNode( - Map.of( - KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE.key(), - KubernetesConfigOptions.ServiceExposedType.LoadBalancer - .name()))); + .getFlinkConfiguration() + .putAllFrom( + Map.of( + KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE.key(), + KubernetesConfigOptions.ServiceExposedType.LoadBalancer.name())); configuration = new FlinkConfigBuilder(deployment, new Configuration()) @@ -718,7 +716,7 @@ public void testApplyJobOrSessionSpec() throws Exception { var dep = ReconciliationUtils.clone(deploymentClone); dep.getSpec().setTaskManager(new TaskManagerSpec()); dep.getSpec().getTaskManager().setReplicas(3); - SpecUtils.addConfigProperty(dep.getSpec(), TaskManagerOptions.NUM_TASK_SLOTS.key(), "4"); + dep.getSpec().getFlinkConfiguration().put(TaskManagerOptions.NUM_TASK_SLOTS.key(), "4"); configuration = new FlinkConfigBuilder(dep, new Configuration()) .applyFlinkConfiguration() @@ -760,10 +758,11 @@ public void testApplyJobOrSessionSpecWithNoJar() throws Exception { @Test public void testAllowNonRestoredStateInSpecOverrideInFlinkConf() throws URISyntaxException { flinkDeployment.getSpec().getJob().setAllowNonRestoredState(false); - SpecUtils.addConfigProperty( - flinkDeployment.getSpec(), - SavepointConfigOptions.SAVEPOINT_IGNORE_UNCLAIMED_STATE.key(), - "true"); + + flinkDeployment + .getSpec() + .getFlinkConfiguration() + .put(SavepointConfigOptions.SAVEPOINT_IGNORE_UNCLAIMED_STATE.key(), "true"); Configuration configuration = new FlinkConfigBuilder(flinkDeployment, new Configuration()) .applyJobOrSessionSpec() @@ -772,10 +771,11 @@ public void testAllowNonRestoredStateInSpecOverrideInFlinkConf() throws URISynta configuration.getBoolean(SavepointConfigOptions.SAVEPOINT_IGNORE_UNCLAIMED_STATE)); flinkDeployment.getSpec().getJob().setAllowNonRestoredState(true); - SpecUtils.addConfigProperty( - flinkDeployment.getSpec(), - SavepointConfigOptions.SAVEPOINT_IGNORE_UNCLAIMED_STATE.key(), - "false"); + + flinkDeployment + .getSpec() + .getFlinkConfiguration() + .put(SavepointConfigOptions.SAVEPOINT_IGNORE_UNCLAIMED_STATE.key(), "false"); configuration = new FlinkConfigBuilder(flinkDeployment, new Configuration()) .applyJobOrSessionSpec() @@ -795,7 +795,7 @@ public void testApplyStandaloneApplicationSpec() throws URISyntaxException, IOEx dep.getSpec().getJob().setJarURI(jarUri); dep.getSpec().setTaskManager(new TaskManagerSpec()); dep.getSpec().getTaskManager().setReplicas(3); - SpecUtils.addConfigProperty(dep.getSpec(), TaskManagerOptions.NUM_TASK_SLOTS.key(), "2"); + dep.getSpec().getFlinkConfiguration().put(TaskManagerOptions.NUM_TASK_SLOTS.key(), "2"); Configuration configuration = new FlinkConfigBuilder(dep, new Configuration()) @@ -835,10 +835,9 @@ public void testApplyStandaloneApplicationSpec() throws URISyntaxException, IOEx configuration.get( StandaloneKubernetesConfigOptionsInternal.KUBERNETES_TASKMANAGER_REPLICAS)); - SpecUtils.addConfigProperty( - dep.getSpec(), - PipelineOptions.PARALLELISM_OVERRIDES.key(), - "vertex1:10,vertex2:20"); + dep.getSpec() + .getFlinkConfiguration() + .put(PipelineOptions.PARALLELISM_OVERRIDES.key(), "vertex1:10,vertex2:20"); configuration = new FlinkConfigBuilder(dep, new Configuration()) .applyFlinkConfiguration() @@ -858,7 +857,7 @@ public void testApplyStandaloneSessionSpec() throws URISyntaxException, IOExcept dep.getSpec().setJob(null); dep.getSpec().setTaskManager(new TaskManagerSpec()); dep.getSpec().getTaskManager().setReplicas(5); - SpecUtils.addConfigProperty(dep.getSpec(), TaskManagerOptions.NUM_TASK_SLOTS.key(), "2"); + dep.getSpec().getFlinkConfiguration().put(TaskManagerOptions.NUM_TASK_SLOTS.key(), "2"); Configuration configuration = new FlinkConfigBuilder(dep, new Configuration()) diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManagerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManagerTest.java index d00c0100a1..39414af44c 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManagerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManagerTest.java @@ -52,8 +52,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Matcher; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperties; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.OPERATOR_WATCHED_NAMESPACES; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -80,21 +78,24 @@ public void testConfigGeneration() { FlinkDeployment deployment = TestUtils.buildApplicationCluster(); var reconciliationStatus = deployment.getStatus().getReconciliationStatus(); - addConfigProperties( - deployment.getSpec(), - Map.of(testConf.key(), "reconciled", opTestConf.key(), "reconciled")); + deployment + .getSpec() + .getFlinkConfiguration() + .putAllFrom(Map.of(testConf.key(), "reconciled", opTestConf.key(), "reconciled")); reconciliationStatus.serializeAndSetLastReconciledSpec(deployment.getSpec(), deployment); reconciliationStatus.markReconciledSpecAsStable(); - addConfigProperties( - deployment.getSpec(), - Map.of( - testConf.key(), - "latest", - opTestConf.key(), - "latest", - AutoScalerOptions.METRICS_WINDOW.key(), - "1234m")); + deployment + .getSpec() + .getFlinkConfiguration() + .putAllFrom( + Map.of( + testConf.key(), + "latest", + opTestConf.key(), + "latest", + AutoScalerOptions.METRICS_WINDOW.key(), + "1234m")); assertEquals( "latest", @@ -112,11 +113,11 @@ public void testConfigGeneration() { Duration.ofMinutes(1234), configManager.getObserveConfig(deployment).get(AutoScalerOptions.METRICS_WINDOW)); - addConfigProperty(deployment.getSpec(), testConf.key(), "stable"); + deployment.getSpec().getFlinkConfiguration().put(testConf.key(), "stable"); reconciliationStatus.serializeAndSetLastReconciledSpec(deployment.getSpec(), deployment); reconciliationStatus.markReconciledSpecAsStable(); - addConfigProperty(deployment.getSpec(), testConf.key(), "rolled-back"); + deployment.getSpec().getFlinkConfiguration().put(testConf.key(), "rolled-back"); reconciliationStatus.serializeAndSetLastReconciledSpec(deployment.getSpec(), deployment); reconciliationStatus.setState(ReconciliationState.ROLLED_BACK); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/DeploymentRecoveryTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/DeploymentRecoveryTest.java index e72d22cf3c..881462fda0 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/DeploymentRecoveryTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/DeploymentRecoveryTest.java @@ -25,7 +25,6 @@ import org.apache.flink.kubernetes.operator.api.spec.FlinkVersion; import org.apache.flink.kubernetes.operator.api.spec.UpgradeMode; import org.apache.flink.kubernetes.operator.api.status.JobManagerDeploymentStatus; -import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.apache.flink.kubernetes.operator.config.FlinkConfigManager; import io.fabric8.kubernetes.client.KubernetesClient; @@ -178,8 +177,11 @@ public void verifyRecoveryWithoutHaData(FlinkVersion flinkVersion, UpgradeMode u // We disable HA for stateless to test recovery without HA metadata if (upgradeMode == UpgradeMode.STATELESS) { - SpecUtils.addConfigProperty( - appCluster.getSpec(), HighAvailabilityOptions.HA_MODE.key(), "none"); + + appCluster + .getSpec() + .getFlinkConfiguration() + .put(HighAvailabilityOptions.HA_MODE.key(), "none"); } testController.reconcile(appCluster, context); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentControllerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentControllerTest.java index dcaa452fcc..c9c18b46b4 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentControllerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentControllerTest.java @@ -65,7 +65,6 @@ import java.util.stream.Collectors; import static org.apache.flink.kubernetes.operator.TestUtils.MAX_RECONCILE_TIMES; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.OPERATOR_JOB_UPGRADE_LAST_STATE_FALLBACK_ENABLED; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.SNAPSHOT_RESOURCE_ENABLED; import static org.apache.flink.kubernetes.operator.utils.EventRecorder.Reason.ValidationError; @@ -198,10 +197,13 @@ public void verifyReconcileLoopForInitialSuspendedDeploymentWithSavepoint( TestUtils.buildApplicationCluster(flinkVersion, JobState.SUSPENDED); appCluster.getSpec().getJob().setUpgradeMode(UpgradeMode.SAVEPOINT); appCluster.getSpec().getJob().setInitialSavepointPath("s0"); - addConfigProperty( - appCluster.getSpec(), - CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), - "file:///flink-data/savepoints"); + + appCluster + .getSpec() + .getFlinkConfiguration() + .put( + CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), + "file:///flink-data/savepoints"); int reconcileTimes = 0; while (reconcileTimes < MAX_RECONCILE_TIMES) { @@ -303,11 +305,14 @@ public void verifyUpgradeFromSavepointLegacyMode(FlinkVersion flinkVersion) thro FlinkDeployment appCluster = TestUtils.buildApplicationCluster(flinkVersion); appCluster.getSpec().getJob().setUpgradeMode(UpgradeMode.SAVEPOINT); appCluster.getSpec().getJob().setInitialSavepointPath("s0"); - addConfigProperty( - appCluster.getSpec(), - CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), - "file:///flink-data/savepoints"); - addConfigProperty(appCluster.getSpec(), SNAPSHOT_RESOURCE_ENABLED.key(), "false"); + + appCluster + .getSpec() + .getFlinkConfiguration() + .put( + CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), + "file:///flink-data/savepoints"); + appCluster.getSpec().getFlinkConfiguration().put(SNAPSHOT_RESOURCE_ENABLED.key(), "false"); testController.reconcile(appCluster, context); var jobs = flinkService.listJobs(); assertEquals(1, jobs.size()); @@ -559,7 +564,7 @@ public void verifyReconcileWithBadConfig() throws Exception { UpdateControl updateControl; // Override rest port, and it should be saved in lastReconciledSpec once a successful // reconcile() finishes. - addConfigProperty(appCluster.getSpec(), RestOptions.PORT.key(), "8088"); + appCluster.getSpec().getFlinkConfiguration().put(RestOptions.PORT.key(), "8088"); updateControl = testController.reconcile(appCluster, context); assertFalse(updateControl.isPatchStatus()); assertEquals( @@ -585,7 +590,7 @@ public void verifyReconcileWithBadConfig() throws Exception { appCluster.getSpec().getJob().setParallelism(0); // Verify the saved rest port in lastReconciledSpec is actually used in observe() by // utilizing listJobConsumer - addConfigProperty(appCluster.getSpec(), RestOptions.PORT.key(), "12345"); + appCluster.getSpec().getFlinkConfiguration().put(RestOptions.PORT.key(), "12345"); flinkService.setListJobConsumer( (configuration) -> assertEquals(8088, configuration.get(RestOptions.PORT))); testController.reconcile(appCluster, context); @@ -793,10 +798,11 @@ private void testUpgradeNotReadyCluster(FlinkDeployment appCluster) throws Excep // triggering upgrade with no last-state fallback on non-healthy app flinkService.setPortReady(false); - addConfigProperty( - appCluster.getSpec(), - OPERATOR_JOB_UPGRADE_LAST_STATE_FALLBACK_ENABLED.key(), - "false"); + + appCluster + .getSpec() + .getFlinkConfiguration() + .put(OPERATOR_JOB_UPGRADE_LAST_STATE_FALLBACK_ENABLED.key(), "false"); appCluster.getSpec().setServiceAccount(appCluster.getSpec().getServiceAccount() + "-5"); // not upgrading the cluster with no last-state fallback testController.reconcile(appCluster, context); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobControllerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobControllerTest.java index c00c43e484..f8a1f8a2f0 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobControllerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobControllerTest.java @@ -56,7 +56,6 @@ import static org.apache.flink.api.common.JobStatus.RECONCILING; import static org.apache.flink.api.common.JobStatus.RUNNING; import static org.apache.flink.kubernetes.operator.TestUtils.MAX_RECONCILE_TIMES; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.SNAPSHOT_RESOURCE_ENABLED; import static org.apache.flink.kubernetes.operator.utils.EventRecorder.Reason.ValidationError; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -195,7 +194,7 @@ public void verifyUpgradeFromSavepointLegacy() throws Exception { sessionJob.getSpec().getJob().setUpgradeMode(UpgradeMode.SAVEPOINT); sessionJob.getSpec().getJob().setInitialSavepointPath("s0"); - addConfigProperty(sessionJob.getSpec(), SNAPSHOT_RESOURCE_ENABLED.key(), "false"); + sessionJob.getSpec().getFlinkConfiguration().put(SNAPSHOT_RESOURCE_ENABLED.key(), "false"); testController.reconcile(sessionJob, context); var jobs = flinkService.listJobs(); assertEquals(1, jobs.size()); @@ -517,10 +516,11 @@ public void verifyReconcileWithBadConfig() throws Exception { UpdateControl updateControl; // Override headers, and it should be saved in lastReconciledSpec once a successful // reconcile() finishes. - addConfigProperty( - sessionJob.getSpec(), - KubernetesOperatorConfigOptions.JAR_ARTIFACT_HTTP_HEADER.key(), - "changed"); + + sessionJob + .getSpec() + .getFlinkConfiguration() + .put(KubernetesOperatorConfigOptions.JAR_ARTIFACT_HTTP_HEADER.key(), "changed"); updateControl = testController.reconcile(sessionJob, context); assertFalse(updateControl.isPatchStatus()); assertEquals(RECONCILING, sessionJob.getStatus().getJobStatus().getState()); @@ -540,10 +540,11 @@ public void verifyReconcileWithBadConfig() throws Exception { // Make sure we do validation before getting effective config in reconcile(). // Verify the saved headers in lastReconciledSpec is actually used in observe() by // utilizing listJobConsumer - addConfigProperty( - sessionJob.getSpec(), - KubernetesOperatorConfigOptions.JAR_ARTIFACT_HTTP_HEADER.key(), - "again"); + + sessionJob + .getSpec() + .getFlinkConfiguration() + .put(KubernetesOperatorConfigOptions.JAR_ARTIFACT_HTTP_HEADER.key(), "again"); flinkService.setListJobConsumer( (configuration) -> assertEquals( diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkStateSnapshotControllerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkStateSnapshotControllerTest.java index 5fb321a47a..2ecf8762a6 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkStateSnapshotControllerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkStateSnapshotControllerTest.java @@ -77,7 +77,6 @@ import static org.apache.flink.kubernetes.operator.api.status.FlinkStateSnapshotStatus.State.FAILED; import static org.apache.flink.kubernetes.operator.api.status.FlinkStateSnapshotStatus.State.IN_PROGRESS; import static org.apache.flink.kubernetes.operator.api.status.FlinkStateSnapshotStatus.State.TRIGGER_PENDING; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.removeConfigProperties; import static org.apache.flink.kubernetes.operator.metrics.FlinkStateSnapshotMetricsUtils.assertSnapshotMetrics; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -751,8 +750,11 @@ private FlinkDeployment createDeployment(FlinkVersion flinkVersion) { .getStatus() .setJobStatus(JobStatus.builder().state(RUNNING).jobId(JOB_ID).build()); deployment.getSpec().setFlinkVersion(flinkVersion); - removeConfigProperties( - deployment.getSpec(), CheckpointingOptions.SAVEPOINT_DIRECTORY.key()); + + deployment + .getSpec() + .getFlinkConfiguration() + .remove(CheckpointingOptions.SAVEPOINT_DIRECTORY.key()); ReconciliationUtils.updateStatusForDeployedSpec(deployment, new Configuration()); client.resource(deployment).create(); return deployment; diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/RollbackTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/RollbackTest.java index 92a4622f4e..6be1729cdd 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/RollbackTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/RollbackTest.java @@ -27,7 +27,6 @@ import org.apache.flink.kubernetes.operator.api.status.ReconciliationState; import org.apache.flink.kubernetes.operator.api.status.Savepoint; import org.apache.flink.kubernetes.operator.api.status.SnapshotTriggerType; -import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.apache.flink.kubernetes.operator.config.FlinkConfigManager; import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; import org.apache.flink.kubernetes.operator.reconciler.deployment.AbstractFlinkResourceReconciler; @@ -47,8 +46,6 @@ import java.util.Map; import static org.apache.flink.api.common.JobStatus.RUNNING; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperties; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotEquals; @@ -93,7 +90,7 @@ public void testStatefulRollback(UpgradeMode upgradeMode) throws Exception { dep, () -> { dep.getSpec().getJob().setParallelism(9999); - addConfigProperty(dep.getSpec(), "test.deploy.config", "roll_back"); + dep.getSpec().getFlinkConfiguration().put("test.deploy.config", "roll_back"); testController.reconcile(dep, context); assertEquals( JobState.SUSPENDED, @@ -133,7 +130,7 @@ public void testSavepointRollbackWithoutHaMetadata() throws Exception { dep, () -> { dep.getSpec().getJob().setParallelism(9999); - addConfigProperty(dep.getSpec(), "test.deploy.config", "roll_back"); + dep.getSpec().getFlinkConfiguration().put("test.deploy.config", "roll_back"); testController.reconcile(dep, context); assertEquals( JobState.SUSPENDED, @@ -174,11 +171,16 @@ public void testSavepointNoRollbackWithoutHaMetadataAndJMWasReady() throws Excep offsetReconcilerClock(deployment, Duration.ZERO); var flinkConfiguration = deployment.getSpec().getFlinkConfiguration(); - addConfigProperties( - deployment.getSpec(), - Map.of( - KubernetesOperatorConfigOptions.DEPLOYMENT_ROLLBACK_ENABLED.key(), "true", - KubernetesOperatorConfigOptions.DEPLOYMENT_READINESS_TIMEOUT.key(), "10s")); + + deployment + .getSpec() + .getFlinkConfiguration() + .putAllFrom( + Map.of( + KubernetesOperatorConfigOptions.DEPLOYMENT_ROLLBACK_ENABLED.key(), + "true", + KubernetesOperatorConfigOptions.DEPLOYMENT_READINESS_TIMEOUT.key(), + "10s")); testController.reconcile(deployment, context); @@ -220,15 +222,15 @@ public void testRollbackFailureWithLastState() throws Exception { var dep = TestUtils.buildApplicationCluster(); dep.getSpec().getJob().setUpgradeMode(UpgradeMode.LAST_STATE); - addConfigProperty(dep.getSpec(), "t", "1"); + dep.getSpec().getFlinkConfiguration().put("t", "1"); offsetReconcilerClock(dep, Duration.ZERO); testRollback( dep, () -> { dep.getSpec().getJob().setParallelism(9999); - addConfigProperty(dep.getSpec(), "test.deploy.config", "roll_back"); - SpecUtils.removeConfigProperties(dep.getSpec(), "t"); + dep.getSpec().getFlinkConfiguration().put("test.deploy.config", "roll_back"); + dep.getSpec().getFlinkConfiguration().remove("t"); testController.reconcile(dep, context); assertEquals( JobState.SUSPENDED, @@ -283,12 +285,14 @@ public void testRollbackStateless() throws Exception { testRollback( dep, () -> { - addConfigProperty( - dep.getSpec(), - KubernetesOperatorConfigOptions.DEPLOYMENT_ROLLBACK_ENABLED.key(), - "false"); + dep.getSpec() + .getFlinkConfiguration() + .put( + KubernetesOperatorConfigOptions.DEPLOYMENT_ROLLBACK_ENABLED + .key(), + "false"); dep.getSpec().getJob().setParallelism(9999); - addConfigProperty(dep.getSpec(), "test.deploy.config", "roll_back"); + dep.getSpec().getFlinkConfiguration().put("test.deploy.config", "roll_back"); testController.reconcile(dep, context); assertEquals( JobState.SUSPENDED, @@ -304,10 +308,13 @@ public void testRollbackStateless() throws Exception { .getSubmittedConf() .getString("test.deploy.config", "unknown")); // Validate that rollback config is picked up from latest deploy conf - addConfigProperty( - dep.getSpec(), - KubernetesOperatorConfigOptions.DEPLOYMENT_ROLLBACK_ENABLED.key(), - "true"); + + dep.getSpec() + .getFlinkConfiguration() + .put( + KubernetesOperatorConfigOptions.DEPLOYMENT_ROLLBACK_ENABLED + .key(), + "true"); // Trigger rollback by delaying the recovery offsetReconcilerClock(dep, Duration.ofSeconds(15)); @@ -335,7 +342,7 @@ public void testRollbackSession() throws Exception { testRollback( dep, () -> { - addConfigProperty(dep.getSpec(), "random", "config"); + dep.getSpec().getFlinkConfiguration().put("random", "config"); testController.reconcile(dep, context); // Trigger rollback by delaying the recovery offsetReconcilerClock(dep, Duration.ofSeconds(15)); @@ -357,15 +364,17 @@ public void testRollback( boolean expectTwoStepRollback) throws Exception { - addConfigProperties( - deployment.getSpec(), - Map.of( - KubernetesOperatorConfigOptions.DEPLOYMENT_ROLLBACK_ENABLED.key(), - "true", - KubernetesOperatorConfigOptions.DEPLOYMENT_READINESS_TIMEOUT.key(), - "10s", - "test.deploy.config", - "stable")); + deployment + .getSpec() + .getFlinkConfiguration() + .putAllFrom( + Map.of( + KubernetesOperatorConfigOptions.DEPLOYMENT_ROLLBACK_ENABLED.key(), + "true", + KubernetesOperatorConfigOptions.DEPLOYMENT_READINESS_TIMEOUT.key(), + "10s", + "test.deploy.config", + "stable")); testController.reconcile(deployment, context); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserverTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserverTest.java index e4f939d061..71176201b2 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserverTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserverTest.java @@ -51,7 +51,6 @@ import java.util.Map; import java.util.stream.Stream; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -746,8 +745,11 @@ private static Stream cancellingArgs() { private static FlinkDeployment initDeployment() { FlinkDeployment deployment = TestUtils.buildApplicationCluster(); var jobId = new JobID().toHexString(); - addConfigProperty( - deployment.getSpec(), PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID.key(), jobId); + + deployment + .getSpec() + .getFlinkConfiguration() + .put(PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID.key(), jobId); deployment.getStatus().getJobStatus().setJobId(jobId); deployment .getStatus() @@ -759,8 +761,10 @@ private static FlinkDeployment initDeployment() { private static FlinkSessionJob initSessionJob() { var job = TestUtils.buildSessionJob(); var jobId = new JobID().toHexString(); - addConfigProperty( - job.getSpec(), PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID.key(), jobId); + + job.getSpec() + .getFlinkConfiguration() + .put(PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID.key(), jobId); job.getStatus().getJobStatus().setJobId(jobId); job.getStatus() diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/ApplicationObserverTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/ApplicationObserverTest.java index e3afa78724..ae49a477fe 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/ApplicationObserverTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/ApplicationObserverTest.java @@ -31,7 +31,6 @@ import org.apache.flink.kubernetes.operator.api.status.ReconciliationState; import org.apache.flink.kubernetes.operator.api.status.SavepointFormatType; import org.apache.flink.kubernetes.operator.api.status.SnapshotTriggerType; -import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.apache.flink.kubernetes.operator.config.FlinkConfigManager; import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; import org.apache.flink.kubernetes.operator.exception.DeploymentFailedException; @@ -56,13 +55,11 @@ import java.time.Duration; import java.time.Instant; import java.util.HashMap; -import java.util.Map; import java.util.stream.Collectors; import java.util.stream.Stream; import static org.apache.flink.kubernetes.operator.api.utils.FlinkResourceUtils.getCheckpointInfo; import static org.apache.flink.kubernetes.operator.api.utils.FlinkResourceUtils.getJobStatus; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.OPERATOR_SAVEPOINT_FORMAT_TYPE; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -85,8 +82,11 @@ public void setup() { readyContext = TestUtils.createContextWithReadyJobManagerDeployment(kubernetesClient); deployment = TestUtils.buildApplicationCluster(); var jobId = new JobID().toHexString(); - addConfigProperty( - deployment.getSpec(), PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID.key(), jobId); + + deployment + .getSpec() + .getFlinkConfiguration() + .put(PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID.key(), jobId); deployment.getStatus().getJobStatus().setJobId(jobId); } @@ -247,10 +247,11 @@ public void observeSavepoint() throws Exception { deployment.getSpec().getJob().setSavepointTriggerNonce(timedOutNonce); Configuration conf = configManager.getDeployConfig(deployment.getMetadata(), deployment.getSpec()); - addConfigProperty( - deployment.getSpec(), - KubernetesOperatorConfigOptions.SNAPSHOT_RESOURCE_ENABLED.key(), - "false"); + + deployment + .getSpec() + .getFlinkConfiguration() + .put(KubernetesOperatorConfigOptions.SNAPSHOT_RESOURCE_ENABLED.key(), "false"); flinkService.submitApplicationCluster(deployment.getSpec().getJob(), conf, false); bringToReadyStatus(deployment); assertTrue(ReconciliationUtils.isJobRunning(deployment.getStatus())); @@ -512,10 +513,12 @@ public void observeSavepoint() throws Exception { .getSavepointHistory() .size()); - addConfigProperty( - deployment.getSpec(), - KubernetesOperatorConfigOptions.OPERATOR_SAVEPOINT_HISTORY_MAX_COUNT.key(), - "1"); + deployment + .getSpec() + .getFlinkConfiguration() + .put( + KubernetesOperatorConfigOptions.OPERATOR_SAVEPOINT_HISTORY_MAX_COUNT.key(), + "1"); observer.observe(deployment, readyContext); assertEquals( 1, @@ -715,12 +718,10 @@ public void testSavepointFormat() throws Exception { deployment.getSpec().getJob().setSavepointTriggerNonce(secondNonce); deployment .getSpec() - .setFlinkConfiguration( - SpecUtils.mapToJsonNode( - Map.of( - OPERATOR_SAVEPOINT_FORMAT_TYPE.key(), - org.apache.flink.core.execution.SavepointFormatType.NATIVE - .name()))); + .getFlinkConfiguration() + .put( + OPERATOR_SAVEPOINT_FORMAT_TYPE.key(), + org.apache.flink.core.execution.SavepointFormatType.NATIVE.name()); conf = configManager.getDeployConfig(deployment.getMetadata(), deployment.getSpec()); flinkService.triggerSavepointLegacy( deployment.getStatus().getJobStatus().getJobId(), diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/SessionObserverTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/SessionObserverTest.java index 66cd6874ae..cfb1910cb5 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/SessionObserverTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/SessionObserverTest.java @@ -23,7 +23,6 @@ import org.apache.flink.kubernetes.operator.api.FlinkDeployment; import org.apache.flink.kubernetes.operator.api.status.JobManagerDeploymentStatus; import org.apache.flink.kubernetes.operator.api.status.ReconciliationState; -import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.apache.flink.kubernetes.operator.config.FlinkConfigManager; import org.apache.flink.kubernetes.operator.observer.TestObserverAdapter; import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; @@ -119,7 +118,7 @@ public void observeAlreadyUpgraded() { ReconciliationUtils.updateStatusForDeployedSpec(deployment, new Configuration()); // Test regular upgrades - SpecUtils.addConfigProperty(deployment.getSpec(), "k", "1"); + deployment.getSpec().getFlinkConfiguration().put("k", "1"); deployment.getMetadata().setGeneration(321L); ReconciliationUtils.updateStatusBeforeDeploymentAttempt( deployment, @@ -143,7 +142,7 @@ public void observeAlreadyUpgraded() { .put(FlinkUtils.CR_GENERATION_LABEL, "321"); deployment.getMetadata().setGeneration(322L); - SpecUtils.addConfigProperty(deployment.getSpec(), "k", "2"); + deployment.getSpec().getFlinkConfiguration().put("k", "2"); observer.observe(deployment, context); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerTest.java index 108802e400..c13c497a19 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerTest.java @@ -124,8 +124,6 @@ import static org.apache.flink.kubernetes.operator.api.utils.FlinkResourceUtils.getReconciledJobSpec; import static org.apache.flink.kubernetes.operator.api.utils.FlinkResourceUtils.getReconciledJobState; import static org.apache.flink.kubernetes.operator.api.utils.FlinkResourceUtils.getSavepointInfo; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperties; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.OPERATOR_CLUSTER_HEALTH_CHECK_ENABLED; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.OPERATOR_JOB_SAVEPOINT_DISPOSE_ON_DELETE; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.SNAPSHOT_RESOURCE_ENABLED; @@ -209,10 +207,11 @@ public void testSubmitAndCleanUpWithSavepoint(FlinkVersion flinkVersion) throws public void testSubmitAndCleanUpWithSavepointOnResource(FlinkVersion flinkVersion) throws Exception { FlinkDeployment deployment = TestUtils.buildApplicationCluster(flinkVersion); - addConfigProperty( - deployment.getSpec(), - KubernetesOperatorConfigOptions.SAVEPOINT_ON_DELETION.key(), - "true"); + + deployment + .getSpec() + .getFlinkConfiguration() + .put(KubernetesOperatorConfigOptions.SAVEPOINT_ON_DELETION.key(), "true"); // session ready reconciler.reconcile( @@ -272,7 +271,7 @@ public void testUpgrade(FlinkVersion flinkVersion, boolean snapshotResource) thr // Test stateless upgrade FlinkDeployment statelessUpgrade = ReconciliationUtils.clone(deployment); getJobSpec(statelessUpgrade).setUpgradeMode(UpgradeMode.STATELESS); - addConfigProperty(statelessUpgrade.getSpec(), "new", "conf"); + statelessUpgrade.getSpec().getFlinkConfiguration().put("new", "conf"); reconciler.reconcile(statelessUpgrade, context); assertFalse( statelessUpgrade @@ -306,7 +305,7 @@ public void testUpgrade(FlinkVersion flinkVersion, boolean snapshotResource) thr // Test stateful upgrade FlinkDeployment statefulUpgrade = ReconciliationUtils.clone(deployment); getJobSpec(statefulUpgrade).setUpgradeMode(UpgradeMode.SAVEPOINT); - addConfigProperty(statefulUpgrade.getSpec(), "new", "conf2"); + statefulUpgrade.getSpec().getFlinkConfiguration().put("new", "conf2"); reconciler.reconcile(statefulUpgrade, context); @@ -428,14 +427,14 @@ private static Savepoint savepointFromSavepointInfo( @Test public void triggerCheckpointLegacy() throws Exception { FlinkDeployment deployment = TestUtils.buildApplicationCluster(); - addConfigProperty(deployment.getSpec(), SNAPSHOT_RESOURCE_ENABLED.key(), "false"); + deployment.getSpec().getFlinkConfiguration().put(SNAPSHOT_RESOURCE_ENABLED.key(), "false"); testSnapshotLegacy(deployment, CHECKPOINT); } @Test public void triggerSavepointLegacy() throws Exception { FlinkDeployment deployment = TestUtils.buildApplicationCluster(); - addConfigProperty(deployment.getSpec(), SNAPSHOT_RESOURCE_ENABLED.key(), "false"); + deployment.getSpec().getFlinkConfiguration().put(SNAPSHOT_RESOURCE_ENABLED.key(), "false"); testSnapshotLegacy(deployment, SAVEPOINT); } @@ -444,8 +443,11 @@ public void triggerSavepointLegacy() throws Exception { public void triggerSavepointWithSnapshotResource(boolean disposeOnDelete) throws Exception { var deployment = TestUtils.buildApplicationCluster(); if (disposeOnDelete) { - addConfigProperty( - deployment.getSpec(), OPERATOR_JOB_SAVEPOINT_DISPOSE_ON_DELETE.key(), "true"); + + deployment + .getSpec() + .getFlinkConfiguration() + .put(OPERATOR_JOB_SAVEPOINT_DISPOSE_ON_DELETE.key(), "true"); } reconciler.reconcile(deployment, context); @@ -666,11 +668,11 @@ private void testSnapshotLegacy(FlinkDeployment deployment, SnapshotType snapsho assertFalse(isSnapshotInProgress.test(getJobStatus(snDeployment))); // trigger by periodic interval settings - addConfigProperty(snDeployment.getSpec(), triggerSnapshotExpression.key(), "1"); + snDeployment.getSpec().getFlinkConfiguration().put(triggerSnapshotExpression.key(), "1"); reconciler.reconcile(snDeployment, context); assertTrue(isSnapshotInProgress.test(getJobStatus(snDeployment))); assertEquals(SnapshotStatus.PENDING, getLastSnapshotStatus(snDeployment, snapshotType)); - addConfigProperty(snDeployment.getSpec(), triggerSnapshotExpression.key(), "0"); + snDeployment.getSpec().getFlinkConfiguration().put(triggerSnapshotExpression.key(), "0"); } @NotNull @@ -740,7 +742,7 @@ private void verifyAndSetRunningJobsToStatus( @Test public void testJobUpgradeIgnorePendingSavepointLegacy() throws Exception { FlinkDeployment deployment = TestUtils.buildApplicationCluster(); - addConfigProperty(deployment.getSpec(), SNAPSHOT_RESOURCE_ENABLED.key(), "false"); + deployment.getSpec().getFlinkConfiguration().put(SNAPSHOT_RESOURCE_ENABLED.key(), "false"); reconciler.reconcile(deployment, context); var runningJobs = flinkService.listJobs(); @@ -755,10 +757,13 @@ public void testJobUpgradeIgnorePendingSavepointLegacy() throws Exception { getJobStatus(spDeployment).getState()); // Force upgrade when savepoint is in progress. - addConfigProperty( - spDeployment.getSpec(), - KubernetesOperatorConfigOptions.JOB_UPGRADE_IGNORE_PENDING_SAVEPOINT.key(), - "true"); + + spDeployment + .getSpec() + .getFlinkConfiguration() + .put( + KubernetesOperatorConfigOptions.JOB_UPGRADE_IGNORE_PENDING_SAVEPOINT.key(), + "true"); spDeployment.getSpec().setImage("flink:greatest"); reconciler.reconcile(spDeployment, context); assertEquals("savepoint_trigger_0", getSavepointInfo(spDeployment).getTriggerId()); @@ -771,8 +776,10 @@ public void testJobUpgradeIgnorePendingSavepointLegacy() throws Exception { public void testRandomJobResultStorePath() throws Exception { FlinkDeployment flinkApp = TestUtils.buildApplicationCluster(); final String haStoragePath = "file:///flink-data/ha"; - addConfigProperty( - flinkApp.getSpec(), HighAvailabilityOptions.HA_STORAGE_PATH.key(), haStoragePath); + + flinkApp.getSpec() + .getFlinkConfiguration() + .put(HighAvailabilityOptions.HA_STORAGE_PATH.key(), haStoragePath); ObjectMeta deployMeta = flinkApp.getMetadata(); FlinkDeploymentStatus status = flinkApp.getStatus(); @@ -840,16 +847,22 @@ public void testScaleWithReactiveModeEnabled() throws Exception { FlinkDeployment deployment = TestUtils.buildApplicationCluster(); deployment.getSpec().setMode(KubernetesDeploymentMode.STANDALONE); - addConfigProperty( - deployment.getSpec(), - JobManagerOptions.SCHEDULER_MODE.key(), - SchedulerExecutionMode.REACTIVE.name()); + + deployment + .getSpec() + .getFlinkConfiguration() + .put( + JobManagerOptions.SCHEDULER_MODE.key(), + SchedulerExecutionMode.REACTIVE.name()); reconciler.reconcile(deployment, context); verifyAndSetRunningJobsToStatus(deployment, flinkService.listJobs()); // the default.parallelism is always ignored - addConfigProperty(deployment.getSpec(), CoreOptions.DEFAULT_PARALLELISM.key(), "100"); + deployment + .getSpec() + .getFlinkConfiguration() + .put(CoreOptions.DEFAULT_PARALLELISM.key(), "100"); reconciler.reconcile(deployment, context); assertEquals(JobState.RUNNING, getReconciledJobState(deployment)); @@ -914,10 +927,13 @@ public CancelResult cancelJob( // Set all the properties required by the rescale api deployment.getSpec().setFlinkVersion(FlinkVersion.v1_18); deployment.getSpec().setMode(KubernetesDeploymentMode.NATIVE); - addConfigProperty( - deployment.getSpec(), - JobManagerOptions.SCHEDULER.key(), - JobManagerOptions.SchedulerType.Adaptive.name()); + + deployment + .getSpec() + .getFlinkConfiguration() + .put( + JobManagerOptions.SCHEDULER.key(), + JobManagerOptions.SchedulerType.Adaptive.name()); deployment.getMetadata().setGeneration(1L); // Deploy the job and update the status accordingly so we can proceed to rescaling it @@ -925,10 +941,11 @@ public CancelResult cancelJob( verifyAndSetRunningJobsToStatus(deployment, flinkService.listJobs()); // Override parallelism for a vertex and trigger rescaling - addConfigProperty( - deployment.getSpec(), - PipelineOptions.PARALLELISM_OVERRIDES.key(), - v1.toHexString() + ":2"); + + deployment + .getSpec() + .getFlinkConfiguration() + .put(PipelineOptions.PARALLELISM_OVERRIDES.key(), v1.toHexString() + ":2"); deployment.getMetadata().setGeneration(2L); appReconciler.reconcile(ctxFactory.getResourceContext(deployment, context)); assertEquals(1, rescaleCounter.get()); @@ -981,13 +998,16 @@ public void cleanup(KubernetesJobAutoScalerContext ctx) { appReconciler = new ApplicationReconciler(eventRecorder, statusRecorder, autoscaler); var deployment = TestUtils.buildApplicationCluster(); - addConfigProperties( - deployment.getSpec(), - Map.of( - AutoScalerOptions.AUTOSCALER_ENABLED.key(), - "true", - PipelineOptions.PARALLELISM_OVERRIDES.key(), - v1 + ":1")); + + deployment + .getSpec() + .getFlinkConfiguration() + .putAllFrom( + Map.of( + AutoScalerOptions.AUTOSCALER_ENABLED.key(), + "true", + PipelineOptions.PARALLELISM_OVERRIDES.key(), + v1 + ":1")); var specCopy = SpecUtils.clone(deployment.getSpec()); @@ -1005,7 +1025,9 @@ public void cleanup(KubernetesJobAutoScalerContext ctx) { // Test overrides are applied correctly overrideFunction.set( - s -> addConfigProperty(s, PipelineOptions.PARALLELISM_OVERRIDES.key(), v1 + ":2")); + s -> + s.getFlinkConfiguration() + .put(PipelineOptions.PARALLELISM_OVERRIDES.key(), v1 + ":2")); appReconciler.reconcile(ctxFactory.getResourceContext(deployment, context)); deployment.setSpec(SpecUtils.clone(specCopy)); @@ -1145,10 +1167,12 @@ public void testTerminalJmTtl(ThrowingConsumer deploymentSetup) var status = deployment.getStatus(); assertEquals(JobManagerDeploymentStatus.READY, status.getJobManagerDeploymentStatus()); - addConfigProperty( - deployment.getSpec(), - KubernetesOperatorConfigOptions.OPERATOR_JM_SHUTDOWN_TTL.key(), - String.valueOf(Duration.ofMinutes(5).toMillis())); + deployment + .getSpec() + .getFlinkConfiguration() + .put( + KubernetesOperatorConfigOptions.OPERATOR_JM_SHUTDOWN_TTL.key(), + String.valueOf(Duration.ofMinutes(5).toMillis())); var now = Instant.now(); status.getJobStatus().setUpdateTime(String.valueOf(now.toEpochMilli())); @@ -1224,8 +1248,11 @@ public void testDeploymentRecoveryEvent() throws Exception { @Test public void testRestartUnhealthyEvent() throws Exception { FlinkDeployment deployment = TestUtils.buildApplicationCluster(); - addConfigProperty( - deployment.getSpec(), OPERATOR_CLUSTER_HEALTH_CHECK_ENABLED.key(), "true"); + + deployment + .getSpec() + .getFlinkConfiguration() + .put(OPERATOR_CLUSTER_HEALTH_CHECK_ENABLED.key(), "true"); reconciler.reconcile(deployment, context); Assertions.assertEquals( @@ -1249,11 +1276,15 @@ public void testReconcileIfUpgradeModeNotAvailable() throws Exception { // We disable last state fallback as we want to test that the deployment is properly // recovered before upgrade - addConfigProperty( - deployment.getSpec(), - KubernetesOperatorConfigOptions.OPERATOR_JOB_UPGRADE_LAST_STATE_FALLBACK_ENABLED - .key(), - "false"); + + deployment + .getSpec() + .getFlinkConfiguration() + .put( + KubernetesOperatorConfigOptions + .OPERATOR_JOB_UPGRADE_LAST_STATE_FALLBACK_ENABLED + .key(), + "false"); // Initial deployment reconciler.reconcile(deployment, context); @@ -1314,7 +1345,7 @@ public void testUpgradeReconciledGeneration() throws Exception { assertEquals(1L, deployment.getStatus().getObservedGeneration()); // Submit no-op upgrade - addConfigProperty(deployment.getSpec(), "kubernetes.operator.test", "value"); + deployment.getSpec().getFlinkConfiguration().put("kubernetes.operator.test", "value"); deployment.getMetadata().setGeneration(2L); reconciler.reconcile(deployment, context); @@ -1328,17 +1359,19 @@ public void testRollbackUpgradeModeHandling(boolean jmStarted) throws Exception deployment.getSpec().getJob().setUpgradeMode(UpgradeMode.SAVEPOINT); offsetReconcilerClock(deployment, Duration.ZERO); - addConfigProperties( - deployment.getSpec(), - Map.of( - KubernetesOperatorConfigOptions.DEPLOYMENT_ROLLBACK_ENABLED.key(), - "true", - KubernetesOperatorConfigOptions.DEPLOYMENT_READINESS_TIMEOUT.key(), - "10s", - KubernetesOperatorConfigOptions - .OPERATOR_JOB_UPGRADE_LAST_STATE_FALLBACK_ENABLED - .key(), - "false")); + deployment + .getSpec() + .getFlinkConfiguration() + .putAllFrom( + Map.of( + KubernetesOperatorConfigOptions.DEPLOYMENT_ROLLBACK_ENABLED.key(), + "true", + KubernetesOperatorConfigOptions.DEPLOYMENT_READINESS_TIMEOUT.key(), + "10s", + KubernetesOperatorConfigOptions + .OPERATOR_JOB_UPGRADE_LAST_STATE_FALLBACK_ENABLED + .key(), + "false")); // Initial deployment, mark as stable reconciler.reconcile(deployment, context); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java index c091189527..52dc837e01 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java @@ -28,6 +28,7 @@ import org.apache.flink.kubernetes.operator.TestUtils; import org.apache.flink.kubernetes.operator.api.CrdConstants; import org.apache.flink.kubernetes.operator.api.FlinkDeployment; +import org.apache.flink.kubernetes.operator.api.spec.ConfigJsonNode; import org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec; import org.apache.flink.kubernetes.operator.api.spec.FlinkVersion; import org.apache.flink.kubernetes.operator.api.spec.JobState; @@ -39,7 +40,6 @@ import org.apache.flink.kubernetes.operator.api.status.Savepoint; import org.apache.flink.kubernetes.operator.api.status.SavepointFormatType; import org.apache.flink.kubernetes.operator.api.status.SnapshotTriggerType; -import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; import org.apache.flink.kubernetes.operator.exception.UpgradeFailureException; import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; @@ -70,10 +70,6 @@ import static org.apache.flink.api.common.JobStatus.RECONCILING; import static org.apache.flink.api.common.JobStatus.RESTARTING; import static org.apache.flink.api.common.JobStatus.RUNNING; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.removeConfigProperties; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.toJsonNode; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.toStringMap; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.SNAPSHOT_RESOURCE_ENABLED; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -166,10 +162,11 @@ private void testUpgradeToSavepoint(FlinkVersion flinkVersion, UpgradeMode fromU FlinkDeployment modifiedDeployment = cloneDeploymentWithUpgradeMode(deployment, UpgradeMode.SAVEPOINT); - addConfigProperty( - modifiedDeployment.getSpec(), - CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), - "test-savepoint-dir"); + + modifiedDeployment + .getSpec() + .getFlinkConfiguration() + .put(CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), "test-savepoint-dir"); reconciler.reconcile(modifiedDeployment, context); assertEquals(0, flinkService.getRunningCount()); @@ -259,10 +256,11 @@ private void testUpgradeToLastState(FlinkVersion flinkVersion, UpgradeMode fromU .getJobStatus() .setState(org.apache.flink.api.common.JobStatus.FINISHED); deployment.getStatus().setJobManagerDeploymentStatus(JobManagerDeploymentStatus.READY); - addConfigProperty( - deployment.getSpec(), - CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), - "test-savepoint-dir"); + + deployment + .getSpec() + .getFlinkConfiguration() + .put(CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), "test-savepoint-dir"); reconciler.reconcile(deployment, context); reconciler.reconcile(deployment, context); @@ -321,7 +319,7 @@ private FlinkDeployment cloneDeploymentWithUpgradeMode( FlinkDeployment result = ReconciliationUtils.clone(deployment); result.getSpec().getJob().setUpgradeMode(upgradeMode); - addConfigProperty(result.getSpec(), "new", "conf"); + result.getSpec().getFlinkConfiguration().put("new", "conf"); return result; } @@ -438,7 +436,7 @@ public void testInitialJmDeployCannotStartLegacy(UpgradeMode upgradeMode, boolea flinkService.setJobManagerReady(false); var deployment = TestUtils.buildApplicationCluster(); - addConfigProperty(deployment.getSpec(), SNAPSHOT_RESOURCE_ENABLED.key(), "false"); + deployment.getSpec().getFlinkConfiguration().put(SNAPSHOT_RESOURCE_ENABLED.key(), "false"); if (initSavepoint) { deployment.getSpec().getJob().setInitialSavepointPath("init-sp"); } @@ -505,10 +503,14 @@ public void testInitialJmDeployCannotStartLegacy(UpgradeMode upgradeMode, boolea @ValueSource(booleans = {true, false}) public void testLastStateMaxCheckpointAge(boolean cancellable) throws Exception { var deployment = TestUtils.buildApplicationCluster(); - addConfigProperty( - deployment.getSpec(), - KubernetesOperatorConfigOptions.OPERATOR_JOB_UPGRADE_LAST_STATE_CANCEL_JOB.key(), - Boolean.toString(cancellable)); + + deployment + .getSpec() + .getFlinkConfiguration() + .put( + KubernetesOperatorConfigOptions.OPERATOR_JOB_UPGRADE_LAST_STATE_CANCEL_JOB + .key(), + Boolean.toString(cancellable)); deployment.getSpec().getJob().setUpgradeMode(UpgradeMode.LAST_STATE); ReconciliationUtils.updateStatusForDeployedSpec(deployment, new Configuration()); @@ -619,14 +621,21 @@ public void testFlinkVersionSwitching( var jobReconciler = (ApplicationReconciler) this.reconciler.getReconciler(); var deployment = TestUtils.buildApplicationCluster(FlinkVersion.v1_18); if (!savepointsEnabled) { - SpecUtils.removeConfigProperties( - deployment.getSpec(), CheckpointingOptions.SAVEPOINT_DIRECTORY.key()); + + deployment + .getSpec() + .getFlinkConfiguration() + .remove(CheckpointingOptions.SAVEPOINT_DIRECTORY.key()); } - addConfigProperty( - deployment.getSpec(), - KubernetesOperatorConfigOptions.OPERATOR_JOB_UPGRADE_LAST_STATE_FALLBACK_ENABLED - .key(), - Boolean.toString(allowFallback)); + + deployment + .getSpec() + .getFlinkConfiguration() + .put( + KubernetesOperatorConfigOptions + .OPERATOR_JOB_UPGRADE_LAST_STATE_FALLBACK_ENABLED + .key(), + Boolean.toString(allowFallback)); deployment.getSpec().getJob().setUpgradeMode(upgradeMode); ReconciliationUtils.updateStatusForDeployedSpec(deployment, new Configuration()); deployment.getSpec().setFlinkVersion(FlinkVersion.v1_19); @@ -676,16 +685,24 @@ public void testLastStateNoHaMeta(UpgradeMode upgradeMode, boolean allowFallback throws Exception { var jobReconciler = (ApplicationReconciler) this.reconciler.getReconciler(); var deployment = TestUtils.buildApplicationCluster(); - addConfigProperty( - deployment.getSpec(), - KubernetesOperatorConfigOptions.OPERATOR_JOB_UPGRADE_LAST_STATE_FALLBACK_ENABLED - .key(), - Boolean.toString(allowFallback)); - removeConfigProperties(deployment.getSpec(), HighAvailabilityOptions.HA_MODE.key()); - addConfigProperty( - deployment.getSpec(), - KubernetesOperatorConfigOptions.OPERATOR_JOB_UPGRADE_LAST_STATE_CANCEL_JOB.key(), - Boolean.toString(false)); + + deployment + .getSpec() + .getFlinkConfiguration() + .put( + KubernetesOperatorConfigOptions + .OPERATOR_JOB_UPGRADE_LAST_STATE_FALLBACK_ENABLED + .key(), + Boolean.toString(allowFallback)); + deployment.getSpec().getFlinkConfiguration().remove(HighAvailabilityOptions.HA_MODE.key()); + + deployment + .getSpec() + .getFlinkConfiguration() + .put( + KubernetesOperatorConfigOptions.OPERATOR_JOB_UPGRADE_LAST_STATE_CANCEL_JOB + .key(), + Boolean.toString(false)); deployment.getSpec().getJob().setUpgradeMode(upgradeMode); ReconciliationUtils.updateStatusForDeployedSpec(deployment, new Configuration()); @@ -734,9 +751,7 @@ public void testLastStateOnDeletedDeployment() throws Exception { verifyAndSetRunningJobsToStatus(deployment, flinkService.listJobs()); // Delete cluster and keep HA metadata - var conf = - Configuration.fromMap( - SpecUtils.toStringMap(deployment.getSpec().getFlinkConfiguration())); + var conf = Configuration.fromMap(deployment.getSpec().getFlinkConfiguration().asFlatMap()); flinkService.deleteClusterDeployment( deployment.getMetadata(), deployment.getStatus(), conf, false); flinkService.setHaDataAvailable(true); @@ -829,7 +844,7 @@ public void testUpgradeModeChangedToLastStateShouldCancelWhileHADisabled() throw flinkService.setHaDataAvailable(false); var deployment = TestUtils.buildApplicationCluster(); - removeConfigProperties(deployment.getSpec(), HighAvailabilityOptions.HA_MODE.key()); + deployment.getSpec().getFlinkConfiguration().remove(HighAvailabilityOptions.HA_MODE.key()); reconciler.reconcile(deployment, context); assertEquals( @@ -917,7 +932,7 @@ public static FlinkDeployment buildApplicationCluster( FlinkVersion flinkVersion, UpgradeMode upgradeMode) { FlinkDeployment deployment = TestUtils.buildApplicationCluster(flinkVersion); deployment.getSpec().getJob().setUpgradeMode(upgradeMode); - Map conf = toStringMap(deployment.getSpec().getFlinkConfiguration()); + Map conf = deployment.getSpec().getFlinkConfiguration().asFlatMap(); switch (upgradeMode) { case STATELESS: @@ -939,7 +954,8 @@ public static FlinkDeployment buildApplicationCluster( default: throw new RuntimeException("Unsupported upgrade mode " + upgradeMode); } - deployment.getSpec().setFlinkConfiguration(toJsonNode(conf)); + deployment.getSpec().setFlinkConfiguration(new ConfigJsonNode()); + deployment.getSpec().getFlinkConfiguration().putAllFrom(conf); return deployment; } diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/SessionReconcilerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/SessionReconcilerTest.java index 429169dab3..fa57522d7e 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/SessionReconcilerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/SessionReconcilerTest.java @@ -49,7 +49,6 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -152,10 +151,11 @@ public void testSetOwnerReference() throws Exception { @Test public void testGetNonTerminalJobs() throws Exception { FlinkDeployment deployment = TestUtils.buildSessionCluster(); - addConfigProperty( - deployment.getSpec(), - KubernetesOperatorConfigOptions.BLOCK_ON_UNMANAGED_JOBS.key(), - "true"); + + deployment + .getSpec() + .getFlinkConfiguration() + .put(KubernetesOperatorConfigOptions.BLOCK_ON_UNMANAGED_JOBS.key(), "true"); assertEquals( true, diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/diff/SpecDiffTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/diff/SpecDiffTest.java index f1dfcb7c89..d760cc9a87 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/diff/SpecDiffTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/diff/SpecDiffTest.java @@ -46,7 +46,6 @@ import java.util.Map; import java.util.stream.Collectors; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.OPERATOR_RECONCILE_INTERVAL; import static org.apache.flink.kubernetes.operator.metrics.KubernetesOperatorMetricOptions.SCOPE_NAMING_KUBERNETES_OPERATOR; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -84,26 +83,27 @@ public void testFlinkDeploymentSpecChanges() { right.getJob().setAllowNonRestoredState(true); right.getJob().setInitialSavepointPath("local:///tmp"); right.getJob().setSavepointTriggerNonce(123L); - SpecUtils.addConfigProperties( - right, - Map.of( - OPERATOR_RECONCILE_INTERVAL.key(), - "100 SECONDS", - SCOPE_NAMING_KUBERNETES_OPERATOR.key(), - "foo.bar", - CoreOptions.DEFAULT_PARALLELISM.key(), - "100", - AutoScalerOptions.METRICS_WINDOW.key(), - "1234m")); + + right.getFlinkConfiguration() + .putAllFrom( + Map.of( + OPERATOR_RECONCILE_INTERVAL.key(), + "100 SECONDS", + SCOPE_NAMING_KUBERNETES_OPERATOR.key(), + "foo.bar", + CoreOptions.DEFAULT_PARALLELISM.key(), + "100", + AutoScalerOptions.METRICS_WINDOW.key(), + "1234m")); diff = new ReflectiveDiffBuilder<>(KubernetesDeploymentMode.NATIVE, left, right).build(); assertEquals(DiffType.IGNORE, diff.getType()); assertEquals(8, diff.getNumDiffs()); - SpecUtils.removeConfigProperties( - right, - SCOPE_NAMING_KUBERNETES_OPERATOR.key(), - AutoScalerOptions.METRICS_WINDOW.key()); + right.getFlinkConfiguration() + .removeAll( + SCOPE_NAMING_KUBERNETES_OPERATOR.key(), + AutoScalerOptions.METRICS_WINDOW.key()); diff = new ReflectiveDiffBuilder<>(KubernetesDeploymentMode.NATIVE, left, right).build(); assertEquals(DiffType.IGNORE, diff.getType()); @@ -150,7 +150,7 @@ public void testFlinkDeploymentSpecChanges() { assertEquals(DiffType.UPGRADE, diff.getType()); assertEquals(21, diff.getNumDiffs()); - addConfigProperty(right, CoreOptions.FLINK_TM_JVM_OPTIONS.key(), "-Dfoo=bar"); + right.getFlinkConfiguration().put(CoreOptions.FLINK_TM_JVM_OPTIONS.key(), "-Dfoo=bar"); diff = new ReflectiveDiffBuilder<>(KubernetesDeploymentMode.NATIVE, left, right).build(); assertEquals(DiffType.UPGRADE, diff.getType()); @@ -176,7 +176,7 @@ public void testFlinkDeploymentSpecChanges() { // verify parallelism override handling for native/standalone left = TestUtils.buildApplicationCluster().getSpec(); right = TestUtils.buildApplicationCluster().getSpec(); - addConfigProperty(left, PipelineOptions.PARALLELISM_OVERRIDES.key(), "new"); + left.getFlinkConfiguration().put(PipelineOptions.PARALLELISM_OVERRIDES.key(), "new"); diff = new ReflectiveDiffBuilder<>(KubernetesDeploymentMode.NATIVE, left, right).build(); assertEquals(DiffType.SCALE, diff.getType()); @@ -209,8 +209,9 @@ public void testFlinkSessionJobSpecChanges() { right.getJob().setAllowNonRestoredState(true); right.getJob().setInitialSavepointPath("local:///tmp"); right.getJob().setSavepointTriggerNonce(123L); - addConfigProperty( - right, KubernetesOperatorConfigOptions.JAR_ARTIFACT_HTTP_HEADER.key(), "changed"); + + right.getFlinkConfiguration() + .put(KubernetesOperatorConfigOptions.JAR_ARTIFACT_HTTP_HEADER.key(), "changed"); diff = new ReflectiveDiffBuilder<>(KubernetesDeploymentMode.NATIVE, left, right).build(); assertEquals(DiffType.IGNORE, diff.getType()); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/sessionjob/SessionJobReconcilerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/sessionjob/SessionJobReconcilerTest.java index b00928930e..5a98068196 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/sessionjob/SessionJobReconcilerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/sessionjob/SessionJobReconcilerTest.java @@ -72,7 +72,6 @@ import static org.apache.flink.kubernetes.operator.api.utils.FlinkResourceUtils.getJobSpec; import static org.apache.flink.kubernetes.operator.api.utils.FlinkResourceUtils.getJobStatus; import static org.apache.flink.kubernetes.operator.api.utils.FlinkResourceUtils.getReconciledJobSpec; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.OPERATOR_JOB_RESTART_FAILED; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.SNAPSHOT_RESOURCE_ENABLED; import static org.apache.flink.kubernetes.operator.reconciler.SnapshotType.CHECKPOINT; @@ -149,13 +148,17 @@ public void testSubmitAndCleanUpWithSavepoint(boolean legacySnapshots) throws Ex public void testSubmitAndCleanUpWithSavepointOnResource(boolean legacySnapshots) throws Exception { FlinkSessionJob sessionJob = TestUtils.buildSessionJob(); - addConfigProperty( - sessionJob.getSpec(), - KubernetesOperatorConfigOptions.SAVEPOINT_ON_DELETION.key(), - "true"); + + sessionJob + .getSpec() + .getFlinkConfiguration() + .put(KubernetesOperatorConfigOptions.SAVEPOINT_ON_DELETION.key(), "true"); if (legacySnapshots) { - addConfigProperty(sessionJob.getSpec(), SNAPSHOT_RESOURCE_ENABLED.key(), "false"); + sessionJob + .getSpec() + .getFlinkConfiguration() + .put(SNAPSHOT_RESOURCE_ENABLED.key(), "false"); } // session ready @@ -362,7 +365,10 @@ public void testStatelessUpgrade() throws Exception { public void testSavepointUpgrade(boolean legacySnapshots) throws Exception { FlinkSessionJob sessionJob = TestUtils.buildSessionJob(); if (legacySnapshots) { - addConfigProperty(sessionJob.getSpec(), SNAPSHOT_RESOURCE_ENABLED.key(), "false"); + sessionJob + .getSpec() + .getFlinkConfiguration() + .put(SNAPSHOT_RESOURCE_ENABLED.key(), "false"); } var readyContext = TestUtils.createContextWithReadyFlinkDeployment(kubernetesClient); @@ -426,7 +432,7 @@ public void testSavepointUpgrade(boolean legacySnapshots) throws Exception { @Test public void testTriggerSavepointLegacyLegacy() throws Exception { FlinkSessionJob sessionJob = TestUtils.buildSessionJob(); - addConfigProperty(sessionJob.getSpec(), SNAPSHOT_RESOURCE_ENABLED.key(), "false"); + sessionJob.getSpec().getFlinkConfiguration().put(SNAPSHOT_RESOURCE_ENABLED.key(), "false"); assertFalse(SnapshotUtils.savepointInProgress(sessionJob.getStatus().getJobStatus())); @@ -553,7 +559,7 @@ public void testTriggerSavepointLegacyLegacy() throws Exception { @Test public void testTriggerCheckpoint() throws Exception { FlinkSessionJob sessionJob = TestUtils.buildSessionJob(); - addConfigProperty(sessionJob.getSpec(), SNAPSHOT_RESOURCE_ENABLED.key(), "false"); + sessionJob.getSpec().getFlinkConfiguration().put(SNAPSHOT_RESOURCE_ENABLED.key(), "false"); assertFalse(SnapshotUtils.checkpointInProgress(getJobStatus(sessionJob))); @@ -786,7 +792,10 @@ public void testJobUpgradeIgnorePendingSavepoint() throws Exception { sessionJob, JobState.RUNNING, RECONCILING, null, flinkService.listJobs()); FlinkSessionJob spSessionJob = ReconciliationUtils.clone(sessionJob); - addConfigProperty(spSessionJob.getSpec(), SNAPSHOT_RESOURCE_ENABLED.key(), "false"); + spSessionJob + .getSpec() + .getFlinkConfiguration() + .put(SNAPSHOT_RESOURCE_ENABLED.key(), "false"); spSessionJob .getSpec() .getJob() diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkServiceTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkServiceTest.java index e07f1a2c3f..a03d7cd9c6 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkServiceTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkServiceTest.java @@ -142,7 +142,6 @@ import static org.apache.flink.api.common.JobStatus.FAILING; import static org.apache.flink.api.common.JobStatus.FINISHED; import static org.apache.flink.api.common.JobStatus.RUNNING; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; import static org.apache.flink.kubernetes.operator.config.FlinkConfigBuilder.FLINK_VERSION; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.OPERATOR_SAVEPOINT_FORMAT_TYPE; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; @@ -383,10 +382,11 @@ public void cancelJobWithSavepointUpgradeModeTest(boolean deleteAfterSavepoint) JobID jobID = JobID.generate(); FlinkDeployment deployment = TestUtils.buildApplicationCluster(); - addConfigProperty( - deployment.getSpec(), - CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), - savepointPath); + + deployment + .getSpec() + .getFlinkConfiguration() + .put(CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), savepointPath); deployment.getStatus().setJobManagerDeploymentStatus(JobManagerDeploymentStatus.READY); JobStatus jobStatus = deployment.getStatus().getJobStatus(); jobStatus.setJobId(jobID.toHexString()); @@ -492,10 +492,11 @@ public void cancelJobWithDrainOnSavepointUpgradeModeTest(boolean drainOnSavepoin JobID jobID = JobID.generate(); FlinkDeployment deployment = TestUtils.buildApplicationCluster(); - addConfigProperty( - deployment.getSpec(), - CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), - savepointPath); + + deployment + .getSpec() + .getFlinkConfiguration() + .put(CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), savepointPath); deployment.getStatus().setJobManagerDeploymentStatus(JobManagerDeploymentStatus.READY); JobStatus jobStatus = deployment.getStatus().getJobStatus(); jobStatus.setJobId(jobID.toHexString()); @@ -503,14 +504,16 @@ public void cancelJobWithDrainOnSavepointUpgradeModeTest(boolean drainOnSavepoin ReconciliationUtils.updateStatusForDeployedSpec(deployment, new Configuration()); if (drainOnSavepoint) { - addConfigProperty( - deployment.getSpec(), - KubernetesOperatorConfigOptions.SAVEPOINT_ON_DELETION.key(), - "true"); - addConfigProperty( - deployment.getSpec(), - KubernetesOperatorConfigOptions.DRAIN_ON_SAVEPOINT_DELETION.key(), - "true"); + + deployment + .getSpec() + .getFlinkConfiguration() + .put(KubernetesOperatorConfigOptions.SAVEPOINT_ON_DELETION.key(), "true"); + + deployment + .getSpec() + .getFlinkConfiguration() + .put(KubernetesOperatorConfigOptions.DRAIN_ON_SAVEPOINT_DELETION.key(), "true"); } var result = @@ -559,8 +562,9 @@ public void cancelSessionJobWithDrainOnSavepointUpgradeModeTest(boolean drainOnS .serializeAndSetLastReconciledSpec(session.getSpec(), session); var job = TestUtils.buildSessionJob(); - addConfigProperty( - job.getSpec(), CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), savepointPath); + job.getSpec() + .getFlinkConfiguration() + .put(CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), savepointPath); JobStatus jobStatus = job.getStatus().getJobStatus(); jobStatus.setJobId(jobID.toHexString()); @@ -568,14 +572,14 @@ public void cancelSessionJobWithDrainOnSavepointUpgradeModeTest(boolean drainOnS ReconciliationUtils.updateStatusForDeployedSpec(job, new Configuration()); if (drainOnSavepoint) { - addConfigProperty( - job.getSpec(), - KubernetesOperatorConfigOptions.SAVEPOINT_ON_DELETION.key(), - "true"); - addConfigProperty( - job.getSpec(), - KubernetesOperatorConfigOptions.DRAIN_ON_SAVEPOINT_DELETION.key(), - "true"); + + job.getSpec() + .getFlinkConfiguration() + .put(KubernetesOperatorConfigOptions.SAVEPOINT_ON_DELETION.key(), "true"); + + job.getSpec() + .getFlinkConfiguration() + .put(KubernetesOperatorConfigOptions.DRAIN_ON_SAVEPOINT_DELETION.key(), "true"); } var deployConf = configManager.getSessionJobConfig( @@ -803,10 +807,11 @@ private void runNativeSavepointFormatTest(boolean failAfterSavepointCompletes) var flinkService = new TestingService(testingClusterClient); final FlinkDeployment deployment = TestUtils.buildApplicationCluster(); - addConfigProperty( - deployment.getSpec(), - CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), - savepointPath); + + deployment + .getSpec() + .getFlinkConfiguration() + .put(CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), savepointPath); deployment.getStatus().setJobManagerDeploymentStatus(JobManagerDeploymentStatus.READY); JobStatus jobStatus = deployment.getStatus().getJobStatus(); jobStatus.setJobId(jobID.toHexString()); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/NativeFlinkServiceTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/NativeFlinkServiceTest.java index c8f13998ca..bbbfdf465e 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/NativeFlinkServiceTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/NativeFlinkServiceTest.java @@ -31,7 +31,6 @@ import org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec; import org.apache.flink.kubernetes.operator.api.spec.FlinkVersion; import org.apache.flink.kubernetes.operator.api.spec.JobSpec; -import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.apache.flink.kubernetes.operator.config.FlinkConfigManager; import org.apache.flink.kubernetes.operator.config.FlinkOperatorConfiguration; import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; @@ -71,9 +70,6 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.configurationToJsonNode; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.removeConfigProperties; import static org.apache.flink.kubernetes.operator.config.FlinkConfigBuilder.FLINK_VERSION; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.OPERATOR_HEALTH_PROBE_PORT; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -281,17 +277,17 @@ protected void updateVertexResources( var spec = flinkDep.getSpec(); spec.setFlinkVersion(FlinkVersion.v1_18); - var appConfig = Configuration.fromMap(SpecUtils.toStringMap(spec.getFlinkConfiguration())); + var appConfig = Configuration.fromMap(spec.getFlinkConfiguration().asFlatMap()); appConfig.set(JobManagerOptions.SCHEDULER, JobManagerOptions.SchedulerType.Adaptive); - spec.setFlinkConfiguration(SpecUtils.mapToJsonNode(appConfig.toMap())); + spec.getFlinkConfiguration().set(appConfig.toMap()); var reconStatus = flinkDep.getStatus().getReconciliationStatus(); reconStatus.serializeAndSetLastReconciledSpec(spec, flinkDep); appConfig.set( PipelineOptions.PARALLELISM_OVERRIDES, Map.of(v1.toHexString(), "4", v2.toHexString(), "1")); - spec.setFlinkConfiguration(SpecUtils.mapToJsonNode(appConfig.toMap())); + spec.getFlinkConfiguration().set(appConfig.toMap()); flinkDep.getStatus().getJobStatus().setState(JobStatus.RUNNING); @@ -325,7 +321,7 @@ protected void updateVertexResources( // Baseline appConfig.set(PipelineOptions.PARALLELISM_OVERRIDES, Map.of(v1.toHexString(), "4")); - spec.setFlinkConfiguration(configurationToJsonNode(appConfig)); + spec.getFlinkConfiguration().set(appConfig.toMap()); testScaleConditionDep(flinkDep, service, d -> {}, true); testScaleConditionLastSpec(flinkDep, service, d -> {}, true); @@ -334,11 +330,13 @@ protected void updateVertexResources( flinkDep, service, d -> - addConfigProperty( - d.getSpec(), - KubernetesOperatorConfigOptions.JOB_UPGRADE_INPLACE_SCALING_ENABLED - .key(), - "false"), + d.getSpec() + .getFlinkConfiguration() + .put( + KubernetesOperatorConfigOptions + .JOB_UPGRADE_INPLACE_SCALING_ENABLED + .key(), + "false"), false); // Do not scale without adaptive scheduler deployed @@ -346,10 +344,10 @@ protected void updateVertexResources( flinkDep, service, ls -> - addConfigProperty( - ls, - JobManagerOptions.SCHEDULER.key(), - JobManagerOptions.SchedulerType.Default.name()), + ls.getFlinkConfiguration() + .put( + JobManagerOptions.SCHEDULER.key(), + JobManagerOptions.SchedulerType.Default.name()), false); // Do not scale without adaptive scheduler deployed @@ -384,7 +382,9 @@ protected void updateVertexResources( testScaleConditionLastSpec( flinkDep, service, - s -> addConfigProperty(s, PipelineOptions.PARALLELISM_OVERRIDES.key(), v2 + ":3"), + s -> + s.getFlinkConfiguration() + .put(PipelineOptions.PARALLELISM_OVERRIDES.key(), v2 + ":3"), false); // Scale if parallelism overrides were removed only from a non-active vertex @@ -392,20 +392,25 @@ protected void updateVertexResources( flinkDep, service, s -> - addConfigProperty( - s, - PipelineOptions.PARALLELISM_OVERRIDES.key(), - v1 + ":1," + new JobVertexID() + ":5"), + s.getFlinkConfiguration() + .put( + PipelineOptions.PARALLELISM_OVERRIDES.key(), + v1 + ":1," + new JobVertexID() + ":5"), true); // Do not scale if parallelism overrides were completely removed var flinkDep2 = ReconciliationUtils.clone(flinkDep); - SpecUtils.removeConfigProperties( - flinkDep2.getSpec(), PipelineOptions.PARALLELISM_OVERRIDES.key()); + + flinkDep2 + .getSpec() + .getFlinkConfiguration() + .remove(PipelineOptions.PARALLELISM_OVERRIDES.key()); testScaleConditionLastSpec( flinkDep2, service, - s -> addConfigProperty(s, PipelineOptions.PARALLELISM_OVERRIDES.key(), v2 + ":3"), + s -> + s.getFlinkConfiguration() + .put(PipelineOptions.PARALLELISM_OVERRIDES.key(), v2 + ":3"), false); // Do not scale if overrides never set @@ -413,8 +418,9 @@ protected void updateVertexResources( flinkDep2, service, d -> - removeConfigProperties( - d.getSpec(), PipelineOptions.PARALLELISM_OVERRIDES.key()), + d.getSpec() + .getFlinkConfiguration() + .remove(PipelineOptions.PARALLELISM_OVERRIDES.key()), false); // Do not scale if non active vertices are overridden only @@ -430,10 +436,9 @@ protected void updateVertexResources( flinkDep, service, d -> - addConfigProperty( - d.getSpec(), - PipelineOptions.PARALLELISM_OVERRIDES.key(), - v2 + ":5"), + d.getSpec() + .getFlinkConfiguration() + .put(PipelineOptions.PARALLELISM_OVERRIDES.key(), v2 + ":5"), true); assertNull(updated.get()); @@ -442,10 +447,11 @@ protected void updateVertexResources( flinkDep, service, d -> - addConfigProperty( - d.getSpec(), - PipelineOptions.PARALLELISM_OVERRIDES.key(), - v2 + ":5," + v1 + ":1"), + d.getSpec() + .getFlinkConfiguration() + .put( + PipelineOptions.PARALLELISM_OVERRIDES.key(), + v2 + ":5," + v1 + ":1"), true); assertNull(updated.get()); @@ -459,10 +465,11 @@ protected void updateVertexResources( flinkDep, service, d -> - addConfigProperty( - d.getSpec(), - PipelineOptions.PARALLELISM_OVERRIDES.key(), - v2 + ":5," + v1 + ":1"), + d.getSpec() + .getFlinkConfiguration() + .put( + PipelineOptions.PARALLELISM_OVERRIDES.key(), + v2 + ":5," + v1 + ":1"), true); assertEquals( new JobVertexResourceRequirements.Parallelism(1, 1), diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/StandaloneFlinkServiceTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/StandaloneFlinkServiceTest.java index 7517057ce6..29662ad888 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/StandaloneFlinkServiceTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/StandaloneFlinkServiceTest.java @@ -26,7 +26,6 @@ import org.apache.flink.kubernetes.operator.api.FlinkDeployment; import org.apache.flink.kubernetes.operator.api.spec.JobSpec; import org.apache.flink.kubernetes.operator.api.spec.KubernetesDeploymentMode; -import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.apache.flink.kubernetes.operator.artifact.ArtifactManager; import org.apache.flink.kubernetes.operator.config.FlinkConfigManager; import org.apache.flink.kubernetes.operator.config.Mode; @@ -44,7 +43,6 @@ import java.util.List; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.OPERATOR_HEALTH_PROBE_PORT; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -135,10 +133,13 @@ public void testTMReplicaScaleApplication() { flinkDeployment.getSpec().setMode(KubernetesDeploymentMode.STANDALONE); // Add parallelism change, verify it is honoured in reactive mode - addConfigProperty( - flinkDeployment.getSpec(), - JobManagerOptions.SCHEDULER_MODE.key(), - SchedulerExecutionMode.REACTIVE.name()); + + flinkDeployment + .getSpec() + .getFlinkConfiguration() + .put( + JobManagerOptions.SCHEDULER_MODE.key(), + SchedulerExecutionMode.REACTIVE.name()); flinkDeployment .getStatus() .getReconciliationStatus() @@ -163,8 +164,10 @@ public void testTMReplicaScaleApplication() { .getSpec() .getReplicas()); - SpecUtils.removeConfigProperties( - flinkDeployment.getSpec(), JobManagerOptions.SCHEDULER_MODE.key()); + flinkDeployment + .getSpec() + .getFlinkConfiguration() + .remove(JobManagerOptions.SCHEDULER_MODE.key()); flinkDeployment .getStatus() .getReconciliationStatus() @@ -189,10 +192,13 @@ public void testTMReplicaScaleSession() { flinkDeployment.getSpec().setMode(KubernetesDeploymentMode.STANDALONE); // Add replicas flinkDeployment.getSpec().getTaskManager().setReplicas(3); - addConfigProperty( - flinkDeployment.getSpec(), - JobManagerOptions.SCHEDULER_MODE.key(), - SchedulerExecutionMode.REACTIVE.name()); + + flinkDeployment + .getSpec() + .getFlinkConfiguration() + .put( + JobManagerOptions.SCHEDULER_MODE.key(), + SchedulerExecutionMode.REACTIVE.name()); flinkDeployment .getStatus() .getReconciliationStatus() diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/SnapshotUtilsTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/SnapshotUtilsTest.java index 3ee0a82310..9e9a222512 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/SnapshotUtilsTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/SnapshotUtilsTest.java @@ -41,7 +41,6 @@ import static org.apache.flink.kubernetes.operator.TestUtils.reconcileSpec; import static org.apache.flink.kubernetes.operator.TestUtils.setupCronTrigger; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.PERIODIC_CHECKPOINT_INTERVAL; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.PERIODIC_SAVEPOINT_INTERVAL; import static org.apache.flink.kubernetes.operator.reconciler.SnapshotType.CHECKPOINT; @@ -83,7 +82,7 @@ public void testCheckpointTriggeringPre1_17() { snapshotType, Instant.MIN)); - addConfigProperty(deployment.getSpec(), PERIODIC_CHECKPOINT_INTERVAL.key(), "10m"); + deployment.getSpec().getFlinkConfiguration().put(PERIODIC_CHECKPOINT_INTERVAL.key(), "10m"); reconcileSpec(deployment); assertEquals( @@ -129,7 +128,10 @@ private void testSnapshotTriggering( snapshotType, Instant.MIN)); - addConfigProperty(deployment.getSpec(), periodicSnapshotIntervalOption.key(), "10m"); + deployment + .getSpec() + .getFlinkConfiguration() + .put(periodicSnapshotIntervalOption.key(), "10m"); reconcileSpec(deployment); @@ -141,7 +143,7 @@ private void testSnapshotTriggering( snapshotType, Instant.MIN)); resetTrigger(deployment, snapshotType); - addConfigProperty(deployment.getSpec(), periodicSnapshotIntervalOption.key(), "0"); + deployment.getSpec().getFlinkConfiguration().put(periodicSnapshotIntervalOption.key(), "0"); reconcileSpec(deployment); setTriggerNonce(deployment, snapshotType, 123L); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/validation/DefaultValidatorTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/validation/DefaultValidatorTest.java index d17ec2b9be..730552ff94 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/validation/DefaultValidatorTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/validation/DefaultValidatorTest.java @@ -46,7 +46,6 @@ import org.apache.flink.kubernetes.operator.api.status.JobStatus; import org.apache.flink.kubernetes.operator.api.status.Savepoint; import org.apache.flink.kubernetes.operator.api.status.SnapshotTriggerType; -import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.apache.flink.kubernetes.operator.config.FlinkConfigManager; import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; @@ -71,9 +70,6 @@ import static org.apache.flink.configuration.TaskManagerOptions.MANAGED_MEMORY_SIZE; import static org.apache.flink.configuration.TaskManagerOptions.TASK_HEAP_MEMORY; import static org.apache.flink.configuration.TaskManagerOptions.TOTAL_FLINK_MEMORY; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.addConfigProperty; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.removeConfigProperties; -import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.toJsonNode; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; @@ -117,8 +113,9 @@ public void testValidationWithoutDefaultConfig() { testError( dep -> { - removeConfigProperties( - dep.getSpec(), CheckpointingOptions.SAVEPOINT_DIRECTORY.key()); + dep.getSpec() + .getFlinkConfiguration() + .remove(CheckpointingOptions.SAVEPOINT_DIRECTORY.key()); dep.getSpec().getJob().setUpgradeMode(UpgradeMode.SAVEPOINT); }, String.format( @@ -127,16 +124,18 @@ public void testValidationWithoutDefaultConfig() { testError( dep -> { - removeConfigProperties( - dep.getSpec(), CheckpointingOptions.CHECKPOINTS_DIRECTORY.key()); + dep.getSpec() + .getFlinkConfiguration() + .remove(CheckpointingOptions.CHECKPOINTS_DIRECTORY.key()); dep.getSpec().getJob().setUpgradeMode(UpgradeMode.SAVEPOINT); }, "Checkpoint directory"); testError( dep -> { - removeConfigProperties( - dep.getSpec(), CheckpointingOptions.CHECKPOINTS_DIRECTORY.key()); + dep.getSpec() + .getFlinkConfiguration() + .remove(CheckpointingOptions.CHECKPOINTS_DIRECTORY.key()); dep.getSpec().getJob().setUpgradeMode(UpgradeMode.LAST_STATE); }, @@ -144,14 +143,15 @@ public void testValidationWithoutDefaultConfig() { testSuccess( dep -> { - removeConfigProperties( - dep.getSpec(), CheckpointingOptions.CHECKPOINTS_DIRECTORY.key()); + dep.getSpec() + .getFlinkConfiguration() + .remove(CheckpointingOptions.CHECKPOINTS_DIRECTORY.key()); dep.getSpec().getJob().setUpgradeMode(UpgradeMode.STATELESS); }); testError( dep -> { - dep.getSpec().setFlinkConfiguration(toJsonNode(new HashMap<>())); + dep.getSpec().getFlinkConfiguration().set(new HashMap<>()); dep.getSpec() .getJob() .setSavepointTriggerNonce(ThreadLocalRandom.current().nextLong()); @@ -163,13 +163,13 @@ public void testValidationWithoutDefaultConfig() { testError( dep -> dep.getSpec() - .setFlinkConfiguration( - toJsonNode( - Map.of( - KubernetesOperatorConfigOptions - .PERIODIC_SAVEPOINT_INTERVAL - .key(), - "1m"))), + .getFlinkConfiguration() + .set( + Map.of( + KubernetesOperatorConfigOptions + .PERIODIC_SAVEPOINT_INTERVAL + .key(), + "1m")), String.format( "Periodic savepoints cannot be enabled when config key[%s] is not set", CheckpointingOptions.SAVEPOINT_DIRECTORY.key())); @@ -177,13 +177,13 @@ public void testValidationWithoutDefaultConfig() { testError( dep -> dep.getSpec() - .setFlinkConfiguration( - toJsonNode( - Map.of( - KubernetesOperatorConfigOptions - .OPERATOR_JOB_UPGRADE_LAST_STATE_CHECKPOINT_MAX_AGE - .key(), - "1m"))), + .getFlinkConfiguration() + .set( + Map.of( + KubernetesOperatorConfigOptions + .OPERATOR_JOB_UPGRADE_LAST_STATE_CHECKPOINT_MAX_AGE + .key(), + "1m")), String.format( "In order to use max-checkpoint age functionality config key[%s] must be set to allow triggering savepoint upgrades.", CheckpointingOptions.SAVEPOINT_DIRECTORY.key())); @@ -192,42 +192,41 @@ public void testValidationWithoutDefaultConfig() { testSuccess( dep -> dep.getSpec() - .setFlinkConfiguration( - toJsonNode(Collections.singletonMap("random", "config")))); + .getFlinkConfiguration() + .set(Collections.singletonMap("random", "config"))); testError( dep -> dep.getSpec() - .setFlinkConfiguration( - toJsonNode( - Collections.singletonMap( - KubernetesConfigOptions.NAMESPACE.key(), - "myns"))), + .getFlinkConfiguration() + .set( + Collections.singletonMap( + KubernetesConfigOptions.NAMESPACE.key(), "myns")), "Forbidden Flink config key"); testError( dep -> dep.getSpec() - .setFlinkConfiguration( - toJsonNode( - Collections.singletonMap( - HighAvailabilityOptions.HA_CLUSTER_ID.key(), - "my-cluster-id"))), + .getFlinkConfiguration() + .set( + Collections.singletonMap( + HighAvailabilityOptions.HA_CLUSTER_ID.key(), + "my-cluster-id")), "Forbidden Flink config key"); testError( dep -> dep.getSpec() - .setFlinkConfiguration( - toJsonNode( - Map.of( - KubernetesOperatorConfigOptions - .OPERATOR_CLUSTER_HEALTH_CHECK_ENABLED - .key(), - "true", - KubernetesOperatorConfigOptions - .OPERATOR_JM_DEPLOYMENT_RECOVERY_ENABLED - .key(), - "false"))), + .getFlinkConfiguration() + .set( + Map.of( + KubernetesOperatorConfigOptions + .OPERATOR_CLUSTER_HEALTH_CHECK_ENABLED + .key(), + "true", + KubernetesOperatorConfigOptions + .OPERATOR_JM_DEPLOYMENT_RECOVERY_ENABLED + .key(), + "false")), "Deployment recovery (" + KubernetesOperatorConfigOptions.OPERATOR_JM_DEPLOYMENT_RECOVERY_ENABLED .key() @@ -266,7 +265,7 @@ public void testValidationWithoutDefaultConfig() { testError( dep -> { - dep.getSpec().setFlinkConfiguration(toJsonNode(new HashMap<>())); + dep.getSpec().getFlinkConfiguration().set(new HashMap<>()); dep.getSpec().getJobManager().setReplicas(2); }, "High availability should be enabled when starting standby JobManagers."); @@ -274,13 +273,13 @@ public void testValidationWithoutDefaultConfig() { testError( dep -> dep.getSpec() - .setFlinkConfiguration( - toJsonNode( - Map.of( - KubernetesOperatorConfigOptions - .DEPLOYMENT_ROLLBACK_ENABLED - .key(), - "true"))), + .getFlinkConfiguration() + .set( + Map.of( + KubernetesOperatorConfigOptions + .DEPLOYMENT_ROLLBACK_ENABLED + .key(), + "true")), "HA must be enabled for rollback support."); testError( @@ -311,8 +310,8 @@ public void testValidationWithoutDefaultConfig() { dep -> { dep.getSpec().getTaskManager().getResource().setMemory(null); dep.getSpec() - .setFlinkConfiguration( - toJsonNode(Map.of(TASK_HEAP_MEMORY.key(), "1024m"))); + .getFlinkConfiguration() + .set(Map.of(TASK_HEAP_MEMORY.key(), "1024m")); }, "TaskManager resource memory must be defined using `spec.taskManager.resource.memory`"); @@ -320,27 +319,27 @@ public void testValidationWithoutDefaultConfig() { dep -> { dep.getSpec().getJobManager().getResource().setMemory(null); dep.getSpec() - .setFlinkConfiguration( - toJsonNode(Map.of(JVM_HEAP_MEMORY.key(), "2048m"))); + .getFlinkConfiguration() + .set(Map.of(JVM_HEAP_MEMORY.key(), "2048m")); }); testSuccess( dep -> { dep.getSpec().getTaskManager().getResource().setMemory(null); dep.getSpec() - .setFlinkConfiguration( - toJsonNode(Map.of(TOTAL_FLINK_MEMORY.key(), "2048m"))); + .getFlinkConfiguration() + .set(Map.of(TOTAL_FLINK_MEMORY.key(), "2048m")); }); testSuccess( dep -> { dep.getSpec().getTaskManager().getResource().setMemory(null); dep.getSpec() - .setFlinkConfiguration( - toJsonNode( - Map.of( - TASK_HEAP_MEMORY.key(), - "1024m", - MANAGED_MEMORY_SIZE.key(), - "1024m"))); + .getFlinkConfiguration() + .set( + Map.of( + TASK_HEAP_MEMORY.key(), + "1024m", + MANAGED_MEMORY_SIZE.key(), + "1024m")); }); // Test savepoint restore validation @@ -362,10 +361,11 @@ public void testValidationWithoutDefaultConfig() { .getReconciliationStatus() .serializeAndSetLastReconciledSpec(spec, dep); - SpecUtils.addConfigProperty( - dep.getSpec(), - CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), - "file:///flink-data/savepoints"); + dep.getSpec() + .getFlinkConfiguration() + .put( + CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), + "file:///flink-data/savepoints"); dep.getSpec().getJob().setUpgradeMode(UpgradeMode.SAVEPOINT); }); @@ -487,12 +487,14 @@ public void testValidationWithoutDefaultConfig() { testSuccess( dep -> { dep.getSpec().getJob().setUpgradeMode(UpgradeMode.LAST_STATE); - addConfigProperty( - dep.getSpec(), - HighAvailabilityOptions.HA_MODE.key(), - // Hardcoded config value should be removed when upgrading Flink - // dependency to 1.16 - "kubernetes"); + + dep.getSpec() + .getFlinkConfiguration() + .put( + HighAvailabilityOptions.HA_MODE.key(), + // Hardcoded config value should be removed when upgrading Flink + // dependency to 1.16 + "kubernetes"); }); testError( @@ -519,7 +521,7 @@ public void testValidationWithDefaultConfig() { new DefaultValidator(new FlinkConfigManager(defaultFlinkConf)); testSuccess( dep -> { - dep.getSpec().setFlinkConfiguration(toJsonNode(new HashMap<>())); + dep.getSpec().getFlinkConfiguration().set(new HashMap<>()); dep.getSpec().getJob().setUpgradeMode(UpgradeMode.LAST_STATE); }, validatorWithDefaultConfig); @@ -705,13 +707,13 @@ public void testSessionJobWithSession() { sessionJob -> sessionJob .getSpec() - .setFlinkConfiguration( - toJsonNode( - Map.of( - KubernetesOperatorConfigOptions - .JAR_ARTIFACT_HTTP_HEADER - .key(), - "headerKey1:headerValue1,headerKey2:headerValue2"))), + .getFlinkConfiguration() + .set( + Map.of( + KubernetesOperatorConfigOptions + .JAR_ARTIFACT_HTTP_HEADER + .key(), + "headerKey1:headerValue1,headerKey2:headerValue2")), flinkDeployment -> {}, null); @@ -719,13 +721,13 @@ public void testSessionJobWithSession() { sessionJob -> sessionJob .getSpec() - .setFlinkConfiguration( - toJsonNode( - Map.of( - KubernetesOperatorConfigOptions - .PERIODIC_SAVEPOINT_INTERVAL - .key(), - "1m"))), + .getFlinkConfiguration() + .set( + Map.of( + KubernetesOperatorConfigOptions + .PERIODIC_SAVEPOINT_INTERVAL + .key(), + "1m")), flinkDeployment -> {}, null); @@ -745,17 +747,16 @@ public void testSessionJobWithSession() { sessionJob.getSpec().getJob().setUpgradeMode(UpgradeMode.SAVEPOINT); sessionJob .getSpec() - .setFlinkConfiguration( - toJsonNode( - Map.of( - CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), - "test-savepoint-dir", - CheckpointingOptions.CHECKPOINTS_DIRECTORY - .key(), - "test-checkpoint-dir"))); + .getFlinkConfiguration() + .set( + Map.of( + CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), + "test-savepoint-dir", + CheckpointingOptions.CHECKPOINTS_DIRECTORY.key(), + "test-checkpoint-dir")); }, flinkDeployment -> { - flinkDeployment.getSpec().setFlinkConfiguration(toJsonNode(Map.of())); + flinkDeployment.getSpec().getFlinkConfiguration().set(Map.of()); }, null); } @@ -1108,7 +1109,7 @@ private Optional testSessionJobAutoScalerConfiguration( var sessionJob = TestUtils.buildSessionJob(); var flinkConfiguration = getDefaultTestAutoScalerFlinkConfigurationMap(); flinkConfigurationModifier.accept(flinkConfiguration); - sessionCluster.getSpec().setFlinkConfiguration(toJsonNode(flinkConfiguration)); + sessionCluster.getSpec().getFlinkConfiguration().set(flinkConfiguration); return validator.validateSessionJob(sessionJob, Optional.of(sessionCluster)); } @@ -1117,7 +1118,7 @@ public Optional testAutoScalerConfiguration( FlinkDeployment deployment = TestUtils.buildApplicationCluster(); var flinkConfiguration = getDefaultTestAutoScalerFlinkConfigurationMap(); flinkConfigurationModifier.accept(flinkConfiguration); - deployment.getSpec().setFlinkConfiguration(toJsonNode(flinkConfiguration)); + deployment.getSpec().getFlinkConfiguration().set(flinkConfiguration); return validator.validateDeployment(deployment); } From 9fc5a8a98181603f0c653c0d1e8c590b0b5051bb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Attila=20M=C3=A9sz=C3=A1ros?= Date: Tue, 19 Aug 2025 18:16:35 +0200 Subject: [PATCH 10/19] naming MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Attila Mészáros --- .../kubernetes/operator/api/spec/AbstractFlinkSpec.java | 4 ++-- .../spec/{ConfigJsonNode.java => ConfigObjectNode.java} | 8 ++++---- ...eserializer.java => ConfigObjectNodeDeserializer.java} | 8 ++++---- .../kubernetes/operator/api/utils/BaseTestUtils.java | 6 +++--- .../operator/reconciler/diff/ReflectiveDiffBuilder.java | 8 ++++---- .../deployment/ApplicationReconcilerUpgradeModeTest.java | 4 ++-- 6 files changed, 19 insertions(+), 19 deletions(-) rename flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/{ConfigJsonNode.java => ConfigObjectNode.java} (92%) rename flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/{ConfigJsonNodeDeserializer.java => ConfigObjectNodeDeserializer.java} (86%) diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/AbstractFlinkSpec.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/AbstractFlinkSpec.java index 17b3591b98..b18adb3b0f 100644 --- a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/AbstractFlinkSpec.java +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/AbstractFlinkSpec.java @@ -56,6 +56,6 @@ public abstract class AbstractFlinkSpec implements Diffable { type = DiffType.SCALE, mode = KubernetesDeploymentMode.NATIVE) }) - @JsonDeserialize(using = ConfigJsonNodeDeserializer.class) - private ConfigJsonNode flinkConfiguration = new ConfigJsonNode(); + @JsonDeserialize(using = ConfigObjectNodeDeserializer.class) + private ConfigObjectNode flinkConfiguration = new ConfigObjectNode(); } diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigJsonNode.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigObjectNode.java similarity index 92% rename from flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigJsonNode.java rename to flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigObjectNode.java index 77c0e46984..5e18335fc8 100644 --- a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigJsonNode.java +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigObjectNode.java @@ -26,17 +26,17 @@ import java.util.Map; /** */ -public class ConfigJsonNode extends ObjectNode { +public class ConfigObjectNode extends ObjectNode { - public ConfigJsonNode() { + public ConfigObjectNode() { this(JsonNodeFactory.instance); } - public ConfigJsonNode(JsonNodeFactory nc, Map kids) { + public ConfigObjectNode(JsonNodeFactory nc, Map kids) { super(nc, kids); } - public ConfigJsonNode(JsonNodeFactory nc) { + public ConfigObjectNode(JsonNodeFactory nc) { super(nc); } diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigJsonNodeDeserializer.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigObjectNodeDeserializer.java similarity index 86% rename from flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigJsonNodeDeserializer.java rename to flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigObjectNodeDeserializer.java index bdb56c57c4..ea7cb0cf51 100644 --- a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigJsonNodeDeserializer.java +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigObjectNodeDeserializer.java @@ -24,15 +24,15 @@ import java.io.IOException; -/** ConfigJsonNode deserializer. */ -public class ConfigJsonNodeDeserializer extends JsonDeserializer { +/** Allows to deserialize to ConfigObjectNode. */ +public class ConfigObjectNodeDeserializer extends JsonDeserializer { @Override - public ConfigJsonNode deserialize( + public ConfigObjectNode deserialize( JsonParser jsonParser, DeserializationContext deserializationContext) throws IOException { ObjectNode tree = jsonParser.readValueAsTree(); - var res = new ConfigJsonNode(); + var res = new ConfigObjectNode(); tree.fields().forEachRemaining(entry -> res.set(entry.getKey(), entry.getValue())); return res; } diff --git a/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/utils/BaseTestUtils.java b/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/utils/BaseTestUtils.java index 8088d7708a..f3bd54f2f2 100644 --- a/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/utils/BaseTestUtils.java +++ b/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/utils/BaseTestUtils.java @@ -25,7 +25,7 @@ import org.apache.flink.kubernetes.operator.api.FlinkSessionJob; import org.apache.flink.kubernetes.operator.api.FlinkStateSnapshot; import org.apache.flink.kubernetes.operator.api.spec.CheckpointSpec; -import org.apache.flink.kubernetes.operator.api.spec.ConfigJsonNode; +import org.apache.flink.kubernetes.operator.api.spec.ConfigObjectNode; import org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec; import org.apache.flink.kubernetes.operator.api.spec.FlinkSessionJobSpec; import org.apache.flink.kubernetes.operator.api.spec.FlinkStateSnapshotSpec; @@ -143,7 +143,7 @@ public static FlinkSessionJob buildSessionJob( .withResourceVersion("1") .build()); - ConfigJsonNode conf = new ConfigJsonNode(); + ConfigObjectNode conf = new ConfigObjectNode(); conf.put("kubernetes.operator.user.artifacts.http.header", "header"); sessionJob.setSpec( FlinkSessionJobSpec.builder() @@ -169,7 +169,7 @@ public static FlinkSessionJob buildSessionJob(JobState state) { } public static FlinkDeploymentSpec getTestFlinkDeploymentSpec(FlinkVersion version) { - ConfigJsonNode conf = new ConfigJsonNode(); + ConfigObjectNode conf = new ConfigObjectNode(); conf.put(TaskManagerOptions.NUM_TASK_SLOTS.key(), "2"); conf.put( HighAvailabilityOptions.HA_MODE.key(), diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/diff/ReflectiveDiffBuilder.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/diff/ReflectiveDiffBuilder.java index 77339adc0a..385322b48e 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/diff/ReflectiveDiffBuilder.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/diff/ReflectiveDiffBuilder.java @@ -21,7 +21,7 @@ import org.apache.flink.kubernetes.operator.api.diff.DiffType; import org.apache.flink.kubernetes.operator.api.diff.Diffable; import org.apache.flink.kubernetes.operator.api.diff.SpecDiff; -import org.apache.flink.kubernetes.operator.api.spec.ConfigJsonNode; +import org.apache.flink.kubernetes.operator.api.spec.ConfigObjectNode; import org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec; import org.apache.flink.kubernetes.operator.api.spec.KubernetesDeploymentMode; @@ -86,13 +86,13 @@ private void appendFields(final Class clazz) { var leftField = readField(field, before, true); var rightField = readField(field, after, true); if (field.getName().equals(FLINK_CONFIGURATION_PROPERTY_NAME)) { - leftField = ((ConfigJsonNode) leftField).asFlatMap(); - rightField = ((ConfigJsonNode) rightField).asFlatMap(); + leftField = ((ConfigObjectNode) leftField).asFlatMap(); + rightField = ((ConfigObjectNode) rightField).asFlatMap(); } if (field.isAnnotationPresent(SpecDiff.Config.class) && (Map.class.isAssignableFrom(field.getType()) - || (field.getType().equals(ConfigJsonNode.class) + || (field.getType().equals(ConfigObjectNode.class) && field.getName().equals("flinkConfiguration")))) { diffBuilder.append( field.getName(), diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java index 52dc837e01..cabacc0188 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java @@ -28,7 +28,7 @@ import org.apache.flink.kubernetes.operator.TestUtils; import org.apache.flink.kubernetes.operator.api.CrdConstants; import org.apache.flink.kubernetes.operator.api.FlinkDeployment; -import org.apache.flink.kubernetes.operator.api.spec.ConfigJsonNode; +import org.apache.flink.kubernetes.operator.api.spec.ConfigObjectNode; import org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec; import org.apache.flink.kubernetes.operator.api.spec.FlinkVersion; import org.apache.flink.kubernetes.operator.api.spec.JobState; @@ -954,7 +954,7 @@ public static FlinkDeployment buildApplicationCluster( default: throw new RuntimeException("Unsupported upgrade mode " + upgradeMode); } - deployment.getSpec().setFlinkConfiguration(new ConfigJsonNode()); + deployment.getSpec().setFlinkConfiguration(new ConfigObjectNode()); deployment.getSpec().getFlinkConfiguration().putAllFrom(conf); return deployment; } From 3ac43393c4bae50c9237a3350986f89065d0ed3a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Attila=20M=C3=A9sz=C3=A1ros?= Date: Tue, 19 Aug 2025 18:21:34 +0200 Subject: [PATCH 11/19] cleanup MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Attila Mészáros --- .../api/spec/JsonNodeNullDeserializer.java | 31 ------------------- 1 file changed, 31 deletions(-) delete mode 100644 flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/JsonNodeNullDeserializer.java diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/JsonNodeNullDeserializer.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/JsonNodeNullDeserializer.java deleted file mode 100644 index 335175c534..0000000000 --- a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/JsonNodeNullDeserializer.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * 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.api.spec; - -import com.fasterxml.jackson.databind.DeserializationContext; -import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.deser.std.JsonNodeDeserializer; - -/** Makes sure JsonNode is properly deserialized to null not NullNode. */ -public class JsonNodeNullDeserializer extends JsonNodeDeserializer { - - @Override - public JsonNode getNullValue(DeserializationContext ctxt) { - return null; - } -} From 381951bf6fa39fd36f9dcbecf97fb50764658cfc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Attila=20M=C3=A9sz=C3=A1ros?= Date: Wed, 20 Aug 2025 12:26:20 +0200 Subject: [PATCH 12/19] improvements + cleanup MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Attila Mészáros --- .../java/org/apache/flink/examples/Basic.java | 2 +- .../operator/api/spec/AbstractFlinkSpec.java | 17 ++++++ .../operator/api/spec/ConfigObjectNode.java | 15 +++-- .../autoscaler/KubernetesScalingRealizer.java | 4 +- .../operator/config/FlinkConfigManager.java | 2 +- .../ApplicationReconcilerUpgradeModeTest.java | 2 +- .../reconciler/diff/SpecDiffTest.java | 2 +- .../service/NativeFlinkServiceTest.java | 8 +-- .../validation/DefaultValidatorTest.java | 55 +++++++------------ 9 files changed, 56 insertions(+), 51 deletions(-) diff --git a/examples/kubernetes-client-examples/src/main/java/org/apache/flink/examples/Basic.java b/examples/kubernetes-client-examples/src/main/java/org/apache/flink/examples/Basic.java index a762d99eb8..2334860ddd 100644 --- a/examples/kubernetes-client-examples/src/main/java/org/apache/flink/examples/Basic.java +++ b/examples/kubernetes-client-examples/src/main/java/org/apache/flink/examples/Basic.java @@ -49,7 +49,7 @@ public static void main(String[] args) { flinkDeploymentSpec.setImage("flink:1.19"); Map flinkConfiguration = Map.ofEntries(entry("taskmanager.numberOfTaskSlots", "2")); - flinkDeploymentSpec.getFlinkConfiguration().set(flinkConfiguration); + flinkDeploymentSpec.setConfiguration(flinkConfiguration); flinkDeployment.setSpec(flinkDeploymentSpec); flinkDeploymentSpec.setServiceAccount("flink"); JobManagerSpec jobManagerSpec = new JobManagerSpec(); diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/AbstractFlinkSpec.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/AbstractFlinkSpec.java index b18adb3b0f..9183e7799f 100644 --- a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/AbstractFlinkSpec.java +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/AbstractFlinkSpec.java @@ -18,15 +18,20 @@ package org.apache.flink.kubernetes.operator.api.spec; import org.apache.flink.annotation.Experimental; +import org.apache.flink.configuration.Configuration; import org.apache.flink.kubernetes.operator.api.diff.DiffType; import org.apache.flink.kubernetes.operator.api.diff.Diffable; import org.apache.flink.kubernetes.operator.api.diff.SpecDiff; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.databind.annotation.JsonDeserialize; import lombok.AllArgsConstructor; import lombok.Data; import lombok.NoArgsConstructor; import lombok.experimental.SuperBuilder; +import lombok.experimental.Tolerate; + +import java.util.Map; /** The common spec. */ @Experimental @@ -58,4 +63,16 @@ public abstract class AbstractFlinkSpec implements Diffable { }) @JsonDeserialize(using = ConfigObjectNodeDeserializer.class) private ConfigObjectNode flinkConfiguration = new ConfigObjectNode(); + + @Tolerate + @JsonIgnore + public void setConfiguration(Map config) { + flinkConfiguration.setAllFrom(config); + } + + @Tolerate + @JsonIgnore + public void setConfiguration(Configuration config) { + setConfiguration(config.toMap()); + } } diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigObjectNode.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigObjectNode.java index 5e18335fc8..7b5356328c 100644 --- a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigObjectNode.java +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigObjectNode.java @@ -17,10 +17,13 @@ package org.apache.flink.kubernetes.operator.api.spec; +import org.apache.flink.configuration.Configuration; + import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.node.JsonNodeFactory; import com.fasterxml.jackson.databind.node.ObjectNode; +import java.util.Arrays; import java.util.HashMap; import java.util.Iterator; import java.util.Map; @@ -40,17 +43,15 @@ public ConfigObjectNode(JsonNodeFactory nc) { super(nc); } - public void removeAll(String... names) { - for (String name : names) { - remove(name); - } + public void remove(String... names) { + remove(Arrays.asList(names)); } public void putAllFrom(Map value) { value.forEach(this::put); } - public void set(Map value) { + public void setAllFrom(Map value) { removeAll(); putAllFrom(value); } @@ -61,6 +62,10 @@ public Map asFlatMap() { return flatMap; } + public Configuration asConfiguration() { + return Configuration.fromMap(asFlatMap()); + } + private static void flattenHelper( JsonNode node, String parentKey, Map flatMap) { if (node.isObject()) { diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/KubernetesScalingRealizer.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/KubernetesScalingRealizer.java index 4d34bfa622..2dc6a4eace 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/KubernetesScalingRealizer.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/autoscaler/KubernetesScalingRealizer.java @@ -20,7 +20,6 @@ import org.apache.flink.autoscaler.realizer.ScalingRealizer; import org.apache.flink.autoscaler.tuning.ConfigChanges; import org.apache.flink.autoscaler.tuning.MemoryTuning; -import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ConfigurationUtils; import org.apache.flink.configuration.MemorySize; import org.apache.flink.configuration.PipelineOptions; @@ -69,8 +68,7 @@ public void realizeConfigOverrides( // Update total memory in spec var totalMemoryOverride = MemoryTuning.getTotalMemory( - Configuration.fromMap( - flinkDeployment.getSpec().getFlinkConfiguration().asFlatMap()), + flinkDeployment.getSpec().getFlinkConfiguration().asConfiguration(), context); if (totalMemoryOverride.compareTo(MemorySize.ZERO) <= 0) { LOG.warn("Total memory override {} is not valid", totalMemoryOverride); diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManager.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManager.java index 4c024e2382..f31fc7ba4b 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManager.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManager.java @@ -372,7 +372,7 @@ private void applyConfigsFromCurrentSpec( AbstractFlinkSpec spec, Configuration conf, ConfigOption... configOptions) { addOperatorConfigsFromSpec(spec, conf); if (spec.getFlinkConfiguration() != null) { - var deployConfig = Configuration.fromMap(spec.getFlinkConfiguration().asFlatMap()); + var deployConfig = spec.getFlinkConfiguration().asConfiguration(); for (ConfigOption configOption : configOptions) { deployConfig.getOptional(configOption).ifPresent(v -> conf.set(configOption, v)); } diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java index cabacc0188..7d164a7be5 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java @@ -751,7 +751,7 @@ public void testLastStateOnDeletedDeployment() throws Exception { verifyAndSetRunningJobsToStatus(deployment, flinkService.listJobs()); // Delete cluster and keep HA metadata - var conf = Configuration.fromMap(deployment.getSpec().getFlinkConfiguration().asFlatMap()); + var conf = deployment.getSpec().getFlinkConfiguration().asConfiguration(); flinkService.deleteClusterDeployment( deployment.getMetadata(), deployment.getStatus(), conf, false); flinkService.setHaDataAvailable(true); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/diff/SpecDiffTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/diff/SpecDiffTest.java index d760cc9a87..e5969d9a4e 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/diff/SpecDiffTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/diff/SpecDiffTest.java @@ -101,7 +101,7 @@ public void testFlinkDeploymentSpecChanges() { assertEquals(8, diff.getNumDiffs()); right.getFlinkConfiguration() - .removeAll( + .remove( SCOPE_NAMING_KUBERNETES_OPERATOR.key(), AutoScalerOptions.METRICS_WINDOW.key()); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/NativeFlinkServiceTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/NativeFlinkServiceTest.java index bbbfdf465e..89b4f8a98f 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/NativeFlinkServiceTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/NativeFlinkServiceTest.java @@ -277,17 +277,17 @@ protected void updateVertexResources( var spec = flinkDep.getSpec(); spec.setFlinkVersion(FlinkVersion.v1_18); - var appConfig = Configuration.fromMap(spec.getFlinkConfiguration().asFlatMap()); + var appConfig = spec.getFlinkConfiguration().asConfiguration(); appConfig.set(JobManagerOptions.SCHEDULER, JobManagerOptions.SchedulerType.Adaptive); - spec.getFlinkConfiguration().set(appConfig.toMap()); + spec.setConfiguration(appConfig); var reconStatus = flinkDep.getStatus().getReconciliationStatus(); reconStatus.serializeAndSetLastReconciledSpec(spec, flinkDep); appConfig.set( PipelineOptions.PARALLELISM_OVERRIDES, Map.of(v1.toHexString(), "4", v2.toHexString(), "1")); - spec.getFlinkConfiguration().set(appConfig.toMap()); + spec.setConfiguration(appConfig); flinkDep.getStatus().getJobStatus().setState(JobStatus.RUNNING); @@ -321,7 +321,7 @@ protected void updateVertexResources( // Baseline appConfig.set(PipelineOptions.PARALLELISM_OVERRIDES, Map.of(v1.toHexString(), "4")); - spec.getFlinkConfiguration().set(appConfig.toMap()); + spec.setConfiguration(appConfig); testScaleConditionDep(flinkDep, service, d -> {}, true); testScaleConditionLastSpec(flinkDep, service, d -> {}, true); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/validation/DefaultValidatorTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/validation/DefaultValidatorTest.java index 730552ff94..8147897a6a 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/validation/DefaultValidatorTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/validation/DefaultValidatorTest.java @@ -151,7 +151,7 @@ public void testValidationWithoutDefaultConfig() { testError( dep -> { - dep.getSpec().getFlinkConfiguration().set(new HashMap<>()); + dep.getSpec().getFlinkConfiguration().setAllFrom(new HashMap<>()); dep.getSpec() .getJob() .setSavepointTriggerNonce(ThreadLocalRandom.current().nextLong()); @@ -163,8 +163,7 @@ public void testValidationWithoutDefaultConfig() { testError( dep -> dep.getSpec() - .getFlinkConfiguration() - .set( + .setConfiguration( Map.of( KubernetesOperatorConfigOptions .PERIODIC_SAVEPOINT_INTERVAL @@ -177,8 +176,7 @@ public void testValidationWithoutDefaultConfig() { testError( dep -> dep.getSpec() - .getFlinkConfiguration() - .set( + .setConfiguration( Map.of( KubernetesOperatorConfigOptions .OPERATOR_JOB_UPGRADE_LAST_STATE_CHECKPOINT_MAX_AGE @@ -192,13 +190,11 @@ public void testValidationWithoutDefaultConfig() { testSuccess( dep -> dep.getSpec() - .getFlinkConfiguration() - .set(Collections.singletonMap("random", "config"))); + .setConfiguration(Collections.singletonMap("random", "config"))); testError( dep -> dep.getSpec() - .getFlinkConfiguration() - .set( + .setConfiguration( Collections.singletonMap( KubernetesConfigOptions.NAMESPACE.key(), "myns")), "Forbidden Flink config key"); @@ -207,7 +203,7 @@ public void testValidationWithoutDefaultConfig() { dep -> dep.getSpec() .getFlinkConfiguration() - .set( + .setAllFrom( Collections.singletonMap( HighAvailabilityOptions.HA_CLUSTER_ID.key(), "my-cluster-id")), @@ -216,8 +212,7 @@ public void testValidationWithoutDefaultConfig() { testError( dep -> dep.getSpec() - .getFlinkConfiguration() - .set( + .setConfiguration( Map.of( KubernetesOperatorConfigOptions .OPERATOR_CLUSTER_HEALTH_CHECK_ENABLED @@ -265,7 +260,7 @@ public void testValidationWithoutDefaultConfig() { testError( dep -> { - dep.getSpec().getFlinkConfiguration().set(new HashMap<>()); + dep.getSpec().setConfiguration(new HashMap<>()); dep.getSpec().getJobManager().setReplicas(2); }, "High availability should be enabled when starting standby JobManagers."); @@ -273,8 +268,7 @@ public void testValidationWithoutDefaultConfig() { testError( dep -> dep.getSpec() - .getFlinkConfiguration() - .set( + .setConfiguration( Map.of( KubernetesOperatorConfigOptions .DEPLOYMENT_ROLLBACK_ENABLED @@ -309,32 +303,25 @@ public void testValidationWithoutDefaultConfig() { testError( dep -> { dep.getSpec().getTaskManager().getResource().setMemory(null); - dep.getSpec() - .getFlinkConfiguration() - .set(Map.of(TASK_HEAP_MEMORY.key(), "1024m")); + dep.getSpec().setConfiguration(Map.of(TASK_HEAP_MEMORY.key(), "1024m")); }, "TaskManager resource memory must be defined using `spec.taskManager.resource.memory`"); testSuccess( dep -> { dep.getSpec().getJobManager().getResource().setMemory(null); - dep.getSpec() - .getFlinkConfiguration() - .set(Map.of(JVM_HEAP_MEMORY.key(), "2048m")); + dep.getSpec().setConfiguration(Map.of(JVM_HEAP_MEMORY.key(), "2048m")); }); testSuccess( dep -> { dep.getSpec().getTaskManager().getResource().setMemory(null); - dep.getSpec() - .getFlinkConfiguration() - .set(Map.of(TOTAL_FLINK_MEMORY.key(), "2048m")); + dep.getSpec().setConfiguration(Map.of(TOTAL_FLINK_MEMORY.key(), "2048m")); }); testSuccess( dep -> { dep.getSpec().getTaskManager().getResource().setMemory(null); dep.getSpec() - .getFlinkConfiguration() - .set( + .setConfiguration( Map.of( TASK_HEAP_MEMORY.key(), "1024m", @@ -521,7 +508,7 @@ public void testValidationWithDefaultConfig() { new DefaultValidator(new FlinkConfigManager(defaultFlinkConf)); testSuccess( dep -> { - dep.getSpec().getFlinkConfiguration().set(new HashMap<>()); + dep.getSpec().setConfiguration(new HashMap<>()); dep.getSpec().getJob().setUpgradeMode(UpgradeMode.LAST_STATE); }, validatorWithDefaultConfig); @@ -707,8 +694,7 @@ public void testSessionJobWithSession() { sessionJob -> sessionJob .getSpec() - .getFlinkConfiguration() - .set( + .setConfiguration( Map.of( KubernetesOperatorConfigOptions .JAR_ARTIFACT_HTTP_HEADER @@ -721,8 +707,7 @@ public void testSessionJobWithSession() { sessionJob -> sessionJob .getSpec() - .getFlinkConfiguration() - .set( + .setConfiguration( Map.of( KubernetesOperatorConfigOptions .PERIODIC_SAVEPOINT_INTERVAL @@ -748,7 +733,7 @@ public void testSessionJobWithSession() { sessionJob .getSpec() .getFlinkConfiguration() - .set( + .setAllFrom( Map.of( CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), "test-savepoint-dir", @@ -756,7 +741,7 @@ public void testSessionJobWithSession() { "test-checkpoint-dir")); }, flinkDeployment -> { - flinkDeployment.getSpec().getFlinkConfiguration().set(Map.of()); + flinkDeployment.getSpec().setConfiguration(Map.of()); }, null); } @@ -1109,7 +1094,7 @@ private Optional testSessionJobAutoScalerConfiguration( var sessionJob = TestUtils.buildSessionJob(); var flinkConfiguration = getDefaultTestAutoScalerFlinkConfigurationMap(); flinkConfigurationModifier.accept(flinkConfiguration); - sessionCluster.getSpec().getFlinkConfiguration().set(flinkConfiguration); + sessionCluster.getSpec().setConfiguration(flinkConfiguration); return validator.validateSessionJob(sessionJob, Optional.of(sessionCluster)); } @@ -1118,7 +1103,7 @@ public Optional testAutoScalerConfiguration( FlinkDeployment deployment = TestUtils.buildApplicationCluster(); var flinkConfiguration = getDefaultTestAutoScalerFlinkConfigurationMap(); flinkConfigurationModifier.accept(flinkConfiguration); - deployment.getSpec().getFlinkConfiguration().set(flinkConfiguration); + deployment.getSpec().setConfiguration(flinkConfiguration); return validator.validateDeployment(deployment); } From 3829654ae4651f242a7f0a52c815c08a1322b71d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Attila=20M=C3=A9sz=C3=A1ros?= Date: Wed, 20 Aug 2025 12:37:02 +0200 Subject: [PATCH 13/19] naming MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Attila Mészáros --- .../java/org/apache/flink/examples/Basic.java | 2 +- .../operator/api/spec/AbstractFlinkSpec.java | 6 ++-- .../service/NativeFlinkServiceTest.java | 6 ++-- .../validation/DefaultValidatorTest.java | 35 ++++++++++--------- 4 files changed, 25 insertions(+), 24 deletions(-) diff --git a/examples/kubernetes-client-examples/src/main/java/org/apache/flink/examples/Basic.java b/examples/kubernetes-client-examples/src/main/java/org/apache/flink/examples/Basic.java index 2334860ddd..17a7358693 100644 --- a/examples/kubernetes-client-examples/src/main/java/org/apache/flink/examples/Basic.java +++ b/examples/kubernetes-client-examples/src/main/java/org/apache/flink/examples/Basic.java @@ -49,7 +49,7 @@ public static void main(String[] args) { flinkDeploymentSpec.setImage("flink:1.19"); Map flinkConfiguration = Map.ofEntries(entry("taskmanager.numberOfTaskSlots", "2")); - flinkDeploymentSpec.setConfiguration(flinkConfiguration); + flinkDeploymentSpec.setFlinkConfiguration(flinkConfiguration); flinkDeployment.setSpec(flinkDeploymentSpec); flinkDeploymentSpec.setServiceAccount("flink"); JobManagerSpec jobManagerSpec = new JobManagerSpec(); diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/AbstractFlinkSpec.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/AbstractFlinkSpec.java index 9183e7799f..4a95eb9beb 100644 --- a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/AbstractFlinkSpec.java +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/AbstractFlinkSpec.java @@ -66,13 +66,13 @@ public abstract class AbstractFlinkSpec implements Diffable { @Tolerate @JsonIgnore - public void setConfiguration(Map config) { + public void setFlinkConfiguration(Map config) { flinkConfiguration.setAllFrom(config); } @Tolerate @JsonIgnore - public void setConfiguration(Configuration config) { - setConfiguration(config.toMap()); + public void setFlinkConfiguration(Configuration config) { + setFlinkConfiguration(config.toMap()); } } diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/NativeFlinkServiceTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/NativeFlinkServiceTest.java index 89b4f8a98f..2dcf843397 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/NativeFlinkServiceTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/NativeFlinkServiceTest.java @@ -280,14 +280,14 @@ protected void updateVertexResources( var appConfig = spec.getFlinkConfiguration().asConfiguration(); appConfig.set(JobManagerOptions.SCHEDULER, JobManagerOptions.SchedulerType.Adaptive); - spec.setConfiguration(appConfig); + spec.setFlinkConfiguration(appConfig); var reconStatus = flinkDep.getStatus().getReconciliationStatus(); reconStatus.serializeAndSetLastReconciledSpec(spec, flinkDep); appConfig.set( PipelineOptions.PARALLELISM_OVERRIDES, Map.of(v1.toHexString(), "4", v2.toHexString(), "1")); - spec.setConfiguration(appConfig); + spec.setFlinkConfiguration(appConfig); flinkDep.getStatus().getJobStatus().setState(JobStatus.RUNNING); @@ -321,7 +321,7 @@ protected void updateVertexResources( // Baseline appConfig.set(PipelineOptions.PARALLELISM_OVERRIDES, Map.of(v1.toHexString(), "4")); - spec.setConfiguration(appConfig); + spec.setFlinkConfiguration(appConfig); testScaleConditionDep(flinkDep, service, d -> {}, true); testScaleConditionLastSpec(flinkDep, service, d -> {}, true); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/validation/DefaultValidatorTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/validation/DefaultValidatorTest.java index 8147897a6a..c0472e1e85 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/validation/DefaultValidatorTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/validation/DefaultValidatorTest.java @@ -163,7 +163,7 @@ public void testValidationWithoutDefaultConfig() { testError( dep -> dep.getSpec() - .setConfiguration( + .setFlinkConfiguration( Map.of( KubernetesOperatorConfigOptions .PERIODIC_SAVEPOINT_INTERVAL @@ -176,7 +176,7 @@ public void testValidationWithoutDefaultConfig() { testError( dep -> dep.getSpec() - .setConfiguration( + .setFlinkConfiguration( Map.of( KubernetesOperatorConfigOptions .OPERATOR_JOB_UPGRADE_LAST_STATE_CHECKPOINT_MAX_AGE @@ -190,11 +190,12 @@ public void testValidationWithoutDefaultConfig() { testSuccess( dep -> dep.getSpec() - .setConfiguration(Collections.singletonMap("random", "config"))); + .setFlinkConfiguration( + Collections.singletonMap("random", "config"))); testError( dep -> dep.getSpec() - .setConfiguration( + .setFlinkConfiguration( Collections.singletonMap( KubernetesConfigOptions.NAMESPACE.key(), "myns")), "Forbidden Flink config key"); @@ -212,7 +213,7 @@ public void testValidationWithoutDefaultConfig() { testError( dep -> dep.getSpec() - .setConfiguration( + .setFlinkConfiguration( Map.of( KubernetesOperatorConfigOptions .OPERATOR_CLUSTER_HEALTH_CHECK_ENABLED @@ -260,7 +261,7 @@ public void testValidationWithoutDefaultConfig() { testError( dep -> { - dep.getSpec().setConfiguration(new HashMap<>()); + dep.getSpec().setFlinkConfiguration(new HashMap<>()); dep.getSpec().getJobManager().setReplicas(2); }, "High availability should be enabled when starting standby JobManagers."); @@ -268,7 +269,7 @@ public void testValidationWithoutDefaultConfig() { testError( dep -> dep.getSpec() - .setConfiguration( + .setFlinkConfiguration( Map.of( KubernetesOperatorConfigOptions .DEPLOYMENT_ROLLBACK_ENABLED @@ -303,25 +304,25 @@ public void testValidationWithoutDefaultConfig() { testError( dep -> { dep.getSpec().getTaskManager().getResource().setMemory(null); - dep.getSpec().setConfiguration(Map.of(TASK_HEAP_MEMORY.key(), "1024m")); + dep.getSpec().setFlinkConfiguration(Map.of(TASK_HEAP_MEMORY.key(), "1024m")); }, "TaskManager resource memory must be defined using `spec.taskManager.resource.memory`"); testSuccess( dep -> { dep.getSpec().getJobManager().getResource().setMemory(null); - dep.getSpec().setConfiguration(Map.of(JVM_HEAP_MEMORY.key(), "2048m")); + dep.getSpec().setFlinkConfiguration(Map.of(JVM_HEAP_MEMORY.key(), "2048m")); }); testSuccess( dep -> { dep.getSpec().getTaskManager().getResource().setMemory(null); - dep.getSpec().setConfiguration(Map.of(TOTAL_FLINK_MEMORY.key(), "2048m")); + dep.getSpec().setFlinkConfiguration(Map.of(TOTAL_FLINK_MEMORY.key(), "2048m")); }); testSuccess( dep -> { dep.getSpec().getTaskManager().getResource().setMemory(null); dep.getSpec() - .setConfiguration( + .setFlinkConfiguration( Map.of( TASK_HEAP_MEMORY.key(), "1024m", @@ -508,7 +509,7 @@ public void testValidationWithDefaultConfig() { new DefaultValidator(new FlinkConfigManager(defaultFlinkConf)); testSuccess( dep -> { - dep.getSpec().setConfiguration(new HashMap<>()); + dep.getSpec().setFlinkConfiguration(new HashMap<>()); dep.getSpec().getJob().setUpgradeMode(UpgradeMode.LAST_STATE); }, validatorWithDefaultConfig); @@ -694,7 +695,7 @@ public void testSessionJobWithSession() { sessionJob -> sessionJob .getSpec() - .setConfiguration( + .setFlinkConfiguration( Map.of( KubernetesOperatorConfigOptions .JAR_ARTIFACT_HTTP_HEADER @@ -707,7 +708,7 @@ public void testSessionJobWithSession() { sessionJob -> sessionJob .getSpec() - .setConfiguration( + .setFlinkConfiguration( Map.of( KubernetesOperatorConfigOptions .PERIODIC_SAVEPOINT_INTERVAL @@ -741,7 +742,7 @@ public void testSessionJobWithSession() { "test-checkpoint-dir")); }, flinkDeployment -> { - flinkDeployment.getSpec().setConfiguration(Map.of()); + flinkDeployment.getSpec().setFlinkConfiguration(Map.of()); }, null); } @@ -1094,7 +1095,7 @@ private Optional testSessionJobAutoScalerConfiguration( var sessionJob = TestUtils.buildSessionJob(); var flinkConfiguration = getDefaultTestAutoScalerFlinkConfigurationMap(); flinkConfigurationModifier.accept(flinkConfiguration); - sessionCluster.getSpec().setConfiguration(flinkConfiguration); + sessionCluster.getSpec().setFlinkConfiguration(flinkConfiguration); return validator.validateSessionJob(sessionJob, Optional.of(sessionCluster)); } @@ -1103,7 +1104,7 @@ public Optional testAutoScalerConfiguration( FlinkDeployment deployment = TestUtils.buildApplicationCluster(); var flinkConfiguration = getDefaultTestAutoScalerFlinkConfigurationMap(); flinkConfigurationModifier.accept(flinkConfiguration); - deployment.getSpec().setConfiguration(flinkConfiguration); + deployment.getSpec().setFlinkConfiguration(flinkConfiguration); return validator.validateDeployment(deployment); } From 447ea75ce7e6a383a5d8fe773dc848f8032008d3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Attila=20M=C3=A9sz=C3=A1ros?= Date: Wed, 20 Aug 2025 13:06:02 +0200 Subject: [PATCH 14/19] remove empty lines MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Attila Mészáros --- .../autoscaler/KubernetesScalingRealizerTest.java | 2 -- .../operator/config/FlinkConfigBuilderTest.java | 2 -- .../operator/controller/DeploymentRecoveryTest.java | 1 - .../controller/FlinkDeploymentControllerTest.java | 3 --- .../controller/FlinkSessionJobControllerTest.java | 2 -- .../controller/FlinkStateSnapshotControllerTest.java | 1 - .../kubernetes/operator/controller/RollbackTest.java | 4 ---- .../operator/observer/JobStatusObserverTest.java | 3 --- .../observer/deployment/ApplicationObserverTest.java | 3 --- .../reconciler/deployment/ApplicationReconcilerTest.java | 9 --------- .../deployment/ApplicationReconcilerUpgradeModeTest.java | 6 ------ .../reconciler/deployment/SessionReconcilerTest.java | 1 - .../operator/reconciler/diff/SpecDiffTest.java | 1 - .../reconciler/sessionjob/SessionJobReconcilerTest.java | 1 - .../operator/service/AbstractFlinkServiceTest.java | 7 ------- .../operator/service/NativeFlinkServiceTest.java | 1 - .../operator/service/StandaloneFlinkServiceTest.java | 2 -- .../kubernetes/operator/utils/SnapshotUtilsTest.java | 1 - .../operator/validation/DefaultValidatorTest.java | 2 -- 19 files changed, 52 deletions(-) diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/autoscaler/KubernetesScalingRealizerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/autoscaler/KubernetesScalingRealizerTest.java index f0ecd9bea2..ba647034fa 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/autoscaler/KubernetesScalingRealizerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/autoscaler/KubernetesScalingRealizerTest.java @@ -96,14 +96,12 @@ private void assertOverridesDoNotChange( FlinkDeployment resource = (FlinkDeployment) ctx.getResource(); // Create resource with existing parallelism overrides - resource.getSpec() .getFlinkConfiguration() .put(PipelineOptions.PARALLELISM_OVERRIDES.key(), currentOverrides); resource.getStatus() .getReconciliationStatus() .serializeAndSetLastReconciledSpec(resource.getSpec(), resource); - resource.getSpec() .getFlinkConfiguration() .remove(PipelineOptions.PARALLELISM_OVERRIDES.key()); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilderTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilderTest.java index e55934136f..674ef4d578 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilderTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilderTest.java @@ -758,7 +758,6 @@ public void testApplyJobOrSessionSpecWithNoJar() throws Exception { @Test public void testAllowNonRestoredStateInSpecOverrideInFlinkConf() throws URISyntaxException { flinkDeployment.getSpec().getJob().setAllowNonRestoredState(false); - flinkDeployment .getSpec() .getFlinkConfiguration() @@ -771,7 +770,6 @@ public void testAllowNonRestoredStateInSpecOverrideInFlinkConf() throws URISynta configuration.getBoolean(SavepointConfigOptions.SAVEPOINT_IGNORE_UNCLAIMED_STATE)); flinkDeployment.getSpec().getJob().setAllowNonRestoredState(true); - flinkDeployment .getSpec() .getFlinkConfiguration() diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/DeploymentRecoveryTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/DeploymentRecoveryTest.java index 881462fda0..07d23a5d5c 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/DeploymentRecoveryTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/DeploymentRecoveryTest.java @@ -177,7 +177,6 @@ public void verifyRecoveryWithoutHaData(FlinkVersion flinkVersion, UpgradeMode u // We disable HA for stateless to test recovery without HA metadata if (upgradeMode == UpgradeMode.STATELESS) { - appCluster .getSpec() .getFlinkConfiguration() diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentControllerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentControllerTest.java index c9c18b46b4..4d379139b6 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentControllerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentControllerTest.java @@ -197,7 +197,6 @@ public void verifyReconcileLoopForInitialSuspendedDeploymentWithSavepoint( TestUtils.buildApplicationCluster(flinkVersion, JobState.SUSPENDED); appCluster.getSpec().getJob().setUpgradeMode(UpgradeMode.SAVEPOINT); appCluster.getSpec().getJob().setInitialSavepointPath("s0"); - appCluster .getSpec() .getFlinkConfiguration() @@ -305,7 +304,6 @@ public void verifyUpgradeFromSavepointLegacyMode(FlinkVersion flinkVersion) thro FlinkDeployment appCluster = TestUtils.buildApplicationCluster(flinkVersion); appCluster.getSpec().getJob().setUpgradeMode(UpgradeMode.SAVEPOINT); appCluster.getSpec().getJob().setInitialSavepointPath("s0"); - appCluster .getSpec() .getFlinkConfiguration() @@ -798,7 +796,6 @@ private void testUpgradeNotReadyCluster(FlinkDeployment appCluster) throws Excep // triggering upgrade with no last-state fallback on non-healthy app flinkService.setPortReady(false); - appCluster .getSpec() .getFlinkConfiguration() diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobControllerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobControllerTest.java index f8a1f8a2f0..09ff806e8c 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobControllerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobControllerTest.java @@ -516,7 +516,6 @@ public void verifyReconcileWithBadConfig() throws Exception { UpdateControl updateControl; // Override headers, and it should be saved in lastReconciledSpec once a successful // reconcile() finishes. - sessionJob .getSpec() .getFlinkConfiguration() @@ -540,7 +539,6 @@ public void verifyReconcileWithBadConfig() throws Exception { // Make sure we do validation before getting effective config in reconcile(). // Verify the saved headers in lastReconciledSpec is actually used in observe() by // utilizing listJobConsumer - sessionJob .getSpec() .getFlinkConfiguration() diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkStateSnapshotControllerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkStateSnapshotControllerTest.java index 2ecf8762a6..cfe6947343 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkStateSnapshotControllerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkStateSnapshotControllerTest.java @@ -750,7 +750,6 @@ private FlinkDeployment createDeployment(FlinkVersion flinkVersion) { .getStatus() .setJobStatus(JobStatus.builder().state(RUNNING).jobId(JOB_ID).build()); deployment.getSpec().setFlinkVersion(flinkVersion); - deployment .getSpec() .getFlinkConfiguration() diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/RollbackTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/RollbackTest.java index 6be1729cdd..2dc00a183a 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/RollbackTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/RollbackTest.java @@ -170,8 +170,6 @@ public void testSavepointNoRollbackWithoutHaMetadataAndJMWasReady() throws Excep deployment.getSpec().getJob().setUpgradeMode(UpgradeMode.SAVEPOINT); offsetReconcilerClock(deployment, Duration.ZERO); - var flinkConfiguration = deployment.getSpec().getFlinkConfiguration(); - deployment .getSpec() .getFlinkConfiguration() @@ -221,7 +219,6 @@ public void testSavepointNoRollbackWithoutHaMetadataAndJMWasReady() throws Excep public void testRollbackFailureWithLastState() throws Exception { var dep = TestUtils.buildApplicationCluster(); dep.getSpec().getJob().setUpgradeMode(UpgradeMode.LAST_STATE); - dep.getSpec().getFlinkConfiguration().put("t", "1"); offsetReconcilerClock(dep, Duration.ZERO); @@ -308,7 +305,6 @@ public void testRollbackStateless() throws Exception { .getSubmittedConf() .getString("test.deploy.config", "unknown")); // Validate that rollback config is picked up from latest deploy conf - dep.getSpec() .getFlinkConfiguration() .put( diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserverTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserverTest.java index 71176201b2..f3ff7a8b73 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserverTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserverTest.java @@ -745,7 +745,6 @@ private static Stream cancellingArgs() { private static FlinkDeployment initDeployment() { FlinkDeployment deployment = TestUtils.buildApplicationCluster(); var jobId = new JobID().toHexString(); - deployment .getSpec() .getFlinkConfiguration() @@ -761,11 +760,9 @@ private static FlinkDeployment initDeployment() { private static FlinkSessionJob initSessionJob() { var job = TestUtils.buildSessionJob(); var jobId = new JobID().toHexString(); - job.getSpec() .getFlinkConfiguration() .put(PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID.key(), jobId); - job.getStatus().getJobStatus().setJobId(jobId); job.getStatus() .getReconciliationStatus() diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/ApplicationObserverTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/ApplicationObserverTest.java index ae49a477fe..31493b5014 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/ApplicationObserverTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/ApplicationObserverTest.java @@ -82,12 +82,10 @@ public void setup() { readyContext = TestUtils.createContextWithReadyJobManagerDeployment(kubernetesClient); deployment = TestUtils.buildApplicationCluster(); var jobId = new JobID().toHexString(); - deployment .getSpec() .getFlinkConfiguration() .put(PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID.key(), jobId); - deployment.getStatus().getJobStatus().setJobId(jobId); } @@ -247,7 +245,6 @@ public void observeSavepoint() throws Exception { deployment.getSpec().getJob().setSavepointTriggerNonce(timedOutNonce); Configuration conf = configManager.getDeployConfig(deployment.getMetadata(), deployment.getSpec()); - deployment .getSpec() .getFlinkConfiguration() diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerTest.java index c13c497a19..a5d87bc946 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerTest.java @@ -207,7 +207,6 @@ public void testSubmitAndCleanUpWithSavepoint(FlinkVersion flinkVersion) throws public void testSubmitAndCleanUpWithSavepointOnResource(FlinkVersion flinkVersion) throws Exception { FlinkDeployment deployment = TestUtils.buildApplicationCluster(flinkVersion); - deployment .getSpec() .getFlinkConfiguration() @@ -443,7 +442,6 @@ public void triggerSavepointLegacy() throws Exception { public void triggerSavepointWithSnapshotResource(boolean disposeOnDelete) throws Exception { var deployment = TestUtils.buildApplicationCluster(); if (disposeOnDelete) { - deployment .getSpec() .getFlinkConfiguration() @@ -757,7 +755,6 @@ public void testJobUpgradeIgnorePendingSavepointLegacy() throws Exception { getJobStatus(spDeployment).getState()); // Force upgrade when savepoint is in progress. - spDeployment .getSpec() .getFlinkConfiguration() @@ -776,7 +773,6 @@ public void testJobUpgradeIgnorePendingSavepointLegacy() throws Exception { public void testRandomJobResultStorePath() throws Exception { FlinkDeployment flinkApp = TestUtils.buildApplicationCluster(); final String haStoragePath = "file:///flink-data/ha"; - flinkApp.getSpec() .getFlinkConfiguration() .put(HighAvailabilityOptions.HA_STORAGE_PATH.key(), haStoragePath); @@ -847,7 +843,6 @@ public void testScaleWithReactiveModeEnabled() throws Exception { FlinkDeployment deployment = TestUtils.buildApplicationCluster(); deployment.getSpec().setMode(KubernetesDeploymentMode.STANDALONE); - deployment .getSpec() .getFlinkConfiguration() @@ -927,7 +922,6 @@ public CancelResult cancelJob( // Set all the properties required by the rescale api deployment.getSpec().setFlinkVersion(FlinkVersion.v1_18); deployment.getSpec().setMode(KubernetesDeploymentMode.NATIVE); - deployment .getSpec() .getFlinkConfiguration() @@ -941,7 +935,6 @@ public CancelResult cancelJob( verifyAndSetRunningJobsToStatus(deployment, flinkService.listJobs()); // Override parallelism for a vertex and trigger rescaling - deployment .getSpec() .getFlinkConfiguration() @@ -1253,7 +1246,6 @@ public void testRestartUnhealthyEvent() throws Exception { .getSpec() .getFlinkConfiguration() .put(OPERATOR_CLUSTER_HEALTH_CHECK_ENABLED.key(), "true"); - reconciler.reconcile(deployment, context); Assertions.assertEquals( MSG_SUBMIT, flinkResourceEventCollector.events.remove().getMessage()); @@ -1276,7 +1268,6 @@ public void testReconcileIfUpgradeModeNotAvailable() throws Exception { // We disable last state fallback as we want to test that the deployment is properly // recovered before upgrade - deployment .getSpec() .getFlinkConfiguration() diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java index 7d164a7be5..3b50af2a76 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java @@ -162,7 +162,6 @@ private void testUpgradeToSavepoint(FlinkVersion flinkVersion, UpgradeMode fromU FlinkDeployment modifiedDeployment = cloneDeploymentWithUpgradeMode(deployment, UpgradeMode.SAVEPOINT); - modifiedDeployment .getSpec() .getFlinkConfiguration() @@ -245,7 +244,6 @@ private void testUpgradeToLastState(FlinkVersion flinkVersion, UpgradeMode fromU .setJobManagerDeploymentStatus(JobManagerDeploymentStatus.ERROR); reconciler.reconcile(deployment, context); }); - flinkService.clear(); deployment.getSpec().getJob().setUpgradeMode(UpgradeMode.LAST_STATE); deployment.getSpec().setRestartNonce(200L); @@ -503,7 +501,6 @@ public void testInitialJmDeployCannotStartLegacy(UpgradeMode upgradeMode, boolea @ValueSource(booleans = {true, false}) public void testLastStateMaxCheckpointAge(boolean cancellable) throws Exception { var deployment = TestUtils.buildApplicationCluster(); - deployment .getSpec() .getFlinkConfiguration() @@ -621,13 +618,11 @@ public void testFlinkVersionSwitching( var jobReconciler = (ApplicationReconciler) this.reconciler.getReconciler(); var deployment = TestUtils.buildApplicationCluster(FlinkVersion.v1_18); if (!savepointsEnabled) { - deployment .getSpec() .getFlinkConfiguration() .remove(CheckpointingOptions.SAVEPOINT_DIRECTORY.key()); } - deployment .getSpec() .getFlinkConfiguration() @@ -685,7 +680,6 @@ public void testLastStateNoHaMeta(UpgradeMode upgradeMode, boolean allowFallback throws Exception { var jobReconciler = (ApplicationReconciler) this.reconciler.getReconciler(); var deployment = TestUtils.buildApplicationCluster(); - deployment .getSpec() .getFlinkConfiguration() diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/SessionReconcilerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/SessionReconcilerTest.java index fa57522d7e..930ad9a4d5 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/SessionReconcilerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/SessionReconcilerTest.java @@ -151,7 +151,6 @@ public void testSetOwnerReference() throws Exception { @Test public void testGetNonTerminalJobs() throws Exception { FlinkDeployment deployment = TestUtils.buildSessionCluster(); - deployment .getSpec() .getFlinkConfiguration() diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/diff/SpecDiffTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/diff/SpecDiffTest.java index e5969d9a4e..13b3c2c473 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/diff/SpecDiffTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/diff/SpecDiffTest.java @@ -209,7 +209,6 @@ public void testFlinkSessionJobSpecChanges() { right.getJob().setAllowNonRestoredState(true); right.getJob().setInitialSavepointPath("local:///tmp"); right.getJob().setSavepointTriggerNonce(123L); - right.getFlinkConfiguration() .put(KubernetesOperatorConfigOptions.JAR_ARTIFACT_HTTP_HEADER.key(), "changed"); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/sessionjob/SessionJobReconcilerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/sessionjob/SessionJobReconcilerTest.java index 5a98068196..b001ed7739 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/sessionjob/SessionJobReconcilerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/sessionjob/SessionJobReconcilerTest.java @@ -148,7 +148,6 @@ public void testSubmitAndCleanUpWithSavepoint(boolean legacySnapshots) throws Ex public void testSubmitAndCleanUpWithSavepointOnResource(boolean legacySnapshots) throws Exception { FlinkSessionJob sessionJob = TestUtils.buildSessionJob(); - sessionJob .getSpec() .getFlinkConfiguration() diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkServiceTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkServiceTest.java index a03d7cd9c6..a57d60a8a3 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkServiceTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkServiceTest.java @@ -382,7 +382,6 @@ public void cancelJobWithSavepointUpgradeModeTest(boolean deleteAfterSavepoint) JobID jobID = JobID.generate(); FlinkDeployment deployment = TestUtils.buildApplicationCluster(); - deployment .getSpec() .getFlinkConfiguration() @@ -492,7 +491,6 @@ public void cancelJobWithDrainOnSavepointUpgradeModeTest(boolean drainOnSavepoin JobID jobID = JobID.generate(); FlinkDeployment deployment = TestUtils.buildApplicationCluster(); - deployment .getSpec() .getFlinkConfiguration() @@ -504,12 +502,10 @@ public void cancelJobWithDrainOnSavepointUpgradeModeTest(boolean drainOnSavepoin ReconciliationUtils.updateStatusForDeployedSpec(deployment, new Configuration()); if (drainOnSavepoint) { - deployment .getSpec() .getFlinkConfiguration() .put(KubernetesOperatorConfigOptions.SAVEPOINT_ON_DELETION.key(), "true"); - deployment .getSpec() .getFlinkConfiguration() @@ -572,11 +568,9 @@ public void cancelSessionJobWithDrainOnSavepointUpgradeModeTest(boolean drainOnS ReconciliationUtils.updateStatusForDeployedSpec(job, new Configuration()); if (drainOnSavepoint) { - job.getSpec() .getFlinkConfiguration() .put(KubernetesOperatorConfigOptions.SAVEPOINT_ON_DELETION.key(), "true"); - job.getSpec() .getFlinkConfiguration() .put(KubernetesOperatorConfigOptions.DRAIN_ON_SAVEPOINT_DELETION.key(), "true"); @@ -807,7 +801,6 @@ private void runNativeSavepointFormatTest(boolean failAfterSavepointCompletes) var flinkService = new TestingService(testingClusterClient); final FlinkDeployment deployment = TestUtils.buildApplicationCluster(); - deployment .getSpec() .getFlinkConfiguration() diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/NativeFlinkServiceTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/NativeFlinkServiceTest.java index 2dcf843397..666f24da81 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/NativeFlinkServiceTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/NativeFlinkServiceTest.java @@ -400,7 +400,6 @@ protected void updateVertexResources( // Do not scale if parallelism overrides were completely removed var flinkDep2 = ReconciliationUtils.clone(flinkDep); - flinkDep2 .getSpec() .getFlinkConfiguration() diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/StandaloneFlinkServiceTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/StandaloneFlinkServiceTest.java index 29662ad888..fde2b12421 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/StandaloneFlinkServiceTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/StandaloneFlinkServiceTest.java @@ -133,7 +133,6 @@ public void testTMReplicaScaleApplication() { flinkDeployment.getSpec().setMode(KubernetesDeploymentMode.STANDALONE); // Add parallelism change, verify it is honoured in reactive mode - flinkDeployment .getSpec() .getFlinkConfiguration() @@ -192,7 +191,6 @@ public void testTMReplicaScaleSession() { flinkDeployment.getSpec().setMode(KubernetesDeploymentMode.STANDALONE); // Add replicas flinkDeployment.getSpec().getTaskManager().setReplicas(3); - flinkDeployment .getSpec() .getFlinkConfiguration() diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/SnapshotUtilsTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/SnapshotUtilsTest.java index 9e9a222512..5b3cb541cc 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/SnapshotUtilsTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/SnapshotUtilsTest.java @@ -132,7 +132,6 @@ private void testSnapshotTriggering( .getSpec() .getFlinkConfiguration() .put(periodicSnapshotIntervalOption.key(), "10m"); - reconcileSpec(deployment); assertEquals( diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/validation/DefaultValidatorTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/validation/DefaultValidatorTest.java index c0472e1e85..5738555bbe 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/validation/DefaultValidatorTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/validation/DefaultValidatorTest.java @@ -136,7 +136,6 @@ public void testValidationWithoutDefaultConfig() { dep.getSpec() .getFlinkConfiguration() .remove(CheckpointingOptions.CHECKPOINTS_DIRECTORY.key()); - dep.getSpec().getJob().setUpgradeMode(UpgradeMode.LAST_STATE); }, "Checkpoint directory"); @@ -475,7 +474,6 @@ public void testValidationWithoutDefaultConfig() { testSuccess( dep -> { dep.getSpec().getJob().setUpgradeMode(UpgradeMode.LAST_STATE); - dep.getSpec() .getFlinkConfiguration() .put( From 17030bea2ea580f21a652e38b6c33ef4440128d8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Attila=20M=C3=A9sz=C3=A1ros?= Date: Wed, 20 Aug 2025 13:58:16 +0200 Subject: [PATCH 15/19] crd checker and changed e2e test MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Attila Mészáros --- e2e-tests/data/autoscaler.yaml | 22 ++-- e2e-tests/data/flinkdep-cr.yaml | 10 +- e2e-tests/data/multi-sessionjob.yaml | 10 +- e2e-tests/data/sessionjob-cr.yaml | 10 +- flink-kubernetes-operator-api/pom.xml | 112 +++++++++--------- .../validation/CrdCompatibilityChecker.java | 20 ++++ .../CrdCompatibilityCheckerTest.java | 17 +++ 7 files changed, 123 insertions(+), 78 deletions(-) diff --git a/e2e-tests/data/autoscaler.yaml b/e2e-tests/data/autoscaler.yaml index ca106ce1cd..2f9a6f3874 100644 --- a/e2e-tests/data/autoscaler.yaml +++ b/e2e-tests/data/autoscaler.yaml @@ -31,16 +31,18 @@ spec: nginx.ingress.kubernetes.io/rewrite-target: "/$2" flinkConfiguration: taskmanager.numberOfTaskSlots: "2" - high-availability.type: kubernetes - high-availability.storageDir: file:///opt/flink/volume/flink-ha - state.checkpoints.dir: file:///opt/flink/volume/flink-cp - state.savepoints.dir: file:///opt/flink/volume/flink-sp - - job.autoscaler.enabled: "true" - job.autoscaler.scaling.enabled: "true" - job.autoscaler.stabilization.interval: "5s" - job.autoscaler.metrics.window: "1m" - job.autoscaler.scale-down.interval: "0m" + high-availability: + type: kubernetes + storageDir: file:///opt/flink/volume/flink-ha + state: + checkpoints.dir: file:///opt/flink/volume/flink-cp + savepoints.dir: file:///opt/flink/volume/flink-sp + job.autoscaler: + enabled: "true" + scaling.enabled: "true" + stabilization.interval: "5s" + metrics.window: "1m" + scale-down.interval: "0m" # Invalid Validations for testing autoscaler configurations # kubernetes.operator.job.autoscaler.scale-down.max-factor: "-0.6" diff --git a/e2e-tests/data/flinkdep-cr.yaml b/e2e-tests/data/flinkdep-cr.yaml index f4847678e0..7c04f7631a 100644 --- a/e2e-tests/data/flinkdep-cr.yaml +++ b/e2e-tests/data/flinkdep-cr.yaml @@ -31,10 +31,12 @@ spec: nginx.ingress.kubernetes.io/rewrite-target: "/$2" flinkConfiguration: taskmanager.numberOfTaskSlots: "2" - high-availability.type: kubernetes - high-availability.storageDir: file:///opt/flink/volume/flink-ha - state.checkpoints.dir: file:///opt/flink/volume/flink-cp - state.savepoints.dir: file:///opt/flink/volume/flink-sp + high-availability: + type: kubernetes + storageDir: file:///opt/flink/volume/flink-ha + state: + checkpoints.dir: file:///opt/flink/volume/flink-cp + savepoints.dir: file:///opt/flink/volume/flink-sp kubernetes.operator.snapshot.resource.enabled: "false" serviceAccount: flink podTemplate: diff --git a/e2e-tests/data/multi-sessionjob.yaml b/e2e-tests/data/multi-sessionjob.yaml index e8f84ce1a8..6173c8c2f7 100644 --- a/e2e-tests/data/multi-sessionjob.yaml +++ b/e2e-tests/data/multi-sessionjob.yaml @@ -31,10 +31,12 @@ spec: nginx.ingress.kubernetes.io/rewrite-target: "/$2" flinkConfiguration: taskmanager.numberOfTaskSlots: "2" - high-availability.type: kubernetes - high-availability.storageDir: file:///opt/flink/volume/flink-ha - state.checkpoints.dir: file:///opt/flink/volume/flink-cp - state.savepoints.dir: file:///opt/flink/volume/flink-sp + high-availability: + type: kubernetes + storageDir: file:///opt/flink/volume/flink-ha + state: + checkpoints.dir: file:///opt/flink/volume/flink-cp + savepoints.dir: file:///opt/flink/volume/flink-sp serviceAccount: flink podTemplate: spec: diff --git a/e2e-tests/data/sessionjob-cr.yaml b/e2e-tests/data/sessionjob-cr.yaml index 1d96988f73..8f9db85b64 100644 --- a/e2e-tests/data/sessionjob-cr.yaml +++ b/e2e-tests/data/sessionjob-cr.yaml @@ -31,10 +31,12 @@ spec: nginx.ingress.kubernetes.io/rewrite-target: "/$2" flinkConfiguration: taskmanager.numberOfTaskSlots: "2" - high-availability.type: kubernetes - high-availability.storageDir: file:///opt/flink/volume/flink-ha - state.checkpoints.dir: file:///opt/flink/volume/flink-cp - state.savepoints.dir: file:///opt/flink/volume/flink-sp + high-availability: + type: kubernetes + storageDir: file:///opt/flink/volume/flink-ha + state: + checkpoints.dir: file:///opt/flink/volume/flink-cp + savepoints.dir: file:///opt/flink/volume/flink-sp kubernetes.operator.snapshot.resource.enabled: "false" serviceAccount: flink podTemplate: diff --git a/flink-kubernetes-operator-api/pom.xml b/flink-kubernetes-operator-api/pom.xml index 0befbb0534..2e83d291d2 100644 --- a/flink-kubernetes-operator-api/pom.xml +++ b/flink-kubernetes-operator-api/pom.xml @@ -254,62 +254,62 @@ under the License. - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - + + maven-antrun-plugin + + + deployment-crd-compatibility-check + package + + run + + + + + + + + + + + + + sessionjob-crd-compatibility-check + package + + run + + + + + + + + + + + + + statesnapshot-crd-compatibility-check + package + + run + + + + + + + + + + + + + org.apache.maven.plugins maven-jar-plugin diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/validation/CrdCompatibilityChecker.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/validation/CrdCompatibilityChecker.java index 3a44ef540a..fa45fc9847 100644 --- a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/validation/CrdCompatibilityChecker.java +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/validation/CrdCompatibilityChecker.java @@ -125,6 +125,16 @@ protected static void checkTypeCompatibility(String path, JsonNode oldNode, Json } } + if (oldNode.get("type") == null && newNode.get("type") != null) { + err("Type mismatch for " + path + ". Old node type is null, while new node is not"); + } + if (oldNode.get("type") != null && newNode.get("type") == null) { + if (isGeneralizingAdditionalPropertiesForYaml(oldNode, newNode)) { + return; + } + err("Type mismatch for " + path + ". Old node type is not null, while new node null"); + } + String oldType = oldNode.get("type").asText(); if (!oldType.equals(newNode.get("type").asText())) { @@ -149,6 +159,16 @@ protected static void checkTypeCompatibility(String path, JsonNode oldNode, Json } } + private static boolean isGeneralizingAdditionalPropertiesForYaml( + JsonNode oldNode, JsonNode newNode) { + var oldAdditionalProperties = oldNode.get("additionalProperties"); + + return oldAdditionalProperties != null + && "object".equals(oldNode.get("type").asText()) + && "string".equals(oldAdditionalProperties.get("type").asText()) + && "true".equals(newNode.get("x-kubernetes-preserve-unknown-fields").asText()); + } + protected static void verifyOtherPropsMatch(String path, JsonNode oldNode, JsonNode newNode) { var oldCopy = (ObjectNode) oldNode.deepCopy(); var newCopy = (ObjectNode) newNode.deepCopy(); diff --git a/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/validation/CrdCompatibilityCheckerTest.java b/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/validation/CrdCompatibilityCheckerTest.java index 90a7924b87..45ce2ccd7a 100644 --- a/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/validation/CrdCompatibilityCheckerTest.java +++ b/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/validation/CrdCompatibilityCheckerTest.java @@ -247,6 +247,23 @@ public void testOtherPropertyMismatch() throws Exception { + " type: object"); } + @Test + void generalizingPropertyMapToAnyType() throws Exception { + expectSuccess( + "openAPIV3Schema:\n" + + " properties:\n" + + " flinkConfiguration:\n" + + " additionalProperties:\n" + + " type: string\n" + + " type: object\n" + + " type: object", + "openAPIV3Schema:\n" + + " properties:\n" + + " flinkConfiguration:\n" + + " x-kubernetes-preserve-unknown-fields: true\n" + + " type: object"); + } + @Test public void testCreateFlinkSessionJobIgnoreUnknownFields() throws IOException { FlinkSessionJob flinkSessionJobWithUnknownFields = From 43c246280c514065c196bc08bb08b9e2261c5ba9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Attila=20M=C3=A9sz=C3=A1ros?= Date: Wed, 20 Aug 2025 14:29:26 +0200 Subject: [PATCH 16/19] javadocs fix MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Attila Mészáros --- .../content/docs/custom-resource/reference.md | 20 +++++++++++++++++-- 1 file changed, 18 insertions(+), 2 deletions(-) diff --git a/docs/content/docs/custom-resource/reference.md b/docs/content/docs/custom-resource/reference.md index 909343c103..250b9ed00b 100644 --- a/docs/content/docs/custom-resource/reference.md +++ b/docs/content/docs/custom-resource/reference.md @@ -57,6 +57,22 @@ This serves as a full reference for FlinkDeployment and FlinkSessionJob custom r | Parameter | Type | Docs | | ----------| ---- | ---- | +### ConfigObjectNode +**Class**: org.apache.flink.kubernetes.operator.api.spec.ConfigObjectNode + +**Description**: + +| Parameter | Type | Docs | +| ----------| ---- | ---- | + +### ConfigObjectNodeDeserializer +**Class**: org.apache.flink.kubernetes.operator.api.spec.ConfigObjectNodeDeserializer + +**Description**: Allows to deserialize to ConfigObjectNode. + +| Parameter | Type | Docs | +| ----------| ---- | ---- | + ### FlinkDeploymentSpec **Class**: org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec @@ -66,7 +82,7 @@ This serves as a full reference for FlinkDeployment and FlinkSessionJob custom r | ----------| ---- | ---- | | job | org.apache.flink.kubernetes.operator.api.spec.JobSpec | Job specification for application deployments/session job. Null for session clusters. | | restartNonce | java.lang.Long | Nonce used to manually trigger restart for the cluster/session job. In order to trigger restart, change the number to a different non-null value. | -| flinkConfiguration | java.util.Map | Flink configuration overrides for the Flink deployment or Flink session job. | +| flinkConfiguration | org.apache.flink.kubernetes.operator.api.spec.ConfigObjectNode | Flink configuration overrides for the Flink deployment or Flink session job. | | image | java.lang.String | Flink docker image used to start the Job and TaskManager pods. | | imagePullPolicy | java.lang.String | Image pull policy of the Flink docker image. | | serviceAccount | java.lang.String | Kubernetes service used by the Flink deployment. | @@ -87,7 +103,7 @@ This serves as a full reference for FlinkDeployment and FlinkSessionJob custom r | ----------| ---- | ---- | | job | org.apache.flink.kubernetes.operator.api.spec.JobSpec | Job specification for application deployments/session job. Null for session clusters. | | restartNonce | java.lang.Long | Nonce used to manually trigger restart for the cluster/session job. In order to trigger restart, change the number to a different non-null value. | -| flinkConfiguration | java.util.Map | Flink configuration overrides for the Flink deployment or Flink session job. | +| flinkConfiguration | org.apache.flink.kubernetes.operator.api.spec.ConfigObjectNode | Flink configuration overrides for the Flink deployment or Flink session job. | | deploymentName | java.lang.String | The name of the target session cluster deployment. | ### FlinkStateSnapshotSpec From 1494b104fd849a407419753bdd3f4cd10bd8d927 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Attila=20M=C3=A9sz=C3=A1ros?= Date: Wed, 20 Aug 2025 14:33:28 +0200 Subject: [PATCH 17/19] add javadoc for ConfigObjectNode MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Attila Mészáros --- docs/content/docs/custom-resource/reference.md | 2 +- .../flink/kubernetes/operator/api/spec/ConfigObjectNode.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/content/docs/custom-resource/reference.md b/docs/content/docs/custom-resource/reference.md index 250b9ed00b..ed8e1d3042 100644 --- a/docs/content/docs/custom-resource/reference.md +++ b/docs/content/docs/custom-resource/reference.md @@ -60,7 +60,7 @@ This serves as a full reference for FlinkDeployment and FlinkSessionJob custom r ### ConfigObjectNode **Class**: org.apache.flink.kubernetes.operator.api.spec.ConfigObjectNode -**Description**: +**Description**: Allows parsing configurations as YAML, and adds related utility methods. | Parameter | Type | Docs | | ----------| ---- | ---- | diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigObjectNode.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigObjectNode.java index 7b5356328c..348a3fb537 100644 --- a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigObjectNode.java +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigObjectNode.java @@ -28,7 +28,7 @@ import java.util.Iterator; import java.util.Map; -/** */ +/** Allows parsing configurations as YAML, and adds related utility methods. */ public class ConfigObjectNode extends ObjectNode { public ConfigObjectNode() { From 96269316ff52edf9281abff408979ef08016c6aa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Attila=20M=C3=A9sz=C3=A1ros?= Date: Wed, 20 Aug 2025 15:52:36 +0200 Subject: [PATCH 18/19] use proper values not string in yaml MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Attila Mészáros --- e2e-tests/data/autoscaler.yaml | 2 +- e2e-tests/data/flinkdep-batch-cr.yaml | 4 ++-- e2e-tests/data/flinkdep-cr.yaml | 4 ++-- e2e-tests/data/multi-sessionjob.yaml | 2 +- e2e-tests/data/sessionjob-cr.yaml | 4 ++-- 5 files changed, 8 insertions(+), 8 deletions(-) diff --git a/e2e-tests/data/autoscaler.yaml b/e2e-tests/data/autoscaler.yaml index 2f9a6f3874..3d83f8359e 100644 --- a/e2e-tests/data/autoscaler.yaml +++ b/e2e-tests/data/autoscaler.yaml @@ -30,7 +30,7 @@ spec: annotations: nginx.ingress.kubernetes.io/rewrite-target: "/$2" flinkConfiguration: - taskmanager.numberOfTaskSlots: "2" + taskmanager.numberOfTaskSlots: 2 high-availability: type: kubernetes storageDir: file:///opt/flink/volume/flink-ha diff --git a/e2e-tests/data/flinkdep-batch-cr.yaml b/e2e-tests/data/flinkdep-batch-cr.yaml index 159199ce4f..9829b41900 100644 --- a/e2e-tests/data/flinkdep-batch-cr.yaml +++ b/e2e-tests/data/flinkdep-batch-cr.yaml @@ -30,8 +30,8 @@ spec: annotations: nginx.ingress.kubernetes.io/rewrite-target: "/$2" flinkConfiguration: - taskmanager.numberOfTaskSlots: "2" - kubernetes.operator.snapshot.resource.enabled: "false" + taskmanager.numberOfTaskSlots: 2 + kubernetes.operator.snapshot.resource.enabled: false serviceAccount: flink podTemplate: spec: diff --git a/e2e-tests/data/flinkdep-cr.yaml b/e2e-tests/data/flinkdep-cr.yaml index 7c04f7631a..3d351dfe66 100644 --- a/e2e-tests/data/flinkdep-cr.yaml +++ b/e2e-tests/data/flinkdep-cr.yaml @@ -30,14 +30,14 @@ spec: annotations: nginx.ingress.kubernetes.io/rewrite-target: "/$2" flinkConfiguration: - taskmanager.numberOfTaskSlots: "2" + taskmanager.numberOfTaskSlots: 2 high-availability: type: kubernetes storageDir: file:///opt/flink/volume/flink-ha state: checkpoints.dir: file:///opt/flink/volume/flink-cp savepoints.dir: file:///opt/flink/volume/flink-sp - kubernetes.operator.snapshot.resource.enabled: "false" + kubernetes.operator.snapshot.resource.enabled: false serviceAccount: flink podTemplate: spec: diff --git a/e2e-tests/data/multi-sessionjob.yaml b/e2e-tests/data/multi-sessionjob.yaml index 6173c8c2f7..41b364a0b2 100644 --- a/e2e-tests/data/multi-sessionjob.yaml +++ b/e2e-tests/data/multi-sessionjob.yaml @@ -30,7 +30,7 @@ spec: annotations: nginx.ingress.kubernetes.io/rewrite-target: "/$2" flinkConfiguration: - taskmanager.numberOfTaskSlots: "2" + taskmanager.numberOfTaskSlots: 2 high-availability: type: kubernetes storageDir: file:///opt/flink/volume/flink-ha diff --git a/e2e-tests/data/sessionjob-cr.yaml b/e2e-tests/data/sessionjob-cr.yaml index 8f9db85b64..13f26b4e9e 100644 --- a/e2e-tests/data/sessionjob-cr.yaml +++ b/e2e-tests/data/sessionjob-cr.yaml @@ -30,14 +30,14 @@ spec: annotations: nginx.ingress.kubernetes.io/rewrite-target: "/$2" flinkConfiguration: - taskmanager.numberOfTaskSlots: "2" + taskmanager.numberOfTaskSlots: 2 high-availability: type: kubernetes storageDir: file:///opt/flink/volume/flink-ha state: checkpoints.dir: file:///opt/flink/volume/flink-cp savepoints.dir: file:///opt/flink/volume/flink-sp - kubernetes.operator.snapshot.resource.enabled: "false" + kubernetes.operator.snapshot.resource.enabled: false serviceAccount: flink podTemplate: spec: From c74e56b05758f4da0e95a9f40cd704ffcc18480a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Attila=20M=C3=A9sz=C3=A1ros?= Date: Thu, 21 Aug 2025 10:52:08 +0200 Subject: [PATCH 19/19] fix duplicate dependency MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Attila Mészáros --- flink-kubernetes-operator-api/pom.xml | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/flink-kubernetes-operator-api/pom.xml b/flink-kubernetes-operator-api/pom.xml index 2e83d291d2..14e7c18beb 100644 --- a/flink-kubernetes-operator-api/pom.xml +++ b/flink-kubernetes-operator-api/pom.xml @@ -125,14 +125,7 @@ under the License. ${flink.version} test - - - org.assertj - assertj-core - ${assertj.version} - test - - + org.apache.flink flink-runtime