diff --git a/docs/content/docs/custom-resource/reference.md b/docs/content/docs/custom-resource/reference.md index 909343c103..ed8e1d3042 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**: Allows parsing configurations as YAML, and adds related utility methods. + +| 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 diff --git a/e2e-tests/data/autoscaler.yaml b/e2e-tests/data/autoscaler.yaml index ca106ce1cd..3d83f8359e 100644 --- a/e2e-tests/data/autoscaler.yaml +++ b/e2e-tests/data/autoscaler.yaml @@ -30,17 +30,19 @@ spec: annotations: 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" + 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 + 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-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 f4847678e0..3d351dfe66 100644 --- a/e2e-tests/data/flinkdep-cr.yaml +++ b/e2e-tests/data/flinkdep-cr.yaml @@ -30,12 +30,14 @@ spec: annotations: 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 - kubernetes.operator.snapshot.resource.enabled: "false" + 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 serviceAccount: flink podTemplate: spec: diff --git a/e2e-tests/data/multi-sessionjob.yaml b/e2e-tests/data/multi-sessionjob.yaml index e8f84ce1a8..41b364a0b2 100644 --- a/e2e-tests/data/multi-sessionjob.yaml +++ b/e2e-tests/data/multi-sessionjob.yaml @@ -30,11 +30,13 @@ spec: annotations: 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 + 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 serviceAccount: flink podTemplate: spec: diff --git a/e2e-tests/data/sessionjob-cr.yaml b/e2e-tests/data/sessionjob-cr.yaml index 1d96988f73..13f26b4e9e 100644 --- a/e2e-tests/data/sessionjob-cr.yaml +++ b/e2e-tests/data/sessionjob-cr.yaml @@ -30,12 +30,14 @@ spec: annotations: 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 - kubernetes.operator.snapshot.resource.enabled: "false" + 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 serviceAccount: flink podTemplate: spec: diff --git a/flink-kubernetes-operator-api/pom.xml b/flink-kubernetes-operator-api/pom.xml index 21b635e5ee..14e7c18beb 100644 --- a/flink-kubernetes-operator-api/pom.xml +++ b/flink-kubernetes-operator-api/pom.xml @@ -125,7 +125,7 @@ under the License. ${flink.version} test - + org.apache.flink flink-runtime @@ -138,6 +138,33 @@ under the License. junit-jupiter-params test + + + org.assertj + assertj-core + ${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/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..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 @@ -18,14 +18,18 @@ 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; @@ -57,5 +61,18 @@ public abstract class AbstractFlinkSpec implements Diffable { type = DiffType.SCALE, mode = KubernetesDeploymentMode.NATIVE) }) - private Map flinkConfiguration; + @JsonDeserialize(using = ConfigObjectNodeDeserializer.class) + private ConfigObjectNode flinkConfiguration = new ConfigObjectNode(); + + @Tolerate + @JsonIgnore + public void setFlinkConfiguration(Map config) { + flinkConfiguration.setAllFrom(config); + } + + @Tolerate + @JsonIgnore + public void setFlinkConfiguration(Configuration config) { + setFlinkConfiguration(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 new file mode 100644 index 0000000000..348a3fb537 --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigObjectNode.java @@ -0,0 +1,89 @@ +/* + * 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 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; + +/** Allows parsing configurations as YAML, and adds related utility methods. */ +public class ConfigObjectNode extends ObjectNode { + + public ConfigObjectNode() { + this(JsonNodeFactory.instance); + } + + public ConfigObjectNode(JsonNodeFactory nc, Map kids) { + super(nc, kids); + } + + public ConfigObjectNode(JsonNodeFactory nc) { + super(nc); + } + + public void remove(String... names) { + remove(Arrays.asList(names)); + } + + public void putAllFrom(Map value) { + value.forEach(this::put); + } + + public void setAllFrom(Map value) { + removeAll(); + putAllFrom(value); + } + + public Map asFlatMap() { + Map flatMap = new HashMap<>(); + flattenHelper(this, "", flatMap); + return flatMap; + } + + public Configuration asConfiguration() { + return Configuration.fromMap(asFlatMap()); + } + + 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/ConfigObjectNodeDeserializer.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigObjectNodeDeserializer.java new file mode 100644 index 0000000000..ea7cb0cf51 --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/ConfigObjectNodeDeserializer.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; + +/** Allows to deserialize to ConfigObjectNode. */ +public class ConfigObjectNodeDeserializer extends JsonDeserializer { + + @Override + public ConfigObjectNode deserialize( + JsonParser jsonParser, DeserializationContext deserializationContext) + throws IOException { + ObjectNode tree = jsonParser.readValueAsTree(); + var res = new ConfigObjectNode(); + 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 458dd69173..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 @@ -99,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(); 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/FlinkConfigurationYamlSupportTest.java b/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/FlinkConfigurationYamlSupportTest.java new file mode 100644 index 0000000000..2302938d59 --- /dev/null +++ b/flink-kubernetes-operator-api/src/test/java/org/apache/flink/kubernetes/operator/api/FlinkConfigurationYamlSupportTest.java @@ -0,0 +1,93 @@ +/* + * 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 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(deployment.getSpec().getFlinkConfiguration().asFlatMap()).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/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..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,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.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; @@ -48,9 +49,7 @@ 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; /** Base Testing utilities. */ @@ -144,7 +143,7 @@ public static FlinkSessionJob buildSessionJob( .withResourceVersion("1") .build()); - Map conf = new HashMap<>(); + ConfigObjectNode conf = new ConfigObjectNode(); conf.put("kubernetes.operator.user.artifacts.http.header", "header"); sessionJob.setSpec( FlinkSessionJobSpec.builder() @@ -170,7 +169,7 @@ public static FlinkSessionJob buildSessionJob(JobState state) { } public static FlinkDeploymentSpec getTestFlinkDeploymentSpec(FlinkVersion version) { - Map conf = new HashMap<>(); + ConfigObjectNode conf = new ConfigObjectNode(); conf.put(TaskManagerOptions.NUM_TASK_SLOTS.key(), "2"); conf.put( HighAvailabilityOptions.HA_MODE.key(), 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..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 @@ -30,10 +30,10 @@ import static org.junit.jupiter.api.Assertions.assertNull; /** Test for {@link SpecUtils}. */ -public class SpecUtilsTest { +class SpecUtilsTest { @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 +56,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); 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 = 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/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..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; @@ -45,7 +44,6 @@ public class KubernetesScalingRealizer @Override public void realizeParallelismOverrides( KubernetesJobAutoScalerContext context, Map parallelismOverrides) { - context.getResource() .getSpec() .getFlinkConfiguration() @@ -63,15 +61,15 @@ 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()); + + flinkDeployment.getSpec().getFlinkConfiguration().remove(configChanges.getRemovals()); + flinkDeployment.getSpec().getFlinkConfiguration().putAllFrom(configChanges.getOverrides()); // Update total memory in spec var totalMemoryOverride = - MemoryTuning.getTotalMemory(Configuration.fromMap(flinkConf), context); + MemoryTuning.getTotalMemory( + flinkDeployment.getSpec().getFlinkConfiguration().asConfiguration(), + 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..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 @@ -140,7 +140,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); + 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 8a2fd2651d..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 @@ -356,6 +356,7 @@ private void addOperatorConfigsFromSpec(AbstractFlinkSpec spec, Configuration co // Observe config should include the latest operator related settings if (spec.getFlinkConfiguration() != null) { spec.getFlinkConfiguration() + .asFlatMap() .forEach( (k, v) -> { if (k.startsWith(K8S_OP_CONF_PREFIX) @@ -371,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()); + var deployConfig = spec.getFlinkConfiguration().asConfiguration(); for (ConfigOption configOption : configOptions) { deployConfig.getOptional(configOption).ifPresent(v -> conf.set(configOption, v)); } @@ -394,7 +395,7 @@ public Configuration getSessionJobConfig( // merge session job specific config var sessionJobFlinkConfiguration = sessionJobSpec.getFlinkConfiguration(); if (sessionJobFlinkConfiguration != null) { - 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 5ab52d7f94..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,6 +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.ConfigObjectNode; import org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec; import org.apache.flink.kubernetes.operator.api.spec.KubernetesDeploymentMode; @@ -50,6 +51,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 +85,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 = ((ConfigObjectNode) leftField).asFlatMap(); + rightField = ((ConfigObjectNode) rightField).asFlatMap(); + } if (field.isAnnotationPresent(SpecDiff.Config.class) - && Map.class.isAssignableFrom(field.getType())) { + && (Map.class.isAssignableFrom(field.getType()) + || (field.getType().equals(ConfigObjectNode.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..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 @@ -102,7 +102,7 @@ public Optional validateDeployment(FlinkDeployment deployment) { deployment.getMetadata().getNamespace(), spec.getFlinkVersion()) .toMap(); if (spec.getFlinkConfiguration() != null) { - effectiveConfig.putAll(spec.getFlinkConfiguration()); + effectiveConfig.putAll(spec.getFlinkConfiguration().asFlatMap()); } return firstPresent( validateDeploymentName(deployment.getMetadata().getName()), @@ -531,11 +531,11 @@ private Optional validateSessionJobWithCluster( sessionCluster.getSpec().getFlinkVersion()) .toMap(); if (sessionCluster.getSpec().getFlinkConfiguration() != null) { - effectiveConfig.putAll(sessionCluster.getSpec().getFlinkConfiguration()); + effectiveConfig.putAll(sessionCluster.getSpec().getFlinkConfiguration().asFlatMap()); } if (sessionJob.getSpec().getFlinkConfiguration() != null) { - effectiveConfig.putAll(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 b961953a53..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 @@ -221,7 +221,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); + session.getSpec().getFlinkConfiguration().putAllFrom(flinkDepConfig); session.getStatus() .getReconciliationStatus() .serializeAndSetLastReconciledSpec(session.getSpec(), session); 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..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 @@ -49,6 +49,7 @@ public void testApplyOverrides() { ctx.getResource() .getSpec() .getFlinkConfiguration() + .asFlatMap() .get(PipelineOptions.PARALLELISM_OVERRIDES.key())) .satisfiesAnyOf( // Currently no enforced order inside the overrides string @@ -111,6 +112,7 @@ private void assertOverridesDoNotChange( 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 7673060890..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 @@ -126,7 +126,8 @@ public void testApplyFlinkConfiguration() { FlinkDeployment deployment = ReconciliationUtils.clone(flinkDeployment); deployment .getSpec() - .setFlinkConfiguration( + .getFlinkConfiguration() + .putAllFrom( Map.of( KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE.key(), KubernetesConfigOptions.ServiceExposedType.LoadBalancer.name())); 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..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 @@ -78,17 +78,24 @@ 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"); + deployment + .getSpec() + .getFlinkConfiguration() + .putAllFrom(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"); + .putAllFrom( + Map.of( + testConf.key(), + "latest", + opTestConf.key(), + "latest", + AutoScalerOptions.METRICS_WINDOW.key(), + "1234m")); assertEquals( "latest", 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..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,11 +170,15 @@ public void testSavepointNoRollbackWithoutHaMetadataAndJMWasReady() throws Excep 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"); + deployment + .getSpec() + .getFlinkConfiguration() + .putAllFrom( + Map.of( + KubernetesOperatorConfigOptions.DEPLOYMENT_ROLLBACK_ENABLED.key(), + "true", + KubernetesOperatorConfigOptions.DEPLOYMENT_READINESS_TIMEOUT.key(), + "10s")); testController.reconcile(deployment, context); @@ -356,12 +360,17 @@ 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"); + 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/deployment/ApplicationObserverTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/ApplicationObserverTest.java index 68233d397e..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 @@ -55,7 +55,6 @@ 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; @@ -716,10 +715,10 @@ public void testSavepointFormat() throws Exception { deployment.getSpec().getJob().setSavepointTriggerNonce(secondNonce); deployment .getSpec() - .setFlinkConfiguration( - 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 1b748ec755..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 @@ -153,6 +153,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..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 @@ -955,7 +955,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 +991,16 @@ 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"); + + deployment + .getSpec() + .getFlinkConfiguration() + .putAllFrom( + Map.of( + AutoScalerOptions.AUTOSCALER_ENABLED.key(), + "true", + PipelineOptions.PARALLELISM_OVERRIDES.key(), + v1 + ":1")); var specCopy = SpecUtils.clone(deployment.getSpec()); @@ -1233,6 +1241,7 @@ public void testDeploymentRecoveryEvent() throws Exception { @Test public void testRestartUnhealthyEvent() throws Exception { FlinkDeployment deployment = TestUtils.buildApplicationCluster(); + deployment .getSpec() .getFlinkConfiguration() @@ -1341,15 +1350,19 @@ 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"); + 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 a208a00f80..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 @@ -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.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; @@ -243,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); @@ -254,6 +254,7 @@ private void testUpgradeToLastState(FlinkVersion flinkVersion, UpgradeMode fromU .getJobStatus() .setState(org.apache.flink.api.common.JobStatus.FINISHED); deployment.getStatus().setJobManagerDeploymentStatus(JobManagerDeploymentStatus.READY); + deployment .getSpec() .getFlinkConfiguration() @@ -267,7 +268,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); @@ -688,6 +689,7 @@ public void testLastStateNoHaMeta(UpgradeMode upgradeMode, boolean allowFallback .key(), Boolean.toString(allowFallback)); deployment.getSpec().getFlinkConfiguration().remove(HighAvailabilityOptions.HA_MODE.key()); + deployment .getSpec() .getFlinkConfiguration() @@ -743,7 +745,7 @@ public void testLastStateOnDeletedDeployment() throws Exception { verifyAndSetRunningJobsToStatus(deployment, flinkService.listJobs()); // Delete cluster and keep HA metadata - var conf = Configuration.fromMap(deployment.getSpec().getFlinkConfiguration()); + var conf = deployment.getSpec().getFlinkConfiguration().asConfiguration(); flinkService.deleteClusterDeployment( deployment.getMetadata(), deployment.getStatus(), conf, false); flinkService.setHaDataAvailable(true); @@ -924,7 +926,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 = deployment.getSpec().getFlinkConfiguration().asFlatMap(); switch (upgradeMode) { case STATELESS: @@ -946,7 +948,8 @@ public static FlinkDeployment buildApplicationCluster( default: throw new RuntimeException("Unsupported upgrade mode " + upgradeMode); } - + deployment.getSpec().setFlinkConfiguration(new ConfigObjectNode()); + 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 284af2213b..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 @@ -157,11 +157,12 @@ public void testGetNonTerminalJobs() throws Exception { .put(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..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 @@ -83,17 +83,27 @@ 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"); + + 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()); - right.getFlinkConfiguration().remove(SCOPE_NAMING_KUBERNETES_OPERATOR.key()); - right.getFlinkConfiguration().remove(AutoScalerOptions.METRICS_WINDOW.key()); + right.getFlinkConfiguration() + .remove( + SCOPE_NAMING_KUBERNETES_OPERATOR.key(), + AutoScalerOptions.METRICS_WINDOW.key()); 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/service/NativeFlinkServiceTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/NativeFlinkServiceTest.java index 64b216324c..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 @@ -277,17 +277,17 @@ protected void updateVertexResources( var spec = flinkDep.getSpec(); spec.setFlinkVersion(FlinkVersion.v1_18); - var appConfig = Configuration.fromMap(spec.getFlinkConfiguration()); + var appConfig = spec.getFlinkConfiguration().asConfiguration(); appConfig.set(JobManagerOptions.SCHEDULER, JobManagerOptions.SchedulerType.Adaptive); - spec.setFlinkConfiguration(appConfig.toMap()); + 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.setFlinkConfiguration(appConfig.toMap()); + 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.setFlinkConfiguration(appConfig.toMap()); + 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 08388b79ea..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 @@ -150,7 +150,7 @@ public void testValidationWithoutDefaultConfig() { testError( dep -> { - dep.getSpec().setFlinkConfiguration(new HashMap<>()); + dep.getSpec().getFlinkConfiguration().setAllFrom(new HashMap<>()); dep.getSpec() .getJob() .setSavepointTriggerNonce(ThreadLocalRandom.current().nextLong()); @@ -202,7 +202,8 @@ public void testValidationWithoutDefaultConfig() { testError( dep -> dep.getSpec() - .setFlinkConfiguration( + .getFlinkConfiguration() + .setAllFrom( Collections.singletonMap( HighAvailabilityOptions.HA_CLUSTER_ID.key(), "my-cluster-id")), @@ -730,7 +731,8 @@ public void testSessionJobWithSession() { sessionJob.getSpec().getJob().setUpgradeMode(UpgradeMode.SAVEPOINT); sessionJob .getSpec() - .setFlinkConfiguration( + .getFlinkConfiguration() + .setAllFrom( Map.of( CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), "test-savepoint-dir", 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: