From 25d70da5daef393ffc81aac990f97f62b71cc73d Mon Sep 17 00:00:00 2001 From: schongloo Date: Mon, 10 Nov 2025 11:41:47 -0800 Subject: [PATCH] First batch for FLIP-504 --- .../operator/api/bluegreen/GateContext.java | 66 + .../api/bluegreen/GateContextOptions.java | 34 + .../api/bluegreen/GateKubernetesService.java | 91 + .../api/bluegreen/GateOutputMode.java | 24 + .../api/bluegreen/TransitionMode.java | 33 + .../api/bluegreen/TransitionStage.java | 27 + .../api/spec/FlinkDeploymentTemplateSpec.java | 5 + .../README.MD | 22 + .../pom.xml | 94 + .../bluegreen/client/GateProcessFunction.java | 214 + .../client/WatermarkGateContext.java | 97 + .../client/WatermarkGateProcessFunction.java | 188 + .../bluegreen/client/WatermarkGateStage.java | 25 + .../WatermarkGateProcessFunctionTest.java | 935 ++ .../FlinkBlueGreenDeploymentController.java | 15 +- .../bluegreen/BlueGreenContext.java | 6 + .../bluegreen/BlueGreenDeploymentService.java | 17 +- .../bluegreen/BlueGreenKubernetesService.java | 63 + .../bluegreen/BlueGreenTransitionUtils.java | 181 + .../flink/kubernetes/operator/TestUtils.java | 13 +- .../controller/BlueGreenTestUtils.java | 167 + ...linkBlueGreenDeploymentControllerTest.java | 299 +- ...ingFlinkBlueGreenDeploymentController.java | 9 - ...uegreendeployments.flink.apache.org-v1.yml | 5 + .../flinkdeployments.flink.apache.org-v1.yml | 9749 ++++++++--------- .../flinksessionjobs.flink.apache.org-v1.yml | 293 +- ...linkstatesnapshots.flink.apache.org-v1.yml | 101 +- pom.xml | 3 + 28 files changed, 7632 insertions(+), 5144 deletions(-) create mode 100644 flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/GateContext.java create mode 100644 flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/GateContextOptions.java create mode 100644 flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/GateKubernetesService.java create mode 100644 flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/GateOutputMode.java create mode 100644 flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/TransitionMode.java create mode 100644 flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/TransitionStage.java create mode 100644 flink-kubernetes-operator-bluegreen-client/README.MD create mode 100644 flink-kubernetes-operator-bluegreen-client/pom.xml create mode 100644 flink-kubernetes-operator-bluegreen-client/src/main/java/org/apache/flink/kubernetes/operator/bluegreen/client/GateProcessFunction.java create mode 100644 flink-kubernetes-operator-bluegreen-client/src/main/java/org/apache/flink/kubernetes/operator/bluegreen/client/WatermarkGateContext.java create mode 100644 flink-kubernetes-operator-bluegreen-client/src/main/java/org/apache/flink/kubernetes/operator/bluegreen/client/WatermarkGateProcessFunction.java create mode 100644 flink-kubernetes-operator-bluegreen-client/src/main/java/org/apache/flink/kubernetes/operator/bluegreen/client/WatermarkGateStage.java create mode 100644 flink-kubernetes-operator-bluegreen-client/src/test/java/org/apache/flink/kubernetes/operator/bluegreen/client/WatermarkGateProcessFunctionTest.java create mode 100644 flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/bluegreen/BlueGreenTransitionUtils.java create mode 100644 flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/BlueGreenTestUtils.java diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/GateContext.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/GateContext.java new file mode 100644 index 0000000000..d67ac63c1b --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/GateContext.java @@ -0,0 +1,66 @@ +/* + * 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.bluegreen; + +import lombok.AllArgsConstructor; +import lombok.Data; + +import java.io.Serializable; +import java.util.Map; + +import static org.apache.flink.kubernetes.operator.api.bluegreen.GateContextOptions.ACTIVE_DEPLOYMENT_TYPE; +import static org.apache.flink.kubernetes.operator.api.bluegreen.GateContextOptions.DEPLOYMENT_DELETION_DELAY; +import static org.apache.flink.kubernetes.operator.api.bluegreen.GateContextOptions.IS_FIRST_DEPLOYMENT; +import static org.apache.flink.kubernetes.operator.api.bluegreen.GateContextOptions.TRANSITION_STAGE; + +/** Base functionality of the Context used for Gate implementations. */ +@Data +@AllArgsConstructor +public class GateContext implements Serializable { + + /** GateContext enum. */ + private final BlueGreenDeploymentType activeBlueGreenDeploymentType; + + private final GateOutputMode outputMode; + private final int deploymentTeardownDelayMs; + private final TransitionStage gateStage; + private final boolean isFirstDeployment; + + public static GateContext create( + Map data, BlueGreenDeploymentType currentBlueGreenDeploymentType) { + var nextActiveDeploymentType = + BlueGreenDeploymentType.valueOf(data.get(ACTIVE_DEPLOYMENT_TYPE.getLabel())); + + var deploymentDeletionDelaySec = + Integer.parseInt(data.get(DEPLOYMENT_DELETION_DELAY.getLabel())); + + var outputMode = + currentBlueGreenDeploymentType == nextActiveDeploymentType + ? GateOutputMode.ACTIVE + : GateOutputMode.STANDBY; + + var isFirstDeployment = Boolean.parseBoolean(data.get(IS_FIRST_DEPLOYMENT.getLabel())); + + return new GateContext( + nextActiveDeploymentType, + outputMode, + deploymentDeletionDelaySec, + TransitionStage.valueOf(data.get(TRANSITION_STAGE.getLabel())), + isFirstDeployment); + } +} diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/GateContextOptions.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/GateContextOptions.java new file mode 100644 index 0000000000..07112012df --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/GateContextOptions.java @@ -0,0 +1,34 @@ +/* + * 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.bluegreen; + +import lombok.Getter; + +/** Options values for the GateContext. */ +public enum GateContextOptions { + IS_FIRST_DEPLOYMENT("is-first-deployment"), + ACTIVE_DEPLOYMENT_TYPE("active-deployment-type"), + DEPLOYMENT_DELETION_DELAY("deployment-deletion-delay-ms"), + TRANSITION_STAGE("stage"); + + @Getter private final String label; + + private GateContextOptions(String label) { + this.label = label; + } +} diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/GateKubernetesService.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/GateKubernetesService.java new file mode 100644 index 0000000000..f60cd2d33c --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/GateKubernetesService.java @@ -0,0 +1,91 @@ +/* + * 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.bluegreen; + +import org.apache.flink.util.Preconditions; + +import io.fabric8.kubernetes.api.model.ConfigMap; +import io.fabric8.kubernetes.client.KubernetesClient; +import io.fabric8.kubernetes.client.KubernetesClientBuilder; +import io.fabric8.kubernetes.client.informers.ResourceEventHandler; +import lombok.Getter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Serializable; +import java.util.Map; + +/** Simple Kubernetes service proxy for Gate operations. */ +public class GateKubernetesService implements Serializable { + + private static final Logger logger = LoggerFactory.getLogger(GateKubernetesService.class); + + @Getter private final KubernetesClient kubernetesClient; + + private final String namespace; + private final String configMapName; + + public GateKubernetesService(String namespace, String configMapName) { + Preconditions.checkNotNull(namespace); + Preconditions.checkNotNull(configMapName); + + try { + kubernetesClient = new KubernetesClientBuilder().build(); + } catch (Exception e) { + logger.error("Error instantiating Kubernetes Client", e); + throw e; + } + + this.namespace = namespace; + this.configMapName = configMapName; + } + + public void setInformers(ResourceEventHandler resourceEventHandler) { + kubernetesClient + .configMaps() + .inNamespace(namespace) + .withName(configMapName) + .inform(resourceEventHandler, 0); + } + + public void updateConfigMapEntries(Map kvps) { + var configMap = parseConfigMap(); + + kvps.forEach((key, value) -> configMap.getData().put(key, value)); + + try { + kubernetesClient.configMaps().inNamespace(namespace).resource(configMap).update(); + } catch (Exception e) { + logger.error("Failed to UPDATE the ConfigMap", e); + throw e; + } + } + + public ConfigMap parseConfigMap() { + try { + return kubernetesClient + .configMaps() + .inNamespace(namespace) + .withName(configMapName) + .get(); + } catch (Exception e) { + logger.error("Failed to GET the ConfigMap", e); + throw e; + } + } +} diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/GateOutputMode.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/GateOutputMode.java new file mode 100644 index 0000000000..6efab11a2b --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/GateOutputMode.java @@ -0,0 +1,24 @@ +/* + * 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.bluegreen; + +/** Gate output enum values. */ +public enum GateOutputMode { + ACTIVE, + STANDBY +} diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/TransitionMode.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/TransitionMode.java new file mode 100644 index 0000000000..15f9285dc0 --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/TransitionMode.java @@ -0,0 +1,33 @@ +/* + * 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.bluegreen; + +/** Possible transition modes supported by the `FlinkBlueGreenDeploymentController`. */ +public enum TransitionMode { + /** + * FLIP-503: simple transition that deletes the previous deployment as soon as the new one is + * RUNNING/STABLE. + */ + BASIC, + + /** + * FLIP-504: advanced coordination between deployment stages during transition. Not supported + * until FLIP-504 is implemented. + */ + ADVANCED; +} diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/TransitionStage.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/TransitionStage.java new file mode 100644 index 0000000000..acd95ce84a --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/TransitionStage.java @@ -0,0 +1,27 @@ +/* + * 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.bluegreen; + +/** Enumeration of the various stages for _ALL_ Blue/Green deployments. */ +public enum TransitionStage { + CLEAR_TO_TEARDOWN, + FAILING, + INITIALIZING, + RUNNING, + TRANSITIONING; +} diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/FlinkDeploymentTemplateSpec.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/FlinkDeploymentTemplateSpec.java index 259e2e3b3b..6f44a99339 100644 --- a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/FlinkDeploymentTemplateSpec.java +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/FlinkDeploymentTemplateSpec.java @@ -17,6 +17,8 @@ package org.apache.flink.kubernetes.operator.api.spec; +import org.apache.flink.kubernetes.operator.api.bluegreen.TransitionMode; + import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import io.fabric8.kubernetes.api.model.ObjectMeta; @@ -38,6 +40,9 @@ public class FlinkDeploymentTemplateSpec { @JsonProperty("metadata") private ObjectMeta metadata; + @JsonProperty("transitionMode") + private TransitionMode transitionMode; + @JsonProperty("spec") private FlinkDeploymentSpec spec; diff --git a/flink-kubernetes-operator-bluegreen-client/README.MD b/flink-kubernetes-operator-bluegreen-client/README.MD new file mode 100644 index 0000000000..81334de989 --- /dev/null +++ b/flink-kubernetes-operator-bluegreen-client/README.MD @@ -0,0 +1,22 @@ + + +# Flink Kubernetes Client Code Client + +TBD \ No newline at end of file diff --git a/flink-kubernetes-operator-bluegreen-client/pom.xml b/flink-kubernetes-operator-bluegreen-client/pom.xml new file mode 100644 index 0000000000..d9d43e2a87 --- /dev/null +++ b/flink-kubernetes-operator-bluegreen-client/pom.xml @@ -0,0 +1,94 @@ + + + 4.0.0 + + + org.apache.flink + flink-kubernetes-operator-parent + 1.14-SNAPSHOT + .. + + + flink-kubernetes-operator-bluegreen-client + Flink Kubernetes BlueGreen Deployment Client + + + + org.apache.flink + flink-kubernetes-operator-api + 1.14-SNAPSHOT + compile + + + + org.apache.flink + flink-streaming-java + ${flink.version} + compile + + + + org.projectlombok + lombok + ${lombok.version} + provided + + + + + org.junit.jupiter + junit-jupiter-engine + ${junit.jupiter.version} + test + + + + org.junit.jupiter + junit-jupiter-api + ${junit.jupiter.version} + test + + + + org.apache.flink + flink-test-utils + ${flink.version} + test + + + + org.apache.flink + flink-streaming-java + ${flink.version} + test-jar + test + + + + io.fabric8 + kubernetes-httpclient-okhttp + ${fabric8.version} + test + + + + diff --git a/flink-kubernetes-operator-bluegreen-client/src/main/java/org/apache/flink/kubernetes/operator/bluegreen/client/GateProcessFunction.java b/flink-kubernetes-operator-bluegreen-client/src/main/java/org/apache/flink/kubernetes/operator/bluegreen/client/GateProcessFunction.java new file mode 100644 index 0000000000..64c109622d --- /dev/null +++ b/flink-kubernetes-operator-bluegreen-client/src/main/java/org/apache/flink/kubernetes/operator/bluegreen/client/GateProcessFunction.java @@ -0,0 +1,214 @@ +/* + * 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.bluegreen.client; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.kubernetes.operator.api.bluegreen.BlueGreenDeploymentType; +import org.apache.flink.kubernetes.operator.api.bluegreen.GateContext; +import org.apache.flink.kubernetes.operator.api.bluegreen.GateContextOptions; +import org.apache.flink.kubernetes.operator.api.bluegreen.GateKubernetesService; +import org.apache.flink.streaming.api.functions.ProcessFunction; +import org.apache.flink.util.Collector; +import org.apache.flink.util.Preconditions; + +import io.fabric8.kubernetes.api.model.ConfigMap; +import io.fabric8.kubernetes.client.informers.ResourceEventHandler; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.flink.kubernetes.operator.api.bluegreen.GateContextOptions.TRANSITION_STAGE; +import static org.apache.flink.kubernetes.operator.api.bluegreen.TransitionStage.CLEAR_TO_TEARDOWN; + +/** Base class for ProcessFunction (streaming) based Gate implementations. */ +abstract class GateProcessFunction extends ProcessFunction implements Serializable { + private static final Logger logger = LoggerFactory.getLogger(GateProcessFunction.class); + + protected final BlueGreenDeploymentType blueGreenDeploymentType; + + // TODO: make this configurable? This cannot be a constant + protected final int subtaskIndexGuide = 1; + + protected GateKubernetesService gateKubernetesService; + protected Boolean clearToTeardown = false; + protected GateContext baseContext; + private String namespace; + private String configMapName; + + protected abstract void onContextUpdate(GateContext baseContext, Map data); + + public GateProcessFunction( + BlueGreenDeploymentType blueGreenDeploymentType, + String namespace, + String configMapName, + GateKubernetesService gateKubernetesService) { + Preconditions.checkArgument( + blueGreenDeploymentType == BlueGreenDeploymentType.BLUE + || blueGreenDeploymentType == BlueGreenDeploymentType.GREEN, + "Invalid deployment type: " + blueGreenDeploymentType); + + this.blueGreenDeploymentType = blueGreenDeploymentType; + this.namespace = namespace; + this.configMapName = configMapName; + this.gateKubernetesService = gateKubernetesService; + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + + // Only create service if not injected + if (gateKubernetesService == null) { + this.gateKubernetesService = new GateKubernetesService(namespace, configMapName); + } + + // Always set up informers, whether service was injected or created + setKubernetesEnvironment(); + processConfigMap(gateKubernetesService.parseConfigMap()); + } + + protected abstract void processElementActive( + I value, ProcessFunction.Context ctx, Collector out) + throws IllegalAccessException; + + protected abstract void processElementStandby( + I value, ProcessFunction.Context ctx, Collector out) + throws IllegalAccessException; + + @Override + public void processElement(I value, ProcessFunction.Context ctx, Collector out) + throws IllegalAccessException { + switch (baseContext.getOutputMode()) { + case ACTIVE: + processElementActive(value, ctx, out); + break; + case STANDBY: + processElementStandby(value, ctx, out); + break; + default: + String error = "Invalid OutputMode caught"; + logger.error(error); + throw new IllegalStateException(error); + } + } + + private void setKubernetesEnvironment() { + logInfo("Preparing Informers..."); + var resourceEventHandler = + new ResourceEventHandler() { + @Override + public void onAdd(ConfigMap obj) { + logger.warn("Unexpected ConfigMap added: " + obj); + } + + @Override + public void onUpdate(ConfigMap oldObj, ConfigMap newObj) { + if (!oldObj.equals(newObj)) { + var oldState = oldObj.getData().get(TRANSITION_STAGE.getLabel()); + var newState = newObj.getData().get(TRANSITION_STAGE.getLabel()); + + logInfo("Update notification 1: " + oldState + " to " + newState); + + processConfigMap(newObj); + } + } + + @Override + public void onDelete(ConfigMap obj, boolean deletedFinalStateUnknown) { + logger.error( + "ConfigMap deleted: " + + obj + + ", final state unknown: " + + deletedFinalStateUnknown); + } + }; + + gateKubernetesService.setInformers(resourceEventHandler); + logInfo("Informers set!"); + } + + private void processConfigMap(ConfigMap configMap) { + this.baseContext = GateContext.create(configMap.getData(), blueGreenDeploymentType); + + // Filtering the "custom" entries only + var baseKeys = + Arrays.stream(GateContextOptions.values()) + .map(GateContextOptions::getLabel) + .collect(Collectors.toSet()); + var allConfigMapKeys = new HashSet<>(configMap.getData().keySet()); + // Set difference: + allConfigMapKeys.removeAll(baseKeys); + + var filteredData = + configMap.getData().entrySet().stream() + .filter(kvp -> allConfigMapKeys.contains(kvp.getKey())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + onContextUpdate(baseContext, filteredData); + } + + protected void notifyClearToTeardown() { + // Notify only once + if (!clearToTeardown && getRuntimeContext().getIndexOfThisSubtask() == subtaskIndexGuide) { + logInfo("Setting " + CLEAR_TO_TEARDOWN); + performConfigMapUpdate( + Map.of(TRANSITION_STAGE.getLabel(), CLEAR_TO_TEARDOWN.toString())); + logInfo(CLEAR_TO_TEARDOWN + " set!"); + this.clearToTeardown = true; + } + } + + /** + * Template method for updating ConfigMap entries. Override in tests to avoid actual K8s + * updates. + */ + protected void performConfigMapUpdate(Map updates) { + gateKubernetesService.updateConfigMapEntries(updates); + } + + protected void updateConfigMapCustomEntries(Map customEntries) + throws IllegalAccessException { + // Validating only "custom" entries/keys can be updated + var keysToUpdate = customEntries.keySet(); + var baseContextKeys = + Arrays.stream(GateContextOptions.values()) + .map(GateContextOptions::getLabel) + .collect(Collectors.toCollection(HashSet::new)); + // Set intersection: + baseContextKeys.retainAll(keysToUpdate); + + if (!baseContextKeys.isEmpty()) { + var error = "Attempted to update read-only base keys" + baseContextKeys; + logger.error(error); + throw new IllegalAccessException(error); + } + logInfo("Updating custom entries: " + customEntries); + performConfigMapUpdate(customEntries); + } + + // Temporary "utility" function for development + protected void logInfo(String message) { + int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask(); + logger.error("[BlueGreen Gate-" + subtaskIdx + "]:" + message); + } +} diff --git a/flink-kubernetes-operator-bluegreen-client/src/main/java/org/apache/flink/kubernetes/operator/bluegreen/client/WatermarkGateContext.java b/flink-kubernetes-operator-bluegreen-client/src/main/java/org/apache/flink/kubernetes/operator/bluegreen/client/WatermarkGateContext.java new file mode 100644 index 0000000000..0a7911b830 --- /dev/null +++ b/flink-kubernetes-operator-bluegreen-client/src/main/java/org/apache/flink/kubernetes/operator/bluegreen/client/WatermarkGateContext.java @@ -0,0 +1,97 @@ +/* + * 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.bluegreen.client; + +import org.apache.flink.kubernetes.operator.api.bluegreen.GateContext; + +import lombok.Getter; +import lombok.Setter; + +import java.io.Serializable; +import java.util.Map; + +/** Watermark based functionality of the GateContext. */ +@Getter +public class WatermarkGateContext implements Serializable { + + static final String WATERMARK_TOGGLE_VALUE = "watermark-toggle-value"; + static final String WATERMARK_STAGE = "watermark-stage"; + + private final GateContext baseContext; + private final WatermarkGateStage watermarkGateStage; + + @Setter private Long watermarkToggleValue; + + public WatermarkGateContext( + GateContext baseContext, + WatermarkGateStage watermarkGateStage, + Long watermarkToggleValue) { + this.baseContext = baseContext; + this.watermarkGateStage = watermarkGateStage; + this.watermarkToggleValue = watermarkToggleValue; + } + + @Override + public boolean equals(Object o) { + if (o instanceof WatermarkGateContext) { + WatermarkGateContext wm = (WatermarkGateContext) o; + return this.baseContext.equals(wm.getBaseContext()) + && this.watermarkGateStage.equals(wm.getWatermarkGateStage()) + && this.watermarkToggleValue.equals(wm.getWatermarkToggleValue()); + } + return false; + } + + public static WatermarkGateContext create(GateContext baseContext, Map data) { + // Possible values: + // null -> indetermined/to be calculated by the active job + // 0 -> For first deployments (undetermined) + // Positive -> For valid transitions + Long watermarkToggleValue = null; + + if (baseContext.isFirstDeployment()) { + watermarkToggleValue = 0L; + } else if (data.containsKey(WATERMARK_TOGGLE_VALUE)) { + watermarkToggleValue = Long.parseLong(data.get(WATERMARK_TOGGLE_VALUE)); + } + + var watermarkState = + data.getOrDefault(WATERMARK_STAGE, WatermarkGateStage.WATERMARK_NOT_SET.toString()); + + return new WatermarkGateContext( + baseContext, WatermarkGateStage.valueOf(watermarkState), watermarkToggleValue); + } + + @Override + public String toString() { + return "WatermarkGateContext: {" + + ", watermarkToggleValue: " + + watermarkToggleValue + + ", watermarkGateStage: " + + watermarkGateStage + + ", gateStage:" + + baseContext.getGateStage() + + ", deploymentTeardownDelaySec: " + + baseContext.getDeploymentTeardownDelayMs() + + ", outputMode: " + + baseContext.getOutputMode() + + ", activeDeploymentType: " + + baseContext.getActiveBlueGreenDeploymentType() + + "}"; + } +} diff --git a/flink-kubernetes-operator-bluegreen-client/src/main/java/org/apache/flink/kubernetes/operator/bluegreen/client/WatermarkGateProcessFunction.java b/flink-kubernetes-operator-bluegreen-client/src/main/java/org/apache/flink/kubernetes/operator/bluegreen/client/WatermarkGateProcessFunction.java new file mode 100644 index 0000000000..6b38481cfd --- /dev/null +++ b/flink-kubernetes-operator-bluegreen-client/src/main/java/org/apache/flink/kubernetes/operator/bluegreen/client/WatermarkGateProcessFunction.java @@ -0,0 +1,188 @@ +/* + * 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.bluegreen.client; + +import org.apache.flink.kubernetes.operator.api.bluegreen.BlueGreenDeploymentType; +import org.apache.flink.kubernetes.operator.api.bluegreen.GateContext; +import org.apache.flink.kubernetes.operator.api.bluegreen.GateKubernetesService; +import org.apache.flink.kubernetes.operator.api.bluegreen.TransitionStage; +import org.apache.flink.streaming.api.functions.ProcessFunction; +import org.apache.flink.util.Collector; +import org.apache.flink.util.Preconditions; + +import java.io.Serializable; +import java.util.Map; +import java.util.function.Function; + +/** Watermark based GateProcessFunction (streaming). */ +public class WatermarkGateProcessFunction extends GateProcessFunction + implements Serializable { + + private final Function watermarkExtractor; + + protected WatermarkGateContext currentWatermarkGateContext; + + protected Boolean waitingForWatermark = false; + + WatermarkGateProcessFunction( + BlueGreenDeploymentType blueGreenDeploymentType, + String namespace, + String configMapName, + Function watermarkExtractor) { + this(blueGreenDeploymentType, namespace, configMapName, watermarkExtractor, null); + } + + WatermarkGateProcessFunction( + BlueGreenDeploymentType blueGreenDeploymentType, + String namespace, + String configMapName, + Function watermarkExtractor, + GateKubernetesService gateKubernetesService) { + super(blueGreenDeploymentType, namespace, configMapName, gateKubernetesService); + + Preconditions.checkNotNull(watermarkExtractor); + + this.watermarkExtractor = watermarkExtractor; + } + + public static WatermarkGateProcessFunction create( + Map flinkConfig, Function watermarkExtractor) { + return new WatermarkGateProcessFunction( + BlueGreenDeploymentType.valueOf( + flinkConfig.get("bluegreen.active-deployment-type")), + flinkConfig.get("kubernetes.namespace"), + flinkConfig.get("bluegreen.configmap.name"), + watermarkExtractor); + } + + @Override + protected void onContextUpdate(GateContext baseContext, Map data) { + var fetchedWatermarkContext = WatermarkGateContext.create(baseContext, data); + logInfo("Refreshing WatermarkGateContext with data: " + data); + + if (currentWatermarkGateContext == null) { + logInfo("currentWatermarkGateContext INITIALIZED: " + fetchedWatermarkContext); + currentWatermarkGateContext = fetchedWatermarkContext; + } else if (!currentWatermarkGateContext.equals(fetchedWatermarkContext)) { + logInfo("currentWatermarkGateContext UPDATED: " + fetchedWatermarkContext); + currentWatermarkGateContext = fetchedWatermarkContext; + } + } + + @Override + protected void processElementActive( + I value, ProcessFunction.Context ctx, Collector out) + throws IllegalAccessException { + Long wmToggleValue = currentWatermarkGateContext.getWatermarkToggleValue(); + if (wmToggleValue != null) { + Long extractedWatermark = watermarkExtractor.apply(value); + if (wmToggleValue <= extractedWatermark) { + // Normal + out.collect(value); + } else { + // Waiting for WM + logInfo( + " -- Waiting to Reach WM: " + + (wmToggleValue - extractedWatermark) + + " ms - "); + } + } else { + // Transitioning to Active + var currentGateStage = currentWatermarkGateContext.getBaseContext().getGateStage(); + if (currentGateStage == TransitionStage.TRANSITIONING) { + logInfo(" -- Waiting for WM to be set - "); + notifyWaitingForWatermark(); + } else { + logInfo("Waiting for the TRANSITIONING state, current: " + currentGateStage); + } + } + } + + @Override + protected void processElementStandby( + I value, ProcessFunction.Context ctx, Collector out) + throws IllegalAccessException { + if (currentWatermarkGateContext.getWatermarkToggleValue() != null) { + var watermarkToggleValue = currentWatermarkGateContext.getWatermarkToggleValue(); + + if (getWatermarkBoundary(ctx) <= watermarkToggleValue) { + if (watermarkToggleValue > watermarkExtractor.apply(value)) { + // Should still output the element + out.collect(value); + } else { + // Went past the Watermark toggle value: BLOCK ELEMENT + logInfo(" -- Past WM -- "); + } + } else { + // Went past the Watermark Boundary: BLOCK ELEMENT + logInfo(" -- Past WM Boundary -- "); + notifyClearToTeardown(); + } + } else { + // This ACTIVE job is transitioning to STANDBY, output elements + out.collect(value); + // Set the watermark when the other new job is ready + updateWatermarkInConfigMap(ctx); + } + } + + private long getWatermarkBoundary(ProcessFunction.Context ctx) { + return ctx.timerService().currentWatermark() > 0 + ? ctx.timerService().currentWatermark() + : ctx.timerService().currentProcessingTime(); + } + + private void updateWatermarkInConfigMap(ProcessFunction.Context ctx) + throws IllegalAccessException { + var curWmCtx = currentWatermarkGateContext; + + if (getRuntimeContext().getIndexOfThisSubtask() == subtaskIndexGuide + && curWmCtx.getWatermarkGateStage() == WatermarkGateStage.WAITING_FOR_WATERMARK) { + var nextWatermarkToggleValue = + getWatermarkBoundary(ctx) + + curWmCtx.getBaseContext().getDeploymentTeardownDelayMs(); + + // Setting the value in advance to avoid subsequent elements in this subtask from + // setting it + // while the changes get reflected in Kubernetes + currentWatermarkGateContext.setWatermarkToggleValue(nextWatermarkToggleValue); + + logInfo("Updating the ConfigMap Watermark value to: " + nextWatermarkToggleValue); + updateConfigMapCustomEntries( + Map.of( + WatermarkGateContext.WATERMARK_TOGGLE_VALUE, + Long.toString(nextWatermarkToggleValue), + WatermarkGateContext.WATERMARK_STAGE, + WatermarkGateStage.WATERMARK_SET.toString())); + logInfo("Watermark updated!"); + } + } + + private void notifyWaitingForWatermark() throws IllegalAccessException { + if (!waitingForWatermark + && getRuntimeContext().getIndexOfThisSubtask() == subtaskIndexGuide) { + logInfo("Setting " + WatermarkGateStage.WAITING_FOR_WATERMARK); + updateConfigMapCustomEntries( + Map.of( + WatermarkGateContext.WATERMARK_STAGE, + WatermarkGateStage.WAITING_FOR_WATERMARK.toString())); + logInfo(WatermarkGateStage.WAITING_FOR_WATERMARK + " set!"); + waitingForWatermark = true; + } + } +} diff --git a/flink-kubernetes-operator-bluegreen-client/src/main/java/org/apache/flink/kubernetes/operator/bluegreen/client/WatermarkGateStage.java b/flink-kubernetes-operator-bluegreen-client/src/main/java/org/apache/flink/kubernetes/operator/bluegreen/client/WatermarkGateStage.java new file mode 100644 index 0000000000..56e610d600 --- /dev/null +++ b/flink-kubernetes-operator-bluegreen-client/src/main/java/org/apache/flink/kubernetes/operator/bluegreen/client/WatermarkGateStage.java @@ -0,0 +1,25 @@ +/* + * 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.bluegreen.client; + +/** Watermark Gate specific stage values. */ +public enum WatermarkGateStage { + WATERMARK_NOT_SET, + WAITING_FOR_WATERMARK, + WATERMARK_SET +} diff --git a/flink-kubernetes-operator-bluegreen-client/src/test/java/org/apache/flink/kubernetes/operator/bluegreen/client/WatermarkGateProcessFunctionTest.java b/flink-kubernetes-operator-bluegreen-client/src/test/java/org/apache/flink/kubernetes/operator/bluegreen/client/WatermarkGateProcessFunctionTest.java new file mode 100644 index 0000000000..acef45c072 --- /dev/null +++ b/flink-kubernetes-operator-bluegreen-client/src/test/java/org/apache/flink/kubernetes/operator/bluegreen/client/WatermarkGateProcessFunctionTest.java @@ -0,0 +1,935 @@ +/* + * 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.bluegreen.client; + +import org.apache.flink.kubernetes.operator.api.bluegreen.BlueGreenDeploymentType; +import org.apache.flink.kubernetes.operator.api.bluegreen.GateContextOptions; +import org.apache.flink.kubernetes.operator.api.bluegreen.GateKubernetesService; +import org.apache.flink.kubernetes.operator.api.bluegreen.TransitionStage; +import org.apache.flink.streaming.api.operators.ProcessOperator; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.streaming.util.ProcessFunctionTestHarnesses; + +import io.fabric8.kubernetes.api.model.ConfigMap; +import io.fabric8.kubernetes.api.model.ConfigMapBuilder; +import io.fabric8.kubernetes.client.informers.ResourceEventHandler; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** Comprehensive unit tests for {@link WatermarkGateProcessFunction}. */ +public class WatermarkGateProcessFunctionTest { + + private static final String TEST_NAMESPACE = "test-namespace"; + private static final String TEST_CONFIGMAP_NAME = "test-configmap"; + private static final long TEST_WATERMARK_VALUE = 1000L; + private static final long TEST_TEARDOWN_DELAY_MS = 5000L; + + /** Serializable Function interface for watermark extraction. */ + private interface SerializableFunction extends Function, java.io.Serializable {} + + /** Static watermark extractor to avoid any serialization issues. */ + private static class TestMessageWatermarkExtractor + implements SerializableFunction { + private static final long serialVersionUID = 1L; + + @Override + public Long apply(TestMessage testMessage) { + return testMessage.getTimestamp(); + } + } + + private TestableWatermarkGateProcessFunction watermarkGateFunction; + private SerializableFunction watermarkExtractor; + private OneInputStreamOperatorTestHarness testHarness; + + @BeforeEach + void setUp() throws Exception { + watermarkExtractor = new TestMessageWatermarkExtractor(); + + // Create mock service that returns initial ConfigMap + MockGateKubernetesService mockService = new MockGateKubernetesService(); + + watermarkGateFunction = + new TestableWatermarkGateProcessFunction<>( + BlueGreenDeploymentType.BLUE, + TEST_NAMESPACE, + TEST_CONFIGMAP_NAME, + watermarkExtractor, + mockService); + + // Set max parallelism to 2 and subtask index to 1 to match subtaskIndexGuide + testHarness = + new OneInputStreamOperatorTestHarness<>( + new ProcessOperator<>(watermarkGateFunction), 2, 2, 1); + + // Disable object reuse and serialization checks to avoid serialization issues in tests + testHarness.getExecutionConfig().disableObjectReuse(); + + testHarness.open(); + } + + @AfterEach + void tearDown() throws Exception { + if (testHarness != null) { + testHarness.close(); + } + } + + // ==================== Kubernetes Informer Callback Tests ==================== + + @Test + void testInformerOnUpdateCallback() throws Exception { + ResourceEventHandler handler = watermarkGateFunction.getCapturedEventHandler(); + assertNotNull(handler, "Event handler should be captured during open()"); + + // Create initial ConfigMap + ConfigMap oldConfigMap = + createConfigMap( + BlueGreenDeploymentType.BLUE, + TransitionStage.RUNNING, + false, + TEST_WATERMARK_VALUE, + WatermarkGateStage.WATERMARK_SET); + + // Create updated ConfigMap with new watermark value + ConfigMap newConfigMap = + createConfigMap( + BlueGreenDeploymentType.BLUE, + TransitionStage.TRANSITIONING, + false, + 2000L, + WatermarkGateStage.WAITING_FOR_WATERMARK); + + // Invoke the informer callback + handler.onUpdate(oldConfigMap, newConfigMap); + + // Verify that the context was updated + WatermarkGateContext context = watermarkGateFunction.getCurrentWatermarkGateContext(); + assertNotNull(context); + assertEquals(2000L, context.getWatermarkToggleValue()); + assertEquals(WatermarkGateStage.WAITING_FOR_WATERMARK, context.getWatermarkGateStage()); + assertEquals(TransitionStage.TRANSITIONING, context.getBaseContext().getGateStage()); + } + + @Test + void testInformerOnUpdateWithIdenticalConfigMaps() throws Exception { + ResourceEventHandler handler = watermarkGateFunction.getCapturedEventHandler(); + + ConfigMap configMap = + createConfigMap( + BlueGreenDeploymentType.BLUE, + TransitionStage.RUNNING, + false, + TEST_WATERMARK_VALUE, + WatermarkGateStage.WATERMARK_SET); + + int initialLogCount = watermarkGateFunction.getLogMessages().size(); + + // Update with identical ConfigMap should not trigger processing + handler.onUpdate(configMap, configMap); + + // No additional logs should be generated for identical updates + assertEquals( + initialLogCount, + watermarkGateFunction.getLogMessages().size(), + "Identical ConfigMaps should not trigger update processing"); + } + + @Test + void testInformerOnAddCallbackLogsWarning() throws Exception { + ResourceEventHandler handler = watermarkGateFunction.getCapturedEventHandler(); + + ConfigMap configMap = + createConfigMap( + BlueGreenDeploymentType.BLUE, + TransitionStage.RUNNING, + false, + TEST_WATERMARK_VALUE, + WatermarkGateStage.WATERMARK_SET); + + // This should log a warning as we don't expect ConfigMaps to be added + handler.onAdd(configMap); + + // The implementation logs this at WARN level, but we can verify it doesn't crash + assertTrue(true, "onAdd should handle unexpected ConfigMap additions gracefully"); + } + + @Test + void testInformerOnDeleteCallbackLogsError() throws Exception { + ResourceEventHandler handler = watermarkGateFunction.getCapturedEventHandler(); + + ConfigMap configMap = + createConfigMap( + BlueGreenDeploymentType.BLUE, + TransitionStage.RUNNING, + false, + TEST_WATERMARK_VALUE, + WatermarkGateStage.WATERMARK_SET); + + // This should log an error as ConfigMap deletion is unexpected + handler.onDelete(configMap, false); + + // The implementation logs this at ERROR level, but we can verify it doesn't crash + assertTrue(true, "onDelete should handle ConfigMap deletion gracefully"); + } + + @Test + void testInformerTriggersContextUpdateInActiveMode() throws Exception { + ResourceEventHandler handler = watermarkGateFunction.getCapturedEventHandler(); + + // Initial state: Active mode with watermark set + ConfigMap initialConfigMap = + createConfigMap( + BlueGreenDeploymentType.BLUE, + TransitionStage.RUNNING, + false, + TEST_WATERMARK_VALUE, + WatermarkGateStage.WATERMARK_SET); + + ConfigMap updatedConfigMap = + createConfigMap( + BlueGreenDeploymentType.BLUE, + TransitionStage.RUNNING, + false, + TEST_WATERMARK_VALUE, + WatermarkGateStage.WATERMARK_SET); + + handler.onUpdate(initialConfigMap, updatedConfigMap); + + // Process a message that should pass through + TestMessage message = new TestMessage(TEST_WATERMARK_VALUE); + testHarness.processElement(message, TEST_WATERMARK_VALUE - 100); + + assertEquals(1, testHarness.extractOutputValues().size()); + assertEquals(message, testHarness.extractOutputValues().get(0)); + } + + @Test + void testInformerTriggersContextUpdateInStandbyMode() throws Exception { + // Create a STANDBY function (GREEN when BLUE is active) + MockGateKubernetesService mockService = new MockGateKubernetesService(); + watermarkGateFunction = + new TestableWatermarkGateProcessFunction<>( + BlueGreenDeploymentType.GREEN, + TEST_NAMESPACE, + TEST_CONFIGMAP_NAME, + watermarkExtractor, + mockService); + testHarness.close(); + testHarness = ProcessFunctionTestHarnesses.forProcessFunction(watermarkGateFunction); + testHarness.open(); + + ResourceEventHandler handler = watermarkGateFunction.getCapturedEventHandler(); + + // BLUE is active, so GREEN is in STANDBY + ConfigMap configMap = + createConfigMap( + BlueGreenDeploymentType.BLUE, + TransitionStage.RUNNING, + false, + null, + WatermarkGateStage.WATERMARK_NOT_SET); + + handler.onUpdate( + createConfigMap( + BlueGreenDeploymentType.BLUE, + TransitionStage.INITIALIZING, + false, + null, + WatermarkGateStage.WATERMARK_NOT_SET), + configMap); + + // Process a message - should output in standby when watermark not set + TestMessage message = new TestMessage(TEST_WATERMARK_VALUE); + testHarness.processElement(message, TEST_WATERMARK_VALUE - 100); + + assertEquals(1, testHarness.extractOutputValues().size()); + } + + // ==================== Active Mode Processing Tests ==================== + + @Test + void testActiveMode_WatermarkSet_MessageAboveWatermark() throws Exception { + updateConfigMapViaInformer( + BlueGreenDeploymentType.BLUE, + TransitionStage.RUNNING, + false, + TEST_WATERMARK_VALUE, + WatermarkGateStage.WATERMARK_SET); + + TestMessage message = new TestMessage(TEST_WATERMARK_VALUE + 100); + testHarness.processElement(message, TEST_WATERMARK_VALUE - 100); + + assertEquals(1, testHarness.extractOutputValues().size()); + assertEquals(message, testHarness.extractOutputValues().get(0)); + } + + @Test + void testActiveMode_WatermarkSet_MessageBelowWatermark() throws Exception { + updateConfigMapViaInformer( + BlueGreenDeploymentType.BLUE, + TransitionStage.RUNNING, + false, + TEST_WATERMARK_VALUE, + WatermarkGateStage.WATERMARK_SET); + + TestMessage message = new TestMessage(TEST_WATERMARK_VALUE - 100); + testHarness.processElement(message, TEST_WATERMARK_VALUE - 200); + + assertEquals(0, testHarness.extractOutputValues().size()); + assertTrue( + watermarkGateFunction.getLogMessages().stream() + .anyMatch(msg -> msg.contains("Waiting to Reach WM"))); + } + + @Test + void testActiveMode_WatermarkSet_MessageExactlyAtWatermark() throws Exception { + updateConfigMapViaInformer( + BlueGreenDeploymentType.BLUE, + TransitionStage.RUNNING, + false, + TEST_WATERMARK_VALUE, + WatermarkGateStage.WATERMARK_SET); + + TestMessage message = new TestMessage(TEST_WATERMARK_VALUE); + testHarness.processElement(message, TEST_WATERMARK_VALUE - 100); + + assertEquals(1, testHarness.extractOutputValues().size()); + assertEquals(message, testHarness.extractOutputValues().get(0)); + } + + @Test + void testActiveMode_TransitioningState_WatermarkNotSet() throws Exception { + updateConfigMapViaInformer( + BlueGreenDeploymentType.BLUE, + TransitionStage.TRANSITIONING, + false, + null, + WatermarkGateStage.WATERMARK_NOT_SET); + + TestMessage message = new TestMessage(TEST_WATERMARK_VALUE); + testHarness.processElement(message, TEST_WATERMARK_VALUE - 100); + + assertEquals(0, testHarness.extractOutputValues().size()); + assertTrue( + watermarkGateFunction.getLogMessages().stream() + .anyMatch(msg -> msg.contains("Waiting for WM to be set"))); + assertTrue(watermarkGateFunction.isWaitingForWatermarkNotified()); + } + + @Test + void testActiveMode_RunningState_WatermarkNotSet() throws Exception { + updateConfigMapViaInformer( + BlueGreenDeploymentType.BLUE, + TransitionStage.RUNNING, + false, + null, + WatermarkGateStage.WATERMARK_NOT_SET); + + TestMessage message = new TestMessage(TEST_WATERMARK_VALUE); + testHarness.processElement(message, TEST_WATERMARK_VALUE - 100); + + assertEquals(0, testHarness.extractOutputValues().size()); + assertTrue( + watermarkGateFunction.getLogMessages().stream() + .anyMatch( + msg -> + msg.contains("Waiting for the TRANSITIONING state") + && msg.contains("RUNNING"))); + } + + @Test + void testActiveMode_WaitingForWatermarkStage() throws Exception { + updateConfigMapViaInformer( + BlueGreenDeploymentType.BLUE, + TransitionStage.TRANSITIONING, + false, + null, + WatermarkGateStage.WAITING_FOR_WATERMARK); + + TestMessage message = new TestMessage(TEST_WATERMARK_VALUE); + testHarness.processElement(message, TEST_WATERMARK_VALUE - 100); + + assertEquals(0, testHarness.extractOutputValues().size()); + assertTrue( + watermarkGateFunction.getLogMessages().stream() + .anyMatch(msg -> msg.contains("Waiting for WM to be set"))); + } + + @Test + void testActiveMode_FirstDeployment() throws Exception { + updateConfigMapViaInformer( + BlueGreenDeploymentType.BLUE, + TransitionStage.INITIALIZING, + true, + 0L, + WatermarkGateStage.WATERMARK_NOT_SET); + + TestMessage message = new TestMessage(TEST_WATERMARK_VALUE); + testHarness.processElement(message, TEST_WATERMARK_VALUE - 100); + + assertEquals(1, testHarness.extractOutputValues().size()); + assertEquals(message, testHarness.extractOutputValues().get(0)); + } + + // ==================== Standby Mode Processing Tests ==================== + + @Test + void testStandbyMode_WatermarkNotSet_ShouldOutputElements() throws Exception { + setupStandbyFunction(); + + updateConfigMapViaInformer( + BlueGreenDeploymentType.BLUE, + TransitionStage.RUNNING, + false, + null, + WatermarkGateStage.WATERMARK_NOT_SET); + + TestMessage message = new TestMessage(TEST_WATERMARK_VALUE); + testHarness.processElement(message, TEST_WATERMARK_VALUE - 100); + + assertEquals(1, testHarness.extractOutputValues().size()); + assertEquals(message, testHarness.extractOutputValues().get(0)); + } + + @Test + void testStandbyMode_WithinWatermarkBoundary_ShouldOutput() throws Exception { + setupStandbyFunction(); + + updateConfigMapViaInformer( + BlueGreenDeploymentType.BLUE, + TransitionStage.RUNNING, + false, + TEST_WATERMARK_VALUE, + WatermarkGateStage.WATERMARK_SET); + + TestMessage message = new TestMessage(TEST_WATERMARK_VALUE - 200); + testHarness.processElement(message, TEST_WATERMARK_VALUE - 300); + + assertEquals(1, testHarness.extractOutputValues().size()); + assertEquals(message, testHarness.extractOutputValues().get(0)); + } + + @Test + void testStandbyMode_PastWatermarkToggleValue_ShouldBlock() throws Exception { + setupStandbyFunction(); + + updateConfigMapViaInformer( + BlueGreenDeploymentType.BLUE, + TransitionStage.RUNNING, + false, + TEST_WATERMARK_VALUE, + WatermarkGateStage.WATERMARK_SET); + + TestMessage message = new TestMessage(TEST_WATERMARK_VALUE + 100); + testHarness.processElement(message, TEST_WATERMARK_VALUE - 100); + + assertEquals(0, testHarness.extractOutputValues().size()); + assertTrue( + watermarkGateFunction.getLogMessages().stream() + .anyMatch(msg -> msg.contains("Past WM"))); + } + + @Test + void testStandbyMode_PastWatermarkBoundary_ShouldBlockAndNotifyClearToTeardown() + throws Exception { + setupStandbyFunction(); + + updateConfigMapViaInformer( + BlueGreenDeploymentType.BLUE, + TransitionStage.RUNNING, + false, + TEST_WATERMARK_VALUE, + WatermarkGateStage.WATERMARK_SET); + + // Advance watermark past the toggle value to trigger "Past WM Boundary" + testHarness.processWatermark( + new org.apache.flink.streaming.api.watermark.Watermark(TEST_WATERMARK_VALUE + 200)); + + // Process message with current watermark past the toggle value + TestMessage message = new TestMessage(TEST_WATERMARK_VALUE + 100); + testHarness.processElement(message, TEST_WATERMARK_VALUE + 100); + + assertEquals(0, testHarness.extractOutputValues().size()); + assertTrue( + watermarkGateFunction.getLogMessages().stream() + .anyMatch(msg -> msg.contains("Past WM Boundary"))); + assertTrue(watermarkGateFunction.isClearToTeardownNotified()); + } + + @Test + void testStandbyMode_MessageAtWatermarkBoundary() throws Exception { + setupStandbyFunction(); + + updateConfigMapViaInformer( + BlueGreenDeploymentType.BLUE, + TransitionStage.RUNNING, + false, + TEST_WATERMARK_VALUE, + WatermarkGateStage.WATERMARK_SET); + + TestMessage message = new TestMessage(TEST_WATERMARK_VALUE); + testHarness.processElement(message, TEST_WATERMARK_VALUE - 100); + + // When message timestamp equals watermark toggle value, standby mode blocks it + // (line 114: watermarkToggleValue > watermarkExtractor.apply(value) is false when equal) + assertEquals(0, testHarness.extractOutputValues().size()); + } + + // ==================== State Transition Tests ==================== + + @Test + void testStateTransition_ActiveToStandby() throws Exception { + // Start in ACTIVE mode + updateConfigMapViaInformer( + BlueGreenDeploymentType.BLUE, + TransitionStage.RUNNING, + false, + TEST_WATERMARK_VALUE, + WatermarkGateStage.WATERMARK_SET); + + TestMessage message1 = new TestMessage(TEST_WATERMARK_VALUE + 100); + testHarness.processElement(message1, TEST_WATERMARK_VALUE - 100); + assertEquals(1, testHarness.extractOutputValues().size()); + + // Transition to STANDBY (GREEN becomes active) + updateConfigMapViaInformer( + BlueGreenDeploymentType.GREEN, + TransitionStage.TRANSITIONING, + false, + TEST_WATERMARK_VALUE + 1000, + WatermarkGateStage.WATERMARK_SET); + + TestMessage message2 = new TestMessage(TEST_WATERMARK_VALUE + 200); + testHarness.processElement(message2, TEST_WATERMARK_VALUE + 100); + assertEquals(2, testHarness.extractOutputValues().size()); + } + + @Test + void testStateTransition_StandbyToActive() throws Exception { + setupStandbyFunction(); + + // Start in STANDBY mode + updateConfigMapViaInformer( + BlueGreenDeploymentType.BLUE, + TransitionStage.RUNNING, + false, + TEST_WATERMARK_VALUE, + WatermarkGateStage.WATERMARK_SET); + + TestMessage message1 = new TestMessage(TEST_WATERMARK_VALUE - 100); + testHarness.processElement(message1, TEST_WATERMARK_VALUE - 200); + assertEquals(1, testHarness.extractOutputValues().size()); + + // Transition to ACTIVE (GREEN becomes active) + updateConfigMapViaInformer( + BlueGreenDeploymentType.GREEN, + TransitionStage.TRANSITIONING, + false, + null, + WatermarkGateStage.WATERMARK_NOT_SET); + + updateConfigMapViaInformer( + BlueGreenDeploymentType.GREEN, + TransitionStage.TRANSITIONING, + false, + TEST_WATERMARK_VALUE + 1000, + WatermarkGateStage.WATERMARK_SET); + + TestMessage message2 = new TestMessage(TEST_WATERMARK_VALUE + 1100); + testHarness.processElement(message2, TEST_WATERMARK_VALUE + 900); + assertEquals(2, testHarness.extractOutputValues().size()); + } + + @Test + void testMultipleConfigMapUpdates() throws Exception { + ResourceEventHandler handler = watermarkGateFunction.getCapturedEventHandler(); + + // First update + ConfigMap config1 = + createConfigMap( + BlueGreenDeploymentType.BLUE, + TransitionStage.INITIALIZING, + false, + null, + WatermarkGateStage.WATERMARK_NOT_SET); + handler.onUpdate( + createConfigMap( + BlueGreenDeploymentType.BLUE, + TransitionStage.INITIALIZING, + true, + 0L, + WatermarkGateStage.WATERMARK_NOT_SET), + config1); + + // Second update + ConfigMap config2 = + createConfigMap( + BlueGreenDeploymentType.BLUE, + TransitionStage.RUNNING, + false, + TEST_WATERMARK_VALUE, + WatermarkGateStage.WATERMARK_SET); + handler.onUpdate(config1, config2); + + // Third update + ConfigMap config3 = + createConfigMap( + BlueGreenDeploymentType.BLUE, + TransitionStage.TRANSITIONING, + false, + TEST_WATERMARK_VALUE + 1000, + WatermarkGateStage.WATERMARK_SET); + handler.onUpdate(config2, config3); + + WatermarkGateContext context = watermarkGateFunction.getCurrentWatermarkGateContext(); + assertEquals(TEST_WATERMARK_VALUE + 1000, context.getWatermarkToggleValue()); + assertEquals(TransitionStage.TRANSITIONING, context.getBaseContext().getGateStage()); + } + + // ==================== Factory Method Tests ==================== + + @Test + void testCreateFromFlinkConfig() { + Map flinkConfig = new HashMap<>(); + flinkConfig.put("bluegreen.active-deployment-type", "BLUE"); + flinkConfig.put("kubernetes.namespace", "test-ns"); + flinkConfig.put("bluegreen.configmap.name", "test-cm"); + + WatermarkGateProcessFunction function = + WatermarkGateProcessFunction.create(flinkConfig, s -> (long) s.length()); + + assertNotNull(function); + } + + // ==================== Helper Methods ==================== + + private void setupStandbyFunction() throws Exception { + MockGateKubernetesService mockService = new MockGateKubernetesService(); + + watermarkGateFunction = + new TestableWatermarkGateProcessFunction<>( + BlueGreenDeploymentType.GREEN, + TEST_NAMESPACE, + TEST_CONFIGMAP_NAME, + watermarkExtractor, + mockService); + testHarness.close(); + testHarness = + new OneInputStreamOperatorTestHarness<>( + new ProcessOperator<>(watermarkGateFunction), 2, 2, 1); + testHarness.getExecutionConfig().disableObjectReuse(); + testHarness.open(); + } + + private void updateConfigMapViaInformer( + BlueGreenDeploymentType activeType, + TransitionStage stage, + boolean isFirstDeployment, + Long watermarkToggleValue, + WatermarkGateStage watermarkStage) + throws Exception { + ResourceEventHandler handler = watermarkGateFunction.getCapturedEventHandler(); + ConfigMap oldConfigMap = + createConfigMap( + activeType, + TransitionStage.INITIALIZING, + true, + 0L, + WatermarkGateStage.WATERMARK_NOT_SET); + ConfigMap newConfigMap = + createConfigMap( + activeType, stage, isFirstDeployment, watermarkToggleValue, watermarkStage); + handler.onUpdate(oldConfigMap, newConfigMap); + } + + private ConfigMap createConfigMap( + BlueGreenDeploymentType activeType, + TransitionStage stage, + boolean isFirstDeployment, + Long watermarkToggleValue, + WatermarkGateStage watermarkStage) { + // Use HashMap explicitly to ensure mutable collection + HashMap data = new HashMap<>(); + data.put(GateContextOptions.ACTIVE_DEPLOYMENT_TYPE.getLabel(), activeType.toString()); + data.put(GateContextOptions.TRANSITION_STAGE.getLabel(), stage.toString()); + data.put( + GateContextOptions.DEPLOYMENT_DELETION_DELAY.getLabel(), + String.valueOf(TEST_TEARDOWN_DELAY_MS)); + data.put( + GateContextOptions.IS_FIRST_DEPLOYMENT.getLabel(), + String.valueOf(isFirstDeployment)); + + if (watermarkToggleValue != null) { + data.put(WatermarkGateContext.WATERMARK_TOGGLE_VALUE, watermarkToggleValue.toString()); + } + data.put(WatermarkGateContext.WATERMARK_STAGE, watermarkStage.toString()); + + // Pass a copy to ConfigMapBuilder to avoid any internal references + return new ConfigMapBuilder() + .withNewMetadata() + .withName(TEST_CONFIGMAP_NAME) + .withNamespace(TEST_NAMESPACE) + .endMetadata() + .withData(new HashMap<>(data)) + .build(); + } + + // ==================== Test Helper Classes ==================== + + /** Simple test message class with timestamp. */ + public static class TestMessage implements java.io.Serializable { + private static final long serialVersionUID = 1L; + + public long timestamp; + + public TestMessage() {} + + public TestMessage(long timestamp) { + this.timestamp = timestamp; + } + + public long getTimestamp() { + return timestamp; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TestMessage that = (TestMessage) o; + return timestamp == that.timestamp; + } + + @Override + public int hashCode() { + return (int) timestamp; + } + + @Override + public String toString() { + return "TestMessage{timestamp=" + timestamp + '}'; + } + } + + /** Mock Kubernetes service for testing. */ + private static class MockGateKubernetesService extends GateKubernetesService { + private ConfigMap initialConfigMap; + + MockGateKubernetesService() { + super(TEST_NAMESPACE, TEST_CONFIGMAP_NAME); + // Create initial ConfigMap + HashMap initialData = new HashMap<>(); + initialData.put( + GateContextOptions.ACTIVE_DEPLOYMENT_TYPE.getLabel(), + BlueGreenDeploymentType.BLUE.toString()); + initialData.put( + GateContextOptions.TRANSITION_STAGE.getLabel(), + TransitionStage.INITIALIZING.toString()); + initialData.put(GateContextOptions.DEPLOYMENT_DELETION_DELAY.getLabel(), "5000"); + initialData.put(GateContextOptions.IS_FIRST_DEPLOYMENT.getLabel(), "true"); + initialData.put("watermark-toggle-value", "0"); + initialData.put("watermark-stage", WatermarkGateStage.WATERMARK_NOT_SET.toString()); + + this.initialConfigMap = + new ConfigMapBuilder() + .withNewMetadata() + .withName(TEST_CONFIGMAP_NAME) + .withNamespace(TEST_NAMESPACE) + .endMetadata() + .withData(new HashMap<>(initialData)) + .build(); + } + + @Override + public void setInformers(ResourceEventHandler resourceEventHandler) { + // No-op - event handler will be captured in TestableWatermarkGateProcessFunction + } + + @Override + public ConfigMap parseConfigMap() { + return initialConfigMap; + } + + @Override + public void updateConfigMapEntries(Map kvps) { + // No-op in tests + } + } + + /** Testable implementation that uses mock service and tracks updates. */ + private static class TestableWatermarkGateProcessFunction + extends WatermarkGateProcessFunction { + private static final long serialVersionUID = 1L; + + private transient List logMessages; + private transient List> configMapUpdates; + private transient ResourceEventHandler capturedEventHandler; + + TestableWatermarkGateProcessFunction( + BlueGreenDeploymentType blueGreenDeploymentType, + String namespace, + String configMapName, + Function watermarkExtractor, + GateKubernetesService mockService) { + super( + blueGreenDeploymentType, + namespace, + configMapName, + watermarkExtractor, + mockService); + initializeTransientFields(); + } + + private void initializeTransientFields() { + if (logMessages == null) { + logMessages = new ArrayList<>(); + } + if (configMapUpdates == null) { + configMapUpdates = new ArrayList<>(); + } + } + + @Override + public void open(org.apache.flink.configuration.Configuration parameters) throws Exception { + super.open(parameters); + // After open, capture the event handler that was created + // We need to access it from the parent's setKubernetesEnvironment + } + + @Override + protected void logInfo(String message) { + initializeTransientFields(); + logMessages.add(message); + } + + @Override + protected void performConfigMapUpdate(Map updates) { + initializeTransientFields(); + configMapUpdates.add(new HashMap<>(updates)); + // Don't call super - avoid actual K8s updates in tests + } + + public ResourceEventHandler getCapturedEventHandler() { + initializeTransientFields(); + if (capturedEventHandler == null) { + // Create a test event handler that delegates to processConfigMap + capturedEventHandler = + new ResourceEventHandler() { + @Override + public void onAdd(ConfigMap obj) { + logInfo("Unexpected ConfigMap added: " + obj); + } + + @Override + public void onUpdate(ConfigMap oldObj, ConfigMap newObj) { + if (!oldObj.equals(newObj)) { + // Directly access protected baseContext field + baseContext = + org.apache.flink.kubernetes.operator.api.bluegreen + .GateContext.create( + newObj.getData(), blueGreenDeploymentType); + + // Extract custom data for watermark context + Map customData = new HashMap<>(); + String watermarkToggleValue = + newObj.getData() + .get( + WatermarkGateContext + .WATERMARK_TOGGLE_VALUE); + String watermarkStage = + newObj.getData() + .get(WatermarkGateContext.WATERMARK_STAGE); + + if (watermarkToggleValue != null) { + customData.put( + WatermarkGateContext.WATERMARK_TOGGLE_VALUE, + watermarkToggleValue); + } + if (watermarkStage != null) { + customData.put( + WatermarkGateContext.WATERMARK_STAGE, + watermarkStage); + } + + onContextUpdate(baseContext, customData); + } + } + + @Override + public void onDelete(ConfigMap obj, boolean deletedFinalStateUnknown) { + logInfo( + "ConfigMap deleted: " + + obj + + ", final state unknown: " + + deletedFinalStateUnknown); + } + }; + } + return capturedEventHandler; + } + + public List getLogMessages() { + initializeTransientFields(); + return logMessages; + } + + public List> getConfigMapUpdates() { + initializeTransientFields(); + return configMapUpdates; + } + + public boolean isWaitingForWatermarkNotified() { + return waitingForWatermark; + } + + public boolean isClearToTeardownNotified() { + return clearToTeardown; + } + + public WatermarkGateContext getCurrentWatermarkGateContext() { + return currentWatermarkGateContext; + } + + // Custom serialization + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + out.defaultWriteObject(); + } + + private void readObject(java.io.ObjectInputStream in) + throws java.io.IOException, ClassNotFoundException { + in.defaultReadObject(); + initializeTransientFields(); + } + } +} diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentController.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentController.java index a35ccb2b1b..6fbcdf9f7a 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentController.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentController.java @@ -27,6 +27,7 @@ import org.apache.flink.kubernetes.operator.controller.bluegreen.handlers.BlueGreenStateHandler; import org.apache.flink.kubernetes.operator.service.FlinkResourceContextFactory; +import io.fabric8.kubernetes.api.model.ConfigMap; import io.javaoperatorsdk.operator.api.config.informer.InformerEventSourceConfiguration; import io.javaoperatorsdk.operator.api.reconciler.Context; import io.javaoperatorsdk.operator.api.reconciler.ControllerConfiguration; @@ -81,7 +82,7 @@ public List> prepareEventSources( EventSourceContext context) { List> eventSources = new ArrayList<>(); - InformerEventSourceConfiguration config = + InformerEventSourceConfiguration flinkDeploymentConfig = InformerEventSourceConfiguration.from( FlinkDeployment.class, FlinkBlueGreenDeployment.class) .withSecondaryToPrimaryMapper( @@ -90,7 +91,17 @@ public List> prepareEventSources( .withFollowControllerNamespacesChanges(true) .build(); - eventSources.add(new InformerEventSource<>(config, context)); + InformerEventSourceConfiguration configMapConfig = + InformerEventSourceConfiguration.from( + ConfigMap.class, FlinkBlueGreenDeployment.class) + .withSecondaryToPrimaryMapper( + Mappers.fromOwnerReferences(context.getPrimaryResourceClass())) + .withNamespacesInheritedFromController() + .withFollowControllerNamespacesChanges(true) + .build(); + + eventSources.add(new InformerEventSource<>(flinkDeploymentConfig, context)); + eventSources.add(new InformerEventSource<>(configMapConfig, context)); return eventSources; } diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/BlueGreenContext.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/BlueGreenContext.java index 054b121f61..512db7975c 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/BlueGreenContext.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/BlueGreenContext.java @@ -24,6 +24,7 @@ import org.apache.flink.kubernetes.operator.controller.FlinkBlueGreenDeployments; import org.apache.flink.kubernetes.operator.service.FlinkResourceContextFactory; +import io.fabric8.kubernetes.api.model.ObjectMeta; import io.javaoperatorsdk.operator.api.reconciler.Context; import lombok.Getter; import lombok.RequiredArgsConstructor; @@ -62,4 +63,9 @@ public BlueGreenDeploymentType getOppositeDeploymentType(BlueGreenDeploymentType ? BlueGreenDeploymentType.GREEN : BlueGreenDeploymentType.BLUE; } + + public String getConfigMapName() { + ObjectMeta bgMeta = bgDeployment.getMetadata(); + return bgMeta.getName() + "-configmap"; + } } diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/BlueGreenDeploymentService.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/BlueGreenDeploymentService.java index 85de365b97..22d6fb2093 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/BlueGreenDeploymentService.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/BlueGreenDeploymentService.java @@ -29,6 +29,7 @@ import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; import org.apache.flink.kubernetes.operator.controller.FlinkBlueGreenDeployments; import org.apache.flink.kubernetes.operator.controller.FlinkResourceContext; +import org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenTransitionUtils; import org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenUtils; import org.apache.flink.util.Preconditions; @@ -47,7 +48,9 @@ import static org.apache.flink.kubernetes.operator.controller.bluegreen.BlueGreenKubernetesService.deployCluster; import static org.apache.flink.kubernetes.operator.controller.bluegreen.BlueGreenKubernetesService.isFlinkDeploymentReady; import static org.apache.flink.kubernetes.operator.controller.bluegreen.BlueGreenKubernetesService.suspendFlinkDeployment; +import static org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenTransitionUtils.updateTransitionStageFromJobStatus; import static org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenUtils.fetchSavepointInfo; +import static org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenUtils.getDeploymentDeletionDelay; import static org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenUtils.getReconciliationReschedInterval; import static org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenUtils.getSpecDiff; import static org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenUtils.hasSpecChanged; @@ -92,6 +95,9 @@ public UpdateControl initiateDeployment( isFirstDeployment, bgMeta); + BlueGreenTransitionUtils.prepareTransitionMetadata( + context, nextBlueGreenDeploymentType, flinkDeployment, isFirstDeployment); + deployCluster(context, flinkDeployment); BlueGreenUtils.setAbortTimestamp(context); @@ -386,6 +392,7 @@ public UpdateControl monitorTransition( determineTransitionState(context, currentBlueGreenDeploymentType); if (isFlinkDeploymentReady(transitionState.nextDeployment)) { + BlueGreenTransitionUtils.moveToFirstTransitionStage(context); return shouldWeDelete( context, transitionState.currentDeployment, @@ -454,6 +461,12 @@ private UpdateControl shouldWeDelete( FlinkDeployment nextDeployment, FlinkBlueGreenDeploymentState nextState) { + if (!BlueGreenTransitionUtils.isClearToTeardown(context)) { + // Wait until CLEAR_TO_TEARDOWN is set by the client + return UpdateControl.noUpdate() + .rescheduleAfter(getReconciliationReschedInterval(context)); + } + var deploymentStatus = context.getDeploymentStatus(); if (currentDeployment == null) { @@ -461,7 +474,7 @@ private UpdateControl shouldWeDelete( return finalizeBlueGreenDeployment(context, nextState); } - long deploymentDeletionDelayMs = BlueGreenUtils.getDeploymentDeletionDelay(context); + long deploymentDeletionDelayMs = getDeploymentDeletionDelay(context); long deploymentReadyTimestamp = instantStrToMillis(deploymentStatus.getDeploymentReadyTimestamp()); @@ -555,6 +568,7 @@ private UpdateControl abortDeployment( FlinkBlueGreenDeploymentState previousState = getPreviousState(nextState, context.getDeployments()); + BlueGreenTransitionUtils.rollbackActiveDeploymentType(context, previousState); context.getDeploymentStatus().setBlueGreenState(previousState); var error = @@ -624,6 +638,7 @@ public static UpdateControl patchStatusUpdateControl( } if (jobState != null) { + updateTransitionStageFromJobStatus(context, jobState); deploymentStatus.getJobStatus().setState(jobState); } diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/BlueGreenKubernetesService.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/BlueGreenKubernetesService.java index ae7492d312..24716d401f 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/BlueGreenKubernetesService.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/BlueGreenKubernetesService.java @@ -23,15 +23,23 @@ import org.apache.flink.kubernetes.operator.api.spec.JobState; import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; +import io.fabric8.kubernetes.api.model.ConfigMap; +import io.fabric8.kubernetes.api.model.ConfigMapBuilder; import io.fabric8.kubernetes.api.model.ObjectMeta; import io.fabric8.kubernetes.api.model.OwnerReference; import io.fabric8.kubernetes.api.model.StatusDetails; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.List; +import java.util.Map; +import java.util.Optional; /** Utility methods for handling Kubernetes operations in Blue/Green deployments. */ public class BlueGreenKubernetesService { + private static final Logger LOG = LoggerFactory.getLogger(BlueGreenKubernetesService.class); + /** * Creates ObjectMeta for a dependent Kubernetes resource with proper owner references. * @@ -116,4 +124,59 @@ public static boolean deleteFlinkDeployment( return deletedStatus.size() == 1 && deletedStatus.get(0).getKind().equals("FlinkDeployment"); } + + // ==================== ConfigMap related methods ==================== + + public static void updateConfigMapEntry(BlueGreenContext context, String key, String value) { + FlinkBlueGreenDeployment bgDeployment = context.getBgDeployment(); + var josdkContext = context.getJosdkContext(); + ConfigMap configMap = getConfigMap(context); + String namespace = bgDeployment.getMetadata().getNamespace(); + configMap.getData().put(key, value); + josdkContext.getClient().configMaps().inNamespace(namespace).resource(configMap).update(); + } + + public static ConfigMap getConfigMap(BlueGreenContext context) { + Optional configMapOpt = + context.getJosdkContext().getSecondaryResources(ConfigMap.class).stream() + .filter(cm -> cm.getMetadata().getName().equals(context.getConfigMapName())) + .findFirst(); + + if (configMapOpt.isEmpty()) { + throw new RuntimeException( + "Expected Blue/Green ConfigMap " + context.getConfigMapName() + " not found"); + } + + return configMapOpt.get(); + } + + public static void upsertConfigMap(BlueGreenContext context, Map data) { + var bgDeployment = context.getBgDeployment(); + var bgMeta = bgDeployment.getMetadata(); + var configMap = new ConfigMapBuilder().addToData(data).build(); + var configMapMeta = getDependentObjectMeta(bgDeployment); + configMapMeta.setName(context.getConfigMapName()); + + // Set metadata BEFORE creating the resource reference + configMap.setMetadata(configMapMeta); + + var configMapResource = + context.getJosdkContext() + .getClient() + .configMaps() + .inNamespace(bgMeta.getNamespace()) + .resource(configMap); + + if (configMapResource.get() == null) { + LOG.info("Creating new Blue/Green ConfigMap for deploymentType: {}", bgMeta.getName()); + configMapResource.create(); + } else { + Map existingData = configMapResource.get().getData(); + LOG.warn( + "Found Blue/Green ConfigMap, existing data: {}, replaced by: {}", + existingData, + data); + configMapResource.update(); + } + } } diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/bluegreen/BlueGreenTransitionUtils.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/bluegreen/BlueGreenTransitionUtils.java new file mode 100644 index 0000000000..5bb48c2ce5 --- /dev/null +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/bluegreen/BlueGreenTransitionUtils.java @@ -0,0 +1,181 @@ +/* + * 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.utils.bluegreen; + +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.kubernetes.operator.api.FlinkDeployment; +import org.apache.flink.kubernetes.operator.api.bluegreen.BlueGreenDeploymentType; +import org.apache.flink.kubernetes.operator.api.bluegreen.GateContextOptions; +import org.apache.flink.kubernetes.operator.api.bluegreen.TransitionMode; +import org.apache.flink.kubernetes.operator.api.bluegreen.TransitionStage; +import org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentState; +import org.apache.flink.kubernetes.operator.controller.bluegreen.BlueGreenContext; + +import io.fabric8.kubernetes.api.model.ConfigMap; +import lombok.SneakyThrows; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.naming.OperationNotSupportedException; + +import java.util.Map; + +import static org.apache.flink.kubernetes.operator.api.bluegreen.GateContextOptions.ACTIVE_DEPLOYMENT_TYPE; +import static org.apache.flink.kubernetes.operator.api.bluegreen.GateContextOptions.IS_FIRST_DEPLOYMENT; +import static org.apache.flink.kubernetes.operator.api.bluegreen.GateContextOptions.TRANSITION_STAGE; +import static org.apache.flink.kubernetes.operator.controller.bluegreen.BlueGreenKubernetesService.getConfigMap; +import static org.apache.flink.kubernetes.operator.controller.bluegreen.BlueGreenKubernetesService.updateConfigMapEntry; +import static org.apache.flink.kubernetes.operator.controller.bluegreen.BlueGreenKubernetesService.upsertConfigMap; +import static org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenUtils.getDeploymentDeletionDelay; + +/** Utility class for Blue/Green transition stage operations. */ +public class BlueGreenTransitionUtils { + + private static final Logger LOG = LoggerFactory.getLogger(BlueGreenTransitionUtils.class); + + @SneakyThrows + public static TransitionMode getTransitionMode(BlueGreenContext context) { + TransitionMode transitionMode = + context.getBgDeployment().getSpec().getTemplate().getTransitionMode(); + + if (transitionMode == null) { + throw new OperationNotSupportedException("Please specify the TransitionMode"); + } + + return transitionMode; + } + + public static void prepareTransitionMetadata( + BlueGreenContext context, + BlueGreenDeploymentType blueGreenDeploymentType, + FlinkDeployment flinkDeployment, + boolean isFirstDeployment) { + if (TransitionMode.ADVANCED != getTransitionMode(context)) { + return; + } + + var transitionDefaultMetadata = + Map.of( + IS_FIRST_DEPLOYMENT.getLabel(), + isFirstDeployment ? "true" : "false", + GateContextOptions.DEPLOYMENT_DELETION_DELAY.getLabel(), + String.valueOf(getDeploymentDeletionDelay(context)), + ACTIVE_DEPLOYMENT_TYPE.getLabel(), + blueGreenDeploymentType.toString(), + TRANSITION_STAGE.getLabel(), + TransitionStage.INITIALIZING.toString()); + + upsertConfigMap(context, transitionDefaultMetadata); + + // Preparing the FlinkConfiguration for the OutputDecider + flinkDeployment + .getSpec() + .getFlinkConfiguration() + .put( + "bluegreen." + ACTIVE_DEPLOYMENT_TYPE.getLabel(), + blueGreenDeploymentType.toString()); + flinkDeployment + .getSpec() + .getFlinkConfiguration() + .put("bluegreen.configmap.name", context.getConfigMapName()); + } + + public static void moveToFirstTransitionStage(BlueGreenContext context) { + if (TransitionMode.ADVANCED != getTransitionMode(context)) { + return; + } + + ConfigMap configMap = getConfigMap(context); + String stage = configMap.getData().get(TRANSITION_STAGE.getLabel()); + + if (stage.equals(TransitionStage.INITIALIZING.toString())) { + TransitionStage nextStage; + + if (context.getDeployments().getNumberOfDeployments() == 2) { + LOG.info("Stage INITIALIZING to TRANSITIONING"); + nextStage = TransitionStage.TRANSITIONING; + } else { + LOG.info( + "No transition between deployments detected, Stage INITIALIZING -> CLEAR_TO_TEARDOWN"); + nextStage = TransitionStage.CLEAR_TO_TEARDOWN; + } + + updateTransitionStage(context, nextStage); + } + } + + public static boolean isClearToTeardown(BlueGreenContext context) { + + if (TransitionMode.ADVANCED == getTransitionMode(context)) { + ConfigMap configMap = getConfigMap(context); + String stage = configMap.getData().get(TRANSITION_STAGE.getLabel()); + + if (!stage.equals(TransitionStage.CLEAR_TO_TEARDOWN.toString())) { + LOG.info("Waiting for CLEAR_TO_TEARDOWN, current stage: " + stage); + return false; + } + } + + return true; + } + + public static void rollbackActiveDeploymentType( + BlueGreenContext context, FlinkBlueGreenDeploymentState previousState) { + + if (TransitionMode.ADVANCED != getTransitionMode(context)) { + return; + } + + var previousDeploymentType = + previousState.name().contains("BLUE") + ? BlueGreenDeploymentType.BLUE + : BlueGreenDeploymentType.GREEN; + + updateConfigMapEntry( + context, ACTIVE_DEPLOYMENT_TYPE.getLabel(), previousDeploymentType.toString()); + } + + public static void updateTransitionStageFromJobStatus( + BlueGreenContext context, JobStatus jobStatus) { + + if (TransitionMode.ADVANCED != getTransitionMode(context)) { + return; + } + + TransitionStage transitionStage; + switch (jobStatus) { + case RUNNING: + transitionStage = TransitionStage.RUNNING; + break; + case FAILING: + transitionStage = TransitionStage.FAILING; + break; + case RECONCILING: + transitionStage = TransitionStage.INITIALIZING; + break; + default: + throw new RuntimeException("Unsupported JobStatus: " + jobStatus); + } + updateTransitionStage(context, transitionStage); + } + + public static void updateTransitionStage( + BlueGreenContext context, TransitionStage transitionStage) { + updateConfigMapEntry(context, TRANSITION_STAGE.getLabel(), transitionStage.toString()); + } +} 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 f3f2e3571d..605f7d9732 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 @@ -497,9 +497,12 @@ public Optional getRetryInfo() { @Override public Set getSecondaryResources(Class aClass) { - // TODO: improve this, even if we only support FlinkDeployment as a secondary resource + KubernetesClient client = getClient(); + if (client == null) { + return new HashSet<>(); + } + if (aClass.getSimpleName().equals(FlinkDeployment.class.getSimpleName())) { - KubernetesClient client = getClient(); var hasMetadata = new HashSet<>( client.resources(FlinkDeployment.class) @@ -507,8 +510,12 @@ public Set getSecondaryResources(Class aClass) { .list() .getItems()); return (Set) hasMetadata; + } else if (aClass.getSimpleName().equals("ConfigMap")) { + var hasMetadata = + new HashSet<>(client.configMaps().inAnyNamespace().list().getItems()); + return (Set) hasMetadata; } else { - return null; + return new HashSet<>(); } } diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/BlueGreenTestUtils.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/BlueGreenTestUtils.java new file mode 100644 index 0000000000..c9e9990cf5 --- /dev/null +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/BlueGreenTestUtils.java @@ -0,0 +1,167 @@ +/* + * 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.controller; + +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.kubernetes.operator.api.FlinkBlueGreenDeployment; +import org.apache.flink.kubernetes.operator.api.bluegreen.TransitionMode; +import org.apache.flink.kubernetes.operator.api.bluegreen.TransitionStage; +import org.apache.flink.kubernetes.operator.api.spec.ConfigObjectNode; +import org.apache.flink.kubernetes.operator.api.spec.FlinkBlueGreenDeploymentSpec; +import org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec; +import org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentTemplateSpec; +import org.apache.flink.kubernetes.operator.api.spec.FlinkVersion; +import org.apache.flink.kubernetes.operator.api.spec.JobManagerSpec; +import org.apache.flink.kubernetes.operator.api.spec.JobSpec; +import org.apache.flink.kubernetes.operator.api.spec.JobState; +import org.apache.flink.kubernetes.operator.api.spec.Resource; +import org.apache.flink.kubernetes.operator.api.spec.TaskManagerSpec; +import org.apache.flink.kubernetes.operator.api.spec.UpgradeMode; + +import io.fabric8.kubernetes.api.model.ConfigMap; +import io.fabric8.kubernetes.api.model.ObjectMetaBuilder; +import io.javaoperatorsdk.operator.api.reconciler.Context; + +import java.time.Instant; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; + +import static org.apache.flink.kubernetes.operator.api.bluegreen.GateContextOptions.TRANSITION_STAGE; +import static org.apache.flink.kubernetes.operator.api.spec.FlinkBlueGreenDeploymentConfigOptions.ABORT_GRACE_PERIOD; +import static org.apache.flink.kubernetes.operator.api.spec.FlinkBlueGreenDeploymentConfigOptions.DEPLOYMENT_DELETION_DELAY; +import static org.apache.flink.kubernetes.operator.api.spec.FlinkBlueGreenDeploymentConfigOptions.RECONCILIATION_RESCHEDULING_INTERVAL; +import static org.apache.flink.kubernetes.operator.api.utils.BaseTestUtils.SAMPLE_JAR; + +/** Shared utilities for Blue/Green testing. */ +public class BlueGreenTestUtils { + + public static final String SERVICE_ACCOUNT = "flink-operator"; + public static final String FLINK_VERSION = "latest"; + public static final String IMAGE = String.format("flink:%s", FLINK_VERSION); + public static final String IMAGE_POLICY = "IfNotPresent"; + private static final int DEFAULT_DELETION_DELAY_VALUE = 500; + + public static FlinkBlueGreenDeployment buildAdvancedDeployment( + String name, + String namespace, + FlinkVersion version, + String initialSavepointPath, + UpgradeMode upgradeMode) { + var deployment = new FlinkBlueGreenDeployment(); + deployment.setMetadata( + new ObjectMetaBuilder() + .withName(name) + .withNamespace(namespace) + .withCreationTimestamp(Instant.now().toString()) + .withUid(UUID.randomUUID().toString()) + .withResourceVersion("1") + .build()); + var bgDeploymentSpec = getTestFlinkDeploymentSpec(version); + + bgDeploymentSpec + .getTemplate() + .getSpec() + .setJob( + JobSpec.builder() + .jarURI(SAMPLE_JAR) + .parallelism(1) + .upgradeMode(upgradeMode) + .state(JobState.RUNNING) + .initialSavepointPath(initialSavepointPath) + .build()); + + bgDeploymentSpec.getTemplate().setTransitionMode(TransitionMode.ADVANCED); + deployment.setSpec(bgDeploymentSpec); + return deployment; + } + + public static ConfigMap getConfigMapFromSecondaryResources( + Context context, String deploymentName) { + var configMaps = context.getSecondaryResources(ConfigMap.class); + return configMaps.stream() + .filter(cm -> cm.getMetadata().getName().equals(deploymentName + "-configmap")) + .findFirst() + .orElseThrow( + () -> + new RuntimeException( + "ConfigMap not found for deployment: " + deploymentName)); + } + + public static TransitionStage getCurrentConfigMapStage( + Context context, String deploymentName) { + ConfigMap configMap = getConfigMapFromSecondaryResources(context, deploymentName); + String stage = configMap.getData().get(TRANSITION_STAGE.getLabel()); + return TransitionStage.valueOf(stage); + } + + public static void assertConfigMapProtocol(ConfigMap configMap) { + Map data = configMap.getData(); + if (!data.containsKey("IS_FIRST_DEPLOYMENT") + || !data.containsKey("ACTIVE_DEPLOYMENT_TYPE") + || !data.containsKey("TRANSITION_STAGE")) { + throw new AssertionError("ConfigMap missing required protocol fields"); + } + } + + public static boolean isValidTransitionStage(String stage) { + try { + TransitionStage.valueOf(stage); + return true; + } catch (IllegalArgumentException e) { + return false; + } + } + + public static FlinkBlueGreenDeploymentSpec getTestFlinkDeploymentSpec(FlinkVersion version) { + Map conf = new HashMap<>(); + conf.put(TaskManagerOptions.NUM_TASK_SLOTS.key(), "2"); + conf.put(CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), "test-savepoint-dir"); + conf.put(CheckpointingOptions.INCREMENTAL_CHECKPOINTS.key(), "true"); + conf.put(CheckpointingOptions.MAX_RETAINED_CHECKPOINTS.key(), "10"); + conf.put( + CheckpointingOptions.CHECKPOINTS_DIRECTORY.key(), + "file:///test/test-checkpoint-dir"); + + var flinkDeploymentSpec = + FlinkDeploymentSpec.builder() + .image(IMAGE) + .imagePullPolicy(IMAGE_POLICY) + .serviceAccount(SERVICE_ACCOUNT) + .flinkVersion(version) + .flinkConfiguration(new ConfigObjectNode()) + .jobManager(new JobManagerSpec(new Resource(1.0, "2048m", "2G"), 1, null)) + .taskManager( + new TaskManagerSpec(new Resource(1.0, "2048m", "2G"), null, null)) + .build(); + + flinkDeploymentSpec.setFlinkConfiguration(conf); + + Map configuration = new HashMap<>(); + configuration.put(ABORT_GRACE_PERIOD.key(), "1"); + configuration.put(RECONCILIATION_RESCHEDULING_INTERVAL.key(), "500"); + configuration.put( + DEPLOYMENT_DELETION_DELAY.key(), String.valueOf(DEFAULT_DELETION_DELAY_VALUE)); + + var flinkDeploymentTemplateSpec = + FlinkDeploymentTemplateSpec.builder().spec(flinkDeploymentSpec).build(); + + return new FlinkBlueGreenDeploymentSpec(configuration, flinkDeploymentTemplateSpec); + } +} diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentControllerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentControllerTest.java index 229406bbc9..13d8e061c4 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentControllerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentControllerTest.java @@ -18,23 +18,20 @@ package org.apache.flink.kubernetes.operator.controller; import org.apache.flink.api.common.JobStatus; -import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.kubernetes.operator.TestUtils; import org.apache.flink.kubernetes.operator.TestingFlinkService; import org.apache.flink.kubernetes.operator.api.FlinkBlueGreenDeployment; import org.apache.flink.kubernetes.operator.api.FlinkDeployment; -import org.apache.flink.kubernetes.operator.api.spec.ConfigObjectNode; +import org.apache.flink.kubernetes.operator.api.bluegreen.GateContextOptions; +import org.apache.flink.kubernetes.operator.api.bluegreen.TransitionMode; +import org.apache.flink.kubernetes.operator.api.bluegreen.TransitionStage; import org.apache.flink.kubernetes.operator.api.spec.FlinkBlueGreenDeploymentSpec; import org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec; import org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentTemplateSpec; import org.apache.flink.kubernetes.operator.api.spec.FlinkVersion; -import org.apache.flink.kubernetes.operator.api.spec.JobManagerSpec; import org.apache.flink.kubernetes.operator.api.spec.JobSpec; import org.apache.flink.kubernetes.operator.api.spec.JobState; -import org.apache.flink.kubernetes.operator.api.spec.Resource; -import org.apache.flink.kubernetes.operator.api.spec.TaskManagerSpec; import org.apache.flink.kubernetes.operator.api.spec.UpgradeMode; import org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentState; import org.apache.flink.kubernetes.operator.api.status.ReconciliationState; @@ -42,6 +39,7 @@ import org.apache.flink.kubernetes.operator.config.FlinkConfigManager; import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; +import io.fabric8.kubernetes.api.model.ConfigMap; import io.fabric8.kubernetes.api.model.ObjectMetaBuilder; import io.fabric8.kubernetes.client.KubernetesClient; import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient; @@ -61,12 +59,16 @@ import java.util.UUID; import java.util.stream.Stream; +import static org.apache.flink.kubernetes.operator.api.bluegreen.GateContextOptions.ACTIVE_DEPLOYMENT_TYPE; +import static org.apache.flink.kubernetes.operator.api.bluegreen.GateContextOptions.IS_FIRST_DEPLOYMENT; +import static org.apache.flink.kubernetes.operator.api.bluegreen.GateContextOptions.TRANSITION_STAGE; import static org.apache.flink.kubernetes.operator.api.spec.FlinkBlueGreenDeploymentConfigOptions.ABORT_GRACE_PERIOD; import static org.apache.flink.kubernetes.operator.api.spec.FlinkBlueGreenDeploymentConfigOptions.DEPLOYMENT_DELETION_DELAY; import static org.apache.flink.kubernetes.operator.api.spec.FlinkBlueGreenDeploymentConfigOptions.RECONCILIATION_RESCHEDULING_INTERVAL; import static org.apache.flink.kubernetes.operator.api.utils.BaseTestUtils.SAMPLE_JAR; import static org.apache.flink.kubernetes.operator.api.utils.BaseTestUtils.TEST_DEPLOYMENT_NAME; import static org.apache.flink.kubernetes.operator.api.utils.BaseTestUtils.TEST_NAMESPACE; +import static org.apache.flink.kubernetes.operator.controller.BlueGreenTestUtils.getTestFlinkDeploymentSpec; import static org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenUtils.instantStrToMillis; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -271,8 +273,13 @@ public void verifyFailureDuringTransition(FlinkVersion flinkVersion) throws Exce rs.reconciledStatus.getBlueGreenState()); assertEquals( customValue, - getFlinkConfigurationValue( - rs.deployment.getSpec().getTemplate().getSpec(), CUSTOM_CONFIG_FIELD)); + rs.deployment + .getSpec() + .getTemplate() + .getSpec() + .getFlinkConfiguration() + .get(CUSTOM_CONFIG_FIELD) + .asText()); // Simulating the Blue deployment doesn't start correctly (status will remain the same) Long reschedDelayMs = 0L; @@ -325,11 +332,6 @@ public void verifyFailureDuringTransition(FlinkVersion flinkVersion) throws Exce testTransitionToGreen(rs, customValue, null); } - private static String getFlinkConfigurationValue( - FlinkDeploymentSpec flinkDeploymentSpec, String propertyName) { - return flinkDeploymentSpec.getFlinkConfiguration().get(propertyName).asText(); - } - @ParameterizedTest @MethodSource("org.apache.flink.kubernetes.operator.TestUtils#flinkVersions") public void verifySpecChangeDuringTransition(FlinkVersion flinkVersion) throws Exception { @@ -704,9 +706,12 @@ public void verifySpecificBehavior( assertEquals(1, deployments.size()); assertEquals( "100 SECONDS", - getFlinkConfigurationValue( - deployments.get(0).getSpec(), - "kubernetes.operator.reconcile.interval")); + deployments + .get(0) + .getSpec() + .getFlinkConfiguration() + .get("kubernetes.operator.reconcile.interval") + .asText()); } } @@ -763,9 +768,12 @@ public void verifySpecificBehavior( // Child spec change should be applied to FlinkDeployment assertEquals( "100 SECONDS", - getFlinkConfigurationValue( - deployments.get(0).getSpec(), - "kubernetes.operator.reconcile.interval")); + deployments + .get(0) + .getSpec() + .getFlinkConfiguration() + .get("kubernetes.operator.reconcile.interval") + .asText()); // Top-level changes should be preserved in reconciled spec assertNotNull(result.rs.reconciledStatus.getLastReconciledSpec()); @@ -1041,8 +1049,13 @@ private void testTransitionToGreen( assertEquals(0, instantStrToMillis(rs.reconciledStatus.getDeploymentReadyTimestamp())); assertEquals( customValue, - getFlinkConfigurationValue( - rs.deployment.getSpec().getTemplate().getSpec(), CUSTOM_CONFIG_FIELD)); + rs.deployment + .getSpec() + .getTemplate() + .getSpec() + .getFlinkConfiguration() + .get(CUSTOM_CONFIG_FIELD) + .asText()); // Initiate and mark the Green deployment ready simulateSuccessfulJobStart(getFlinkDeployments().get(1)); @@ -1187,43 +1200,223 @@ private static FlinkBlueGreenDeployment buildSessionCluster( .initialSavepointPath(initialSavepointPath) .build()); + bgDeploymentSpec.getTemplate().setTransitionMode(TransitionMode.BASIC); deployment.setSpec(bgDeploymentSpec); return deployment; } - private static FlinkBlueGreenDeploymentSpec getTestFlinkDeploymentSpec(FlinkVersion version) { - Map conf = new HashMap<>(); - conf.put(TaskManagerOptions.NUM_TASK_SLOTS.key(), "2"); - conf.put(CheckpointingOptions.SAVEPOINT_DIRECTORY.key(), "test-savepoint-dir"); - conf.put(CheckpointingOptions.INCREMENTAL_CHECKPOINTS.key(), "true"); - conf.put(CheckpointingOptions.MAX_RETAINED_CHECKPOINTS.key(), "10"); - conf.put( - CheckpointingOptions.CHECKPOINTS_DIRECTORY.key(), - "file:///test/test-checkpoint-dir"); - - var flinkDeploymentSpec = - FlinkDeploymentSpec.builder() - .image(IMAGE) - .imagePullPolicy(IMAGE_POLICY) - .serviceAccount(SERVICE_ACCOUNT) - .flinkVersion(version) - .flinkConfiguration(new ConfigObjectNode()) - .jobManager(new JobManagerSpec(new Resource(1.0, "2048m", "2G"), 1, null)) - .taskManager( - new TaskManagerSpec(new Resource(1.0, "2048m", "2G"), null, null)) - .build(); - - flinkDeploymentSpec.setFlinkConfiguration(conf); - - Map configuration = new HashMap<>(); - configuration.put(ABORT_GRACE_PERIOD.key(), "1"); - configuration.put(RECONCILIATION_RESCHEDULING_INTERVAL.key(), "500"); - configuration.put( - DEPLOYMENT_DELETION_DELAY.key(), String.valueOf(DEFAULT_DELETION_DELAY_VALUE)); + // ==================== ADVANCED Mode Infrastructure Tests ==================== + + @ParameterizedTest + @MethodSource("org.apache.flink.kubernetes.operator.TestUtils#flinkVersions") + public void verifyAdvancedModeConfigMapCreation(FlinkVersion flinkVersion) throws Exception { + var deployment = + buildAdvancedSessionCluster( + TEST_DEPLOYMENT_NAME, + TEST_NAMESPACE, + flinkVersion, + null, + UpgradeMode.STATELESS); + var rs = executeAdvancedDeployment(deployment); + + // Verify ConfigMap is created with correct schema + ConfigMap configMap = + BlueGreenTestUtils.getConfigMapFromSecondaryResources( + context, TEST_DEPLOYMENT_NAME); + assertNotNull(configMap); + + Map data = configMap.getData(); + assertEquals("true", data.get(IS_FIRST_DEPLOYMENT.getLabel())); + assertEquals("BLUE", data.get(ACTIVE_DEPLOYMENT_TYPE.getLabel())); + assertEquals( + String.valueOf(DEFAULT_DELETION_DELAY_VALUE), + data.get(GateContextOptions.DEPLOYMENT_DELETION_DELAY.getLabel())); + + // Verify ConfigMap is available as secondary resource + var configMaps = context.getSecondaryResources(ConfigMap.class); + assertEquals(1, configMaps.size()); + assertEquals( + configMap.getMetadata().getName(), + configMaps.iterator().next().getMetadata().getName()); + } + + @ParameterizedTest + @MethodSource("org.apache.flink.kubernetes.operator.TestUtils#flinkVersions") + public void verifyAdvancedModeStageProgression(FlinkVersion flinkVersion) throws Exception { + var deployment = + buildAdvancedSessionCluster( + TEST_DEPLOYMENT_NAME, + TEST_NAMESPACE, + flinkVersion, + null, + UpgradeMode.STATELESS); + var rs = executeAdvancedDeployment(deployment); + + // After executeAdvancedDeployment, should be in RUNNING state + assertEquals( + TransitionStage.RUNNING, + BlueGreenTestUtils.getCurrentConfigMapStage(context, TEST_DEPLOYMENT_NAME)); + assertEquals( + FlinkBlueGreenDeploymentState.ACTIVE_BLUE, rs.reconciledStatus.getBlueGreenState()); + } + + @ParameterizedTest + @MethodSource("org.apache.flink.kubernetes.operator.TestUtils#flinkVersions") + public void verifyAdvancedModeBlocking(FlinkVersion flinkVersion) throws Exception { + var deployment = + buildAdvancedSessionCluster( + TEST_DEPLOYMENT_NAME, + TEST_NAMESPACE, + flinkVersion, + null, + UpgradeMode.STATELESS); + + // Complete first deployment + var rs = executeAdvancedDeployment(deployment); + assertEquals( + FlinkBlueGreenDeploymentState.ACTIVE_BLUE, rs.reconciledStatus.getBlueGreenState()); + + // Trigger Blue->Green transition + simulateChangeInSpec(rs.deployment, "green-config", ALT_DELETION_DELAY_VALUE, null); + rs = reconcile(rs.deployment); + + // Initial stage should be INITIALIZING + assertEquals( + TransitionStage.INITIALIZING, + BlueGreenTestUtils.getCurrentConfigMapStage(context, TEST_DEPLOYMENT_NAME)); + + simulateSuccessfulJobStart(getFlinkDeployments().get(1)); + rs = reconcile(rs.deployment); + + // Should be in TRANSITIONING stage and blocked + assertEquals( + TransitionStage.TRANSITIONING, + BlueGreenTestUtils.getCurrentConfigMapStage(context, TEST_DEPLOYMENT_NAME)); + assertEquals( + FlinkBlueGreenDeploymentState.TRANSITIONING_TO_GREEN, + rs.reconciledStatus.getBlueGreenState()); + + // Multiple reconciles should remain blocked (no-op) + rs = reconcile(rs.deployment); + assertTrue(rs.updateControl.isNoUpdate()); - var flinkDeploymentTemplateSpec = - FlinkDeploymentTemplateSpec.builder().spec(flinkDeploymentSpec).build(); + rs = reconcile(rs.deployment); + assertTrue(rs.updateControl.isNoUpdate()); + + // Verify GREEN deployment is created but transition is blocked + assertEquals(2, getFlinkDeployments().size()); + + // Simulate GREEN deployment becoming ready + simulateSuccessfulJobStart(getFlinkDeployments().get(1)); // Green deployment + rs = reconcile(rs.deployment); + + // Should still be blocked waiting for CLEAR_TO_TEARDOWN + assertEquals( + TransitionStage.TRANSITIONING, + BlueGreenTestUtils.getCurrentConfigMapStage(context, TEST_DEPLOYMENT_NAME)); + assertTrue(rs.updateControl.isNoUpdate()); + + // Simulate external system (gate) updating ConfigMap to CLEAR_TO_TEARDOWN + simulateExternalConfigMapUpdate( + TEST_DEPLOYMENT_NAME, + TRANSITION_STAGE.getLabel(), + TransitionStage.CLEAR_TO_TEARDOWN.toString()); + + // Now reconciliation should proceed with the transition + rs = reconcile(rs.deployment); + assertEquals( + TransitionStage.CLEAR_TO_TEARDOWN, + BlueGreenTestUtils.getCurrentConfigMapStage(context, TEST_DEPLOYMENT_NAME)); + + // Verify transition can continue - should schedule for deletion delay + assertTrue(rs.updateControl.isPatchStatus()); + assertTrue(rs.updateControl.getScheduleDelay().isPresent()); + assertEquals(ALT_DELETION_DELAY_VALUE, rs.updateControl.getScheduleDelay().get()); + + // Complete the transition after deletion delay + Thread.sleep(ALT_DELETION_DELAY_VALUE); + rs = reconcile(rs.deployment); + + // Verify successful transition completion + assertEquals(1, getFlinkDeployments().size()); // Blue deployment deleted + rs = reconcile(rs.deployment); // Final reconciliation + assertEquals( + FlinkBlueGreenDeploymentState.ACTIVE_GREEN, + rs.reconciledStatus.getBlueGreenState()); + assertEquals( + TransitionStage.RUNNING, + BlueGreenTestUtils.getCurrentConfigMapStage(context, TEST_DEPLOYMENT_NAME)); + } + + @ParameterizedTest + @MethodSource("org.apache.flink.kubernetes.operator.TestUtils#flinkVersions") + public void verifyAdvancedModeJobStatusMapping(FlinkVersion flinkVersion) throws Exception { + var deployment = + buildAdvancedSessionCluster( + TEST_DEPLOYMENT_NAME, + TEST_NAMESPACE, + flinkVersion, + null, + UpgradeMode.STATELESS); + var rs = executeAdvancedDeployment(deployment); + + // Test JobStatus.RUNNING -> TransitionStage.RUNNING + assertEquals( + TransitionStage.RUNNING, + BlueGreenTestUtils.getCurrentConfigMapStage(context, TEST_DEPLOYMENT_NAME)); + + // Test JobStatus.FAILING -> TransitionStage.FAILING + simulateChangeInSpec(rs.deployment, UUID.randomUUID().toString(), 0, null); + simulateJobFailure(getFlinkDeployments().get(0)); + rs = reconcile(rs.deployment); + assertEquals( + TransitionStage.FAILING, + BlueGreenTestUtils.getCurrentConfigMapStage(context, TEST_DEPLOYMENT_NAME)); + + // Test JobStatus.RECONCILING -> TransitionStage.INITIALIZING + simulateChangeInSpec(rs.deployment, UUID.randomUUID().toString(), 0, null); + simulateSuccessfulJobStart(getFlinkDeployments().get(0)); + rs = reconcile(rs.deployment); + assertEquals( + TransitionStage.INITIALIZING, + BlueGreenTestUtils.getCurrentConfigMapStage(context, TEST_DEPLOYMENT_NAME)); + } + + // ==================== ADVANCED Mode Helper Methods ==================== + + private FlinkBlueGreenDeployment buildAdvancedSessionCluster( + String name, + String namespace, + FlinkVersion version, + String initialSavepointPath, + UpgradeMode upgradeMode) { + var deployment = + buildSessionCluster(name, namespace, version, initialSavepointPath, upgradeMode); + deployment.getSpec().getTemplate().setTransitionMode(TransitionMode.ADVANCED); + return deployment; + } + + private TestingFlinkBlueGreenDeploymentController.BlueGreenReconciliationResult + executeAdvancedDeployment(FlinkBlueGreenDeployment deployment) throws Exception { + var rs = reconcile(deployment); // Initialize + rs = reconcile(rs.deployment); // Transition to Blue + simulateSuccessfulJobStart(getFlinkDeployments().get(0)); + rs = reconcile(rs.deployment); // Complete the deployment and reach ACTIVE_BLUE + return rs; + } - return new FlinkBlueGreenDeploymentSpec(configuration, flinkDeploymentTemplateSpec); + private void simulateExternalConfigMapUpdate(String deploymentName, String key, String value) { + String configMapName = + deploymentName + + "-configmap"; // Using actual naming convention from BlueGreenContext + ConfigMap configMap = + kubernetesClient + .configMaps() + .inNamespace(TEST_NAMESPACE) + .withName(configMapName) + .get(); + + configMap.getData().put(key, value); + kubernetesClient.configMaps().inNamespace(TEST_NAMESPACE).resource(configMap).update(); } } diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/TestingFlinkBlueGreenDeploymentController.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/TestingFlinkBlueGreenDeploymentController.java index 6e8e058ce2..0af9a3a5f3 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/TestingFlinkBlueGreenDeploymentController.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/TestingFlinkBlueGreenDeploymentController.java @@ -27,7 +27,6 @@ import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.api.reconciler.ErrorStatusUpdateControl; import io.javaoperatorsdk.operator.api.reconciler.Reconciler; import io.javaoperatorsdk.operator.api.reconciler.UpdateControl; import lombok.AllArgsConstructor; @@ -62,14 +61,6 @@ public UpdateControl reconcile( return flinkBlueGreenDeploymentController.reconcile(cloned, context); } - @Override - public ErrorStatusUpdateControl updateErrorStatus( - FlinkBlueGreenDeployment flinkBlueGreenDeployment, - Context context, - Exception e) { - return null; - } - /** A simple DTO to handle common reconciliation results for tests. */ @AllArgsConstructor public static class BlueGreenReconciliationResult { diff --git a/helm/flink-kubernetes-operator/crds/flinkbluegreendeployments.flink.apache.org-v1.yml b/helm/flink-kubernetes-operator/crds/flinkbluegreendeployments.flink.apache.org-v1.yml index d31d14aa24..48a74cd156 100644 --- a/helm/flink-kubernetes-operator/crds/flinkbluegreendeployments.flink.apache.org-v1.yml +++ b/helm/flink-kubernetes-operator/crds/flinkbluegreendeployments.flink.apache.org-v1.yml @@ -10751,6 +10751,11 @@ spec: type: "object" type: "object" type: "object" + transitionMode: + enum: + - "ADVANCED" + - "BASIC" + type: "string" type: "object" type: "object" status: 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 33ab49e82d..1aa1969dc6 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 @@ -1,31 +1,28 @@ # Generated by Fabric8 CRDGenerator, manual edits might get overwritten! -apiVersion: apiextensions.k8s.io/v1 -kind: CustomResourceDefinition +apiVersion: "apiextensions.k8s.io/v1" +kind: "CustomResourceDefinition" metadata: - name: flinkdeployments.flink.apache.org + name: "flinkdeployments.flink.apache.org" spec: - group: flink.apache.org + group: "flink.apache.org" names: - kind: FlinkDeployment - plural: flinkdeployments + kind: "FlinkDeployment" + plural: "flinkdeployments" shortNames: - - flinkdep - singular: flinkdeployment - scope: Namespaced + - "flinkdep" + singular: "flinkdeployment" + scope: "Namespaced" versions: - additionalPrinterColumns: - - description: Last observed state of the job. - jsonPath: .status.jobStatus.state - name: Job Status + - 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 + type: "string" + - jsonPath: ".status.lifecycleState" + name: "Lifecycle State" priority: 0 - type: string - name: v1beta1 + type: "string" + name: "v1beta1" schema: openAPIV3Schema: properties: @@ -35,167 +32,167 @@ spec: x-kubernetes-preserve-unknown-fields: true flinkVersion: enum: - - v1_13 - - v1_14 - - v1_15 - - v1_16 - - v1_17 - - v1_18 - - v1_19 - - v1_20 - - v2_0 - - v2_1 - - v2_2 - type: string + - "v1_13" + - "v1_14" + - "v1_15" + - "v1_16" + - "v1_17" + - "v1_18" + - "v1_19" + - "v1_20" + - "v2_0" + - "v2_1" + - "v2_2" + type: "string" image: - type: string + type: "string" imagePullPolicy: - type: string + type: "string" ingress: properties: annotations: additionalProperties: - type: string - type: object + type: "string" + type: "object" className: - type: string + type: "string" labels: additionalProperties: - type: string - type: object + type: "string" + type: "object" template: - type: string + type: "string" tls: items: properties: hosts: items: - type: string - type: array + type: "string" + type: "array" secretName: - type: string - type: object - type: array - type: object + type: "string" + type: "object" + type: "array" + type: "object" job: properties: allowNonRestoredState: - type: boolean + type: "boolean" args: items: - type: string - type: array + type: "string" + type: "array" autoscalerResetNonce: - type: integer + type: "integer" checkpointTriggerNonce: - type: integer + type: "integer" entryClass: - type: string + type: "string" initialSavepointPath: - type: string + type: "string" jarURI: - type: string + type: "string" parallelism: - type: integer + type: "integer" savepointRedeployNonce: - type: integer + type: "integer" savepointTriggerNonce: - type: integer + type: "integer" state: enum: - - running - - suspended - type: string + - "running" + - "suspended" + type: "string" upgradeMode: enum: - - last-state - - savepoint - - stateless - type: string - type: object + - "last-state" + - "savepoint" + - "stateless" + type: "string" + type: "object" jobManager: properties: podTemplate: properties: apiVersion: - type: string + type: "string" kind: - type: string + type: "string" metadata: properties: annotations: additionalProperties: - type: string - type: object + type: "string" + type: "object" creationTimestamp: - type: string + type: "string" deletionGracePeriodSeconds: - type: integer + type: "integer" deletionTimestamp: - type: string + type: "string" finalizers: items: - type: string - type: array + type: "string" + type: "array" generateName: - type: string + type: "string" generation: - type: integer + type: "integer" labels: additionalProperties: - type: string - type: object + type: "string" + type: "object" managedFields: items: properties: apiVersion: - type: string + type: "string" fieldsType: - type: string + type: "string" fieldsV1: - type: object + type: "object" manager: - type: string + type: "string" operation: - type: string + type: "string" subresource: - type: string + type: "string" time: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" name: - type: string + type: "string" namespace: - type: string + type: "string" ownerReferences: items: properties: apiVersion: - type: string + type: "string" blockOwnerDeletion: - type: boolean + type: "boolean" controller: - type: boolean + type: "boolean" kind: - type: string + type: "string" name: - type: string + type: "string" uid: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" resourceVersion: - type: string + type: "string" selfLink: - type: string + type: "string" uid: - type: string - type: object + type: "string" + type: "object" spec: properties: activeDeadlineSeconds: - type: integer + type: "integer" affinity: properties: nodeAffinity: @@ -209,33 +206,33 @@ spec: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchFields: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array - type: object + type: "string" + type: "array" + type: "object" + type: "array" + type: "object" weight: - type: integer - type: object - type: array + type: "integer" + type: "object" + type: "array" requiredDuringSchedulingIgnoredDuringExecution: properties: nodeSelectorTerms: @@ -245,32 +242,32 @@ spec: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchFields: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array - type: object - type: array - type: object - type: object + type: "string" + type: "array" + type: "object" + type: "array" + type: "object" + type: "array" + type: "object" + type: "object" podAffinity: properties: preferredDuringSchedulingIgnoredDuringExecution: @@ -284,59 +281,59 @@ spec: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" matchLabelKeys: items: - type: string - type: array + type: "string" + type: "array" mismatchLabelKeys: items: - type: string - type: array + type: "string" + type: "array" namespaceSelector: properties: matchExpressions: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" namespaces: items: - type: string - type: array + type: "string" + type: "array" topologyKey: - type: string - type: object + type: "string" + type: "object" weight: - type: integer - type: object - type: array + type: "integer" + type: "object" + type: "array" requiredDuringSchedulingIgnoredDuringExecution: items: properties: @@ -346,57 +343,57 @@ spec: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" matchLabelKeys: items: - type: string - type: array + type: "string" + type: "array" mismatchLabelKeys: items: - type: string - type: array + type: "string" + type: "array" namespaceSelector: properties: matchExpressions: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" namespaces: items: - type: string - type: array + type: "string" + type: "array" topologyKey: - type: string - type: object - type: array - type: object + type: "string" + type: "object" + type: "array" + type: "object" podAntiAffinity: properties: preferredDuringSchedulingIgnoredDuringExecution: @@ -410,59 +407,59 @@ spec: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" matchLabelKeys: items: - type: string - type: array + type: "string" + type: "array" mismatchLabelKeys: items: - type: string - type: array + type: "string" + type: "array" namespaceSelector: properties: matchExpressions: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" namespaces: items: - type: string - type: array + type: "string" + type: "array" topologyKey: - type: string - type: object + type: "string" + type: "object" weight: - type: integer - type: object - type: array + type: "integer" + type: "object" + type: "array" requiredDuringSchedulingIgnoredDuringExecution: items: properties: @@ -472,145 +469,145 @@ spec: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" matchLabelKeys: items: - type: string - type: array + type: "string" + type: "array" mismatchLabelKeys: items: - type: string - type: array + type: "string" + type: "array" namespaceSelector: properties: matchExpressions: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" namespaces: items: - type: string - type: array + type: "string" + type: "array" topologyKey: - type: string - type: object - type: array - type: object - type: object + type: "string" + type: "object" + type: "array" + type: "object" + type: "object" automountServiceAccountToken: - type: boolean + type: "boolean" containers: items: properties: args: items: - type: string - type: array + type: "string" + type: "array" command: items: - type: string - type: array + type: "string" + type: "array" env: items: properties: name: - type: string + type: "string" value: - type: string + type: "string" valueFrom: properties: configMapKeyRef: properties: key: - type: string + type: "string" name: - type: string + type: "string" optional: - type: boolean - type: object + type: "boolean" + type: "object" fieldRef: properties: apiVersion: - type: string + type: "string" fieldPath: - type: string - type: object + type: "string" + type: "object" resourceFieldRef: properties: containerName: - type: string + type: "string" divisor: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true resource: - type: string - type: object + type: "string" + type: "object" secretKeyRef: properties: key: - type: string + type: "string" name: - type: string + type: "string" optional: - type: boolean - type: object - type: object - type: object - type: array + type: "boolean" + type: "object" + type: "object" + type: "object" + type: "array" envFrom: items: properties: configMapRef: properties: name: - type: string + type: "string" optional: - type: boolean - type: object + type: "boolean" + type: "object" prefix: - type: string + type: "string" secretRef: properties: name: - type: string + type: "string" optional: - type: boolean - type: object - type: object - type: array + type: "boolean" + type: "object" + type: "object" + type: "array" image: - type: string + type: "string" imagePullPolicy: - type: string + type: "string" lifecycle: properties: postStart: @@ -619,554 +616,554 @@ spec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" sleep: properties: seconds: - type: integer - type: object + type: "integer" + type: "object" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" preStop: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" sleep: properties: seconds: - type: integer - type: object + type: "integer" + type: "object" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" stopSignal: - type: string - type: object + type: "string" + type: "object" livenessProbe: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" failureThreshold: - type: integer + type: "integer" grpc: properties: port: - type: integer + type: "integer" service: - type: string - type: object + type: "string" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" initialDelaySeconds: - type: integer + type: "integer" periodSeconds: - type: integer + type: "integer" successThreshold: - type: integer + type: "integer" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" terminationGracePeriodSeconds: - type: integer + type: "integer" timeoutSeconds: - type: integer - type: object + type: "integer" + type: "object" name: - type: string + type: "string" ports: items: properties: containerPort: - type: integer + type: "integer" hostIP: - type: string + type: "string" hostPort: - type: integer + type: "integer" name: - type: string + type: "string" protocol: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" readinessProbe: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" failureThreshold: - type: integer + type: "integer" grpc: properties: port: - type: integer + type: "integer" service: - type: string - type: object + type: "string" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" initialDelaySeconds: - type: integer + type: "integer" periodSeconds: - type: integer + type: "integer" successThreshold: - type: integer + type: "integer" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" terminationGracePeriodSeconds: - type: integer + type: "integer" timeoutSeconds: - type: integer - type: object + type: "integer" + type: "object" resizePolicy: items: properties: resourceName: - type: string + type: "string" restartPolicy: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" resources: properties: claims: items: properties: name: - type: string + type: "string" request: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" limits: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" requests: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" restartPolicy: - type: string + type: "string" securityContext: properties: allowPrivilegeEscalation: - type: boolean + type: "boolean" appArmorProfile: properties: localhostProfile: - type: string + type: "string" type: - type: string - type: object + type: "string" + type: "object" capabilities: properties: add: items: - type: string - type: array + type: "string" + type: "array" drop: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" privileged: - type: boolean + type: "boolean" procMount: - type: string + type: "string" readOnlyRootFilesystem: - type: boolean + type: "boolean" runAsGroup: - type: integer + type: "integer" runAsNonRoot: - type: boolean + type: "boolean" runAsUser: - type: integer + type: "integer" seLinuxOptions: properties: level: - type: string + type: "string" role: - type: string + type: "string" type: - type: string + type: "string" user: - type: string - type: object + type: "string" + type: "object" seccompProfile: properties: localhostProfile: - type: string + type: "string" type: - type: string - type: object + type: "string" + type: "object" windowsOptions: properties: gmsaCredentialSpec: - type: string + type: "string" gmsaCredentialSpecName: - type: string + type: "string" hostProcess: - type: boolean + type: "boolean" runAsUserName: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" startupProbe: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" failureThreshold: - type: integer + type: "integer" grpc: properties: port: - type: integer + type: "integer" service: - type: string - type: object + type: "string" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" initialDelaySeconds: - type: integer + type: "integer" periodSeconds: - type: integer + type: "integer" successThreshold: - type: integer + type: "integer" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" terminationGracePeriodSeconds: - type: integer + type: "integer" timeoutSeconds: - type: integer - type: object + type: "integer" + type: "object" stdin: - type: boolean + type: "boolean" stdinOnce: - type: boolean + type: "boolean" terminationMessagePath: - type: string + type: "string" terminationMessagePolicy: - type: string + type: "string" tty: - type: boolean + type: "boolean" volumeDevices: items: properties: devicePath: - type: string + type: "string" name: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" volumeMounts: items: properties: mountPath: - type: string + type: "string" mountPropagation: - type: string + type: "string" name: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" recursiveReadOnly: - type: string + type: "string" subPath: - type: string + type: "string" subPathExpr: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" workingDir: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" dnsConfig: properties: nameservers: items: - type: string - type: array + type: "string" + type: "array" options: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" searches: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" dnsPolicy: - type: string + type: "string" enableServiceLinks: - type: boolean + type: "boolean" ephemeralContainers: items: properties: args: items: - type: string - type: array + type: "string" + type: "array" command: items: - type: string - type: array + type: "string" + type: "array" env: items: properties: name: - type: string + type: "string" value: - type: string + type: "string" valueFrom: properties: configMapKeyRef: properties: key: - type: string + type: "string" name: - type: string + type: "string" optional: - type: boolean - type: object + type: "boolean" + type: "object" fieldRef: properties: apiVersion: - type: string + type: "string" fieldPath: - type: string - type: object + type: "string" + type: "object" resourceFieldRef: properties: containerName: - type: string + type: "string" divisor: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true resource: - type: string - type: object + type: "string" + type: "object" secretKeyRef: properties: key: - type: string + type: "string" name: - type: string + type: "string" optional: - type: boolean - type: object - type: object - type: object - type: array + type: "boolean" + type: "object" + type: "object" + type: "object" + type: "array" envFrom: items: properties: configMapRef: properties: name: - type: string + type: "string" optional: - type: boolean - type: object + type: "boolean" + type: "object" prefix: - type: string + type: "string" secretRef: properties: name: - type: string + type: "string" optional: - type: boolean - type: object - type: object - type: array + type: "boolean" + type: "object" + type: "object" + type: "array" image: - type: string + type: "string" imagePullPolicy: - type: string + type: "string" lifecycle: properties: postStart: @@ -1175,560 +1172,560 @@ spec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" sleep: properties: seconds: - type: integer - type: object + type: "integer" + type: "object" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" preStop: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" sleep: properties: seconds: - type: integer - type: object + type: "integer" + type: "object" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" stopSignal: - type: string - type: object + type: "string" + type: "object" livenessProbe: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" failureThreshold: - type: integer + type: "integer" grpc: properties: port: - type: integer + type: "integer" service: - type: string - type: object + type: "string" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" initialDelaySeconds: - type: integer + type: "integer" periodSeconds: - type: integer + type: "integer" successThreshold: - type: integer + type: "integer" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" terminationGracePeriodSeconds: - type: integer + type: "integer" timeoutSeconds: - type: integer - type: object + type: "integer" + type: "object" name: - type: string + type: "string" ports: items: properties: containerPort: - type: integer + type: "integer" hostIP: - type: string + type: "string" hostPort: - type: integer + type: "integer" name: - type: string + type: "string" protocol: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" readinessProbe: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" failureThreshold: - type: integer + type: "integer" grpc: properties: port: - type: integer + type: "integer" service: - type: string - type: object + type: "string" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" initialDelaySeconds: - type: integer + type: "integer" periodSeconds: - type: integer + type: "integer" successThreshold: - type: integer + type: "integer" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" terminationGracePeriodSeconds: - type: integer + type: "integer" timeoutSeconds: - type: integer - type: object + type: "integer" + type: "object" resizePolicy: items: properties: resourceName: - type: string + type: "string" restartPolicy: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" resources: properties: claims: items: properties: name: - type: string + type: "string" request: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" limits: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" requests: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" restartPolicy: - type: string + type: "string" securityContext: properties: allowPrivilegeEscalation: - type: boolean + type: "boolean" appArmorProfile: properties: localhostProfile: - type: string + type: "string" type: - type: string - type: object + type: "string" + type: "object" capabilities: properties: add: items: - type: string - type: array + type: "string" + type: "array" drop: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" privileged: - type: boolean + type: "boolean" procMount: - type: string + type: "string" readOnlyRootFilesystem: - type: boolean + type: "boolean" runAsGroup: - type: integer + type: "integer" runAsNonRoot: - type: boolean + type: "boolean" runAsUser: - type: integer + type: "integer" seLinuxOptions: properties: level: - type: string + type: "string" role: - type: string + type: "string" type: - type: string + type: "string" user: - type: string - type: object + type: "string" + type: "object" seccompProfile: properties: localhostProfile: - type: string + type: "string" type: - type: string - type: object + type: "string" + type: "object" windowsOptions: properties: gmsaCredentialSpec: - type: string + type: "string" gmsaCredentialSpecName: - type: string + type: "string" hostProcess: - type: boolean + type: "boolean" runAsUserName: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" startupProbe: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" failureThreshold: - type: integer + type: "integer" grpc: properties: port: - type: integer + type: "integer" service: - type: string - type: object + type: "string" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" initialDelaySeconds: - type: integer + type: "integer" periodSeconds: - type: integer + type: "integer" successThreshold: - type: integer + type: "integer" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" terminationGracePeriodSeconds: - type: integer + type: "integer" timeoutSeconds: - type: integer - type: object + type: "integer" + type: "object" stdin: - type: boolean + type: "boolean" stdinOnce: - type: boolean + type: "boolean" targetContainerName: - type: string + type: "string" terminationMessagePath: - type: string + type: "string" terminationMessagePolicy: - type: string + type: "string" tty: - type: boolean + type: "boolean" volumeDevices: items: properties: devicePath: - type: string + type: "string" name: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" volumeMounts: items: properties: mountPath: - type: string + type: "string" mountPropagation: - type: string + type: "string" name: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" recursiveReadOnly: - type: string + type: "string" subPath: - type: string + type: "string" subPathExpr: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" workingDir: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" hostAliases: items: properties: hostnames: items: - type: string - type: array + type: "string" + type: "array" ip: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" hostIPC: - type: boolean + type: "boolean" hostNetwork: - type: boolean + type: "boolean" hostPID: - type: boolean + type: "boolean" hostUsers: - type: boolean + type: "boolean" hostname: - type: string + type: "string" imagePullSecrets: items: properties: name: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" initContainers: items: properties: args: items: - type: string - type: array + type: "string" + type: "array" command: items: - type: string - type: array + type: "string" + type: "array" env: items: properties: name: - type: string + type: "string" value: - type: string + type: "string" valueFrom: properties: configMapKeyRef: properties: key: - type: string + type: "string" name: - type: string + type: "string" optional: - type: boolean - type: object + type: "boolean" + type: "object" fieldRef: properties: apiVersion: - type: string + type: "string" fieldPath: - type: string - type: object + type: "string" + type: "object" resourceFieldRef: properties: containerName: - type: string + type: "string" divisor: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true resource: - type: string - type: object + type: "string" + type: "object" secretKeyRef: properties: key: - type: string + type: "string" name: - type: string + type: "string" optional: - type: boolean - type: object - type: object - type: object - type: array + type: "boolean" + type: "object" + type: "object" + type: "object" + type: "array" envFrom: items: properties: configMapRef: properties: name: - type: string + type: "string" optional: - type: boolean - type: object + type: "boolean" + type: "object" prefix: - type: string + type: "string" secretRef: properties: name: - type: string + type: "string" optional: - type: boolean - type: object - type: object - type: array + type: "boolean" + type: "object" + type: "object" + type: "array" image: - type: string + type: "string" imagePullPolicy: - type: string + type: "string" lifecycle: properties: postStart: @@ -1737,619 +1734,619 @@ spec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" sleep: properties: seconds: - type: integer - type: object + type: "integer" + type: "object" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" preStop: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" sleep: properties: seconds: - type: integer - type: object + type: "integer" + type: "object" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" stopSignal: - type: string - type: object + type: "string" + type: "object" livenessProbe: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" failureThreshold: - type: integer + type: "integer" grpc: properties: port: - type: integer + type: "integer" service: - type: string - type: object + type: "string" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" initialDelaySeconds: - type: integer + type: "integer" periodSeconds: - type: integer + type: "integer" successThreshold: - type: integer + type: "integer" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" terminationGracePeriodSeconds: - type: integer + type: "integer" timeoutSeconds: - type: integer - type: object + type: "integer" + type: "object" name: - type: string + type: "string" ports: items: properties: containerPort: - type: integer + type: "integer" hostIP: - type: string + type: "string" hostPort: - type: integer + type: "integer" name: - type: string + type: "string" protocol: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" readinessProbe: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" failureThreshold: - type: integer + type: "integer" grpc: properties: port: - type: integer + type: "integer" service: - type: string - type: object + type: "string" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" initialDelaySeconds: - type: integer + type: "integer" periodSeconds: - type: integer + type: "integer" successThreshold: - type: integer + type: "integer" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" terminationGracePeriodSeconds: - type: integer + type: "integer" timeoutSeconds: - type: integer - type: object + type: "integer" + type: "object" resizePolicy: items: properties: resourceName: - type: string + type: "string" restartPolicy: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" resources: properties: claims: items: properties: name: - type: string + type: "string" request: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" limits: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" requests: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" restartPolicy: - type: string + type: "string" securityContext: properties: allowPrivilegeEscalation: - type: boolean + type: "boolean" appArmorProfile: properties: localhostProfile: - type: string + type: "string" type: - type: string - type: object + type: "string" + type: "object" capabilities: properties: add: items: - type: string - type: array + type: "string" + type: "array" drop: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" privileged: - type: boolean + type: "boolean" procMount: - type: string + type: "string" readOnlyRootFilesystem: - type: boolean + type: "boolean" runAsGroup: - type: integer + type: "integer" runAsNonRoot: - type: boolean + type: "boolean" runAsUser: - type: integer + type: "integer" seLinuxOptions: properties: level: - type: string + type: "string" role: - type: string + type: "string" type: - type: string + type: "string" user: - type: string - type: object + type: "string" + type: "object" seccompProfile: properties: localhostProfile: - type: string + type: "string" type: - type: string - type: object + type: "string" + type: "object" windowsOptions: properties: gmsaCredentialSpec: - type: string + type: "string" gmsaCredentialSpecName: - type: string + type: "string" hostProcess: - type: boolean + type: "boolean" runAsUserName: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" startupProbe: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" failureThreshold: - type: integer + type: "integer" grpc: properties: port: - type: integer + type: "integer" service: - type: string - type: object + type: "string" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" initialDelaySeconds: - type: integer + type: "integer" periodSeconds: - type: integer + type: "integer" successThreshold: - type: integer + type: "integer" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" terminationGracePeriodSeconds: - type: integer + type: "integer" timeoutSeconds: - type: integer - type: object + type: "integer" + type: "object" stdin: - type: boolean + type: "boolean" stdinOnce: - type: boolean + type: "boolean" terminationMessagePath: - type: string + type: "string" terminationMessagePolicy: - type: string + type: "string" tty: - type: boolean + type: "boolean" volumeDevices: items: properties: devicePath: - type: string + type: "string" name: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" volumeMounts: items: properties: mountPath: - type: string + type: "string" mountPropagation: - type: string + type: "string" name: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" recursiveReadOnly: - type: string + type: "string" subPath: - type: string + type: "string" subPathExpr: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" workingDir: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" nodeName: - type: string + type: "string" nodeSelector: additionalProperties: - type: string - type: object + type: "string" + type: "object" os: properties: name: - type: string - type: object + type: "string" + type: "object" overhead: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" preemptionPolicy: - type: string + type: "string" priority: - type: integer + type: "integer" priorityClassName: - type: string + type: "string" readinessGates: items: properties: conditionType: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" resourceClaims: items: properties: name: - type: string + type: "string" resourceClaimName: - type: string + type: "string" resourceClaimTemplateName: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" resources: properties: claims: items: properties: name: - type: string + type: "string" request: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" limits: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" requests: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" restartPolicy: - type: string + type: "string" runtimeClassName: - type: string + type: "string" schedulerName: - type: string + type: "string" schedulingGates: items: properties: name: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" securityContext: properties: appArmorProfile: properties: localhostProfile: - type: string + type: "string" type: - type: string - type: object + type: "string" + type: "object" fsGroup: - type: integer + type: "integer" fsGroupChangePolicy: - type: string + type: "string" runAsGroup: - type: integer + type: "integer" runAsNonRoot: - type: boolean + type: "boolean" runAsUser: - type: integer + type: "integer" seLinuxChangePolicy: - type: string + type: "string" seLinuxOptions: properties: level: - type: string + type: "string" role: - type: string + type: "string" type: - type: string + type: "string" user: - type: string - type: object + type: "string" + type: "object" seccompProfile: properties: localhostProfile: - type: string + type: "string" type: - type: string - type: object + type: "string" + type: "object" supplementalGroups: items: - type: integer - type: array + type: "integer" + type: "array" supplementalGroupsPolicy: - type: string + type: "string" sysctls: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" windowsOptions: properties: gmsaCredentialSpec: - type: string + type: "string" gmsaCredentialSpecName: - type: string + type: "string" hostProcess: - type: boolean + type: "boolean" runAsUserName: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" serviceAccount: - type: string + type: "string" serviceAccountName: - type: string + type: "string" setHostnameAsFQDN: - type: boolean + type: "boolean" shareProcessNamespace: - type: boolean + type: "boolean" subdomain: - type: string + type: "string" terminationGracePeriodSeconds: - type: integer + type: "integer" tolerations: items: properties: effect: - type: string + type: "string" key: - type: string + type: "string" operator: - type: string + type: "string" tolerationSeconds: - type: integer + type: "integer" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" topologySpreadConstraints: items: properties: @@ -2359,191 +2356,191 @@ spec: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" matchLabelKeys: items: - type: string - type: array + type: "string" + type: "array" maxSkew: - type: integer + type: "integer" minDomains: - type: integer + type: "integer" nodeAffinityPolicy: - type: string + type: "string" nodeTaintsPolicy: - type: string + type: "string" topologyKey: - type: string + type: "string" whenUnsatisfiable: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" volumes: items: properties: awsElasticBlockStore: properties: fsType: - type: string + type: "string" partition: - type: integer + type: "integer" readOnly: - type: boolean + type: "boolean" volumeID: - type: string - type: object + type: "string" + type: "object" azureDisk: properties: cachingMode: - type: string + type: "string" diskName: - type: string + type: "string" diskURI: - type: string + type: "string" fsType: - type: string + type: "string" kind: - type: string + type: "string" readOnly: - type: boolean - type: object + type: "boolean" + type: "object" azureFile: properties: readOnly: - type: boolean + type: "boolean" secretName: - type: string + type: "string" shareName: - type: string - type: object + type: "string" + type: "object" cephfs: properties: monitors: items: - type: string - type: array + type: "string" + type: "array" path: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" secretFile: - type: string + type: "string" secretRef: properties: name: - type: string - type: object + type: "string" + type: "object" user: - type: string - type: object + type: "string" + type: "object" cinder: properties: fsType: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" secretRef: properties: name: - type: string - type: object + type: "string" + type: "object" volumeID: - type: string - type: object + type: "string" + type: "object" configMap: properties: defaultMode: - type: integer + type: "integer" items: items: properties: key: - type: string + type: "string" mode: - type: integer + type: "integer" path: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" name: - type: string + type: "string" optional: - type: boolean - type: object + type: "boolean" + type: "object" csi: properties: driver: - type: string + type: "string" fsType: - type: string + type: "string" nodePublishSecretRef: properties: name: - type: string - type: object + type: "string" + type: "object" readOnly: - type: boolean + type: "boolean" volumeAttributes: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" downwardAPI: properties: defaultMode: - type: integer + type: "integer" items: items: properties: fieldRef: properties: apiVersion: - type: string + type: "string" fieldPath: - type: string - type: object + type: "string" + type: "object" mode: - type: integer + type: "integer" path: - type: string + type: "string" resourceFieldRef: properties: containerName: - type: string + type: "string" divisor: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true resource: - type: string - type: object - type: object - type: array - type: object + type: "string" + type: "object" + type: "object" + type: "array" + type: "object" emptyDir: properties: medium: - type: string + type: "string" sizeLimit: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" ephemeral: properties: volumeClaimTemplate: @@ -2552,300 +2549,300 @@ spec: properties: annotations: additionalProperties: - type: string - type: object + type: "string" + type: "object" creationTimestamp: - type: string + type: "string" deletionGracePeriodSeconds: - type: integer + type: "integer" deletionTimestamp: - type: string + type: "string" finalizers: items: - type: string - type: array + type: "string" + type: "array" generateName: - type: string + type: "string" generation: - type: integer + type: "integer" labels: additionalProperties: - type: string - type: object + type: "string" + type: "object" managedFields: items: properties: apiVersion: - type: string + type: "string" fieldsType: - type: string + type: "string" fieldsV1: - type: object + type: "object" manager: - type: string + type: "string" operation: - type: string + type: "string" subresource: - type: string + type: "string" time: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" name: - type: string + type: "string" namespace: - type: string + type: "string" ownerReferences: items: properties: apiVersion: - type: string + type: "string" blockOwnerDeletion: - type: boolean + type: "boolean" controller: - type: boolean + type: "boolean" kind: - type: string + type: "string" name: - type: string + type: "string" uid: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" resourceVersion: - type: string + type: "string" selfLink: - type: string + type: "string" uid: - type: string - type: object + type: "string" + type: "object" spec: properties: accessModes: items: - type: string - type: array + type: "string" + type: "array" dataSource: properties: apiGroup: - type: string + type: "string" kind: - type: string + type: "string" name: - type: string - type: object + type: "string" + type: "object" dataSourceRef: properties: apiGroup: - type: string + type: "string" kind: - type: string + type: "string" name: - type: string + type: "string" namespace: - type: string - type: object + type: "string" + type: "object" resources: properties: limits: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" requests: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" selector: properties: matchExpressions: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" storageClassName: - type: string + type: "string" volumeAttributesClassName: - type: string + type: "string" volumeMode: - type: string + type: "string" volumeName: - type: string - type: object - type: object - type: object + type: "string" + type: "object" + type: "object" + type: "object" fc: properties: fsType: - type: string + type: "string" lun: - type: integer + type: "integer" readOnly: - type: boolean + type: "boolean" targetWWNs: items: - type: string - type: array + type: "string" + type: "array" wwids: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" flexVolume: properties: driver: - type: string + type: "string" fsType: - type: string + type: "string" options: additionalProperties: - type: string - type: object + type: "string" + type: "object" readOnly: - type: boolean + type: "boolean" secretRef: properties: name: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" flocker: properties: datasetName: - type: string + type: "string" datasetUUID: - type: string - type: object + type: "string" + type: "object" gcePersistentDisk: properties: fsType: - type: string + type: "string" partition: - type: integer + type: "integer" pdName: - type: string + type: "string" readOnly: - type: boolean - type: object + type: "boolean" + type: "object" gitRepo: properties: directory: - type: string + type: "string" repository: - type: string + type: "string" revision: - type: string - type: object + type: "string" + type: "object" glusterfs: properties: endpoints: - type: string + type: "string" path: - type: string + type: "string" readOnly: - type: boolean - type: object + type: "boolean" + type: "object" hostPath: properties: path: - type: string + type: "string" type: - type: string - type: object + type: "string" + type: "object" image: properties: pullPolicy: - type: string + type: "string" reference: - type: string - type: object + type: "string" + type: "object" iscsi: properties: chapAuthDiscovery: - type: boolean + type: "boolean" chapAuthSession: - type: boolean + type: "boolean" fsType: - type: string + type: "string" initiatorName: - type: string + type: "string" iqn: - type: string + type: "string" iscsiInterface: - type: string + type: "string" lun: - type: integer + type: "integer" portals: items: - type: string - type: array + type: "string" + type: "array" readOnly: - type: boolean + type: "boolean" secretRef: properties: name: - type: string - type: object + type: "string" + type: "object" targetPortal: - type: string - type: object + type: "string" + type: "object" name: - type: string + type: "string" nfs: properties: path: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" server: - type: string - type: object + type: "string" + type: "object" persistentVolumeClaim: properties: claimName: - type: string + type: "string" readOnly: - type: boolean - type: object + type: "boolean" + type: "object" photonPersistentDisk: properties: fsType: - type: string + type: "string" pdID: - type: string - type: object + type: "string" + type: "object" portworxVolume: properties: fsType: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" volumeID: - type: string - type: object + type: "string" + type: "object" projected: properties: defaultMode: - type: integer + type: "integer" sources: items: properties: @@ -2857,47 +2854,47 @@ spec: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" name: - type: string + type: "string" optional: - type: boolean + type: "boolean" path: - type: string + type: "string" signerName: - type: string - type: object + type: "string" + type: "object" configMap: properties: items: items: properties: key: - type: string + type: "string" mode: - type: integer + type: "integer" path: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" name: - type: string + type: "string" optional: - type: boolean - type: object + type: "boolean" + type: "object" downwardAPI: properties: items: @@ -2906,825 +2903,825 @@ spec: fieldRef: properties: apiVersion: - type: string + type: "string" fieldPath: - type: string - type: object + type: "string" + type: "object" mode: - type: integer + type: "integer" path: - type: string + type: "string" resourceFieldRef: properties: containerName: - type: string + type: "string" divisor: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true resource: - type: string - type: object - type: object - type: array - type: object + type: "string" + type: "object" + type: "object" + type: "array" + type: "object" secret: properties: items: items: properties: key: - type: string + type: "string" mode: - type: integer + type: "integer" path: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" name: - type: string + type: "string" optional: - type: boolean - type: object + type: "boolean" + type: "object" serviceAccountToken: properties: audience: - type: string + type: "string" expirationSeconds: - type: integer + type: "integer" path: - type: string - type: object - type: object - type: array - type: object + type: "string" + type: "object" + type: "object" + type: "array" + type: "object" quobyte: properties: group: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" registry: - type: string + type: "string" tenant: - type: string + type: "string" user: - type: string + type: "string" volume: - type: string - type: object + type: "string" + type: "object" rbd: properties: fsType: - type: string + type: "string" image: - type: string + type: "string" keyring: - type: string + type: "string" monitors: items: - type: string - type: array + type: "string" + type: "array" pool: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" secretRef: properties: name: - type: string - type: object + type: "string" + type: "object" user: - type: string - type: object + type: "string" + type: "object" scaleIO: properties: fsType: - type: string + type: "string" gateway: - type: string + type: "string" protectionDomain: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" secretRef: properties: name: - type: string - type: object + type: "string" + type: "object" sslEnabled: - type: boolean + type: "boolean" storageMode: - type: string + type: "string" storagePool: - type: string + type: "string" system: - type: string + type: "string" volumeName: - type: string - type: object + type: "string" + type: "object" secret: properties: defaultMode: - type: integer + type: "integer" items: items: properties: key: - type: string + type: "string" mode: - type: integer + type: "integer" path: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" optional: - type: boolean + type: "boolean" secretName: - type: string - type: object + type: "string" + type: "object" storageos: properties: fsType: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" secretRef: properties: name: - type: string - type: object + type: "string" + type: "object" volumeName: - type: string + type: "string" volumeNamespace: - type: string - type: object + type: "string" + type: "object" vsphereVolume: properties: fsType: - type: string + type: "string" storagePolicyID: - type: string + type: "string" storagePolicyName: - type: string + type: "string" volumePath: - type: string - type: object - type: object - type: array - type: object + type: "string" + type: "object" + type: "object" + type: "array" + type: "object" status: properties: conditions: items: properties: lastProbeTime: - type: string + type: "string" lastTransitionTime: - type: string + type: "string" message: - type: string + type: "string" observedGeneration: - type: integer + type: "integer" reason: - type: string + type: "string" status: - type: string + type: "string" type: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" containerStatuses: items: properties: allocatedResources: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" allocatedResourcesStatus: items: properties: name: - type: string + type: "string" resources: items: properties: health: - type: string + type: "string" resourceID: - type: string - type: object - type: array - type: object - type: array + type: "string" + type: "object" + type: "array" + type: "object" + type: "array" containerID: - type: string + type: "string" image: - type: string + type: "string" imageID: - type: string + type: "string" lastState: properties: running: properties: startedAt: - type: string - type: object + type: "string" + type: "object" terminated: properties: containerID: - type: string + type: "string" exitCode: - type: integer + type: "integer" finishedAt: - type: string + type: "string" message: - type: string + type: "string" reason: - type: string + type: "string" signal: - type: integer + type: "integer" startedAt: - type: string - type: object + type: "string" + type: "object" waiting: properties: message: - type: string + type: "string" reason: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" name: - type: string + type: "string" ready: - type: boolean + type: "boolean" resources: properties: claims: items: properties: name: - type: string + type: "string" request: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" limits: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" requests: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" restartCount: - type: integer + type: "integer" started: - type: boolean + type: "boolean" state: properties: running: properties: startedAt: - type: string - type: object + type: "string" + type: "object" terminated: properties: containerID: - type: string + type: "string" exitCode: - type: integer + type: "integer" finishedAt: - type: string + type: "string" message: - type: string + type: "string" reason: - type: string + type: "string" signal: - type: integer + type: "integer" startedAt: - type: string - type: object + type: "string" + type: "object" waiting: properties: message: - type: string + type: "string" reason: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" stopSignal: - type: string + type: "string" user: properties: linux: properties: gid: - type: integer + type: "integer" supplementalGroups: items: - type: integer - type: array + type: "integer" + type: "array" uid: - type: integer - type: object - type: object + type: "integer" + type: "object" + type: "object" volumeMounts: items: properties: mountPath: - type: string + type: "string" name: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" recursiveReadOnly: - type: string - type: object - type: array - type: object - type: array + type: "string" + type: "object" + type: "array" + type: "object" + type: "array" ephemeralContainerStatuses: items: properties: allocatedResources: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" allocatedResourcesStatus: items: properties: name: - type: string + type: "string" resources: items: properties: health: - type: string + type: "string" resourceID: - type: string - type: object - type: array - type: object - type: array + type: "string" + type: "object" + type: "array" + type: "object" + type: "array" containerID: - type: string + type: "string" image: - type: string + type: "string" imageID: - type: string + type: "string" lastState: properties: running: properties: startedAt: - type: string - type: object + type: "string" + type: "object" terminated: properties: containerID: - type: string + type: "string" exitCode: - type: integer + type: "integer" finishedAt: - type: string + type: "string" message: - type: string + type: "string" reason: - type: string + type: "string" signal: - type: integer + type: "integer" startedAt: - type: string - type: object + type: "string" + type: "object" waiting: properties: message: - type: string + type: "string" reason: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" name: - type: string + type: "string" ready: - type: boolean + type: "boolean" resources: properties: claims: items: properties: name: - type: string + type: "string" request: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" limits: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" requests: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" restartCount: - type: integer + type: "integer" started: - type: boolean + type: "boolean" state: properties: running: properties: startedAt: - type: string - type: object + type: "string" + type: "object" terminated: properties: containerID: - type: string + type: "string" exitCode: - type: integer + type: "integer" finishedAt: - type: string + type: "string" message: - type: string + type: "string" reason: - type: string + type: "string" signal: - type: integer + type: "integer" startedAt: - type: string - type: object + type: "string" + type: "object" waiting: properties: message: - type: string + type: "string" reason: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" stopSignal: - type: string + type: "string" user: properties: linux: properties: gid: - type: integer + type: "integer" supplementalGroups: items: - type: integer - type: array + type: "integer" + type: "array" uid: - type: integer - type: object - type: object + type: "integer" + type: "object" + type: "object" volumeMounts: items: properties: mountPath: - type: string + type: "string" name: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" recursiveReadOnly: - type: string - type: object - type: array - type: object - type: array + type: "string" + type: "object" + type: "array" + type: "object" + type: "array" hostIP: - type: string + type: "string" hostIPs: items: properties: ip: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" initContainerStatuses: items: properties: allocatedResources: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" allocatedResourcesStatus: items: properties: name: - type: string + type: "string" resources: items: properties: health: - type: string + type: "string" resourceID: - type: string - type: object - type: array - type: object - type: array + type: "string" + type: "object" + type: "array" + type: "object" + type: "array" containerID: - type: string + type: "string" image: - type: string + type: "string" imageID: - type: string + type: "string" lastState: properties: running: properties: startedAt: - type: string - type: object + type: "string" + type: "object" terminated: properties: containerID: - type: string + type: "string" exitCode: - type: integer + type: "integer" finishedAt: - type: string + type: "string" message: - type: string + type: "string" reason: - type: string + type: "string" signal: - type: integer + type: "integer" startedAt: - type: string - type: object + type: "string" + type: "object" waiting: properties: message: - type: string + type: "string" reason: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" name: - type: string + type: "string" ready: - type: boolean + type: "boolean" resources: properties: claims: items: properties: name: - type: string + type: "string" request: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" limits: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" requests: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" restartCount: - type: integer + type: "integer" started: - type: boolean + type: "boolean" state: properties: running: properties: startedAt: - type: string - type: object + type: "string" + type: "object" terminated: properties: containerID: - type: string + type: "string" exitCode: - type: integer + type: "integer" finishedAt: - type: string + type: "string" message: - type: string + type: "string" reason: - type: string + type: "string" signal: - type: integer + type: "integer" startedAt: - type: string - type: object + type: "string" + type: "object" waiting: properties: message: - type: string + type: "string" reason: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" stopSignal: - type: string + type: "string" user: properties: linux: properties: gid: - type: integer + type: "integer" supplementalGroups: items: - type: integer - type: array + type: "integer" + type: "array" uid: - type: integer - type: object - type: object + type: "integer" + type: "object" + type: "object" volumeMounts: items: properties: mountPath: - type: string + type: "string" name: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" recursiveReadOnly: - type: string - type: object - type: array - type: object - type: array + type: "string" + type: "object" + type: "array" + type: "object" + type: "array" message: - type: string + type: "string" nominatedNodeName: - type: string + type: "string" observedGeneration: - type: integer + type: "integer" phase: - type: string + type: "string" podIP: - type: string + type: "string" podIPs: items: properties: ip: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" qosClass: - type: string + type: "string" reason: - type: string + type: "string" resize: - type: string + type: "string" resourceClaimStatuses: items: properties: name: - type: string + type: "string" resourceClaimName: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" startTime: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" replicas: - type: integer + type: "integer" resource: properties: cpu: - type: number + type: "number" ephemeralStorage: - type: string + type: "string" memory: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" logConfiguration: additionalProperties: - type: string - type: object + type: "string" + type: "object" mode: enum: - - native - - standalone - type: string + - "native" + - "standalone" + type: "string" podTemplate: properties: apiVersion: - type: string + type: "string" kind: - type: string + type: "string" metadata: properties: annotations: additionalProperties: - type: string - type: object + type: "string" + type: "object" creationTimestamp: - type: string + type: "string" deletionGracePeriodSeconds: - type: integer + type: "integer" deletionTimestamp: - type: string + type: "string" finalizers: items: - type: string - type: array + type: "string" + type: "array" generateName: - type: string + type: "string" generation: - type: integer + type: "integer" labels: additionalProperties: - type: string - type: object + type: "string" + type: "object" managedFields: items: properties: apiVersion: - type: string + type: "string" fieldsType: - type: string + type: "string" fieldsV1: - type: object + type: "object" manager: - type: string + type: "string" operation: - type: string + type: "string" subresource: - type: string + type: "string" time: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" name: - type: string + type: "string" namespace: - type: string + type: "string" ownerReferences: items: properties: apiVersion: - type: string + type: "string" blockOwnerDeletion: - type: boolean + type: "boolean" controller: - type: boolean + type: "boolean" kind: - type: string + type: "string" name: - type: string + type: "string" uid: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" resourceVersion: - type: string + type: "string" selfLink: - type: string + type: "string" uid: - type: string - type: object + type: "string" + type: "object" spec: properties: activeDeadlineSeconds: - type: integer + type: "integer" affinity: properties: nodeAffinity: @@ -3738,33 +3735,33 @@ spec: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchFields: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array - type: object + type: "string" + type: "array" + type: "object" + type: "array" + type: "object" weight: - type: integer - type: object - type: array + type: "integer" + type: "object" + type: "array" requiredDuringSchedulingIgnoredDuringExecution: properties: nodeSelectorTerms: @@ -3774,32 +3771,32 @@ spec: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchFields: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array - type: object - type: array - type: object - type: object + type: "string" + type: "array" + type: "object" + type: "array" + type: "object" + type: "array" + type: "object" + type: "object" podAffinity: properties: preferredDuringSchedulingIgnoredDuringExecution: @@ -3813,59 +3810,59 @@ spec: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" matchLabelKeys: items: - type: string - type: array + type: "string" + type: "array" mismatchLabelKeys: items: - type: string - type: array + type: "string" + type: "array" namespaceSelector: properties: matchExpressions: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" namespaces: items: - type: string - type: array + type: "string" + type: "array" topologyKey: - type: string - type: object + type: "string" + type: "object" weight: - type: integer - type: object - type: array + type: "integer" + type: "object" + type: "array" requiredDuringSchedulingIgnoredDuringExecution: items: properties: @@ -3875,57 +3872,57 @@ spec: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" matchLabelKeys: items: - type: string - type: array + type: "string" + type: "array" mismatchLabelKeys: items: - type: string - type: array + type: "string" + type: "array" namespaceSelector: properties: matchExpressions: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" namespaces: items: - type: string - type: array + type: "string" + type: "array" topologyKey: - type: string - type: object - type: array - type: object + type: "string" + type: "object" + type: "array" + type: "object" podAntiAffinity: properties: preferredDuringSchedulingIgnoredDuringExecution: @@ -3939,59 +3936,59 @@ spec: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" matchLabelKeys: items: - type: string - type: array + type: "string" + type: "array" mismatchLabelKeys: items: - type: string - type: array + type: "string" + type: "array" namespaceSelector: properties: matchExpressions: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" namespaces: items: - type: string - type: array + type: "string" + type: "array" topologyKey: - type: string - type: object + type: "string" + type: "object" weight: - type: integer - type: object - type: array + type: "integer" + type: "object" + type: "array" requiredDuringSchedulingIgnoredDuringExecution: items: properties: @@ -4001,145 +3998,145 @@ spec: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" matchLabelKeys: items: - type: string - type: array + type: "string" + type: "array" mismatchLabelKeys: items: - type: string - type: array + type: "string" + type: "array" namespaceSelector: properties: matchExpressions: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" namespaces: items: - type: string - type: array + type: "string" + type: "array" topologyKey: - type: string - type: object - type: array - type: object - type: object + type: "string" + type: "object" + type: "array" + type: "object" + type: "object" automountServiceAccountToken: - type: boolean + type: "boolean" containers: items: properties: args: items: - type: string - type: array + type: "string" + type: "array" command: items: - type: string - type: array + type: "string" + type: "array" env: items: properties: name: - type: string + type: "string" value: - type: string + type: "string" valueFrom: properties: configMapKeyRef: properties: key: - type: string + type: "string" name: - type: string + type: "string" optional: - type: boolean - type: object + type: "boolean" + type: "object" fieldRef: properties: apiVersion: - type: string + type: "string" fieldPath: - type: string - type: object + type: "string" + type: "object" resourceFieldRef: properties: containerName: - type: string + type: "string" divisor: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true resource: - type: string - type: object + type: "string" + type: "object" secretKeyRef: properties: key: - type: string + type: "string" name: - type: string + type: "string" optional: - type: boolean - type: object - type: object - type: object - type: array + type: "boolean" + type: "object" + type: "object" + type: "object" + type: "array" envFrom: items: properties: configMapRef: properties: name: - type: string + type: "string" optional: - type: boolean - type: object + type: "boolean" + type: "object" prefix: - type: string + type: "string" secretRef: properties: name: - type: string + type: "string" optional: - type: boolean - type: object - type: object - type: array + type: "boolean" + type: "object" + type: "object" + type: "array" image: - type: string + type: "string" imagePullPolicy: - type: string + type: "string" lifecycle: properties: postStart: @@ -4148,554 +4145,554 @@ spec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" sleep: properties: seconds: - type: integer - type: object + type: "integer" + type: "object" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" preStop: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" sleep: properties: seconds: - type: integer - type: object + type: "integer" + type: "object" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" stopSignal: - type: string - type: object + type: "string" + type: "object" livenessProbe: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" failureThreshold: - type: integer + type: "integer" grpc: properties: port: - type: integer + type: "integer" service: - type: string - type: object + type: "string" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" initialDelaySeconds: - type: integer + type: "integer" periodSeconds: - type: integer + type: "integer" successThreshold: - type: integer + type: "integer" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" terminationGracePeriodSeconds: - type: integer + type: "integer" timeoutSeconds: - type: integer - type: object + type: "integer" + type: "object" name: - type: string + type: "string" ports: items: properties: containerPort: - type: integer + type: "integer" hostIP: - type: string + type: "string" hostPort: - type: integer + type: "integer" name: - type: string + type: "string" protocol: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" readinessProbe: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" failureThreshold: - type: integer + type: "integer" grpc: properties: port: - type: integer + type: "integer" service: - type: string - type: object + type: "string" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" initialDelaySeconds: - type: integer + type: "integer" periodSeconds: - type: integer + type: "integer" successThreshold: - type: integer + type: "integer" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" terminationGracePeriodSeconds: - type: integer + type: "integer" timeoutSeconds: - type: integer - type: object + type: "integer" + type: "object" resizePolicy: items: properties: resourceName: - type: string + type: "string" restartPolicy: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" resources: properties: claims: items: properties: name: - type: string + type: "string" request: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" limits: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" requests: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" restartPolicy: - type: string + type: "string" securityContext: properties: allowPrivilegeEscalation: - type: boolean + type: "boolean" appArmorProfile: properties: localhostProfile: - type: string + type: "string" type: - type: string - type: object + type: "string" + type: "object" capabilities: properties: add: items: - type: string - type: array + type: "string" + type: "array" drop: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" privileged: - type: boolean + type: "boolean" procMount: - type: string + type: "string" readOnlyRootFilesystem: - type: boolean + type: "boolean" runAsGroup: - type: integer + type: "integer" runAsNonRoot: - type: boolean + type: "boolean" runAsUser: - type: integer + type: "integer" seLinuxOptions: properties: level: - type: string + type: "string" role: - type: string + type: "string" type: - type: string + type: "string" user: - type: string - type: object + type: "string" + type: "object" seccompProfile: properties: localhostProfile: - type: string + type: "string" type: - type: string - type: object + type: "string" + type: "object" windowsOptions: properties: gmsaCredentialSpec: - type: string + type: "string" gmsaCredentialSpecName: - type: string + type: "string" hostProcess: - type: boolean + type: "boolean" runAsUserName: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" startupProbe: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" failureThreshold: - type: integer + type: "integer" grpc: properties: port: - type: integer + type: "integer" service: - type: string - type: object + type: "string" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" initialDelaySeconds: - type: integer + type: "integer" periodSeconds: - type: integer + type: "integer" successThreshold: - type: integer + type: "integer" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" terminationGracePeriodSeconds: - type: integer + type: "integer" timeoutSeconds: - type: integer - type: object + type: "integer" + type: "object" stdin: - type: boolean + type: "boolean" stdinOnce: - type: boolean + type: "boolean" terminationMessagePath: - type: string + type: "string" terminationMessagePolicy: - type: string + type: "string" tty: - type: boolean + type: "boolean" volumeDevices: items: properties: devicePath: - type: string + type: "string" name: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" volumeMounts: items: properties: mountPath: - type: string + type: "string" mountPropagation: - type: string + type: "string" name: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" recursiveReadOnly: - type: string + type: "string" subPath: - type: string + type: "string" subPathExpr: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" workingDir: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" dnsConfig: properties: nameservers: items: - type: string - type: array + type: "string" + type: "array" options: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" searches: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" dnsPolicy: - type: string + type: "string" enableServiceLinks: - type: boolean + type: "boolean" ephemeralContainers: items: properties: args: items: - type: string - type: array + type: "string" + type: "array" command: items: - type: string - type: array + type: "string" + type: "array" env: items: properties: name: - type: string + type: "string" value: - type: string + type: "string" valueFrom: properties: configMapKeyRef: properties: key: - type: string + type: "string" name: - type: string + type: "string" optional: - type: boolean - type: object + type: "boolean" + type: "object" fieldRef: properties: apiVersion: - type: string + type: "string" fieldPath: - type: string - type: object + type: "string" + type: "object" resourceFieldRef: properties: containerName: - type: string + type: "string" divisor: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true resource: - type: string - type: object + type: "string" + type: "object" secretKeyRef: properties: key: - type: string + type: "string" name: - type: string + type: "string" optional: - type: boolean - type: object - type: object - type: object - type: array + type: "boolean" + type: "object" + type: "object" + type: "object" + type: "array" envFrom: items: properties: configMapRef: properties: name: - type: string + type: "string" optional: - type: boolean - type: object + type: "boolean" + type: "object" prefix: - type: string + type: "string" secretRef: properties: name: - type: string + type: "string" optional: - type: boolean - type: object - type: object - type: array + type: "boolean" + type: "object" + type: "object" + type: "array" image: - type: string + type: "string" imagePullPolicy: - type: string + type: "string" lifecycle: properties: postStart: @@ -4704,560 +4701,560 @@ spec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" sleep: properties: seconds: - type: integer - type: object + type: "integer" + type: "object" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" preStop: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" sleep: properties: seconds: - type: integer - type: object + type: "integer" + type: "object" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" stopSignal: - type: string - type: object + type: "string" + type: "object" livenessProbe: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" failureThreshold: - type: integer + type: "integer" grpc: properties: port: - type: integer + type: "integer" service: - type: string - type: object + type: "string" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" initialDelaySeconds: - type: integer + type: "integer" periodSeconds: - type: integer + type: "integer" successThreshold: - type: integer + type: "integer" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" terminationGracePeriodSeconds: - type: integer + type: "integer" timeoutSeconds: - type: integer - type: object + type: "integer" + type: "object" name: - type: string + type: "string" ports: items: properties: containerPort: - type: integer + type: "integer" hostIP: - type: string + type: "string" hostPort: - type: integer + type: "integer" name: - type: string + type: "string" protocol: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" readinessProbe: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" failureThreshold: - type: integer + type: "integer" grpc: properties: port: - type: integer + type: "integer" service: - type: string - type: object + type: "string" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" initialDelaySeconds: - type: integer + type: "integer" periodSeconds: - type: integer + type: "integer" successThreshold: - type: integer + type: "integer" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" terminationGracePeriodSeconds: - type: integer + type: "integer" timeoutSeconds: - type: integer - type: object + type: "integer" + type: "object" resizePolicy: items: properties: resourceName: - type: string + type: "string" restartPolicy: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" resources: properties: claims: items: properties: name: - type: string + type: "string" request: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" limits: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" requests: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" restartPolicy: - type: string + type: "string" securityContext: properties: allowPrivilegeEscalation: - type: boolean + type: "boolean" appArmorProfile: properties: localhostProfile: - type: string + type: "string" type: - type: string - type: object + type: "string" + type: "object" capabilities: properties: add: items: - type: string - type: array + type: "string" + type: "array" drop: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" privileged: - type: boolean + type: "boolean" procMount: - type: string + type: "string" readOnlyRootFilesystem: - type: boolean + type: "boolean" runAsGroup: - type: integer + type: "integer" runAsNonRoot: - type: boolean + type: "boolean" runAsUser: - type: integer + type: "integer" seLinuxOptions: properties: level: - type: string + type: "string" role: - type: string + type: "string" type: - type: string + type: "string" user: - type: string - type: object + type: "string" + type: "object" seccompProfile: properties: localhostProfile: - type: string + type: "string" type: - type: string - type: object + type: "string" + type: "object" windowsOptions: properties: gmsaCredentialSpec: - type: string + type: "string" gmsaCredentialSpecName: - type: string + type: "string" hostProcess: - type: boolean + type: "boolean" runAsUserName: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" startupProbe: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" failureThreshold: - type: integer + type: "integer" grpc: properties: port: - type: integer + type: "integer" service: - type: string - type: object + type: "string" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" initialDelaySeconds: - type: integer + type: "integer" periodSeconds: - type: integer + type: "integer" successThreshold: - type: integer + type: "integer" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" terminationGracePeriodSeconds: - type: integer + type: "integer" timeoutSeconds: - type: integer - type: object + type: "integer" + type: "object" stdin: - type: boolean + type: "boolean" stdinOnce: - type: boolean + type: "boolean" targetContainerName: - type: string + type: "string" terminationMessagePath: - type: string + type: "string" terminationMessagePolicy: - type: string + type: "string" tty: - type: boolean + type: "boolean" volumeDevices: items: properties: devicePath: - type: string + type: "string" name: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" volumeMounts: items: properties: mountPath: - type: string + type: "string" mountPropagation: - type: string + type: "string" name: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" recursiveReadOnly: - type: string + type: "string" subPath: - type: string + type: "string" subPathExpr: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" workingDir: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" hostAliases: items: properties: hostnames: items: - type: string - type: array + type: "string" + type: "array" ip: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" hostIPC: - type: boolean + type: "boolean" hostNetwork: - type: boolean + type: "boolean" hostPID: - type: boolean + type: "boolean" hostUsers: - type: boolean + type: "boolean" hostname: - type: string + type: "string" imagePullSecrets: items: properties: name: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" initContainers: items: properties: args: items: - type: string - type: array + type: "string" + type: "array" command: items: - type: string - type: array + type: "string" + type: "array" env: items: properties: name: - type: string + type: "string" value: - type: string + type: "string" valueFrom: properties: configMapKeyRef: properties: key: - type: string + type: "string" name: - type: string + type: "string" optional: - type: boolean - type: object + type: "boolean" + type: "object" fieldRef: properties: apiVersion: - type: string + type: "string" fieldPath: - type: string - type: object + type: "string" + type: "object" resourceFieldRef: properties: containerName: - type: string + type: "string" divisor: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true resource: - type: string - type: object + type: "string" + type: "object" secretKeyRef: properties: key: - type: string + type: "string" name: - type: string + type: "string" optional: - type: boolean - type: object - type: object - type: object - type: array + type: "boolean" + type: "object" + type: "object" + type: "object" + type: "array" envFrom: items: properties: configMapRef: properties: name: - type: string + type: "string" optional: - type: boolean - type: object + type: "boolean" + type: "object" prefix: - type: string + type: "string" secretRef: properties: name: - type: string + type: "string" optional: - type: boolean - type: object - type: object - type: array + type: "boolean" + type: "object" + type: "object" + type: "array" image: - type: string + type: "string" imagePullPolicy: - type: string + type: "string" lifecycle: properties: postStart: @@ -5266,619 +5263,619 @@ spec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" sleep: properties: seconds: - type: integer - type: object + type: "integer" + type: "object" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" preStop: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" sleep: properties: seconds: - type: integer - type: object + type: "integer" + type: "object" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" stopSignal: - type: string - type: object + type: "string" + type: "object" livenessProbe: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" failureThreshold: - type: integer + type: "integer" grpc: properties: port: - type: integer + type: "integer" service: - type: string - type: object + type: "string" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" initialDelaySeconds: - type: integer + type: "integer" periodSeconds: - type: integer + type: "integer" successThreshold: - type: integer + type: "integer" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" terminationGracePeriodSeconds: - type: integer + type: "integer" timeoutSeconds: - type: integer - type: object + type: "integer" + type: "object" name: - type: string + type: "string" ports: items: properties: containerPort: - type: integer + type: "integer" hostIP: - type: string + type: "string" hostPort: - type: integer + type: "integer" name: - type: string + type: "string" protocol: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" readinessProbe: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" failureThreshold: - type: integer + type: "integer" grpc: properties: port: - type: integer + type: "integer" service: - type: string - type: object + type: "string" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" initialDelaySeconds: - type: integer + type: "integer" periodSeconds: - type: integer + type: "integer" successThreshold: - type: integer + type: "integer" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" terminationGracePeriodSeconds: - type: integer + type: "integer" timeoutSeconds: - type: integer - type: object + type: "integer" + type: "object" resizePolicy: items: properties: resourceName: - type: string + type: "string" restartPolicy: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" resources: properties: claims: items: properties: name: - type: string + type: "string" request: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" limits: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" requests: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" restartPolicy: - type: string + type: "string" securityContext: properties: allowPrivilegeEscalation: - type: boolean + type: "boolean" appArmorProfile: properties: localhostProfile: - type: string + type: "string" type: - type: string - type: object + type: "string" + type: "object" capabilities: properties: add: items: - type: string - type: array + type: "string" + type: "array" drop: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" privileged: - type: boolean + type: "boolean" procMount: - type: string + type: "string" readOnlyRootFilesystem: - type: boolean + type: "boolean" runAsGroup: - type: integer + type: "integer" runAsNonRoot: - type: boolean + type: "boolean" runAsUser: - type: integer + type: "integer" seLinuxOptions: properties: level: - type: string + type: "string" role: - type: string + type: "string" type: - type: string + type: "string" user: - type: string - type: object + type: "string" + type: "object" seccompProfile: properties: localhostProfile: - type: string + type: "string" type: - type: string - type: object + type: "string" + type: "object" windowsOptions: properties: gmsaCredentialSpec: - type: string + type: "string" gmsaCredentialSpecName: - type: string + type: "string" hostProcess: - type: boolean + type: "boolean" runAsUserName: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" startupProbe: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" failureThreshold: - type: integer + type: "integer" grpc: properties: port: - type: integer + type: "integer" service: - type: string - type: object + type: "string" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" initialDelaySeconds: - type: integer + type: "integer" periodSeconds: - type: integer + type: "integer" successThreshold: - type: integer + type: "integer" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" terminationGracePeriodSeconds: - type: integer + type: "integer" timeoutSeconds: - type: integer - type: object + type: "integer" + type: "object" stdin: - type: boolean + type: "boolean" stdinOnce: - type: boolean + type: "boolean" terminationMessagePath: - type: string + type: "string" terminationMessagePolicy: - type: string + type: "string" tty: - type: boolean + type: "boolean" volumeDevices: items: properties: devicePath: - type: string + type: "string" name: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" volumeMounts: items: properties: mountPath: - type: string + type: "string" mountPropagation: - type: string + type: "string" name: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" recursiveReadOnly: - type: string + type: "string" subPath: - type: string + type: "string" subPathExpr: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" workingDir: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" nodeName: - type: string + type: "string" nodeSelector: additionalProperties: - type: string - type: object + type: "string" + type: "object" os: properties: name: - type: string - type: object + type: "string" + type: "object" overhead: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" preemptionPolicy: - type: string + type: "string" priority: - type: integer + type: "integer" priorityClassName: - type: string + type: "string" readinessGates: items: properties: conditionType: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" resourceClaims: items: properties: name: - type: string + type: "string" resourceClaimName: - type: string + type: "string" resourceClaimTemplateName: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" resources: properties: claims: items: properties: name: - type: string + type: "string" request: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" limits: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" requests: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" restartPolicy: - type: string + type: "string" runtimeClassName: - type: string + type: "string" schedulerName: - type: string + type: "string" schedulingGates: items: properties: name: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" securityContext: properties: appArmorProfile: properties: localhostProfile: - type: string + type: "string" type: - type: string - type: object + type: "string" + type: "object" fsGroup: - type: integer + type: "integer" fsGroupChangePolicy: - type: string + type: "string" runAsGroup: - type: integer + type: "integer" runAsNonRoot: - type: boolean + type: "boolean" runAsUser: - type: integer + type: "integer" seLinuxChangePolicy: - type: string + type: "string" seLinuxOptions: properties: level: - type: string + type: "string" role: - type: string + type: "string" type: - type: string + type: "string" user: - type: string - type: object + type: "string" + type: "object" seccompProfile: properties: localhostProfile: - type: string + type: "string" type: - type: string - type: object + type: "string" + type: "object" supplementalGroups: items: - type: integer - type: array + type: "integer" + type: "array" supplementalGroupsPolicy: - type: string + type: "string" sysctls: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" windowsOptions: properties: gmsaCredentialSpec: - type: string + type: "string" gmsaCredentialSpecName: - type: string + type: "string" hostProcess: - type: boolean + type: "boolean" runAsUserName: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" serviceAccount: - type: string + type: "string" serviceAccountName: - type: string + type: "string" setHostnameAsFQDN: - type: boolean + type: "boolean" shareProcessNamespace: - type: boolean + type: "boolean" subdomain: - type: string + type: "string" terminationGracePeriodSeconds: - type: integer + type: "integer" tolerations: items: properties: effect: - type: string + type: "string" key: - type: string + type: "string" operator: - type: string + type: "string" tolerationSeconds: - type: integer + type: "integer" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" topologySpreadConstraints: items: properties: @@ -5888,191 +5885,191 @@ spec: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" matchLabelKeys: items: - type: string - type: array + type: "string" + type: "array" maxSkew: - type: integer + type: "integer" minDomains: - type: integer + type: "integer" nodeAffinityPolicy: - type: string + type: "string" nodeTaintsPolicy: - type: string + type: "string" topologyKey: - type: string + type: "string" whenUnsatisfiable: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" volumes: items: properties: awsElasticBlockStore: properties: fsType: - type: string + type: "string" partition: - type: integer + type: "integer" readOnly: - type: boolean + type: "boolean" volumeID: - type: string - type: object + type: "string" + type: "object" azureDisk: properties: cachingMode: - type: string + type: "string" diskName: - type: string + type: "string" diskURI: - type: string + type: "string" fsType: - type: string + type: "string" kind: - type: string + type: "string" readOnly: - type: boolean - type: object + type: "boolean" + type: "object" azureFile: properties: readOnly: - type: boolean + type: "boolean" secretName: - type: string + type: "string" shareName: - type: string - type: object + type: "string" + type: "object" cephfs: properties: monitors: items: - type: string - type: array + type: "string" + type: "array" path: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" secretFile: - type: string + type: "string" secretRef: properties: name: - type: string - type: object + type: "string" + type: "object" user: - type: string - type: object + type: "string" + type: "object" cinder: properties: fsType: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" secretRef: properties: name: - type: string - type: object + type: "string" + type: "object" volumeID: - type: string - type: object + type: "string" + type: "object" configMap: properties: defaultMode: - type: integer + type: "integer" items: items: properties: key: - type: string + type: "string" mode: - type: integer + type: "integer" path: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" name: - type: string + type: "string" optional: - type: boolean - type: object + type: "boolean" + type: "object" csi: properties: driver: - type: string + type: "string" fsType: - type: string + type: "string" nodePublishSecretRef: properties: name: - type: string - type: object + type: "string" + type: "object" readOnly: - type: boolean + type: "boolean" volumeAttributes: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" downwardAPI: properties: defaultMode: - type: integer + type: "integer" items: items: properties: fieldRef: properties: apiVersion: - type: string + type: "string" fieldPath: - type: string - type: object + type: "string" + type: "object" mode: - type: integer + type: "integer" path: - type: string + type: "string" resourceFieldRef: properties: containerName: - type: string + type: "string" divisor: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true resource: - type: string - type: object - type: object - type: array - type: object + type: "string" + type: "object" + type: "object" + type: "array" + type: "object" emptyDir: properties: medium: - type: string + type: "string" sizeLimit: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" ephemeral: properties: volumeClaimTemplate: @@ -6081,300 +6078,300 @@ spec: properties: annotations: additionalProperties: - type: string - type: object + type: "string" + type: "object" creationTimestamp: - type: string + type: "string" deletionGracePeriodSeconds: - type: integer + type: "integer" deletionTimestamp: - type: string + type: "string" finalizers: items: - type: string - type: array + type: "string" + type: "array" generateName: - type: string + type: "string" generation: - type: integer + type: "integer" labels: additionalProperties: - type: string - type: object + type: "string" + type: "object" managedFields: items: properties: apiVersion: - type: string + type: "string" fieldsType: - type: string + type: "string" fieldsV1: - type: object + type: "object" manager: - type: string + type: "string" operation: - type: string + type: "string" subresource: - type: string + type: "string" time: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" name: - type: string + type: "string" namespace: - type: string + type: "string" ownerReferences: items: properties: apiVersion: - type: string + type: "string" blockOwnerDeletion: - type: boolean + type: "boolean" controller: - type: boolean + type: "boolean" kind: - type: string + type: "string" name: - type: string + type: "string" uid: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" resourceVersion: - type: string + type: "string" selfLink: - type: string + type: "string" uid: - type: string - type: object + type: "string" + type: "object" spec: properties: accessModes: items: - type: string - type: array + type: "string" + type: "array" dataSource: properties: apiGroup: - type: string + type: "string" kind: - type: string + type: "string" name: - type: string - type: object + type: "string" + type: "object" dataSourceRef: properties: apiGroup: - type: string + type: "string" kind: - type: string + type: "string" name: - type: string + type: "string" namespace: - type: string - type: object + type: "string" + type: "object" resources: properties: limits: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" requests: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" selector: properties: matchExpressions: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" storageClassName: - type: string + type: "string" volumeAttributesClassName: - type: string + type: "string" volumeMode: - type: string + type: "string" volumeName: - type: string - type: object - type: object - type: object + type: "string" + type: "object" + type: "object" + type: "object" fc: properties: fsType: - type: string + type: "string" lun: - type: integer + type: "integer" readOnly: - type: boolean + type: "boolean" targetWWNs: items: - type: string - type: array + type: "string" + type: "array" wwids: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" flexVolume: properties: driver: - type: string + type: "string" fsType: - type: string + type: "string" options: additionalProperties: - type: string - type: object + type: "string" + type: "object" readOnly: - type: boolean + type: "boolean" secretRef: properties: name: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" flocker: properties: datasetName: - type: string + type: "string" datasetUUID: - type: string - type: object + type: "string" + type: "object" gcePersistentDisk: properties: fsType: - type: string + type: "string" partition: - type: integer + type: "integer" pdName: - type: string + type: "string" readOnly: - type: boolean - type: object + type: "boolean" + type: "object" gitRepo: properties: directory: - type: string + type: "string" repository: - type: string + type: "string" revision: - type: string - type: object + type: "string" + type: "object" glusterfs: properties: endpoints: - type: string + type: "string" path: - type: string + type: "string" readOnly: - type: boolean - type: object + type: "boolean" + type: "object" hostPath: properties: path: - type: string + type: "string" type: - type: string - type: object + type: "string" + type: "object" image: properties: pullPolicy: - type: string + type: "string" reference: - type: string - type: object + type: "string" + type: "object" iscsi: properties: chapAuthDiscovery: - type: boolean + type: "boolean" chapAuthSession: - type: boolean + type: "boolean" fsType: - type: string + type: "string" initiatorName: - type: string + type: "string" iqn: - type: string + type: "string" iscsiInterface: - type: string + type: "string" lun: - type: integer + type: "integer" portals: items: - type: string - type: array + type: "string" + type: "array" readOnly: - type: boolean + type: "boolean" secretRef: properties: name: - type: string - type: object + type: "string" + type: "object" targetPortal: - type: string - type: object + type: "string" + type: "object" name: - type: string + type: "string" nfs: properties: path: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" server: - type: string - type: object + type: "string" + type: "object" persistentVolumeClaim: properties: claimName: - type: string + type: "string" readOnly: - type: boolean - type: object + type: "boolean" + type: "object" photonPersistentDisk: properties: fsType: - type: string + type: "string" pdID: - type: string - type: object + type: "string" + type: "object" portworxVolume: properties: fsType: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" volumeID: - type: string - type: object + type: "string" + type: "object" projected: properties: defaultMode: - type: integer + type: "integer" sources: items: properties: @@ -6386,47 +6383,47 @@ spec: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" name: - type: string + type: "string" optional: - type: boolean + type: "boolean" path: - type: string + type: "string" signerName: - type: string - type: object + type: "string" + type: "object" configMap: properties: items: items: properties: key: - type: string + type: "string" mode: - type: integer + type: "integer" path: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" name: - type: string + type: "string" optional: - type: boolean - type: object + type: "boolean" + type: "object" downwardAPI: properties: items: @@ -6435,810 +6432,810 @@ spec: fieldRef: properties: apiVersion: - type: string + type: "string" fieldPath: - type: string - type: object + type: "string" + type: "object" mode: - type: integer + type: "integer" path: - type: string + type: "string" resourceFieldRef: properties: containerName: - type: string + type: "string" divisor: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true resource: - type: string - type: object - type: object - type: array - type: object + type: "string" + type: "object" + type: "object" + type: "array" + type: "object" secret: properties: items: items: properties: key: - type: string + type: "string" mode: - type: integer + type: "integer" path: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" name: - type: string + type: "string" optional: - type: boolean - type: object + type: "boolean" + type: "object" serviceAccountToken: properties: audience: - type: string + type: "string" expirationSeconds: - type: integer + type: "integer" path: - type: string - type: object - type: object - type: array - type: object + type: "string" + type: "object" + type: "object" + type: "array" + type: "object" quobyte: properties: group: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" registry: - type: string + type: "string" tenant: - type: string + type: "string" user: - type: string + type: "string" volume: - type: string - type: object + type: "string" + type: "object" rbd: properties: fsType: - type: string + type: "string" image: - type: string + type: "string" keyring: - type: string + type: "string" monitors: items: - type: string - type: array + type: "string" + type: "array" pool: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" secretRef: properties: name: - type: string - type: object + type: "string" + type: "object" user: - type: string - type: object + type: "string" + type: "object" scaleIO: properties: fsType: - type: string + type: "string" gateway: - type: string + type: "string" protectionDomain: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" secretRef: properties: name: - type: string - type: object + type: "string" + type: "object" sslEnabled: - type: boolean + type: "boolean" storageMode: - type: string + type: "string" storagePool: - type: string + type: "string" system: - type: string + type: "string" volumeName: - type: string - type: object + type: "string" + type: "object" secret: properties: defaultMode: - type: integer + type: "integer" items: items: properties: key: - type: string + type: "string" mode: - type: integer + type: "integer" path: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" optional: - type: boolean + type: "boolean" secretName: - type: string - type: object + type: "string" + type: "object" storageos: properties: fsType: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" secretRef: properties: name: - type: string - type: object + type: "string" + type: "object" volumeName: - type: string + type: "string" volumeNamespace: - type: string - type: object + type: "string" + type: "object" vsphereVolume: properties: fsType: - type: string + type: "string" storagePolicyID: - type: string + type: "string" storagePolicyName: - type: string + type: "string" volumePath: - type: string - type: object - type: object - type: array - type: object + type: "string" + type: "object" + type: "object" + type: "array" + type: "object" status: properties: conditions: items: properties: lastProbeTime: - type: string + type: "string" lastTransitionTime: - type: string + type: "string" message: - type: string + type: "string" observedGeneration: - type: integer + type: "integer" reason: - type: string + type: "string" status: - type: string + type: "string" type: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" containerStatuses: items: properties: allocatedResources: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" allocatedResourcesStatus: items: properties: name: - type: string + type: "string" resources: items: properties: health: - type: string + type: "string" resourceID: - type: string - type: object - type: array - type: object - type: array + type: "string" + type: "object" + type: "array" + type: "object" + type: "array" containerID: - type: string + type: "string" image: - type: string + type: "string" imageID: - type: string + type: "string" lastState: properties: running: properties: startedAt: - type: string - type: object + type: "string" + type: "object" terminated: properties: containerID: - type: string + type: "string" exitCode: - type: integer + type: "integer" finishedAt: - type: string + type: "string" message: - type: string + type: "string" reason: - type: string + type: "string" signal: - type: integer + type: "integer" startedAt: - type: string - type: object + type: "string" + type: "object" waiting: properties: message: - type: string + type: "string" reason: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" name: - type: string + type: "string" ready: - type: boolean + type: "boolean" resources: properties: claims: items: properties: name: - type: string + type: "string" request: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" limits: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" requests: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" restartCount: - type: integer + type: "integer" started: - type: boolean + type: "boolean" state: properties: running: properties: startedAt: - type: string - type: object + type: "string" + type: "object" terminated: properties: containerID: - type: string + type: "string" exitCode: - type: integer + type: "integer" finishedAt: - type: string + type: "string" message: - type: string + type: "string" reason: - type: string + type: "string" signal: - type: integer + type: "integer" startedAt: - type: string - type: object + type: "string" + type: "object" waiting: properties: message: - type: string + type: "string" reason: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" stopSignal: - type: string + type: "string" user: properties: linux: properties: gid: - type: integer + type: "integer" supplementalGroups: items: - type: integer - type: array + type: "integer" + type: "array" uid: - type: integer - type: object - type: object + type: "integer" + type: "object" + type: "object" volumeMounts: items: properties: mountPath: - type: string + type: "string" name: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" recursiveReadOnly: - type: string - type: object - type: array - type: object - type: array + type: "string" + type: "object" + type: "array" + type: "object" + type: "array" ephemeralContainerStatuses: items: properties: allocatedResources: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" allocatedResourcesStatus: items: properties: name: - type: string + type: "string" resources: items: properties: health: - type: string + type: "string" resourceID: - type: string - type: object - type: array - type: object - type: array + type: "string" + type: "object" + type: "array" + type: "object" + type: "array" containerID: - type: string + type: "string" image: - type: string + type: "string" imageID: - type: string + type: "string" lastState: properties: running: properties: startedAt: - type: string - type: object + type: "string" + type: "object" terminated: properties: containerID: - type: string + type: "string" exitCode: - type: integer + type: "integer" finishedAt: - type: string + type: "string" message: - type: string + type: "string" reason: - type: string + type: "string" signal: - type: integer + type: "integer" startedAt: - type: string - type: object + type: "string" + type: "object" waiting: properties: message: - type: string + type: "string" reason: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" name: - type: string + type: "string" ready: - type: boolean + type: "boolean" resources: properties: claims: items: properties: name: - type: string + type: "string" request: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" limits: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" requests: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" restartCount: - type: integer + type: "integer" started: - type: boolean + type: "boolean" state: properties: running: properties: startedAt: - type: string - type: object + type: "string" + type: "object" terminated: properties: containerID: - type: string + type: "string" exitCode: - type: integer + type: "integer" finishedAt: - type: string + type: "string" message: - type: string + type: "string" reason: - type: string + type: "string" signal: - type: integer + type: "integer" startedAt: - type: string - type: object + type: "string" + type: "object" waiting: properties: message: - type: string + type: "string" reason: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" stopSignal: - type: string + type: "string" user: properties: linux: properties: gid: - type: integer + type: "integer" supplementalGroups: items: - type: integer - type: array + type: "integer" + type: "array" uid: - type: integer - type: object - type: object + type: "integer" + type: "object" + type: "object" volumeMounts: items: properties: mountPath: - type: string + type: "string" name: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" recursiveReadOnly: - type: string - type: object - type: array - type: object - type: array + type: "string" + type: "object" + type: "array" + type: "object" + type: "array" hostIP: - type: string + type: "string" hostIPs: items: properties: ip: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" initContainerStatuses: items: properties: allocatedResources: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" allocatedResourcesStatus: items: properties: name: - type: string + type: "string" resources: items: properties: health: - type: string + type: "string" resourceID: - type: string - type: object - type: array - type: object - type: array + type: "string" + type: "object" + type: "array" + type: "object" + type: "array" containerID: - type: string + type: "string" image: - type: string + type: "string" imageID: - type: string + type: "string" lastState: properties: running: properties: startedAt: - type: string - type: object + type: "string" + type: "object" terminated: properties: containerID: - type: string + type: "string" exitCode: - type: integer + type: "integer" finishedAt: - type: string + type: "string" message: - type: string + type: "string" reason: - type: string + type: "string" signal: - type: integer + type: "integer" startedAt: - type: string - type: object + type: "string" + type: "object" waiting: properties: message: - type: string + type: "string" reason: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" name: - type: string + type: "string" ready: - type: boolean + type: "boolean" resources: properties: claims: items: properties: name: - type: string + type: "string" request: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" limits: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" requests: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" restartCount: - type: integer + type: "integer" started: - type: boolean + type: "boolean" state: properties: running: properties: startedAt: - type: string - type: object + type: "string" + type: "object" terminated: properties: containerID: - type: string + type: "string" exitCode: - type: integer + type: "integer" finishedAt: - type: string + type: "string" message: - type: string + type: "string" reason: - type: string + type: "string" signal: - type: integer + type: "integer" startedAt: - type: string - type: object + type: "string" + type: "object" waiting: properties: message: - type: string + type: "string" reason: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" stopSignal: - type: string + type: "string" user: properties: linux: properties: gid: - type: integer + type: "integer" supplementalGroups: items: - type: integer - type: array + type: "integer" + type: "array" uid: - type: integer - type: object - type: object + type: "integer" + type: "object" + type: "object" volumeMounts: items: properties: mountPath: - type: string + type: "string" name: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" recursiveReadOnly: - type: string - type: object - type: array - type: object - type: array + type: "string" + type: "object" + type: "array" + type: "object" + type: "array" message: - type: string + type: "string" nominatedNodeName: - type: string + type: "string" observedGeneration: - type: integer + type: "integer" phase: - type: string + type: "string" podIP: - type: string + type: "string" podIPs: items: properties: ip: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" qosClass: - type: string + type: "string" reason: - type: string + type: "string" resize: - type: string + type: "string" resourceClaimStatuses: items: properties: name: - type: string + type: "string" resourceClaimName: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" startTime: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" restartNonce: - type: integer + type: "integer" serviceAccount: - type: string + type: "string" taskManager: properties: podTemplate: properties: apiVersion: - type: string + type: "string" kind: - type: string + type: "string" metadata: properties: annotations: additionalProperties: - type: string - type: object + type: "string" + type: "object" creationTimestamp: - type: string + type: "string" deletionGracePeriodSeconds: - type: integer + type: "integer" deletionTimestamp: - type: string + type: "string" finalizers: items: - type: string - type: array + type: "string" + type: "array" generateName: - type: string + type: "string" generation: - type: integer + type: "integer" labels: additionalProperties: - type: string - type: object + type: "string" + type: "object" managedFields: items: properties: apiVersion: - type: string + type: "string" fieldsType: - type: string + type: "string" fieldsV1: - type: object + type: "object" manager: - type: string + type: "string" operation: - type: string + type: "string" subresource: - type: string + type: "string" time: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" name: - type: string + type: "string" namespace: - type: string + type: "string" ownerReferences: items: properties: apiVersion: - type: string + type: "string" blockOwnerDeletion: - type: boolean + type: "boolean" controller: - type: boolean + type: "boolean" kind: - type: string + type: "string" name: - type: string + type: "string" uid: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" resourceVersion: - type: string + type: "string" selfLink: - type: string + type: "string" uid: - type: string - type: object + type: "string" + type: "object" spec: properties: activeDeadlineSeconds: - type: integer + type: "integer" affinity: properties: nodeAffinity: @@ -7252,33 +7249,33 @@ spec: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchFields: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array - type: object + type: "string" + type: "array" + type: "object" + type: "array" + type: "object" weight: - type: integer - type: object - type: array + type: "integer" + type: "object" + type: "array" requiredDuringSchedulingIgnoredDuringExecution: properties: nodeSelectorTerms: @@ -7288,32 +7285,32 @@ spec: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchFields: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array - type: object - type: array - type: object - type: object + type: "string" + type: "array" + type: "object" + type: "array" + type: "object" + type: "array" + type: "object" + type: "object" podAffinity: properties: preferredDuringSchedulingIgnoredDuringExecution: @@ -7327,59 +7324,59 @@ spec: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" matchLabelKeys: items: - type: string - type: array + type: "string" + type: "array" mismatchLabelKeys: items: - type: string - type: array + type: "string" + type: "array" namespaceSelector: properties: matchExpressions: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" namespaces: items: - type: string - type: array + type: "string" + type: "array" topologyKey: - type: string - type: object + type: "string" + type: "object" weight: - type: integer - type: object - type: array + type: "integer" + type: "object" + type: "array" requiredDuringSchedulingIgnoredDuringExecution: items: properties: @@ -7389,57 +7386,57 @@ spec: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" matchLabelKeys: items: - type: string - type: array + type: "string" + type: "array" mismatchLabelKeys: items: - type: string - type: array + type: "string" + type: "array" namespaceSelector: properties: matchExpressions: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" namespaces: items: - type: string - type: array + type: "string" + type: "array" topologyKey: - type: string - type: object - type: array - type: object + type: "string" + type: "object" + type: "array" + type: "object" podAntiAffinity: properties: preferredDuringSchedulingIgnoredDuringExecution: @@ -7453,59 +7450,59 @@ spec: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" matchLabelKeys: items: - type: string - type: array + type: "string" + type: "array" mismatchLabelKeys: items: - type: string - type: array + type: "string" + type: "array" namespaceSelector: properties: matchExpressions: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" namespaces: items: - type: string - type: array + type: "string" + type: "array" topologyKey: - type: string - type: object + type: "string" + type: "object" weight: - type: integer - type: object - type: array + type: "integer" + type: "object" + type: "array" requiredDuringSchedulingIgnoredDuringExecution: items: properties: @@ -7515,145 +7512,145 @@ spec: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" matchLabelKeys: items: - type: string - type: array + type: "string" + type: "array" mismatchLabelKeys: items: - type: string - type: array + type: "string" + type: "array" namespaceSelector: properties: matchExpressions: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" namespaces: items: - type: string - type: array + type: "string" + type: "array" topologyKey: - type: string - type: object - type: array - type: object - type: object + type: "string" + type: "object" + type: "array" + type: "object" + type: "object" automountServiceAccountToken: - type: boolean + type: "boolean" containers: items: properties: args: items: - type: string - type: array + type: "string" + type: "array" command: items: - type: string - type: array + type: "string" + type: "array" env: items: properties: name: - type: string + type: "string" value: - type: string + type: "string" valueFrom: properties: configMapKeyRef: properties: key: - type: string + type: "string" name: - type: string + type: "string" optional: - type: boolean - type: object + type: "boolean" + type: "object" fieldRef: properties: apiVersion: - type: string + type: "string" fieldPath: - type: string - type: object + type: "string" + type: "object" resourceFieldRef: properties: containerName: - type: string + type: "string" divisor: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true resource: - type: string - type: object + type: "string" + type: "object" secretKeyRef: properties: key: - type: string + type: "string" name: - type: string + type: "string" optional: - type: boolean - type: object - type: object - type: object - type: array + type: "boolean" + type: "object" + type: "object" + type: "object" + type: "array" envFrom: items: properties: configMapRef: properties: name: - type: string + type: "string" optional: - type: boolean - type: object + type: "boolean" + type: "object" prefix: - type: string + type: "string" secretRef: properties: name: - type: string + type: "string" optional: - type: boolean - type: object - type: object - type: array + type: "boolean" + type: "object" + type: "object" + type: "array" image: - type: string + type: "string" imagePullPolicy: - type: string + type: "string" lifecycle: properties: postStart: @@ -7662,554 +7659,554 @@ spec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" sleep: properties: seconds: - type: integer - type: object + type: "integer" + type: "object" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" preStop: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" sleep: properties: seconds: - type: integer - type: object + type: "integer" + type: "object" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" stopSignal: - type: string - type: object + type: "string" + type: "object" livenessProbe: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" failureThreshold: - type: integer + type: "integer" grpc: properties: port: - type: integer + type: "integer" service: - type: string - type: object + type: "string" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" initialDelaySeconds: - type: integer + type: "integer" periodSeconds: - type: integer + type: "integer" successThreshold: - type: integer + type: "integer" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" terminationGracePeriodSeconds: - type: integer + type: "integer" timeoutSeconds: - type: integer - type: object + type: "integer" + type: "object" name: - type: string + type: "string" ports: items: properties: containerPort: - type: integer + type: "integer" hostIP: - type: string + type: "string" hostPort: - type: integer + type: "integer" name: - type: string + type: "string" protocol: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" readinessProbe: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" failureThreshold: - type: integer + type: "integer" grpc: properties: port: - type: integer + type: "integer" service: - type: string - type: object + type: "string" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" initialDelaySeconds: - type: integer + type: "integer" periodSeconds: - type: integer + type: "integer" successThreshold: - type: integer + type: "integer" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" terminationGracePeriodSeconds: - type: integer + type: "integer" timeoutSeconds: - type: integer - type: object + type: "integer" + type: "object" resizePolicy: items: properties: resourceName: - type: string + type: "string" restartPolicy: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" resources: properties: claims: items: properties: name: - type: string + type: "string" request: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" limits: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" requests: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" restartPolicy: - type: string + type: "string" securityContext: properties: allowPrivilegeEscalation: - type: boolean + type: "boolean" appArmorProfile: properties: localhostProfile: - type: string + type: "string" type: - type: string - type: object + type: "string" + type: "object" capabilities: properties: add: items: - type: string - type: array + type: "string" + type: "array" drop: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" privileged: - type: boolean + type: "boolean" procMount: - type: string + type: "string" readOnlyRootFilesystem: - type: boolean + type: "boolean" runAsGroup: - type: integer + type: "integer" runAsNonRoot: - type: boolean + type: "boolean" runAsUser: - type: integer + type: "integer" seLinuxOptions: properties: level: - type: string + type: "string" role: - type: string + type: "string" type: - type: string + type: "string" user: - type: string - type: object + type: "string" + type: "object" seccompProfile: properties: localhostProfile: - type: string + type: "string" type: - type: string - type: object + type: "string" + type: "object" windowsOptions: properties: gmsaCredentialSpec: - type: string + type: "string" gmsaCredentialSpecName: - type: string + type: "string" hostProcess: - type: boolean + type: "boolean" runAsUserName: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" startupProbe: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" failureThreshold: - type: integer + type: "integer" grpc: properties: port: - type: integer + type: "integer" service: - type: string - type: object + type: "string" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" initialDelaySeconds: - type: integer + type: "integer" periodSeconds: - type: integer + type: "integer" successThreshold: - type: integer + type: "integer" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" terminationGracePeriodSeconds: - type: integer + type: "integer" timeoutSeconds: - type: integer - type: object + type: "integer" + type: "object" stdin: - type: boolean + type: "boolean" stdinOnce: - type: boolean + type: "boolean" terminationMessagePath: - type: string + type: "string" terminationMessagePolicy: - type: string + type: "string" tty: - type: boolean + type: "boolean" volumeDevices: items: properties: devicePath: - type: string + type: "string" name: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" volumeMounts: items: properties: mountPath: - type: string + type: "string" mountPropagation: - type: string + type: "string" name: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" recursiveReadOnly: - type: string + type: "string" subPath: - type: string + type: "string" subPathExpr: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" workingDir: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" dnsConfig: properties: nameservers: items: - type: string - type: array + type: "string" + type: "array" options: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" searches: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" dnsPolicy: - type: string + type: "string" enableServiceLinks: - type: boolean + type: "boolean" ephemeralContainers: items: properties: args: items: - type: string - type: array + type: "string" + type: "array" command: items: - type: string - type: array + type: "string" + type: "array" env: items: properties: name: - type: string + type: "string" value: - type: string + type: "string" valueFrom: properties: configMapKeyRef: properties: key: - type: string + type: "string" name: - type: string + type: "string" optional: - type: boolean - type: object + type: "boolean" + type: "object" fieldRef: properties: apiVersion: - type: string + type: "string" fieldPath: - type: string - type: object + type: "string" + type: "object" resourceFieldRef: properties: containerName: - type: string + type: "string" divisor: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true resource: - type: string - type: object + type: "string" + type: "object" secretKeyRef: properties: key: - type: string + type: "string" name: - type: string + type: "string" optional: - type: boolean - type: object - type: object - type: object - type: array + type: "boolean" + type: "object" + type: "object" + type: "object" + type: "array" envFrom: items: properties: configMapRef: properties: name: - type: string + type: "string" optional: - type: boolean - type: object + type: "boolean" + type: "object" prefix: - type: string + type: "string" secretRef: properties: name: - type: string + type: "string" optional: - type: boolean - type: object - type: object - type: array + type: "boolean" + type: "object" + type: "object" + type: "array" image: - type: string + type: "string" imagePullPolicy: - type: string + type: "string" lifecycle: properties: postStart: @@ -8218,560 +8215,560 @@ spec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" sleep: properties: seconds: - type: integer - type: object + type: "integer" + type: "object" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" preStop: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" sleep: properties: seconds: - type: integer - type: object + type: "integer" + type: "object" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" stopSignal: - type: string - type: object + type: "string" + type: "object" livenessProbe: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" failureThreshold: - type: integer + type: "integer" grpc: properties: port: - type: integer + type: "integer" service: - type: string - type: object + type: "string" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" initialDelaySeconds: - type: integer + type: "integer" periodSeconds: - type: integer + type: "integer" successThreshold: - type: integer + type: "integer" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" terminationGracePeriodSeconds: - type: integer + type: "integer" timeoutSeconds: - type: integer - type: object + type: "integer" + type: "object" name: - type: string + type: "string" ports: items: properties: containerPort: - type: integer + type: "integer" hostIP: - type: string + type: "string" hostPort: - type: integer + type: "integer" name: - type: string + type: "string" protocol: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" readinessProbe: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" failureThreshold: - type: integer + type: "integer" grpc: properties: port: - type: integer + type: "integer" service: - type: string - type: object + type: "string" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" initialDelaySeconds: - type: integer + type: "integer" periodSeconds: - type: integer + type: "integer" successThreshold: - type: integer + type: "integer" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" terminationGracePeriodSeconds: - type: integer + type: "integer" timeoutSeconds: - type: integer - type: object + type: "integer" + type: "object" resizePolicy: items: properties: resourceName: - type: string + type: "string" restartPolicy: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" resources: properties: claims: items: properties: name: - type: string + type: "string" request: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" limits: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" requests: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" restartPolicy: - type: string + type: "string" securityContext: properties: allowPrivilegeEscalation: - type: boolean + type: "boolean" appArmorProfile: properties: localhostProfile: - type: string + type: "string" type: - type: string - type: object + type: "string" + type: "object" capabilities: properties: add: items: - type: string - type: array + type: "string" + type: "array" drop: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" privileged: - type: boolean + type: "boolean" procMount: - type: string + type: "string" readOnlyRootFilesystem: - type: boolean + type: "boolean" runAsGroup: - type: integer + type: "integer" runAsNonRoot: - type: boolean + type: "boolean" runAsUser: - type: integer + type: "integer" seLinuxOptions: properties: level: - type: string + type: "string" role: - type: string + type: "string" type: - type: string + type: "string" user: - type: string - type: object + type: "string" + type: "object" seccompProfile: properties: localhostProfile: - type: string + type: "string" type: - type: string - type: object + type: "string" + type: "object" windowsOptions: properties: gmsaCredentialSpec: - type: string + type: "string" gmsaCredentialSpecName: - type: string + type: "string" hostProcess: - type: boolean + type: "boolean" runAsUserName: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" startupProbe: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" failureThreshold: - type: integer + type: "integer" grpc: properties: port: - type: integer + type: "integer" service: - type: string - type: object + type: "string" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" initialDelaySeconds: - type: integer + type: "integer" periodSeconds: - type: integer + type: "integer" successThreshold: - type: integer + type: "integer" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" terminationGracePeriodSeconds: - type: integer + type: "integer" timeoutSeconds: - type: integer - type: object + type: "integer" + type: "object" stdin: - type: boolean + type: "boolean" stdinOnce: - type: boolean + type: "boolean" targetContainerName: - type: string + type: "string" terminationMessagePath: - type: string + type: "string" terminationMessagePolicy: - type: string + type: "string" tty: - type: boolean + type: "boolean" volumeDevices: items: properties: devicePath: - type: string + type: "string" name: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" volumeMounts: items: properties: mountPath: - type: string + type: "string" mountPropagation: - type: string + type: "string" name: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" recursiveReadOnly: - type: string + type: "string" subPath: - type: string + type: "string" subPathExpr: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" workingDir: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" hostAliases: items: properties: hostnames: items: - type: string - type: array + type: "string" + type: "array" ip: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" hostIPC: - type: boolean + type: "boolean" hostNetwork: - type: boolean + type: "boolean" hostPID: - type: boolean + type: "boolean" hostUsers: - type: boolean + type: "boolean" hostname: - type: string + type: "string" imagePullSecrets: items: properties: name: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" initContainers: items: properties: args: items: - type: string - type: array + type: "string" + type: "array" command: items: - type: string - type: array + type: "string" + type: "array" env: items: properties: name: - type: string + type: "string" value: - type: string + type: "string" valueFrom: properties: configMapKeyRef: properties: key: - type: string + type: "string" name: - type: string + type: "string" optional: - type: boolean - type: object + type: "boolean" + type: "object" fieldRef: properties: apiVersion: - type: string + type: "string" fieldPath: - type: string - type: object + type: "string" + type: "object" resourceFieldRef: properties: containerName: - type: string + type: "string" divisor: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true resource: - type: string - type: object + type: "string" + type: "object" secretKeyRef: properties: key: - type: string + type: "string" name: - type: string + type: "string" optional: - type: boolean - type: object - type: object - type: object - type: array + type: "boolean" + type: "object" + type: "object" + type: "object" + type: "array" envFrom: items: properties: configMapRef: properties: name: - type: string + type: "string" optional: - type: boolean - type: object + type: "boolean" + type: "object" prefix: - type: string + type: "string" secretRef: properties: name: - type: string + type: "string" optional: - type: boolean - type: object - type: object - type: array + type: "boolean" + type: "object" + type: "object" + type: "array" image: - type: string + type: "string" imagePullPolicy: - type: string + type: "string" lifecycle: properties: postStart: @@ -8780,619 +8777,619 @@ spec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" sleep: properties: seconds: - type: integer - type: object + type: "integer" + type: "object" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" preStop: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" sleep: properties: seconds: - type: integer - type: object + type: "integer" + type: "object" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" stopSignal: - type: string - type: object + type: "string" + type: "object" livenessProbe: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" failureThreshold: - type: integer + type: "integer" grpc: properties: port: - type: integer + type: "integer" service: - type: string - type: object + type: "string" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" initialDelaySeconds: - type: integer + type: "integer" periodSeconds: - type: integer + type: "integer" successThreshold: - type: integer + type: "integer" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" terminationGracePeriodSeconds: - type: integer + type: "integer" timeoutSeconds: - type: integer - type: object + type: "integer" + type: "object" name: - type: string + type: "string" ports: items: properties: containerPort: - type: integer + type: "integer" hostIP: - type: string + type: "string" hostPort: - type: integer + type: "integer" name: - type: string + type: "string" protocol: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" readinessProbe: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" failureThreshold: - type: integer + type: "integer" grpc: properties: port: - type: integer + type: "integer" service: - type: string - type: object + type: "string" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" initialDelaySeconds: - type: integer + type: "integer" periodSeconds: - type: integer + type: "integer" successThreshold: - type: integer + type: "integer" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" terminationGracePeriodSeconds: - type: integer + type: "integer" timeoutSeconds: - type: integer - type: object + type: "integer" + type: "object" resizePolicy: items: properties: resourceName: - type: string + type: "string" restartPolicy: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" resources: properties: claims: items: properties: name: - type: string + type: "string" request: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" limits: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" requests: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" restartPolicy: - type: string + type: "string" securityContext: properties: allowPrivilegeEscalation: - type: boolean + type: "boolean" appArmorProfile: properties: localhostProfile: - type: string + type: "string" type: - type: string - type: object + type: "string" + type: "object" capabilities: properties: add: items: - type: string - type: array + type: "string" + type: "array" drop: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" privileged: - type: boolean + type: "boolean" procMount: - type: string + type: "string" readOnlyRootFilesystem: - type: boolean + type: "boolean" runAsGroup: - type: integer + type: "integer" runAsNonRoot: - type: boolean + type: "boolean" runAsUser: - type: integer + type: "integer" seLinuxOptions: properties: level: - type: string + type: "string" role: - type: string + type: "string" type: - type: string + type: "string" user: - type: string - type: object + type: "string" + type: "object" seccompProfile: properties: localhostProfile: - type: string + type: "string" type: - type: string - type: object + type: "string" + type: "object" windowsOptions: properties: gmsaCredentialSpec: - type: string + type: "string" gmsaCredentialSpecName: - type: string + type: "string" hostProcess: - type: boolean + type: "boolean" runAsUserName: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" startupProbe: properties: exec: properties: command: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" failureThreshold: - type: integer + type: "integer" grpc: properties: port: - type: integer + type: "integer" service: - type: string - type: object + type: "string" + type: "object" httpGet: properties: host: - type: string + type: "string" httpHeaders: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" path: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true scheme: - type: string - type: object + type: "string" + type: "object" initialDelaySeconds: - type: integer + type: "integer" periodSeconds: - type: integer + type: "integer" successThreshold: - type: integer + type: "integer" tcpSocket: properties: host: - type: string + type: "string" port: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" terminationGracePeriodSeconds: - type: integer + type: "integer" timeoutSeconds: - type: integer - type: object + type: "integer" + type: "object" stdin: - type: boolean + type: "boolean" stdinOnce: - type: boolean + type: "boolean" terminationMessagePath: - type: string + type: "string" terminationMessagePolicy: - type: string + type: "string" tty: - type: boolean + type: "boolean" volumeDevices: items: properties: devicePath: - type: string + type: "string" name: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" volumeMounts: items: properties: mountPath: - type: string + type: "string" mountPropagation: - type: string + type: "string" name: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" recursiveReadOnly: - type: string + type: "string" subPath: - type: string + type: "string" subPathExpr: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" workingDir: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" nodeName: - type: string + type: "string" nodeSelector: additionalProperties: - type: string - type: object + type: "string" + type: "object" os: properties: name: - type: string - type: object + type: "string" + type: "object" overhead: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" preemptionPolicy: - type: string + type: "string" priority: - type: integer + type: "integer" priorityClassName: - type: string + type: "string" readinessGates: items: properties: conditionType: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" resourceClaims: items: properties: name: - type: string + type: "string" resourceClaimName: - type: string + type: "string" resourceClaimTemplateName: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" resources: properties: claims: items: properties: name: - type: string + type: "string" request: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" limits: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" requests: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" restartPolicy: - type: string + type: "string" runtimeClassName: - type: string + type: "string" schedulerName: - type: string + type: "string" schedulingGates: items: properties: name: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" securityContext: properties: appArmorProfile: properties: localhostProfile: - type: string + type: "string" type: - type: string - type: object + type: "string" + type: "object" fsGroup: - type: integer + type: "integer" fsGroupChangePolicy: - type: string + type: "string" runAsGroup: - type: integer + type: "integer" runAsNonRoot: - type: boolean + type: "boolean" runAsUser: - type: integer + type: "integer" seLinuxChangePolicy: - type: string + type: "string" seLinuxOptions: properties: level: - type: string + type: "string" role: - type: string + type: "string" type: - type: string + type: "string" user: - type: string - type: object + type: "string" + type: "object" seccompProfile: properties: localhostProfile: - type: string + type: "string" type: - type: string - type: object + type: "string" + type: "object" supplementalGroups: items: - type: integer - type: array + type: "integer" + type: "array" supplementalGroupsPolicy: - type: string + type: "string" sysctls: items: properties: name: - type: string + type: "string" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" windowsOptions: properties: gmsaCredentialSpec: - type: string + type: "string" gmsaCredentialSpecName: - type: string + type: "string" hostProcess: - type: boolean + type: "boolean" runAsUserName: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" serviceAccount: - type: string + type: "string" serviceAccountName: - type: string + type: "string" setHostnameAsFQDN: - type: boolean + type: "boolean" shareProcessNamespace: - type: boolean + type: "boolean" subdomain: - type: string + type: "string" terminationGracePeriodSeconds: - type: integer + type: "integer" tolerations: items: properties: effect: - type: string + type: "string" key: - type: string + type: "string" operator: - type: string + type: "string" tolerationSeconds: - type: integer + type: "integer" value: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" topologySpreadConstraints: items: properties: @@ -9402,191 +9399,191 @@ spec: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" matchLabelKeys: items: - type: string - type: array + type: "string" + type: "array" maxSkew: - type: integer + type: "integer" minDomains: - type: integer + type: "integer" nodeAffinityPolicy: - type: string + type: "string" nodeTaintsPolicy: - type: string + type: "string" topologyKey: - type: string + type: "string" whenUnsatisfiable: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" volumes: items: properties: awsElasticBlockStore: properties: fsType: - type: string + type: "string" partition: - type: integer + type: "integer" readOnly: - type: boolean + type: "boolean" volumeID: - type: string - type: object + type: "string" + type: "object" azureDisk: properties: cachingMode: - type: string + type: "string" diskName: - type: string + type: "string" diskURI: - type: string + type: "string" fsType: - type: string + type: "string" kind: - type: string + type: "string" readOnly: - type: boolean - type: object + type: "boolean" + type: "object" azureFile: properties: readOnly: - type: boolean + type: "boolean" secretName: - type: string + type: "string" shareName: - type: string - type: object + type: "string" + type: "object" cephfs: properties: monitors: items: - type: string - type: array + type: "string" + type: "array" path: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" secretFile: - type: string + type: "string" secretRef: properties: name: - type: string - type: object + type: "string" + type: "object" user: - type: string - type: object + type: "string" + type: "object" cinder: properties: fsType: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" secretRef: properties: name: - type: string - type: object + type: "string" + type: "object" volumeID: - type: string - type: object + type: "string" + type: "object" configMap: properties: defaultMode: - type: integer + type: "integer" items: items: properties: key: - type: string + type: "string" mode: - type: integer + type: "integer" path: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" name: - type: string + type: "string" optional: - type: boolean - type: object + type: "boolean" + type: "object" csi: properties: driver: - type: string + type: "string" fsType: - type: string + type: "string" nodePublishSecretRef: properties: name: - type: string - type: object + type: "string" + type: "object" readOnly: - type: boolean + type: "boolean" volumeAttributes: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" downwardAPI: properties: defaultMode: - type: integer + type: "integer" items: items: properties: fieldRef: properties: apiVersion: - type: string + type: "string" fieldPath: - type: string - type: object + type: "string" + type: "object" mode: - type: integer + type: "integer" path: - type: string + type: "string" resourceFieldRef: properties: containerName: - type: string + type: "string" divisor: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true resource: - type: string - type: object - type: object - type: array - type: object + type: "string" + type: "object" + type: "object" + type: "array" + type: "object" emptyDir: properties: medium: - type: string + type: "string" sizeLimit: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" ephemeral: properties: volumeClaimTemplate: @@ -9595,300 +9592,300 @@ spec: properties: annotations: additionalProperties: - type: string - type: object + type: "string" + type: "object" creationTimestamp: - type: string + type: "string" deletionGracePeriodSeconds: - type: integer + type: "integer" deletionTimestamp: - type: string + type: "string" finalizers: items: - type: string - type: array + type: "string" + type: "array" generateName: - type: string + type: "string" generation: - type: integer + type: "integer" labels: additionalProperties: - type: string - type: object + type: "string" + type: "object" managedFields: items: properties: apiVersion: - type: string + type: "string" fieldsType: - type: string + type: "string" fieldsV1: - type: object + type: "object" manager: - type: string + type: "string" operation: - type: string + type: "string" subresource: - type: string + type: "string" time: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" name: - type: string + type: "string" namespace: - type: string + type: "string" ownerReferences: items: properties: apiVersion: - type: string + type: "string" blockOwnerDeletion: - type: boolean + type: "boolean" controller: - type: boolean + type: "boolean" kind: - type: string + type: "string" name: - type: string + type: "string" uid: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" resourceVersion: - type: string + type: "string" selfLink: - type: string + type: "string" uid: - type: string - type: object + type: "string" + type: "object" spec: properties: accessModes: items: - type: string - type: array + type: "string" + type: "array" dataSource: properties: apiGroup: - type: string + type: "string" kind: - type: string + type: "string" name: - type: string - type: object + type: "string" + type: "object" dataSourceRef: properties: apiGroup: - type: string + type: "string" kind: - type: string + type: "string" name: - type: string + type: "string" namespace: - type: string - type: object + type: "string" + type: "object" resources: properties: limits: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" requests: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" selector: properties: matchExpressions: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" storageClassName: - type: string + type: "string" volumeAttributesClassName: - type: string + type: "string" volumeMode: - type: string + type: "string" volumeName: - type: string - type: object - type: object - type: object + type: "string" + type: "object" + type: "object" + type: "object" fc: properties: fsType: - type: string + type: "string" lun: - type: integer + type: "integer" readOnly: - type: boolean + type: "boolean" targetWWNs: items: - type: string - type: array + type: "string" + type: "array" wwids: items: - type: string - type: array - type: object + type: "string" + type: "array" + type: "object" flexVolume: properties: driver: - type: string + type: "string" fsType: - type: string + type: "string" options: additionalProperties: - type: string - type: object + type: "string" + type: "object" readOnly: - type: boolean + type: "boolean" secretRef: properties: name: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" flocker: properties: datasetName: - type: string + type: "string" datasetUUID: - type: string - type: object + type: "string" + type: "object" gcePersistentDisk: properties: fsType: - type: string + type: "string" partition: - type: integer + type: "integer" pdName: - type: string + type: "string" readOnly: - type: boolean - type: object + type: "boolean" + type: "object" gitRepo: properties: directory: - type: string + type: "string" repository: - type: string + type: "string" revision: - type: string - type: object + type: "string" + type: "object" glusterfs: properties: endpoints: - type: string + type: "string" path: - type: string + type: "string" readOnly: - type: boolean - type: object + type: "boolean" + type: "object" hostPath: properties: path: - type: string + type: "string" type: - type: string - type: object + type: "string" + type: "object" image: properties: pullPolicy: - type: string + type: "string" reference: - type: string - type: object + type: "string" + type: "object" iscsi: properties: chapAuthDiscovery: - type: boolean + type: "boolean" chapAuthSession: - type: boolean + type: "boolean" fsType: - type: string + type: "string" initiatorName: - type: string + type: "string" iqn: - type: string + type: "string" iscsiInterface: - type: string + type: "string" lun: - type: integer + type: "integer" portals: items: - type: string - type: array + type: "string" + type: "array" readOnly: - type: boolean + type: "boolean" secretRef: properties: name: - type: string - type: object + type: "string" + type: "object" targetPortal: - type: string - type: object + type: "string" + type: "object" name: - type: string + type: "string" nfs: properties: path: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" server: - type: string - type: object + type: "string" + type: "object" persistentVolumeClaim: properties: claimName: - type: string + type: "string" readOnly: - type: boolean - type: object + type: "boolean" + type: "object" photonPersistentDisk: properties: fsType: - type: string + type: "string" pdID: - type: string - type: object + type: "string" + type: "object" portworxVolume: properties: fsType: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" volumeID: - type: string - type: object + type: "string" + type: "object" projected: properties: defaultMode: - type: integer + type: "integer" sources: items: properties: @@ -9900,47 +9897,47 @@ spec: items: properties: key: - type: string + type: "string" operator: - type: string + type: "string" values: items: - type: string - type: array - type: object - type: array + type: "string" + type: "array" + type: "object" + type: "array" matchLabels: additionalProperties: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" name: - type: string + type: "string" optional: - type: boolean + type: "boolean" path: - type: string + type: "string" signerName: - type: string - type: object + type: "string" + type: "object" configMap: properties: items: items: properties: key: - type: string + type: "string" mode: - type: integer + type: "integer" path: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" name: - type: string + type: "string" optional: - type: boolean - type: object + type: "boolean" + type: "object" downwardAPI: properties: items: @@ -9949,934 +9946,934 @@ spec: fieldRef: properties: apiVersion: - type: string + type: "string" fieldPath: - type: string - type: object + type: "string" + type: "object" mode: - type: integer + type: "integer" path: - type: string + type: "string" resourceFieldRef: properties: containerName: - type: string + type: "string" divisor: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true resource: - type: string - type: object - type: object - type: array - type: object + type: "string" + type: "object" + type: "object" + type: "array" + type: "object" secret: properties: items: items: properties: key: - type: string + type: "string" mode: - type: integer + type: "integer" path: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" name: - type: string + type: "string" optional: - type: boolean - type: object + type: "boolean" + type: "object" serviceAccountToken: properties: audience: - type: string + type: "string" expirationSeconds: - type: integer + type: "integer" path: - type: string - type: object - type: object - type: array - type: object + type: "string" + type: "object" + type: "object" + type: "array" + type: "object" quobyte: properties: group: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" registry: - type: string + type: "string" tenant: - type: string + type: "string" user: - type: string + type: "string" volume: - type: string - type: object + type: "string" + type: "object" rbd: properties: fsType: - type: string + type: "string" image: - type: string + type: "string" keyring: - type: string + type: "string" monitors: items: - type: string - type: array + type: "string" + type: "array" pool: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" secretRef: properties: name: - type: string - type: object + type: "string" + type: "object" user: - type: string - type: object + type: "string" + type: "object" scaleIO: properties: fsType: - type: string + type: "string" gateway: - type: string + type: "string" protectionDomain: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" secretRef: properties: name: - type: string - type: object + type: "string" + type: "object" sslEnabled: - type: boolean + type: "boolean" storageMode: - type: string + type: "string" storagePool: - type: string + type: "string" system: - type: string + type: "string" volumeName: - type: string - type: object + type: "string" + type: "object" secret: properties: defaultMode: - type: integer + type: "integer" items: items: properties: key: - type: string + type: "string" mode: - type: integer + type: "integer" path: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" optional: - type: boolean + type: "boolean" secretName: - type: string - type: object + type: "string" + type: "object" storageos: properties: fsType: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" secretRef: properties: name: - type: string - type: object + type: "string" + type: "object" volumeName: - type: string + type: "string" volumeNamespace: - type: string - type: object + type: "string" + type: "object" vsphereVolume: properties: fsType: - type: string + type: "string" storagePolicyID: - type: string + type: "string" storagePolicyName: - type: string + type: "string" volumePath: - type: string - type: object - type: object - type: array - type: object + type: "string" + type: "object" + type: "object" + type: "array" + type: "object" status: properties: conditions: items: properties: lastProbeTime: - type: string + type: "string" lastTransitionTime: - type: string + type: "string" message: - type: string + type: "string" observedGeneration: - type: integer + type: "integer" reason: - type: string + type: "string" status: - type: string + type: "string" type: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" containerStatuses: items: properties: allocatedResources: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" allocatedResourcesStatus: items: properties: name: - type: string + type: "string" resources: items: properties: health: - type: string + type: "string" resourceID: - type: string - type: object - type: array - type: object - type: array + type: "string" + type: "object" + type: "array" + type: "object" + type: "array" containerID: - type: string + type: "string" image: - type: string + type: "string" imageID: - type: string + type: "string" lastState: properties: running: properties: startedAt: - type: string - type: object + type: "string" + type: "object" terminated: properties: containerID: - type: string + type: "string" exitCode: - type: integer + type: "integer" finishedAt: - type: string + type: "string" message: - type: string + type: "string" reason: - type: string + type: "string" signal: - type: integer + type: "integer" startedAt: - type: string - type: object + type: "string" + type: "object" waiting: properties: message: - type: string + type: "string" reason: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" name: - type: string + type: "string" ready: - type: boolean + type: "boolean" resources: properties: claims: items: properties: name: - type: string + type: "string" request: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" limits: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" requests: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" restartCount: - type: integer + type: "integer" started: - type: boolean + type: "boolean" state: properties: running: properties: startedAt: - type: string - type: object + type: "string" + type: "object" terminated: properties: containerID: - type: string + type: "string" exitCode: - type: integer + type: "integer" finishedAt: - type: string + type: "string" message: - type: string + type: "string" reason: - type: string + type: "string" signal: - type: integer + type: "integer" startedAt: - type: string - type: object + type: "string" + type: "object" waiting: properties: message: - type: string + type: "string" reason: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" stopSignal: - type: string + type: "string" user: properties: linux: properties: gid: - type: integer + type: "integer" supplementalGroups: items: - type: integer - type: array + type: "integer" + type: "array" uid: - type: integer - type: object - type: object + type: "integer" + type: "object" + type: "object" volumeMounts: items: properties: mountPath: - type: string + type: "string" name: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" recursiveReadOnly: - type: string - type: object - type: array - type: object - type: array + type: "string" + type: "object" + type: "array" + type: "object" + type: "array" ephemeralContainerStatuses: items: properties: allocatedResources: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" allocatedResourcesStatus: items: properties: name: - type: string + type: "string" resources: items: properties: health: - type: string + type: "string" resourceID: - type: string - type: object - type: array - type: object - type: array + type: "string" + type: "object" + type: "array" + type: "object" + type: "array" containerID: - type: string + type: "string" image: - type: string + type: "string" imageID: - type: string + type: "string" lastState: properties: running: properties: startedAt: - type: string - type: object + type: "string" + type: "object" terminated: properties: containerID: - type: string + type: "string" exitCode: - type: integer + type: "integer" finishedAt: - type: string + type: "string" message: - type: string + type: "string" reason: - type: string + type: "string" signal: - type: integer + type: "integer" startedAt: - type: string - type: object + type: "string" + type: "object" waiting: properties: message: - type: string + type: "string" reason: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" name: - type: string + type: "string" ready: - type: boolean + type: "boolean" resources: properties: claims: items: properties: name: - type: string + type: "string" request: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" limits: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" requests: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" restartCount: - type: integer + type: "integer" started: - type: boolean + type: "boolean" state: properties: running: properties: startedAt: - type: string - type: object + type: "string" + type: "object" terminated: properties: containerID: - type: string + type: "string" exitCode: - type: integer + type: "integer" finishedAt: - type: string + type: "string" message: - type: string + type: "string" reason: - type: string + type: "string" signal: - type: integer + type: "integer" startedAt: - type: string - type: object + type: "string" + type: "object" waiting: properties: message: - type: string + type: "string" reason: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" stopSignal: - type: string + type: "string" user: properties: linux: properties: gid: - type: integer + type: "integer" supplementalGroups: items: - type: integer - type: array + type: "integer" + type: "array" uid: - type: integer - type: object - type: object + type: "integer" + type: "object" + type: "object" volumeMounts: items: properties: mountPath: - type: string + type: "string" name: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" recursiveReadOnly: - type: string - type: object - type: array - type: object - type: array + type: "string" + type: "object" + type: "array" + type: "object" + type: "array" hostIP: - type: string + type: "string" hostIPs: items: properties: ip: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" initContainerStatuses: items: properties: allocatedResources: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" allocatedResourcesStatus: items: properties: name: - type: string + type: "string" resources: items: properties: health: - type: string + type: "string" resourceID: - type: string - type: object - type: array - type: object - type: array + type: "string" + type: "object" + type: "array" + type: "object" + type: "array" containerID: - type: string + type: "string" image: - type: string + type: "string" imageID: - type: string + type: "string" lastState: properties: running: properties: startedAt: - type: string - type: object + type: "string" + type: "object" terminated: properties: containerID: - type: string + type: "string" exitCode: - type: integer + type: "integer" finishedAt: - type: string + type: "string" message: - type: string + type: "string" reason: - type: string + type: "string" signal: - type: integer + type: "integer" startedAt: - type: string - type: object + type: "string" + type: "object" waiting: properties: message: - type: string + type: "string" reason: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" name: - type: string + type: "string" ready: - type: boolean + type: "boolean" resources: properties: claims: items: properties: name: - type: string + type: "string" request: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" limits: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object + type: "object" requests: additionalProperties: anyOf: - - type: integer - - type: string + - type: "integer" + - type: "string" x-kubernetes-int-or-string: true - type: object - type: object + type: "object" + type: "object" restartCount: - type: integer + type: "integer" started: - type: boolean + type: "boolean" state: properties: running: properties: startedAt: - type: string - type: object + type: "string" + type: "object" terminated: properties: containerID: - type: string + type: "string" exitCode: - type: integer + type: "integer" finishedAt: - type: string + type: "string" message: - type: string + type: "string" reason: - type: string + type: "string" signal: - type: integer + type: "integer" startedAt: - type: string - type: object + type: "string" + type: "object" waiting: properties: message: - type: string + type: "string" reason: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" stopSignal: - type: string + type: "string" user: properties: linux: properties: gid: - type: integer + type: "integer" supplementalGroups: items: - type: integer - type: array + type: "integer" + type: "array" uid: - type: integer - type: object - type: object + type: "integer" + type: "object" + type: "object" volumeMounts: items: properties: mountPath: - type: string + type: "string" name: - type: string + type: "string" readOnly: - type: boolean + type: "boolean" recursiveReadOnly: - type: string - type: object - type: array - type: object - type: array + type: "string" + type: "object" + type: "array" + type: "object" + type: "array" message: - type: string + type: "string" nominatedNodeName: - type: string + type: "string" observedGeneration: - type: integer + type: "integer" phase: - type: string + type: "string" podIP: - type: string + type: "string" podIPs: items: properties: ip: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" qosClass: - type: string + type: "string" reason: - type: string + type: "string" resize: - type: string + type: "string" resourceClaimStatuses: items: properties: name: - type: string + type: "string" resourceClaimName: - type: string - type: object - type: array + type: "string" + type: "object" + type: "array" startTime: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" replicas: - type: integer + type: "integer" resource: properties: cpu: - type: number + type: "number" ephemeralStorage: - type: string + type: "string" memory: - type: string - type: object - type: object - type: object + type: "string" + type: "object" + type: "object" + type: "object" status: properties: clusterInfo: additionalProperties: - type: string - type: object + type: "string" + type: "object" error: - type: string + type: "string" jobManagerDeploymentStatus: enum: - - DEPLOYED_NOT_READY - - DEPLOYING - - ERROR - - MISSING - - READY - type: string + - "DEPLOYED_NOT_READY" + - "DEPLOYING" + - "ERROR" + - "MISSING" + - "READY" + type: "string" jobStatus: properties: checkpointInfo: properties: formatType: enum: - - FULL - - INCREMENTAL - - UNKNOWN - type: string + - "FULL" + - "INCREMENTAL" + - "UNKNOWN" + type: "string" lastCheckpoint: properties: formatType: enum: - - FULL - - INCREMENTAL - - UNKNOWN - type: string + - "FULL" + - "INCREMENTAL" + - "UNKNOWN" + type: "string" timeStamp: - type: integer + type: "integer" triggerNonce: - type: integer + type: "integer" triggerType: enum: - - MANUAL - - PERIODIC - - UNKNOWN - - UPGRADE - type: string - type: object + - "MANUAL" + - "PERIODIC" + - "UNKNOWN" + - "UPGRADE" + type: "string" + type: "object" lastPeriodicCheckpointTimestamp: - type: integer + type: "integer" triggerId: - type: string + type: "string" triggerTimestamp: - type: integer + type: "integer" triggerType: enum: - - MANUAL - - PERIODIC - - UNKNOWN - - UPGRADE - type: string - type: object + - "MANUAL" + - "PERIODIC" + - "UNKNOWN" + - "UPGRADE" + type: "string" + type: "object" jobId: - type: string + type: "string" jobName: - type: string + type: "string" savepointInfo: properties: formatType: enum: - - CANONICAL - - NATIVE - - UNKNOWN - type: string + - "CANONICAL" + - "NATIVE" + - "UNKNOWN" + type: "string" lastPeriodicSavepointTimestamp: - type: integer + type: "integer" lastSavepoint: properties: formatType: enum: - - CANONICAL - - NATIVE - - UNKNOWN - type: string + - "CANONICAL" + - "NATIVE" + - "UNKNOWN" + type: "string" location: - type: string + type: "string" timeStamp: - type: integer + type: "integer" triggerNonce: - type: integer + type: "integer" triggerType: enum: - - MANUAL - - PERIODIC - - UNKNOWN - - UPGRADE - type: string - type: object + - "MANUAL" + - "PERIODIC" + - "UNKNOWN" + - "UPGRADE" + type: "string" + type: "object" savepointHistory: items: properties: formatType: enum: - - CANONICAL - - NATIVE - - UNKNOWN - type: string + - "CANONICAL" + - "NATIVE" + - "UNKNOWN" + type: "string" location: - type: string + type: "string" timeStamp: - type: integer + type: "integer" triggerNonce: - type: integer + type: "integer" triggerType: enum: - - MANUAL - - PERIODIC - - UNKNOWN - - UPGRADE - type: string - type: object - type: array + - "MANUAL" + - "PERIODIC" + - "UNKNOWN" + - "UPGRADE" + type: "string" + type: "object" + type: "array" triggerId: - type: string + type: "string" triggerTimestamp: - type: integer + type: "integer" triggerType: enum: - - MANUAL - - PERIODIC - - UNKNOWN - - UPGRADE - type: string - type: object + - "MANUAL" + - "PERIODIC" + - "UNKNOWN" + - "UPGRADE" + type: "string" + type: "object" startTime: - type: string + type: "string" state: enum: - - CANCELED - - CANCELLING - - CREATED - - FAILED - - FAILING - - FINISHED - - INITIALIZING - - RECONCILING - - RESTARTING - - RUNNING - - SUSPENDED - type: string + - "CANCELED" + - "CANCELLING" + - "CREATED" + - "FAILED" + - "FAILING" + - "FINISHED" + - "INITIALIZING" + - "RECONCILING" + - "RESTARTING" + - "RUNNING" + - "SUSPENDED" + type: "string" updateTime: - type: string + type: "string" upgradeSavepointPath: - type: string - type: object + type: "string" + type: "object" lifecycleState: enum: - - CREATED - - DELETED - - DELETING - - DEPLOYED - - FAILED - - ROLLED_BACK - - ROLLING_BACK - - STABLE - - SUSPENDED - - UPGRADING - type: string + - "CREATED" + - "DELETED" + - "DELETING" + - "DEPLOYED" + - "FAILED" + - "ROLLED_BACK" + - "ROLLING_BACK" + - "STABLE" + - "SUSPENDED" + - "UPGRADING" + type: "string" observedGeneration: - type: integer + type: "integer" reconciliationStatus: properties: lastReconciledSpec: - type: string + type: "string" lastStableSpec: - type: string + type: "string" reconciliationTimestamp: - type: integer + type: "integer" state: enum: - - DEPLOYED - - ROLLED_BACK - - ROLLING_BACK - - UPGRADING - type: string - type: object + - "DEPLOYED" + - "ROLLED_BACK" + - "ROLLING_BACK" + - "UPGRADING" + type: "string" + type: "object" taskManager: properties: labelSelector: - type: string + type: "string" replicas: - type: integer - type: object - type: object - type: object + 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 + labelSelectorPath: ".status.taskManager.labelSelector" + specReplicasPath: ".spec.taskManager.replicas" + statusReplicasPath: ".status.taskManager.replicas" status: {} 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 668a84b6c7..758118c108 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 @@ -1,253 +1,250 @@ # Generated by Fabric8 CRDGenerator, manual edits might get overwritten! -apiVersion: apiextensions.k8s.io/v1 -kind: CustomResourceDefinition +apiVersion: "apiextensions.k8s.io/v1" +kind: "CustomResourceDefinition" metadata: - name: flinksessionjobs.flink.apache.org + name: "flinksessionjobs.flink.apache.org" spec: - group: flink.apache.org + group: "flink.apache.org" names: - kind: FlinkSessionJob - plural: flinksessionjobs + kind: "FlinkSessionJob" + plural: "flinksessionjobs" shortNames: - - sessionjob - singular: flinksessionjob - scope: Namespaced + - "sessionjob" + singular: "flinksessionjob" + scope: "Namespaced" versions: - additionalPrinterColumns: - - description: Last observed state of the job. - jsonPath: .status.jobStatus.state - name: Job Status + - 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 + type: "string" + - jsonPath: ".status.lifecycleState" + name: "Lifecycle State" priority: 0 - type: string - name: v1beta1 + type: "string" + name: "v1beta1" schema: openAPIV3Schema: properties: spec: properties: deploymentName: - type: string + type: "string" flinkConfiguration: x-kubernetes-preserve-unknown-fields: true job: properties: allowNonRestoredState: - type: boolean + type: "boolean" args: items: - type: string - type: array + type: "string" + type: "array" autoscalerResetNonce: - type: integer + type: "integer" checkpointTriggerNonce: - type: integer + type: "integer" entryClass: - type: string + type: "string" initialSavepointPath: - type: string + type: "string" jarURI: - type: string + type: "string" parallelism: - type: integer + type: "integer" savepointRedeployNonce: - type: integer + type: "integer" savepointTriggerNonce: - type: integer + type: "integer" state: enum: - - running - - suspended - type: string + - "running" + - "suspended" + type: "string" upgradeMode: enum: - - last-state - - savepoint - - stateless - type: string - type: object + - "last-state" + - "savepoint" + - "stateless" + type: "string" + type: "object" restartNonce: - type: integer - type: object + type: "integer" + type: "object" status: properties: error: - type: string + type: "string" jobStatus: properties: checkpointInfo: properties: formatType: enum: - - FULL - - INCREMENTAL - - UNKNOWN - type: string + - "FULL" + - "INCREMENTAL" + - "UNKNOWN" + type: "string" lastCheckpoint: properties: formatType: enum: - - FULL - - INCREMENTAL - - UNKNOWN - type: string + - "FULL" + - "INCREMENTAL" + - "UNKNOWN" + type: "string" timeStamp: - type: integer + type: "integer" triggerNonce: - type: integer + type: "integer" triggerType: enum: - - MANUAL - - PERIODIC - - UNKNOWN - - UPGRADE - type: string - type: object + - "MANUAL" + - "PERIODIC" + - "UNKNOWN" + - "UPGRADE" + type: "string" + type: "object" lastPeriodicCheckpointTimestamp: - type: integer + type: "integer" triggerId: - type: string + type: "string" triggerTimestamp: - type: integer + type: "integer" triggerType: enum: - - MANUAL - - PERIODIC - - UNKNOWN - - UPGRADE - type: string - type: object + - "MANUAL" + - "PERIODIC" + - "UNKNOWN" + - "UPGRADE" + type: "string" + type: "object" jobId: - type: string + type: "string" jobName: - type: string + type: "string" savepointInfo: properties: formatType: enum: - - CANONICAL - - NATIVE - - UNKNOWN - type: string + - "CANONICAL" + - "NATIVE" + - "UNKNOWN" + type: "string" lastPeriodicSavepointTimestamp: - type: integer + type: "integer" lastSavepoint: properties: formatType: enum: - - CANONICAL - - NATIVE - - UNKNOWN - type: string + - "CANONICAL" + - "NATIVE" + - "UNKNOWN" + type: "string" location: - type: string + type: "string" timeStamp: - type: integer + type: "integer" triggerNonce: - type: integer + type: "integer" triggerType: enum: - - MANUAL - - PERIODIC - - UNKNOWN - - UPGRADE - type: string - type: object + - "MANUAL" + - "PERIODIC" + - "UNKNOWN" + - "UPGRADE" + type: "string" + type: "object" savepointHistory: items: properties: formatType: enum: - - CANONICAL - - NATIVE - - UNKNOWN - type: string + - "CANONICAL" + - "NATIVE" + - "UNKNOWN" + type: "string" location: - type: string + type: "string" timeStamp: - type: integer + type: "integer" triggerNonce: - type: integer + type: "integer" triggerType: enum: - - MANUAL - - PERIODIC - - UNKNOWN - - UPGRADE - type: string - type: object - type: array + - "MANUAL" + - "PERIODIC" + - "UNKNOWN" + - "UPGRADE" + type: "string" + type: "object" + type: "array" triggerId: - type: string + type: "string" triggerTimestamp: - type: integer + type: "integer" triggerType: enum: - - MANUAL - - PERIODIC - - UNKNOWN - - UPGRADE - type: string - type: object + - "MANUAL" + - "PERIODIC" + - "UNKNOWN" + - "UPGRADE" + type: "string" + type: "object" startTime: - type: string + type: "string" state: enum: - - CANCELED - - CANCELLING - - CREATED - - FAILED - - FAILING - - FINISHED - - INITIALIZING - - RECONCILING - - RESTARTING - - RUNNING - - SUSPENDED - type: string + - "CANCELED" + - "CANCELLING" + - "CREATED" + - "FAILED" + - "FAILING" + - "FINISHED" + - "INITIALIZING" + - "RECONCILING" + - "RESTARTING" + - "RUNNING" + - "SUSPENDED" + type: "string" updateTime: - type: string + type: "string" upgradeSavepointPath: - type: string - type: object + type: "string" + type: "object" lifecycleState: enum: - - CREATED - - DELETED - - DELETING - - DEPLOYED - - FAILED - - ROLLED_BACK - - ROLLING_BACK - - STABLE - - SUSPENDED - - UPGRADING - type: string + - "CREATED" + - "DELETED" + - "DELETING" + - "DEPLOYED" + - "FAILED" + - "ROLLED_BACK" + - "ROLLING_BACK" + - "STABLE" + - "SUSPENDED" + - "UPGRADING" + type: "string" observedGeneration: - type: integer + type: "integer" reconciliationStatus: properties: lastReconciledSpec: - type: string + type: "string" lastStableSpec: - type: string + type: "string" reconciliationTimestamp: - type: integer + type: "integer" state: enum: - - DEPLOYED - - ROLLED_BACK - - ROLLING_BACK - - UPGRADING - type: string - type: object - type: object - type: object + - "DEPLOYED" + - "ROLLED_BACK" + - "ROLLING_BACK" + - "UPGRADING" + type: "string" + type: "object" + type: "object" + type: "object" served: true storage: true subresources: diff --git a/helm/flink-kubernetes-operator/crds/flinkstatesnapshots.flink.apache.org-v1.yml b/helm/flink-kubernetes-operator/crds/flinkstatesnapshots.flink.apache.org-v1.yml index 3f2f3475de..06a05f7e2a 100644 --- a/helm/flink-kubernetes-operator/crds/flinkstatesnapshots.flink.apache.org-v1.yml +++ b/helm/flink-kubernetes-operator/crds/flinkstatesnapshots.flink.apache.org-v1.yml @@ -1,94 +1,91 @@ # Generated by Fabric8 CRDGenerator, manual edits might get overwritten! -apiVersion: apiextensions.k8s.io/v1 -kind: CustomResourceDefinition +apiVersion: "apiextensions.k8s.io/v1" +kind: "CustomResourceDefinition" metadata: - name: flinkstatesnapshots.flink.apache.org + name: "flinkstatesnapshots.flink.apache.org" spec: - group: flink.apache.org + group: "flink.apache.org" names: - kind: FlinkStateSnapshot - plural: flinkstatesnapshots + kind: "FlinkStateSnapshot" + plural: "flinkstatesnapshots" shortNames: - - flinksnp - singular: flinkstatesnapshot - scope: Namespaced + - "flinksnp" + singular: "flinkstatesnapshot" + scope: "Namespaced" versions: - additionalPrinterColumns: - - description: Final path of the snapshot. - jsonPath: .status.path - name: Path + - jsonPath: ".status.path" + name: "Path" priority: 0 - type: string - - description: Timestamp when the snapshot was last created/failed. - jsonPath: .status.resultTimestamp - name: Result Timestamp + type: "string" + - jsonPath: ".status.resultTimestamp" + name: "Result Timestamp" priority: 0 - type: string - - description: Current state of the snapshot. - jsonPath: .status.state - name: Snapshot State + type: "string" + - jsonPath: ".status.state" + name: "Snapshot State" priority: 0 - type: string - name: v1beta1 + type: "string" + name: "v1beta1" schema: openAPIV3Schema: properties: spec: properties: backoffLimit: - type: integer + type: "integer" checkpoint: - type: object + type: "object" jobReference: properties: kind: enum: - - FlinkDeployment - - FlinkSessionJob - type: string + - "FlinkDeployment" + - "FlinkSessionJob" + type: "string" name: - type: string - type: object + type: "string" + type: "object" savepoint: properties: alreadyExists: - type: boolean + type: "boolean" disposeOnDelete: - type: boolean + type: "boolean" formatType: enum: - - CANONICAL - - NATIVE - - UNKNOWN - type: string + - "CANONICAL" + - "NATIVE" + - "UNKNOWN" + type: "string" path: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" status: properties: error: - type: string + type: "string" failures: - type: integer + type: "integer" path: - type: string + type: "string" resultTimestamp: - type: string + type: "string" state: enum: - - ABANDONED - - COMPLETED - - FAILED - - IN_PROGRESS - - TRIGGER_PENDING - type: string + - "ABANDONED" + - "COMPLETED" + - "FAILED" + - "IN_PROGRESS" + - "TRIGGER_PENDING" + type: "string" triggerId: - type: string + type: "string" triggerTimestamp: - type: string - type: object - type: object + type: "string" + type: "object" + type: "object" served: true storage: true subresources: diff --git a/pom.xml b/pom.xml index 44a1fffd26..e2724c019c 100644 --- a/pom.xml +++ b/pom.xml @@ -51,6 +51,7 @@ under the License. + flink-kubernetes-operator-bluegreen-client flink-kubernetes-standalone flink-kubernetes-operator flink-kubernetes-operator-api @@ -85,6 +86,8 @@ under the License. 2.17.0 1.20.1 + 5.8.2 + 1.7.36 2.23.1