T checkNotNull(T object) {
if (object == null) {
throw new NullPointerException();
diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/FlinkOperator.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/FlinkOperator.java
index 3e21ab456d..96473ddb8f 100644
--- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/FlinkOperator.java
+++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/FlinkOperator.java
@@ -30,6 +30,7 @@
import org.apache.flink.kubernetes.operator.config.FlinkConfigManager;
import org.apache.flink.kubernetes.operator.config.FlinkOperatorConfiguration;
import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions;
+import org.apache.flink.kubernetes.operator.controller.FlinkBlueGreenDeploymentController;
import org.apache.flink.kubernetes.operator.controller.FlinkDeploymentController;
import org.apache.flink.kubernetes.operator.controller.FlinkSessionJobController;
import org.apache.flink.kubernetes.operator.controller.FlinkStateSnapshotController;
@@ -254,6 +255,12 @@ void registerSnapshotController() {
registeredControllers.add(operator.register(controller, this::overrideControllerConfigs));
}
+ @VisibleForTesting
+ void registerBlueGreenController() {
+ var controller = new FlinkBlueGreenDeploymentController(ctxFactory);
+ registeredControllers.add(operator.register(controller, this::overrideControllerConfigs));
+ }
+
private void overrideControllerConfigs(ControllerConfigurationOverrider> overrider) {
var operatorConf = configManager.getOperatorConfiguration();
var watchNamespaces = operatorConf.getWatchedNamespaces();
@@ -274,6 +281,7 @@ public void run() {
registerDeploymentController();
registerSessionJobController();
registerSnapshotController();
+ registerBlueGreenController();
operator.installShutdownHook(
baseConfig.get(KubernetesOperatorConfigOptions.OPERATOR_TERMINATION_TIMEOUT));
operator.start();
diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/BlueGreenDiffType.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/BlueGreenDiffType.java
new file mode 100644
index 0000000000..5581493f7e
--- /dev/null
+++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/BlueGreenDiffType.java
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/** Enum representing different types of differences in Blue/Green deployment specifications. */
+public enum BlueGreenDiffType {
+ /** No changes detected, should be ignored. */
+ IGNORE,
+
+ /** Changes that require a Blue/Green transition. */
+ TRANSITION,
+
+ /** Changes that only affect the child FlinkDeploymentSpec. */
+ PATCH_CHILD
+}
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
new file mode 100644
index 0000000000..a35ccb2b1b
--- /dev/null
+++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentController.java
@@ -0,0 +1,142 @@
+/*
+ * 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.kubernetes.operator.api.FlinkBlueGreenDeployment;
+import org.apache.flink.kubernetes.operator.api.FlinkDeployment;
+import org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentState;
+import org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentStatus;
+import org.apache.flink.kubernetes.operator.controller.bluegreen.BlueGreenContext;
+import org.apache.flink.kubernetes.operator.controller.bluegreen.BlueGreenDeploymentService;
+import org.apache.flink.kubernetes.operator.controller.bluegreen.BlueGreenStateHandlerRegistry;
+import org.apache.flink.kubernetes.operator.controller.bluegreen.handlers.BlueGreenStateHandler;
+import org.apache.flink.kubernetes.operator.service.FlinkResourceContextFactory;
+
+import io.javaoperatorsdk.operator.api.config.informer.InformerEventSourceConfiguration;
+import io.javaoperatorsdk.operator.api.reconciler.Context;
+import io.javaoperatorsdk.operator.api.reconciler.ControllerConfiguration;
+import io.javaoperatorsdk.operator.api.reconciler.EventSourceContext;
+import io.javaoperatorsdk.operator.api.reconciler.Reconciler;
+import io.javaoperatorsdk.operator.api.reconciler.UpdateControl;
+import io.javaoperatorsdk.operator.processing.event.source.EventSource;
+import io.javaoperatorsdk.operator.processing.event.source.informer.InformerEventSource;
+import io.javaoperatorsdk.operator.processing.event.source.informer.Mappers;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentState.INITIALIZING_BLUE;
+
+/**
+ * Controller that runs the main reconcile loop for Flink Blue/Green deployments.
+ *
+ * State Machine Flow
+ *
+ *
Deployment States
+ *
+ *
1. INITIALIZING_BLUE - First-time deployment setup 2. ACTIVE_BLUE - Blue environment serving
+ * traffic, monitoring for updates 3. TRANSITIONING_TO_GREEN - Deploying Green environment while
+ * Blue serves traffic 4. ACTIVE_GREEN - Green environment serving traffic, monitoring for updates
+ * 5. TRANSITIONING_TO_BLUE - Deploying Blue environment while Green serves traffic
+ *
+ *
Orchestration Process
+ *
+ *
FlinkBlueGreenDeploymentController.reconcile() 1. Create BlueGreenContext with current
+ * deployment state 2. Query StateHandlerRegistry for appropriate handler 3. Delegate to specific
+ * StateHandler.handle(context) 4. StateHandler invokes BlueGreenDeploymentService operations 5.
+ * Return UpdateControl with next reconciliation schedule
+ */
+@ControllerConfiguration
+public class FlinkBlueGreenDeploymentController implements Reconciler {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkDeploymentController.class);
+
+ private final FlinkResourceContextFactory ctxFactory;
+ private final BlueGreenStateHandlerRegistry handlerRegistry;
+
+ public FlinkBlueGreenDeploymentController(FlinkResourceContextFactory ctxFactory) {
+ this.ctxFactory = ctxFactory;
+ this.handlerRegistry = new BlueGreenStateHandlerRegistry();
+ }
+
+ @Override
+ public List> prepareEventSources(
+ EventSourceContext context) {
+ List> eventSources = new ArrayList<>();
+
+ InformerEventSourceConfiguration config =
+ InformerEventSourceConfiguration.from(
+ FlinkDeployment.class, FlinkBlueGreenDeployment.class)
+ .withSecondaryToPrimaryMapper(
+ Mappers.fromOwnerReferences(context.getPrimaryResourceClass()))
+ .withNamespacesInheritedFromController()
+ .withFollowControllerNamespacesChanges(true)
+ .build();
+
+ eventSources.add(new InformerEventSource<>(config, context));
+
+ return eventSources;
+ }
+
+ @Override
+ public UpdateControl reconcile(
+ FlinkBlueGreenDeployment bgDeployment, Context josdkContext)
+ throws Exception {
+
+ FlinkBlueGreenDeploymentStatus deploymentStatus = bgDeployment.getStatus();
+
+ if (deploymentStatus == null) {
+ var context =
+ new BlueGreenContext(
+ bgDeployment,
+ new FlinkBlueGreenDeploymentStatus(),
+ josdkContext,
+ null,
+ ctxFactory);
+ return BlueGreenDeploymentService.patchStatusUpdateControl(
+ context, INITIALIZING_BLUE, null, null)
+ .rescheduleAfter(0);
+ } else {
+ FlinkBlueGreenDeploymentState currentState = deploymentStatus.getBlueGreenState();
+ var context =
+ new BlueGreenContext(
+ bgDeployment,
+ deploymentStatus,
+ josdkContext,
+ currentState == INITIALIZING_BLUE
+ ? null
+ : FlinkBlueGreenDeployments.fromSecondaryResources(
+ josdkContext),
+ ctxFactory);
+
+ LOG.debug(
+ "Processing state: {} for deployment: {}",
+ currentState,
+ context.getDeploymentName());
+
+ BlueGreenStateHandler handler = handlerRegistry.getHandler(currentState);
+ return handler.handle(context);
+ }
+ }
+
+ public static void logAndThrow(String message) {
+ throw new RuntimeException(message);
+ }
+}
diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeployments.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeployments.java
new file mode 100644
index 0000000000..9f074df43f
--- /dev/null
+++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeployments.java
@@ -0,0 +1,81 @@
+/*
+ * 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.kubernetes.operator.api.FlinkBlueGreenDeployment;
+import org.apache.flink.kubernetes.operator.api.FlinkDeployment;
+import org.apache.flink.kubernetes.operator.api.bluegreen.BlueGreenDeploymentType;
+
+import io.javaoperatorsdk.operator.api.reconciler.Context;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+
+import java.util.Set;
+
+/** Utility to handle A/B deployments. */
+@Data
+@NoArgsConstructor
+public class FlinkBlueGreenDeployments {
+ private FlinkDeployment flinkDeploymentBlue;
+ private FlinkDeployment flinkDeploymentGreen;
+
+ public int getNumberOfDeployments() {
+ var counter = 0;
+ if (flinkDeploymentBlue != null) {
+ counter++;
+ }
+ if (flinkDeploymentGreen != null) {
+ counter++;
+ }
+ return counter;
+ }
+
+ static FlinkBlueGreenDeployments fromSecondaryResources(
+ Context context) {
+ Set secondaryResources =
+ context.getSecondaryResources(FlinkDeployment.class);
+
+ if (secondaryResources.isEmpty() || secondaryResources.size() > 2) {
+ FlinkBlueGreenDeploymentController.logAndThrow(
+ "Unexpected number of dependent deployments: " + secondaryResources.size());
+ }
+
+ FlinkBlueGreenDeployments flinkBlueGreenDeployments = new FlinkBlueGreenDeployments();
+
+ for (FlinkDeployment dependentDeployment : secondaryResources) {
+ var flinkBlueGreenDeploymentType =
+ BlueGreenDeploymentType.fromDeployment(dependentDeployment);
+
+ if (flinkBlueGreenDeploymentType == BlueGreenDeploymentType.BLUE) {
+ if (flinkBlueGreenDeployments.getFlinkDeploymentBlue() != null) {
+ FlinkBlueGreenDeploymentController.logAndThrow(
+ "Detected multiple Dependent Deployments of type BLUE");
+ }
+ flinkBlueGreenDeployments.setFlinkDeploymentBlue(dependentDeployment);
+ } else {
+ if (flinkBlueGreenDeployments.getFlinkDeploymentGreen() != null) {
+ FlinkBlueGreenDeploymentController.logAndThrow(
+ "Detected multiple Dependent Deployments of type GREEN");
+ }
+ flinkBlueGreenDeployments.setFlinkDeploymentGreen(dependentDeployment);
+ }
+ }
+
+ return flinkBlueGreenDeployments;
+ }
+}
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
new file mode 100644
index 0000000000..054b121f61
--- /dev/null
+++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/BlueGreenContext.java
@@ -0,0 +1,65 @@
+/*
+ * 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.bluegreen;
+
+import org.apache.flink.kubernetes.operator.api.FlinkBlueGreenDeployment;
+import org.apache.flink.kubernetes.operator.api.FlinkDeployment;
+import org.apache.flink.kubernetes.operator.api.bluegreen.BlueGreenDeploymentType;
+import org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentStatus;
+import org.apache.flink.kubernetes.operator.controller.FlinkBlueGreenDeployments;
+import org.apache.flink.kubernetes.operator.service.FlinkResourceContextFactory;
+
+import io.javaoperatorsdk.operator.api.reconciler.Context;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+
+/**
+ * Simplified context object containing all the necessary state and dependencies for Blue/Green
+ * deployment state transitions.
+ */
+@Getter
+@RequiredArgsConstructor
+public class BlueGreenContext {
+ private final FlinkBlueGreenDeployment bgDeployment;
+ private final FlinkBlueGreenDeploymentStatus deploymentStatus;
+ private final Context josdkContext;
+ private final FlinkBlueGreenDeployments deployments;
+ private final FlinkResourceContextFactory ctxFactory;
+
+ public String getDeploymentName() {
+ return bgDeployment.getMetadata().getName();
+ }
+
+ public FlinkDeployment getBlueDeployment() {
+ return deployments != null ? deployments.getFlinkDeploymentBlue() : null;
+ }
+
+ public FlinkDeployment getGreenDeployment() {
+ return deployments != null ? deployments.getFlinkDeploymentGreen() : null;
+ }
+
+ public FlinkDeployment getDeploymentByType(BlueGreenDeploymentType type) {
+ return type == BlueGreenDeploymentType.BLUE ? getBlueDeployment() : getGreenDeployment();
+ }
+
+ public BlueGreenDeploymentType getOppositeDeploymentType(BlueGreenDeploymentType type) {
+ return type == BlueGreenDeploymentType.BLUE
+ ? BlueGreenDeploymentType.GREEN
+ : BlueGreenDeploymentType.BLUE;
+ }
+}
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
new file mode 100644
index 0000000000..85de365b97
--- /dev/null
+++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/BlueGreenDeploymentService.java
@@ -0,0 +1,659 @@
+/*
+ * 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.bluegreen;
+
+import org.apache.flink.api.common.JobStatus;
+import org.apache.flink.kubernetes.operator.api.FlinkBlueGreenDeployment;
+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.BlueGreenDiffType;
+import org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentState;
+import org.apache.flink.kubernetes.operator.api.status.Savepoint;
+import org.apache.flink.kubernetes.operator.api.status.SavepointFormatType;
+import org.apache.flink.kubernetes.operator.api.status.SnapshotTriggerType;
+import org.apache.flink.kubernetes.operator.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.BlueGreenUtils;
+import org.apache.flink.util.Preconditions;
+
+import io.fabric8.kubernetes.api.model.ObjectMeta;
+import io.javaoperatorsdk.operator.api.reconciler.UpdateControl;
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.time.Instant;
+
+import static org.apache.flink.kubernetes.operator.controller.bluegreen.BlueGreenKubernetesService.deleteFlinkDeployment;
+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.BlueGreenUtils.fetchSavepointInfo;
+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;
+import static org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenUtils.instantStrToMillis;
+import static org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenUtils.isSavepointRequired;
+import static org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenUtils.millisToInstantStr;
+import static org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenUtils.prepareFlinkDeployment;
+import static org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenUtils.setLastReconciledSpec;
+import static org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenUtils.triggerSavepoint;
+
+/** Consolidated service for all Blue/Green deployment operations. */
+public class BlueGreenDeploymentService {
+
+ private static final Logger LOG = LoggerFactory.getLogger(BlueGreenDeploymentService.class);
+ private static final long RETRY_DELAY_MS = 500;
+
+ // ==================== Deployment Initiation Methods ====================
+
+ /**
+ * Initiates a new Blue/Green deployment.
+ *
+ * @param context the transition context
+ * @param nextBlueGreenDeploymentType the type of deployment to create
+ * @param nextState the next state to transition to
+ * @param lastCheckpoint the checkpoint to restore from (can be null)
+ * @param isFirstDeployment whether this is the first deployment
+ * @return UpdateControl for the deployment
+ */
+ public UpdateControl initiateDeployment(
+ BlueGreenContext context,
+ BlueGreenDeploymentType nextBlueGreenDeploymentType,
+ FlinkBlueGreenDeploymentState nextState,
+ Savepoint lastCheckpoint,
+ boolean isFirstDeployment) {
+ ObjectMeta bgMeta = context.getBgDeployment().getMetadata();
+
+ FlinkDeployment flinkDeployment =
+ prepareFlinkDeployment(
+ context,
+ nextBlueGreenDeploymentType,
+ lastCheckpoint,
+ isFirstDeployment,
+ bgMeta);
+
+ deployCluster(context, flinkDeployment);
+
+ BlueGreenUtils.setAbortTimestamp(context);
+
+ return patchStatusUpdateControl(context, nextState, JobStatus.RECONCILING, null)
+ .rescheduleAfter(BlueGreenUtils.getReconciliationReschedInterval(context));
+ }
+
+ /**
+ * Checks if a full transition can be initiated and initiates it if conditions are met.
+ *
+ * @param context the transition context
+ * @param currentBlueGreenDeploymentType the current deployment type
+ * @return UpdateControl for the deployment
+ */
+ public UpdateControl checkAndInitiateDeployment(
+ BlueGreenContext context, BlueGreenDeploymentType currentBlueGreenDeploymentType) {
+ BlueGreenDiffType specDiff = getSpecDiff(context);
+
+ if (specDiff != BlueGreenDiffType.IGNORE) {
+ FlinkDeployment currentFlinkDeployment =
+ context.getDeploymentByType(currentBlueGreenDeploymentType);
+
+ if (isFlinkDeploymentReady(currentFlinkDeployment)) {
+ if (specDiff == BlueGreenDiffType.TRANSITION) {
+ boolean savepointTriggered = false;
+ try {
+ savepointTriggered = handleSavepoint(context, currentFlinkDeployment);
+ } catch (Exception e) {
+ var error = "Could not trigger Savepoint. Details: " + e.getMessage();
+ return markDeploymentFailing(context, error);
+ }
+
+ if (savepointTriggered) {
+ // Spec is intentionally not marked as reconciled here to allow
+ // reprocessing the TRANSITION once savepoint creation completes
+ var savepointingState =
+ calculateSavepointingState(currentBlueGreenDeploymentType);
+ return patchStatusUpdateControl(context, savepointingState, null, null)
+ .rescheduleAfter(getReconciliationReschedInterval(context));
+ }
+
+ setLastReconciledSpec(context);
+ try {
+ return startTransition(
+ context, currentBlueGreenDeploymentType, currentFlinkDeployment);
+ } catch (Exception e) {
+ var error = "Could not start Transition. Details: " + e.getMessage();
+ context.getDeploymentStatus().setSavepointTriggerId(null);
+ return markDeploymentFailing(context, error);
+ }
+ } else {
+ setLastReconciledSpec(context);
+ LOG.info(
+ "Patching FlinkDeployment '{}' during checkAndInitiateDeployment",
+ currentFlinkDeployment.getMetadata().getName());
+ return patchFlinkDeployment(context, currentBlueGreenDeploymentType);
+ }
+ } else {
+ if (context.getDeploymentStatus().getJobStatus().getState() != JobStatus.FAILING) {
+ setLastReconciledSpec(context);
+ var error =
+ String.format(
+ "Transition to %s not possible, current Flink Deployment '%s' is not READY. FAILING '%s'",
+ calculateTransition(currentBlueGreenDeploymentType)
+ .nextBlueGreenDeploymentType,
+ currentFlinkDeployment.getMetadata().getName(),
+ context.getBgDeployment().getMetadata().getName());
+ return markDeploymentFailing(context, error);
+ }
+ }
+ }
+
+ return UpdateControl.noUpdate();
+ }
+
+ private UpdateControl patchFlinkDeployment(
+ BlueGreenContext context, BlueGreenDeploymentType blueGreenDeploymentTypeToPatch) {
+
+ String childDeploymentName =
+ context.getBgDeployment().getMetadata().getName()
+ + "-"
+ + blueGreenDeploymentTypeToPatch.toString().toLowerCase();
+
+ // We want to patch, therefore the transition should point to the existing deployment
+ // details
+ var patchingState = calculatePatchingState(blueGreenDeploymentTypeToPatch);
+
+ // If we're not transitioning between deployments, mark as a single deployment to have it
+ // not wait for synchronization
+ var isFirstDeployment = context.getDeployments().getNumberOfDeployments() != 2;
+
+ // TODO: if the resource failed right after being deployed with an initialSavepointPath,
+ // will it be used by this patching? otherwise this is unnecessary, keep lastSavepoint =
+ // null.
+ Savepoint lastSavepoint =
+ carryOverSavepoint(context, blueGreenDeploymentTypeToPatch, childDeploymentName);
+
+ return initiateDeployment(
+ context,
+ blueGreenDeploymentTypeToPatch,
+ patchingState,
+ lastSavepoint,
+ isFirstDeployment);
+ }
+
+ @Nullable
+ private static Savepoint carryOverSavepoint(
+ BlueGreenContext context,
+ BlueGreenDeploymentType blueGreenDeploymentTypeToPatch,
+ String childDeploymentName) {
+ var deploymentToPatch = context.getDeploymentByType(blueGreenDeploymentTypeToPatch);
+ var initialSavepointPath = deploymentToPatch.getSpec().getJob().getInitialSavepointPath();
+
+ if (initialSavepointPath == null || initialSavepointPath.isEmpty()) {
+ initialSavepointPath =
+ deploymentToPatch.getStatus().getJobStatus().getUpgradeSavepointPath();
+ }
+
+ Savepoint lastSavepoint = null;
+ if (initialSavepointPath != null && !initialSavepointPath.isEmpty()) {
+ var ctx =
+ context.getCtxFactory()
+ .getResourceContext(deploymentToPatch, context.getJosdkContext());
+
+ lastSavepoint = getSavepointObject(ctx, initialSavepointPath);
+
+ LOG.info(
+ "Patching FlinkDeployment '{}', carrying over Savepoint at: '{}'",
+ childDeploymentName,
+ initialSavepointPath);
+ } else {
+ LOG.info("Patching FlinkDeployment '{}'", childDeploymentName);
+ }
+
+ return lastSavepoint;
+ }
+
+ private UpdateControl startTransition(
+ BlueGreenContext context,
+ BlueGreenDeploymentType currentBlueGreenDeploymentType,
+ FlinkDeployment currentFlinkDeployment) {
+ DeploymentTransition transition = calculateTransition(currentBlueGreenDeploymentType);
+
+ Savepoint lastCheckpoint = configureInitialSavepoint(context, currentFlinkDeployment);
+
+ return initiateDeployment(
+ context,
+ transition.nextBlueGreenDeploymentType,
+ transition.nextState,
+ lastCheckpoint,
+ false);
+ }
+
+ private DeploymentTransition calculateTransition(BlueGreenDeploymentType currentType) {
+ if (BlueGreenDeploymentType.BLUE == currentType) {
+ return new DeploymentTransition(
+ BlueGreenDeploymentType.GREEN,
+ FlinkBlueGreenDeploymentState.TRANSITIONING_TO_GREEN);
+ } else {
+ return new DeploymentTransition(
+ BlueGreenDeploymentType.BLUE,
+ FlinkBlueGreenDeploymentState.TRANSITIONING_TO_BLUE);
+ }
+ }
+
+ private FlinkBlueGreenDeploymentState calculatePatchingState(
+ BlueGreenDeploymentType currentType) {
+ if (BlueGreenDeploymentType.BLUE == currentType) {
+ return FlinkBlueGreenDeploymentState.TRANSITIONING_TO_BLUE;
+ } else {
+ return FlinkBlueGreenDeploymentState.TRANSITIONING_TO_GREEN;
+ }
+ }
+
+ // ==================== Savepointing Methods ====================
+
+ public boolean monitorSavepoint(
+ BlueGreenContext context, BlueGreenDeploymentType currentBlueGreenDeploymentType) {
+
+ FlinkResourceContext ctx =
+ context.getCtxFactory()
+ .getResourceContext(
+ context.getDeploymentByType(currentBlueGreenDeploymentType),
+ context.getJosdkContext());
+
+ String savepointTriggerId = context.getDeploymentStatus().getSavepointTriggerId();
+ var savepointFetchResult = fetchSavepointInfo(ctx, savepointTriggerId);
+
+ return !savepointFetchResult.isPending();
+ }
+
+ private Savepoint configureInitialSavepoint(
+ BlueGreenContext context, FlinkDeployment currentFlinkDeployment) {
+ // Create savepoint for all upgrade modes except STATELESS
+ // (originally only SAVEPOINT mode required savepoints)
+ if (isSavepointRequired(context)) {
+ FlinkResourceContext ctx =
+ context.getCtxFactory()
+ .getResourceContext(currentFlinkDeployment, context.getJosdkContext());
+
+ String triggerId = context.getDeploymentStatus().getSavepointTriggerId();
+ var savepointFetchResult = fetchSavepointInfo(ctx, triggerId);
+
+ if (savepointFetchResult.getError() != null
+ && !savepointFetchResult.getError().isEmpty()) {
+ throw new RuntimeException(
+ String.format(
+ "Could not fetch savepoint with triggerId: %s. Error: %s",
+ triggerId, savepointFetchResult.getError()));
+ }
+
+ return getSavepointObject(ctx, savepointFetchResult.getLocation());
+ }
+
+ // Currently not using last checkpoint recovery for LAST_STATE upgrade mode
+ // This could be re-enabled in the future by uncommenting the logic below
+ return null;
+
+ // if (!lookForCheckpoint(context)) {
+ // return null;
+ // }
+ //
+ // return getLastCheckpoint(ctx);
+ }
+
+ @NotNull
+ private static Savepoint getSavepointObject(
+ FlinkResourceContext ctx, String savepointLocation) {
+ org.apache.flink.core.execution.SavepointFormatType coreSavepointFormatType =
+ ctx.getObserveConfig()
+ .get(KubernetesOperatorConfigOptions.OPERATOR_SAVEPOINT_FORMAT_TYPE);
+
+ var savepointFormatType = SavepointFormatType.valueOf(coreSavepointFormatType.toString());
+
+ return Savepoint.of(savepointLocation, SnapshotTriggerType.MANUAL, savepointFormatType);
+ }
+
+ private boolean handleSavepoint(
+ BlueGreenContext context, FlinkDeployment currentFlinkDeployment) throws Exception {
+
+ if (!isSavepointRequired(context)) {
+ return false;
+ }
+
+ FlinkResourceContext ctx =
+ context.getCtxFactory()
+ .getResourceContext(currentFlinkDeployment, context.getJosdkContext());
+
+ String savepointTriggerId = context.getDeploymentStatus().getSavepointTriggerId();
+
+ if (savepointTriggerId == null || savepointTriggerId.isEmpty()) {
+ String triggerId = triggerSavepoint(ctx);
+ LOG.info("Savepoint requested (triggerId: {}", triggerId);
+ context.getDeploymentStatus().setSavepointTriggerId(triggerId);
+ return true;
+ }
+
+ LOG.info("Savepoint previously requested (triggerId: {})", savepointTriggerId);
+ return false;
+ }
+
+ private FlinkBlueGreenDeploymentState calculateSavepointingState(
+ BlueGreenDeploymentType currentType) {
+ if (BlueGreenDeploymentType.BLUE == currentType) {
+ return FlinkBlueGreenDeploymentState.SAVEPOINTING_BLUE;
+ } else {
+ return FlinkBlueGreenDeploymentState.SAVEPOINTING_GREEN;
+ }
+ }
+
+ // ==================== Transition Monitoring Methods ====================
+
+ /**
+ * Monitors an ongoing Blue/Green deployment transition.
+ *
+ * @param context the transition context
+ * @param currentBlueGreenDeploymentType the current deployment type being transitioned from
+ * @return UpdateControl for the transition
+ */
+ public UpdateControl monitorTransition(
+ BlueGreenContext context, BlueGreenDeploymentType currentBlueGreenDeploymentType) {
+
+ var updateControl =
+ handleSpecChangesDuringTransition(context, currentBlueGreenDeploymentType);
+
+ if (updateControl != null) {
+ return updateControl;
+ }
+
+ TransitionState transitionState =
+ determineTransitionState(context, currentBlueGreenDeploymentType);
+
+ if (isFlinkDeploymentReady(transitionState.nextDeployment)) {
+ return shouldWeDelete(
+ context,
+ transitionState.currentDeployment,
+ transitionState.nextDeployment,
+ transitionState.nextState);
+ } else {
+ return shouldWeAbort(
+ context, transitionState.nextDeployment, transitionState.nextState);
+ }
+ }
+
+ private UpdateControl handleSpecChangesDuringTransition(
+ BlueGreenContext context, BlueGreenDeploymentType currentBlueGreenDeploymentType) {
+ if (hasSpecChanged(context)) {
+ BlueGreenDiffType diffType = getSpecDiff(context);
+
+ if (diffType != BlueGreenDiffType.IGNORE) {
+ setLastReconciledSpec(context);
+ var oppositeDeploymentType =
+ context.getOppositeDeploymentType(currentBlueGreenDeploymentType);
+ LOG.info(
+ "Patching FlinkDeployment '{}' during handleSpecChangesDuringTransition",
+ context.getDeploymentByType(oppositeDeploymentType)
+ .getMetadata()
+ .getName());
+ return patchFlinkDeployment(context, oppositeDeploymentType);
+ }
+ }
+
+ return null;
+ }
+
+ private TransitionState determineTransitionState(
+ BlueGreenContext context, BlueGreenDeploymentType currentBlueGreenDeploymentType) {
+ TransitionState transitionState;
+
+ if (BlueGreenDeploymentType.BLUE == currentBlueGreenDeploymentType) {
+ transitionState =
+ new TransitionState(
+ context.getBlueDeployment(), // currentDeployment
+ context.getGreenDeployment(), // nextDeployment
+ FlinkBlueGreenDeploymentState.ACTIVE_GREEN); // next State
+ } else {
+ transitionState =
+ new TransitionState(
+ context.getGreenDeployment(), // currentDeployment
+ context.getBlueDeployment(), // nextDeployment
+ FlinkBlueGreenDeploymentState.ACTIVE_BLUE); // next State
+ }
+
+ Preconditions.checkNotNull(
+ transitionState.nextDeployment,
+ "Target Dependent Deployment resource not found. Blue/Green deployment name: "
+ + context.getDeploymentName()
+ + ", current deployment type: "
+ + currentBlueGreenDeploymentType);
+
+ return transitionState;
+ }
+
+ // ==================== Deployment Deletion Methods ====================
+
+ private UpdateControl shouldWeDelete(
+ BlueGreenContext context,
+ FlinkDeployment currentDeployment,
+ FlinkDeployment nextDeployment,
+ FlinkBlueGreenDeploymentState nextState) {
+
+ var deploymentStatus = context.getDeploymentStatus();
+
+ if (currentDeployment == null) {
+ deploymentStatus.setDeploymentReadyTimestamp(Instant.now().toString());
+ return finalizeBlueGreenDeployment(context, nextState);
+ }
+
+ long deploymentDeletionDelayMs = BlueGreenUtils.getDeploymentDeletionDelay(context);
+ long deploymentReadyTimestamp =
+ instantStrToMillis(deploymentStatus.getDeploymentReadyTimestamp());
+
+ if (deploymentReadyTimestamp == 0) {
+ LOG.info(
+ "FlinkDeployment '{}' marked ready, rescheduling reconciliation in {} seconds.",
+ nextDeployment.getMetadata().getName(),
+ deploymentDeletionDelayMs / 1000);
+
+ deploymentStatus.setDeploymentReadyTimestamp(Instant.now().toString());
+ return patchStatusUpdateControl(context, null, null, null)
+ .rescheduleAfter(deploymentDeletionDelayMs);
+ }
+
+ long deletionTimestamp = deploymentReadyTimestamp + deploymentDeletionDelayMs;
+
+ if (deletionTimestamp < System.currentTimeMillis()) {
+ return deleteDeployment(currentDeployment, context);
+ } else {
+ return waitBeforeDeleting(currentDeployment, deletionTimestamp);
+ }
+ }
+
+ private UpdateControl waitBeforeDeleting(
+ FlinkDeployment currentDeployment, long deletionTimestamp) {
+
+ long delay = deletionTimestamp - System.currentTimeMillis();
+ LOG.info(
+ "Awaiting deletion delay for FlinkDeployment '{}', rescheduling reconciliation in {} seconds.",
+ currentDeployment.getMetadata().getName(),
+ delay / 1000);
+
+ return UpdateControl.noUpdate().rescheduleAfter(delay);
+ }
+
+ private UpdateControl deleteDeployment(
+ FlinkDeployment currentDeployment, BlueGreenContext context) {
+
+ boolean deleted = deleteFlinkDeployment(currentDeployment, context);
+
+ if (!deleted) {
+ LOG.info("FlinkDeployment '{}' not deleted, will retry", currentDeployment);
+ } else {
+ LOG.info("FlinkDeployment '{}' deleted!", currentDeployment);
+ }
+
+ return UpdateControl.noUpdate().rescheduleAfter(RETRY_DELAY_MS);
+ }
+
+ // ==================== Abort and Retry Methods ====================
+
+ private UpdateControl shouldWeAbort(
+ BlueGreenContext context,
+ FlinkDeployment nextDeployment,
+ FlinkBlueGreenDeploymentState nextState) {
+
+ String deploymentName = nextDeployment.getMetadata().getName();
+ long abortTimestamp = instantStrToMillis(context.getDeploymentStatus().getAbortTimestamp());
+
+ if (abortTimestamp == 0) {
+ throw new IllegalStateException("Unexpected abortTimestamp == 0");
+ }
+
+ if (abortTimestamp < System.currentTimeMillis()) {
+ return abortDeployment(context, nextDeployment, nextState, deploymentName);
+ } else {
+ return retryDeployment(context, deploymentName);
+ }
+ }
+
+ private UpdateControl retryDeployment(
+ BlueGreenContext context, String deploymentName) {
+
+ long delay = getReconciliationReschedInterval(context);
+
+ LOG.info(
+ "FlinkDeployment '{}' not ready yet, retrying in {} seconds.",
+ deploymentName,
+ delay / 1000);
+
+ return patchStatusUpdateControl(context, null, null, null).rescheduleAfter(delay);
+ }
+
+ private UpdateControl abortDeployment(
+ BlueGreenContext context,
+ FlinkDeployment nextDeployment,
+ FlinkBlueGreenDeploymentState nextState,
+ String deploymentName) {
+
+ suspendFlinkDeployment(context, nextDeployment);
+
+ FlinkBlueGreenDeploymentState previousState =
+ getPreviousState(nextState, context.getDeployments());
+ context.getDeploymentStatus().setBlueGreenState(previousState);
+
+ var error =
+ String.format(
+ "Aborting deployment '%s', rolling B/G deployment back to %s",
+ deploymentName, previousState);
+ return markDeploymentFailing(context, error);
+ }
+
+ @NotNull
+ private static UpdateControl markDeploymentFailing(
+ BlueGreenContext context, String error) {
+ LOG.error(error);
+ return patchStatusUpdateControl(context, null, JobStatus.FAILING, error);
+ }
+
+ private static FlinkBlueGreenDeploymentState getPreviousState(
+ FlinkBlueGreenDeploymentState nextState, FlinkBlueGreenDeployments deployments) {
+ FlinkBlueGreenDeploymentState previousState;
+ if (deployments.getNumberOfDeployments() == 1) {
+ previousState = FlinkBlueGreenDeploymentState.INITIALIZING_BLUE;
+ } else if (deployments.getNumberOfDeployments() == 2) {
+ previousState =
+ nextState == FlinkBlueGreenDeploymentState.ACTIVE_BLUE
+ ? FlinkBlueGreenDeploymentState.ACTIVE_GREEN
+ : FlinkBlueGreenDeploymentState.ACTIVE_BLUE;
+ } else {
+ throw new IllegalStateException("No blue/green FlinkDeployments found!");
+ }
+ return previousState;
+ }
+
+ // ==================== Finalization Methods ====================
+
+ /**
+ * Finalizes a Blue/Green deployment transition.
+ *
+ * @param context the transition context
+ * @param nextState the next state to transition to
+ * @return UpdateControl for finalization
+ */
+ public UpdateControl finalizeBlueGreenDeployment(
+ BlueGreenContext context, FlinkBlueGreenDeploymentState nextState) {
+
+ LOG.info("Finalizing deployment '{}' to {} state", context.getDeploymentName(), nextState);
+
+ context.getDeploymentStatus().setDeploymentReadyTimestamp(millisToInstantStr(0));
+ context.getDeploymentStatus().setAbortTimestamp(millisToInstantStr(0));
+ context.getDeploymentStatus().setSavepointTriggerId(null);
+
+ return patchStatusUpdateControl(context, nextState, JobStatus.RUNNING, null);
+ }
+
+ // ==================== Common Utility Methods ====================
+
+ public static UpdateControl patchStatusUpdateControl(
+ BlueGreenContext context,
+ FlinkBlueGreenDeploymentState deploymentState,
+ JobStatus jobState,
+ String error) {
+
+ var deploymentStatus = context.getDeploymentStatus();
+ var flinkBlueGreenDeployment = context.getBgDeployment();
+
+ if (deploymentState != null) {
+ deploymentStatus.setBlueGreenState(deploymentState);
+ }
+
+ if (jobState != null) {
+ deploymentStatus.getJobStatus().setState(jobState);
+ }
+
+ if (jobState == JobStatus.FAILING) {
+ deploymentStatus.setError(error);
+ }
+
+ if (jobState == JobStatus.RECONCILING || jobState == JobStatus.RUNNING) {
+ deploymentStatus.setError(null);
+ }
+
+ deploymentStatus.setLastReconciledTimestamp(java.time.Instant.now().toString());
+ flinkBlueGreenDeployment.setStatus(deploymentStatus);
+ return UpdateControl.patchStatus(flinkBlueGreenDeployment);
+ }
+
+ // ==================== DTO/Result Classes ====================
+
+ @Getter
+ @AllArgsConstructor
+ private static class DeploymentTransition {
+ final BlueGreenDeploymentType nextBlueGreenDeploymentType;
+ final FlinkBlueGreenDeploymentState nextState;
+ }
+
+ @Getter
+ @AllArgsConstructor
+ private static class TransitionState {
+ final FlinkDeployment currentDeployment;
+ final FlinkDeployment nextDeployment;
+ final FlinkBlueGreenDeploymentState nextState;
+ }
+}
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
new file mode 100644
index 0000000000..ae7492d312
--- /dev/null
+++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/BlueGreenKubernetesService.java
@@ -0,0 +1,119 @@
+/*
+ * 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.bluegreen;
+
+import org.apache.flink.kubernetes.operator.api.FlinkBlueGreenDeployment;
+import org.apache.flink.kubernetes.operator.api.FlinkDeployment;
+import org.apache.flink.kubernetes.operator.api.lifecycle.ResourceLifecycleState;
+import org.apache.flink.kubernetes.operator.api.spec.JobState;
+import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils;
+
+import io.fabric8.kubernetes.api.model.ObjectMeta;
+import io.fabric8.kubernetes.api.model.OwnerReference;
+import io.fabric8.kubernetes.api.model.StatusDetails;
+
+import java.util.List;
+
+/** Utility methods for handling Kubernetes operations in Blue/Green deployments. */
+public class BlueGreenKubernetesService {
+
+ /**
+ * Creates ObjectMeta for a dependent Kubernetes resource with proper owner references.
+ *
+ * @param bgDeployment the parent Blue/Green deployment
+ * @return ObjectMeta configured with namespace and owner references
+ */
+ public static ObjectMeta getDependentObjectMeta(FlinkBlueGreenDeployment bgDeployment) {
+ ObjectMeta bgMeta = bgDeployment.getMetadata();
+ ObjectMeta objectMeta = new ObjectMeta();
+ objectMeta.setNamespace(bgMeta.getNamespace());
+ objectMeta.setOwnerReferences(
+ List.of(
+ new OwnerReference(
+ bgDeployment.getApiVersion(),
+ true,
+ false,
+ bgDeployment.getKind(),
+ bgMeta.getName(),
+ bgMeta.getUid())));
+ return objectMeta;
+ }
+
+ public static void deployCluster(BlueGreenContext context, FlinkDeployment flinkDeployment) {
+ // Deploy
+ context.getJosdkContext().getClient().resource(flinkDeployment).createOrReplace();
+ }
+
+ /**
+ * Checks if a FlinkDeployment is ready (STABLE lifecycle state and RUNNING job status).
+ *
+ * @param deployment the FlinkDeployment to check
+ * @return true if the deployment is ready, false otherwise
+ */
+ public static boolean isFlinkDeploymentReady(FlinkDeployment deployment) {
+ return ResourceLifecycleState.STABLE == deployment.getStatus().getLifecycleState()
+ && ReconciliationUtils.isJobRunning(deployment.getStatus());
+ }
+
+ public static void suspendFlinkDeployment(
+ BlueGreenContext context, FlinkDeployment nextDeployment) {
+ nextDeployment.getSpec().getJob().setState(JobState.SUSPENDED);
+ updateFlinkDeployment(nextDeployment, context);
+ }
+
+ public static void updateFlinkDeployment(
+ FlinkDeployment nextDeployment, BlueGreenContext context) {
+ String namespace = context.getBgDeployment().getMetadata().getNamespace();
+ context.getJosdkContext()
+ .getClient()
+ .resource(nextDeployment)
+ .inNamespace(namespace)
+ .update();
+ }
+
+ public static void replaceFlinkBlueGreenDeployment(BlueGreenContext context) {
+ String namespace = context.getBgDeployment().getMetadata().getNamespace();
+ context.getJosdkContext()
+ .getClient()
+ .resource(context.getBgDeployment())
+ .inNamespace(namespace)
+ .replace();
+ }
+
+ /**
+ * Deletes a Kubernetes FlinkDeployment resource.
+ *
+ * @param currentDeployment the FlinkDeployment to delete
+ * @param context the Blue/Green transition context
+ * @return true if the deployment was successfully deleted, false otherwise
+ */
+ public static boolean deleteFlinkDeployment(
+ FlinkDeployment currentDeployment, BlueGreenContext context) {
+ String deploymentName = currentDeployment.getMetadata().getName();
+ List deletedStatus =
+ context.getJosdkContext()
+ .getClient()
+ .resources(FlinkDeployment.class)
+ .inNamespace(currentDeployment.getMetadata().getNamespace())
+ .withName(deploymentName)
+ .delete();
+
+ return deletedStatus.size() == 1
+ && deletedStatus.get(0).getKind().equals("FlinkDeployment");
+ }
+}
diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/BlueGreenStateHandlerRegistry.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/BlueGreenStateHandlerRegistry.java
new file mode 100644
index 0000000000..20dd01644d
--- /dev/null
+++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/BlueGreenStateHandlerRegistry.java
@@ -0,0 +1,75 @@
+/*
+ * 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.bluegreen;
+
+import org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentState;
+import org.apache.flink.kubernetes.operator.controller.bluegreen.handlers.ActiveStateHandler;
+import org.apache.flink.kubernetes.operator.controller.bluegreen.handlers.BlueGreenStateHandler;
+import org.apache.flink.kubernetes.operator.controller.bluegreen.handlers.InitializingBlueStateHandler;
+import org.apache.flink.kubernetes.operator.controller.bluegreen.handlers.SavepointingStateHandler;
+import org.apache.flink.kubernetes.operator.controller.bluegreen.handlers.TransitioningStateHandler;
+
+import java.util.Map;
+
+import static org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentState.ACTIVE_BLUE;
+import static org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentState.ACTIVE_GREEN;
+import static org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentState.INITIALIZING_BLUE;
+import static org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentState.SAVEPOINTING_BLUE;
+import static org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentState.SAVEPOINTING_GREEN;
+import static org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentState.TRANSITIONING_TO_BLUE;
+import static org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentState.TRANSITIONING_TO_GREEN;
+
+/** Registry for Blue/Green deployment state handlers. */
+public class BlueGreenStateHandlerRegistry {
+
+ private final Map handlers;
+
+ public BlueGreenStateHandlerRegistry() {
+ // Create consolidated service
+ BlueGreenDeploymentService deploymentService = new BlueGreenDeploymentService();
+
+ // Create handlers
+ this.handlers =
+ Map.of(
+ INITIALIZING_BLUE, new InitializingBlueStateHandler(deploymentService),
+ ACTIVE_BLUE, new ActiveStateHandler(ACTIVE_BLUE, deploymentService),
+ ACTIVE_GREEN, new ActiveStateHandler(ACTIVE_GREEN, deploymentService),
+ TRANSITIONING_TO_BLUE,
+ new TransitioningStateHandler(
+ TRANSITIONING_TO_BLUE, deploymentService),
+ TRANSITIONING_TO_GREEN,
+ new TransitioningStateHandler(
+ TRANSITIONING_TO_GREEN, deploymentService),
+ SAVEPOINTING_BLUE,
+ new SavepointingStateHandler(SAVEPOINTING_BLUE, deploymentService),
+ SAVEPOINTING_GREEN,
+ new SavepointingStateHandler(
+ SAVEPOINTING_GREEN, deploymentService));
+ }
+
+ /**
+ * Gets the appropriate handler for the given state.
+ *
+ * @param state the Blue/Green deployment state
+ * @return the corresponding state handler
+ * @throws IllegalStateException if no handler is found for the state
+ */
+ public BlueGreenStateHandler getHandler(FlinkBlueGreenDeploymentState state) {
+ return handlers.get(state);
+ }
+}
diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/handlers/AbstractBlueGreenStateHandler.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/handlers/AbstractBlueGreenStateHandler.java
new file mode 100644
index 0000000000..8d95ea3490
--- /dev/null
+++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/handlers/AbstractBlueGreenStateHandler.java
@@ -0,0 +1,47 @@
+/*
+ * 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.bluegreen.handlers;
+
+import org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentState;
+import org.apache.flink.kubernetes.operator.controller.bluegreen.BlueGreenDeploymentService;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Abstract base class providing common functionality for Blue/Green state handlers. */
+public abstract class AbstractBlueGreenStateHandler implements BlueGreenStateHandler {
+
+ private final FlinkBlueGreenDeploymentState supportedState;
+
+ protected static final Logger LOG =
+ LoggerFactory.getLogger(AbstractBlueGreenStateHandler.class);
+
+ protected final BlueGreenDeploymentService deploymentService;
+
+ protected AbstractBlueGreenStateHandler(
+ FlinkBlueGreenDeploymentState supportedState,
+ BlueGreenDeploymentService deploymentService) {
+ this.supportedState = supportedState;
+ this.deploymentService = deploymentService;
+ }
+
+ @Override
+ public FlinkBlueGreenDeploymentState getSupportedState() {
+ return supportedState;
+ }
+}
diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/handlers/ActiveStateHandler.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/handlers/ActiveStateHandler.java
new file mode 100644
index 0000000000..176b94fb7f
--- /dev/null
+++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/handlers/ActiveStateHandler.java
@@ -0,0 +1,48 @@
+/*
+ * 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.bluegreen.handlers;
+
+import org.apache.flink.kubernetes.operator.api.FlinkBlueGreenDeployment;
+import org.apache.flink.kubernetes.operator.api.bluegreen.BlueGreenDeploymentType;
+import org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentState;
+import org.apache.flink.kubernetes.operator.controller.bluegreen.BlueGreenContext;
+import org.apache.flink.kubernetes.operator.controller.bluegreen.BlueGreenDeploymentService;
+
+import io.javaoperatorsdk.operator.api.reconciler.UpdateControl;
+
+/** Consolidated state handler for both ACTIVE_BLUE and ACTIVE_GREEN states. */
+public class ActiveStateHandler extends AbstractBlueGreenStateHandler {
+
+ public ActiveStateHandler(
+ FlinkBlueGreenDeploymentState supportedState,
+ BlueGreenDeploymentService deploymentService) {
+ super(supportedState, deploymentService);
+ }
+
+ @Override
+ public UpdateControl handle(BlueGreenContext context) {
+ BlueGreenDeploymentType currentType = getCurrentDeploymentType();
+ return deploymentService.checkAndInitiateDeployment(context, currentType);
+ }
+
+ private BlueGreenDeploymentType getCurrentDeploymentType() {
+ return getSupportedState() == FlinkBlueGreenDeploymentState.ACTIVE_BLUE
+ ? BlueGreenDeploymentType.BLUE
+ : BlueGreenDeploymentType.GREEN;
+ }
+}
diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/handlers/BlueGreenStateHandler.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/handlers/BlueGreenStateHandler.java
new file mode 100644
index 0000000000..eaf4000b7d
--- /dev/null
+++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/handlers/BlueGreenStateHandler.java
@@ -0,0 +1,53 @@
+/*
+ * 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.bluegreen.handlers;
+
+import org.apache.flink.kubernetes.operator.api.FlinkBlueGreenDeployment;
+import org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentState;
+import org.apache.flink.kubernetes.operator.controller.bluegreen.BlueGreenContext;
+
+import io.javaoperatorsdk.operator.api.reconciler.UpdateControl;
+
+/** Interface for handling specific Blue/Green deployment states. */
+public interface BlueGreenStateHandler {
+
+ /**
+ * Handles the processing logic for a specific Blue/Green deployment state.
+ *
+ * @param context the transition context containing all necessary dependencies
+ * @return UpdateControl indicating the next action
+ */
+ UpdateControl handle(BlueGreenContext context);
+
+ /**
+ * Gets the deployment state that this handler supports.
+ *
+ * @return the supported FlinkBlueGreenDeploymentState
+ */
+ FlinkBlueGreenDeploymentState getSupportedState();
+
+ /**
+ * Validates if this handler can process the given state.
+ *
+ * @param state the state to validate
+ * @return true if this handler can process the state, false otherwise
+ */
+ default boolean canHandle(FlinkBlueGreenDeploymentState state) {
+ return getSupportedState() == state;
+ }
+}
diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/handlers/InitializingBlueStateHandler.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/handlers/InitializingBlueStateHandler.java
new file mode 100644
index 0000000000..f2882d46f7
--- /dev/null
+++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/handlers/InitializingBlueStateHandler.java
@@ -0,0 +1,65 @@
+/*
+ * 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.bluegreen.handlers;
+
+import org.apache.flink.api.common.JobStatus;
+import org.apache.flink.kubernetes.operator.api.FlinkBlueGreenDeployment;
+import org.apache.flink.kubernetes.operator.api.bluegreen.BlueGreenDeploymentType;
+import org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentState;
+import org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentStatus;
+import org.apache.flink.kubernetes.operator.controller.bluegreen.BlueGreenContext;
+import org.apache.flink.kubernetes.operator.controller.bluegreen.BlueGreenDeploymentService;
+
+import io.javaoperatorsdk.operator.api.reconciler.UpdateControl;
+
+import static org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenUtils.hasSpecChanged;
+import static org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenUtils.setLastReconciledSpec;
+
+/** State handler for the INITIALIZING_BLUE state. */
+public class InitializingBlueStateHandler extends AbstractBlueGreenStateHandler {
+
+ public InitializingBlueStateHandler(BlueGreenDeploymentService deploymentService) {
+ super(FlinkBlueGreenDeploymentState.INITIALIZING_BLUE, deploymentService);
+ }
+
+ @Override
+ public UpdateControl handle(BlueGreenContext context) {
+ FlinkBlueGreenDeploymentStatus deploymentStatus = context.getDeploymentStatus();
+
+ // Deploy only if this is the initial deployment (no previous spec exists)
+ // or if we're recovering from a failure and the spec has changed since the last attempt
+ if (deploymentStatus.getLastReconciledSpec() == null
+ || (deploymentStatus.getJobStatus().getState().equals(JobStatus.FAILING)
+ && hasSpecChanged(context))) {
+
+ setLastReconciledSpec(context);
+ return deploymentService.initiateDeployment(
+ context,
+ BlueGreenDeploymentType.BLUE,
+ FlinkBlueGreenDeploymentState.TRANSITIONING_TO_BLUE,
+ null,
+ true);
+ } else {
+ LOG.warn(
+ "Ignoring initial deployment. Last Reconciled Spec null: {}. BG Status: {}.",
+ deploymentStatus.getLastReconciledSpec() == null,
+ deploymentStatus.getJobStatus().getState());
+ return UpdateControl.noUpdate();
+ }
+ }
+}
diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/handlers/SavepointingStateHandler.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/handlers/SavepointingStateHandler.java
new file mode 100644
index 0000000000..ba805a2e00
--- /dev/null
+++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/handlers/SavepointingStateHandler.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.controller.bluegreen.handlers;
+
+import org.apache.flink.kubernetes.operator.api.FlinkBlueGreenDeployment;
+import org.apache.flink.kubernetes.operator.api.bluegreen.BlueGreenDeploymentType;
+import org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentState;
+import org.apache.flink.kubernetes.operator.controller.bluegreen.BlueGreenContext;
+import org.apache.flink.kubernetes.operator.controller.bluegreen.BlueGreenDeploymentService;
+
+import io.javaoperatorsdk.operator.api.reconciler.UpdateControl;
+
+import static org.apache.flink.kubernetes.operator.controller.bluegreen.BlueGreenDeploymentService.patchStatusUpdateControl;
+import static org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenUtils.getReconciliationReschedInterval;
+
+/** State handler for managing Blue/Green deployment savepointing transitions. */
+public class SavepointingStateHandler extends AbstractBlueGreenStateHandler {
+
+ public SavepointingStateHandler(
+ FlinkBlueGreenDeploymentState supportedState,
+ BlueGreenDeploymentService deploymentService) {
+ super(supportedState, deploymentService);
+ }
+
+ @Override
+ public UpdateControl handle(BlueGreenContext context) {
+ BlueGreenDeploymentType currentType = getCurrentDeploymentType();
+ boolean isSavepointReady = deploymentService.monitorSavepoint(context, currentType);
+
+ // Savepoint creation completed, transition back to active state to continue deployment
+ if (isSavepointReady) {
+ var nextState =
+ getSupportedState() == FlinkBlueGreenDeploymentState.SAVEPOINTING_BLUE
+ ? FlinkBlueGreenDeploymentState.ACTIVE_BLUE
+ : FlinkBlueGreenDeploymentState.ACTIVE_GREEN;
+ return patchStatusUpdateControl(context, nextState, null, null).rescheduleAfter(0);
+ }
+
+ // TODO: this will wait indefinitely for a savepoint to complete,
+ // we could abort the transition, WITHOUT SUSPENDING the FlinkDeployment,
+ // if the grace period is exceeded.
+ return UpdateControl.noUpdate()
+ .rescheduleAfter(getReconciliationReschedInterval(context));
+ }
+
+ private BlueGreenDeploymentType getCurrentDeploymentType() {
+ return getSupportedState() == FlinkBlueGreenDeploymentState.SAVEPOINTING_BLUE
+ ? BlueGreenDeploymentType.BLUE
+ : BlueGreenDeploymentType.GREEN;
+ }
+}
diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/handlers/TransitioningStateHandler.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/handlers/TransitioningStateHandler.java
new file mode 100644
index 0000000000..0334720c18
--- /dev/null
+++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/handlers/TransitioningStateHandler.java
@@ -0,0 +1,48 @@
+/*
+ * 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.bluegreen.handlers;
+
+import org.apache.flink.kubernetes.operator.api.FlinkBlueGreenDeployment;
+import org.apache.flink.kubernetes.operator.api.bluegreen.BlueGreenDeploymentType;
+import org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentState;
+import org.apache.flink.kubernetes.operator.controller.bluegreen.BlueGreenContext;
+import org.apache.flink.kubernetes.operator.controller.bluegreen.BlueGreenDeploymentService;
+
+import io.javaoperatorsdk.operator.api.reconciler.UpdateControl;
+
+/** Consolidated state handler for both TRANSITIONING_TO_BLUE and TRANSITIONING_TO_GREEN states. */
+public class TransitioningStateHandler extends AbstractBlueGreenStateHandler {
+
+ public TransitioningStateHandler(
+ FlinkBlueGreenDeploymentState supportedState,
+ BlueGreenDeploymentService deploymentService) {
+ super(supportedState, deploymentService);
+ }
+
+ @Override
+ public UpdateControl handle(BlueGreenContext context) {
+ BlueGreenDeploymentType currentType = getCurrentDeploymentType();
+ return deploymentService.monitorTransition(context, currentType);
+ }
+
+ private BlueGreenDeploymentType getCurrentDeploymentType() {
+ return getSupportedState() == FlinkBlueGreenDeploymentState.TRANSITIONING_TO_BLUE
+ ? BlueGreenDeploymentType.GREEN // Transitioning FROM green TO blue
+ : BlueGreenDeploymentType.BLUE; // Transitioning FROM blue TO green
+ }
+}
diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/metrics/OperatorJosdkMetrics.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/metrics/OperatorJosdkMetrics.java
index 66c191d6ee..ecc97dbb10 100644
--- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/metrics/OperatorJosdkMetrics.java
+++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/metrics/OperatorJosdkMetrics.java
@@ -17,6 +17,7 @@
package org.apache.flink.kubernetes.operator.metrics;
+import org.apache.flink.kubernetes.operator.api.FlinkBlueGreenDeployment;
import org.apache.flink.kubernetes.operator.api.FlinkDeployment;
import org.apache.flink.kubernetes.operator.api.FlinkSessionJob;
import org.apache.flink.kubernetes.operator.api.FlinkStateSnapshot;
@@ -221,6 +222,8 @@ private KubernetesResourceNamespaceMetricGroup getResourceNsMg(
resourceClass = FlinkSessionJob.class;
} else if (resourceGvk.getKind().equals(FlinkStateSnapshot.class.getSimpleName())) {
resourceClass = FlinkStateSnapshot.class;
+ } else if (resourceGvk.getKind().equals(FlinkBlueGreenDeployment.class.getSimpleName())) {
+ resourceClass = FlinkBlueGreenDeployment.class;
} else {
return Optional.empty();
}
diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/diff/FlinkBlueGreenDeploymentSpecDiff.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/diff/FlinkBlueGreenDeploymentSpecDiff.java
new file mode 100644
index 0000000000..f11f71c54b
--- /dev/null
+++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/diff/FlinkBlueGreenDeploymentSpecDiff.java
@@ -0,0 +1,100 @@
+/*
+ * 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.reconciler.diff;
+
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.kubernetes.operator.api.bluegreen.BlueGreenDiffType;
+import org.apache.flink.kubernetes.operator.api.diff.DiffType;
+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.KubernetesDeploymentMode;
+
+import lombok.NonNull;
+
+/**
+ * Diff class for comparing FlinkBlueGreenDeploymentSpec objects. Provides specialized comparison
+ * logic that delegates nested FlinkDeploymentSpec comparison to ReflectiveDiffBuilder.
+ */
+@Experimental
+public class FlinkBlueGreenDeploymentSpecDiff {
+
+ private final FlinkBlueGreenDeploymentSpec left;
+ private final FlinkBlueGreenDeploymentSpec right;
+ private final KubernetesDeploymentMode deploymentMode;
+
+ public FlinkBlueGreenDeploymentSpecDiff(
+ KubernetesDeploymentMode deploymentMode,
+ @NonNull FlinkBlueGreenDeploymentSpec left,
+ @NonNull FlinkBlueGreenDeploymentSpec right) {
+ this.deploymentMode = deploymentMode;
+ this.left = left;
+ this.right = right;
+
+ // Validate that neither spec is null
+ validateSpecs();
+ }
+
+ /**
+ * Compares the Blue/Green deployment specs and returns the appropriate diff type. The
+ * comparison focuses solely on the nested FlinkDeploymentSpec differences.
+ *
+ * @return BlueGreenDiffType indicating the type of difference found
+ */
+ public BlueGreenDiffType compare() {
+ FlinkDeploymentSpec leftSpec = left.getTemplate().getSpec();
+ FlinkDeploymentSpec rightSpec = right.getTemplate().getSpec();
+
+ // Case 1: FlinkDeploymentSpecs are identical
+ if (leftSpec.equals(rightSpec)) {
+ return BlueGreenDiffType.IGNORE;
+ }
+
+ // Case 2 & 3: Delegate to ReflectiveDiffBuilder for nested spec comparison
+ DiffResult diffResult =
+ new ReflectiveDiffBuilder<>(deploymentMode, leftSpec, rightSpec).build();
+
+ DiffType diffType = diffResult.getType();
+
+ // Case 2: ReflectiveDiffBuilder returns IGNORE
+ if (diffType == DiffType.IGNORE) {
+ return BlueGreenDiffType.PATCH_CHILD;
+ } else {
+ // Case 3: ReflectiveDiffBuilder returns anything else map it to TRANSITION as well
+ return BlueGreenDiffType.TRANSITION;
+ }
+ }
+
+ /**
+ * Validates that the specs and their nested components are not null. Throws
+ * IllegalArgumentException if any required component is null.
+ */
+ private void validateSpecs() {
+ if (left.getTemplate() == null) {
+ throw new IllegalArgumentException("Left spec template cannot be null");
+ }
+ if (right.getTemplate() == null) {
+ throw new IllegalArgumentException("Right spec template cannot be null");
+ }
+ if (left.getTemplate().getSpec() == null) {
+ throw new IllegalArgumentException("Left spec template.spec cannot be null");
+ }
+ if (right.getTemplate().getSpec() == null) {
+ throw new IllegalArgumentException("Right spec template.spec cannot be null");
+ }
+ }
+}
diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/bluegreen/BlueGreenUtils.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/bluegreen/BlueGreenUtils.java
new file mode 100644
index 0000000000..cd2d9ae235
--- /dev/null
+++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/bluegreen/BlueGreenUtils.java
@@ -0,0 +1,387 @@
+/*
+ * 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.JobID;
+import org.apache.flink.configuration.CheckpointingOptions;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.kubernetes.operator.api.FlinkBlueGreenDeployment;
+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.BlueGreenDiffType;
+import org.apache.flink.kubernetes.operator.api.spec.FlinkBlueGreenDeploymentSpec;
+import org.apache.flink.kubernetes.operator.api.spec.KubernetesDeploymentMode;
+import org.apache.flink.kubernetes.operator.api.spec.UpgradeMode;
+import org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentStatus;
+import org.apache.flink.kubernetes.operator.api.status.Savepoint;
+import org.apache.flink.kubernetes.operator.api.utils.SpecUtils;
+import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions;
+import org.apache.flink.kubernetes.operator.controller.FlinkResourceContext;
+import org.apache.flink.kubernetes.operator.controller.bluegreen.BlueGreenContext;
+import org.apache.flink.kubernetes.operator.observer.SavepointFetchResult;
+import org.apache.flink.kubernetes.operator.reconciler.diff.FlinkBlueGreenDeploymentSpecDiff;
+import org.apache.flink.util.Preconditions;
+
+import io.fabric8.kubernetes.api.model.ObjectMeta;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.time.Instant;
+import java.util.Map;
+import java.util.Optional;
+
+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.controller.bluegreen.BlueGreenKubernetesService.getDependentObjectMeta;
+import static org.apache.flink.kubernetes.operator.controller.bluegreen.BlueGreenKubernetesService.replaceFlinkBlueGreenDeployment;
+
+/** Consolidated utility methods for Blue/Green deployment operations. */
+public class BlueGreenUtils {
+
+ private static final Logger LOG = LoggerFactory.getLogger(BlueGreenUtils.class);
+
+ // ==================== Spec Operations ====================
+
+ /**
+ * Adjusts name references in a spec by replacing deployment names with child deployment names.
+ *
+ * @param spec the spec to adjust
+ * @param deploymentName the original deployment name
+ * @param childDeploymentName the child deployment name to replace with
+ * @param wrapperKey the JSON wrapper key
+ * @param valueType the spec type
+ * @return adjusted spec with name references updated
+ */
+ public static T adjustNameReferences(
+ T spec,
+ String deploymentName,
+ String childDeploymentName,
+ String wrapperKey,
+ Class valueType) {
+ String serializedSpec = SpecUtils.writeSpecAsJSON(spec, wrapperKey);
+ String replacedSerializedSpec = serializedSpec.replace(deploymentName, childDeploymentName);
+ return SpecUtils.readSpecFromJSON(replacedSerializedSpec, wrapperKey, valueType);
+ }
+
+ /**
+ * Checks if the Blue/Green deployment spec has changed compared to the last reconciled spec.
+ *
+ * @param context the Blue/Green transition context
+ * @return true if the spec has changed, false otherwise
+ */
+ public static boolean hasSpecChanged(BlueGreenContext context) {
+ BlueGreenDiffType diffType = getSpecDiff(context);
+ return diffType != BlueGreenDiffType.IGNORE;
+ }
+
+ public static BlueGreenDiffType getSpecDiff(BlueGreenContext context) {
+ FlinkBlueGreenDeploymentStatus deploymentStatus = context.getDeploymentStatus();
+ String lastReconciledSpec = deploymentStatus.getLastReconciledSpec();
+ FlinkBlueGreenDeploymentSpec lastSpec =
+ SpecUtils.readSpecFromJSON(
+ lastReconciledSpec, "spec", FlinkBlueGreenDeploymentSpec.class);
+
+ FlinkBlueGreenDeploymentSpecDiff diff =
+ new FlinkBlueGreenDeploymentSpecDiff(
+ KubernetesDeploymentMode.NATIVE,
+ lastSpec,
+ context.getBgDeployment().getSpec());
+
+ return diff.compare();
+ }
+
+ public static void setLastReconciledSpec(BlueGreenContext context) {
+ FlinkBlueGreenDeploymentStatus deploymentStatus = context.getDeploymentStatus();
+ deploymentStatus.setLastReconciledSpec(
+ SpecUtils.writeSpecAsJSON(context.getBgDeployment().getSpec(), "spec"));
+ deploymentStatus.setLastReconciledTimestamp(Instant.now().toString());
+ }
+
+ public static void revertToLastSpec(BlueGreenContext context) {
+ context.getBgDeployment()
+ .setSpec(
+ SpecUtils.readSpecFromJSON(
+ context.getDeploymentStatus().getLastReconciledSpec(),
+ "spec",
+ FlinkBlueGreenDeploymentSpec.class));
+ replaceFlinkBlueGreenDeployment(context);
+ }
+
+ /**
+ * Extracts a configuration option value from the Blue/Green deployment spec.
+ *
+ * @param bgDeployment the Blue/Green deployment
+ * @param option the configuration option to extract
+ * @return the configuration value or default if not found
+ */
+ public static T getConfigOption(
+ FlinkBlueGreenDeployment bgDeployment, ConfigOption option) {
+ Map configuration = bgDeployment.getSpec().getConfiguration();
+
+ if (configuration == null) {
+ return option.defaultValue();
+ }
+
+ return Configuration.fromMap(configuration).get(option);
+ }
+
+ // ==================== Time Utilities ====================
+
+ /**
+ * Converts milliseconds to ISO instant string.
+ *
+ * @param millis the milliseconds since epoch
+ * @return ISO instant string representation
+ */
+ public static String millisToInstantStr(long millis) {
+ return Instant.ofEpochMilli(millis).toString();
+ }
+
+ /**
+ * Converts ISO instant string to milliseconds.
+ *
+ * @param instant the ISO instant string
+ * @return milliseconds since epoch, or 0 if instant is null
+ */
+ public static long instantStrToMillis(String instant) {
+ if (instant == null) {
+ return 0;
+ }
+ return Instant.parse(instant).toEpochMilli();
+ }
+
+ /**
+ * Gets the reconciliation rescheduling interval for the Blue/Green deployment.
+ *
+ * @param context the Blue/Green transition context
+ * @return reconciliation interval in milliseconds
+ */
+ public static long getReconciliationReschedInterval(BlueGreenContext context) {
+ return Math.max(
+ getConfigOption(context.getBgDeployment(), RECONCILIATION_RESCHEDULING_INTERVAL)
+ .toMillis(),
+ 0);
+ }
+
+ /**
+ * Gets the deployment deletion delay for the Blue/Green deployment.
+ *
+ * @param context the Blue/Green transition context
+ * @return deletion delay in milliseconds
+ */
+ public static long getDeploymentDeletionDelay(BlueGreenContext context) {
+ return Math.max(
+ getConfigOption(context.getBgDeployment(), DEPLOYMENT_DELETION_DELAY).toMillis(),
+ 0);
+ }
+
+ /**
+ * Gets the abort grace period for the Blue/Green deployment.
+ *
+ * @param context the Blue/Green transition context
+ * @return abort grace period in milliseconds
+ */
+ public static long getAbortGracePeriod(BlueGreenContext context) {
+ long abortGracePeriod =
+ getConfigOption(context.getBgDeployment(), ABORT_GRACE_PERIOD).toMillis();
+ return abortGracePeriod;
+ }
+
+ /**
+ * Sets the abort timestamp in the deployment status based on current time and grace period.
+ *
+ * @param context the Blue/Green transition context
+ */
+ public static void setAbortTimestamp(BlueGreenContext context) {
+ context.getDeploymentStatus()
+ .setAbortTimestamp(
+ millisToInstantStr(
+ System.currentTimeMillis() + getAbortGracePeriod(context)));
+ }
+
+ // ==================== Savepoint/Checkpoint Operations ====================
+
+ /**
+ * Determines if a savepoint is required based on the deployment's upgrade mode. Currently all
+ * upgrade modes except STATELESS require savepoints.
+ *
+ * @param context the Blue/Green transition context
+ * @return true if savepoint is required, false otherwise
+ */
+ public static boolean isSavepointRequired(BlueGreenContext context) {
+ UpgradeMode upgradeMode =
+ context.getBgDeployment()
+ .getSpec()
+ .getTemplate()
+ .getSpec()
+ .getJob()
+ .getUpgradeMode();
+ // return UpgradeMode.SAVEPOINT == upgradeMode;
+ // Currently taking savepoints for all modes except STATELESS
+ // (previously only SAVEPOINT mode required savepoints)
+ return UpgradeMode.STATELESS != upgradeMode;
+ }
+
+ public static boolean lookForCheckpoint(BlueGreenContext context) {
+ FlinkBlueGreenDeploymentStatus deploymentStatus = context.getDeploymentStatus();
+ String lastReconciledSpec = deploymentStatus.getLastReconciledSpec();
+ FlinkBlueGreenDeploymentSpec lastSpec =
+ SpecUtils.readSpecFromJSON(
+ lastReconciledSpec, "spec", FlinkBlueGreenDeploymentSpec.class);
+
+ var previousUpgradeMode = lastSpec.getTemplate().getSpec().getJob().getUpgradeMode();
+ var nextUpgradeMode =
+ context.getBgDeployment()
+ .getSpec()
+ .getTemplate()
+ .getSpec()
+ .getJob()
+ .getUpgradeMode();
+
+ return previousUpgradeMode == nextUpgradeMode && nextUpgradeMode == UpgradeMode.LAST_STATE;
+ }
+
+ public static String triggerSavepoint(FlinkResourceContext ctx)
+ throws Exception {
+ var jobId = ctx.getResource().getStatus().getJobStatus().getJobId();
+ var conf = ctx.getObserveConfig();
+ var savepointFormatType =
+ conf.get(KubernetesOperatorConfigOptions.OPERATOR_SAVEPOINT_FORMAT_TYPE);
+ var savepointDirectory =
+ Preconditions.checkNotNull(conf.get(CheckpointingOptions.SAVEPOINT_DIRECTORY));
+
+ LOG.info("About to trigger savepoint info for jobId: {}", jobId);
+
+ String triggerId =
+ ctx.getFlinkService()
+ .triggerSavepoint(jobId, savepointFormatType, savepointDirectory, conf);
+
+ LOG.info(
+ "Triggered savepoint for jobId: {}, triggerId: {}, savepoint dir: {}",
+ jobId,
+ triggerId,
+ savepointDirectory);
+
+ return triggerId;
+ }
+
+ public static SavepointFetchResult fetchSavepointInfo(
+ FlinkResourceContext ctx, String triggerId) {
+ String jobId = ctx.getResource().getStatus().getJobStatus().getJobId();
+ LOG.info("About to fetch savepoint info for jobId: {}, triggerId: {}", jobId, triggerId);
+
+ var savepointFetchResult =
+ ctx.getFlinkService().fetchSavepointInfo(triggerId, jobId, ctx.getObserveConfig());
+
+ LOG.info("Fetched savepoint info for jobId: {}, triggerId: {}", jobId, triggerId);
+ return savepointFetchResult;
+ }
+
+ public static Savepoint getLastCheckpoint(
+ FlinkResourceContext resourceContext) {
+
+ Optional lastCheckpoint =
+ resourceContext
+ .getFlinkService()
+ .getLastCheckpoint(
+ JobID.fromHexString(
+ resourceContext
+ .getResource()
+ .getStatus()
+ .getJobStatus()
+ .getJobId()),
+ resourceContext.getObserveConfig());
+
+ // Alternative fallback if no checkpoint is available could be implemented here
+ if (lastCheckpoint.isEmpty()) {
+ throw new IllegalStateException(
+ "Last Checkpoint for Job "
+ + resourceContext.getResource().getMetadata().getName()
+ + " not found!");
+ }
+
+ return lastCheckpoint.get();
+ }
+
+ // ==================== Deployment Preparation Utilities ====================
+
+ /**
+ * Creates a new FlinkDeployment resource for a Blue/Green deployment transition. This method
+ * prepares the deployment with proper metadata, specs, and savepoint configuration.
+ *
+ * @param context the Blue/Green transition context
+ * @param blueGreenDeploymentType the type of deployment (BLUE or GREEN)
+ * @param lastCheckpoint the savepoint/checkpoint to restore from (can be null)
+ * @param isFirstDeployment whether this is the initial deployment
+ * @param bgMeta the metadata of the parent Blue/Green deployment
+ * @return configured FlinkDeployment ready for deployment
+ */
+ public static FlinkDeployment prepareFlinkDeployment(
+ BlueGreenContext context,
+ BlueGreenDeploymentType blueGreenDeploymentType,
+ Savepoint lastCheckpoint,
+ boolean isFirstDeployment,
+ ObjectMeta bgMeta) {
+ // Deployment
+ FlinkDeployment flinkDeployment = new FlinkDeployment();
+ FlinkBlueGreenDeploymentSpec spec = context.getBgDeployment().getSpec();
+
+ String childDeploymentName =
+ bgMeta.getName() + "-" + blueGreenDeploymentType.toString().toLowerCase();
+
+ FlinkBlueGreenDeploymentSpec adjustedSpec =
+ adjustNameReferences(
+ spec,
+ bgMeta.getName(),
+ childDeploymentName,
+ "spec",
+ FlinkBlueGreenDeploymentSpec.class);
+
+ // The Blue/Green initialSavepointPath is only used for first-time deployments
+ if (isFirstDeployment) {
+ String initialSavepointPath =
+ adjustedSpec.getTemplate().getSpec().getJob().getInitialSavepointPath();
+ if (initialSavepointPath != null && !initialSavepointPath.isEmpty()) {
+ LOG.info("Using initialSavepointPath: " + initialSavepointPath);
+ adjustedSpec
+ .getTemplate()
+ .getSpec()
+ .getJob()
+ .setInitialSavepointPath(initialSavepointPath);
+ } else {
+ LOG.info("Clean startup with no checkpoint/savepoint restoration");
+ }
+ } else if (lastCheckpoint != null) {
+ String location = lastCheckpoint.getLocation().replace("file:", "");
+ LOG.info("Using Blue/Green savepoint/checkpoint: " + location);
+ adjustedSpec.getTemplate().getSpec().getJob().setInitialSavepointPath(location);
+ }
+
+ flinkDeployment.setSpec(adjustedSpec.getTemplate().getSpec());
+
+ // Deployment metadata
+ ObjectMeta flinkDeploymentMeta = getDependentObjectMeta(context.getBgDeployment());
+ flinkDeploymentMeta.setName(childDeploymentName);
+ flinkDeploymentMeta.setLabels(
+ Map.of(BlueGreenDeploymentType.LABEL_KEY, blueGreenDeploymentType.toString()));
+ flinkDeployment.setMetadata(flinkDeploymentMeta);
+ return flinkDeployment;
+ }
+}
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 aa06e303ee..f3f2e3571d 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
@@ -73,6 +73,7 @@
import java.util.Calendar;
import java.util.Collections;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
@@ -496,7 +497,19 @@ public Optional getRetryInfo() {
@Override
public Set getSecondaryResources(Class aClass) {
- return null;
+ // TODO: improve this, even if we only support FlinkDeployment as a secondary resource
+ if (aClass.getSimpleName().equals(FlinkDeployment.class.getSimpleName())) {
+ KubernetesClient client = getClient();
+ var hasMetadata =
+ new HashSet<>(
+ client.resources(FlinkDeployment.class)
+ .inAnyNamespace()
+ .list()
+ .getItems());
+ return (Set) hasMetadata;
+ } else {
+ return null;
+ }
}
@Override
diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/TestingFlinkService.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/TestingFlinkService.java
index c020c53d98..8440ffe8e7 100644
--- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/TestingFlinkService.java
+++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/TestingFlinkService.java
@@ -139,6 +139,8 @@ public class TestingFlinkService extends AbstractFlinkService {
@Setter private boolean deployFailure = false;
@Setter private Exception makeItFailWith;
@Setter private boolean triggerSavepointFailure = false;
+ @Setter private Exception savepointTriggerException = null;
+ @Setter private String savepointFetchError = null;
@Setter private boolean disposeSavepointFailure = false;
@Setter private Runnable sessionJobSubmittedCallback;
@Setter private PodList podList = new PodList();
@@ -375,6 +377,9 @@ public String triggerSavepoint(
String savepointDirectory,
Configuration conf)
throws Exception {
+ if (savepointTriggerException != null) {
+ throw savepointTriggerException;
+ }
if (triggerSavepointFailure) {
throw new Exception(SNAPSHOT_ERROR_MESSAGE);
}
@@ -398,6 +403,10 @@ public String triggerCheckpoint(
public SavepointFetchResult fetchSavepointInfo(
String triggerId, String jobId, Configuration conf) {
+ if (savepointFetchError != null) {
+ return SavepointFetchResult.error(savepointFetchError);
+ }
+
if (savepointTriggers.containsKey(triggerId)) {
if (savepointTriggers.get(triggerId)) {
return SavepointFetchResult.completed("savepoint_" + savepointCounter++);
@@ -626,10 +635,13 @@ public Optional getLastCheckpoint(JobID jobId, Configuration conf) {
.findAny()
.ifPresent(
t -> {
- if (!t.f1.getJobState().isGloballyTerminalState()) {
- throw new RuntimeException(
- "Checkpoint should not be queried if job is not in terminal state");
- }
+ // TODO: check this... for example getting the SP/CP
+ // in RUNNING state should be valid
+ // if (!t.f1.getJobState().isGloballyTerminalState()) {
+ // throw new RuntimeException(
+ // "Checkpoint should not be
+ // queried if job is not in terminal state");
+ // }
});
return super.getLastCheckpoint(jobId, conf);
@@ -771,4 +783,20 @@ public void addExceptionHistory(
new JobExceptionsInfoWithHistory(exceptionHistory);
jobExceptionsMap.put(jobId, newExceptionHistory);
}
+
+ public void setSavepointTriggerException(Exception exception) {
+ this.savepointTriggerException = exception;
+ }
+
+ public void clearSavepointTriggerException() {
+ this.savepointTriggerException = null;
+ }
+
+ public void setSavepointFetchError(String error) {
+ this.savepointFetchError = error;
+ }
+
+ public void clearSavepointFetchError() {
+ this.savepointFetchError = null;
+ }
}
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
new file mode 100644
index 0000000000..229406bbc9
--- /dev/null
+++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentControllerTest.java
@@ -0,0 +1,1229 @@
+/*
+ * 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.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.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;
+import org.apache.flink.kubernetes.operator.api.utils.SpecUtils;
+import org.apache.flink.kubernetes.operator.config.FlinkConfigManager;
+import org.apache.flink.runtime.jobgraph.SavepointConfigOptions;
+
+import io.fabric8.kubernetes.api.model.ObjectMetaBuilder;
+import io.fabric8.kubernetes.client.KubernetesClient;
+import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient;
+import io.javaoperatorsdk.operator.api.reconciler.Context;
+import io.javaoperatorsdk.operator.api.reconciler.UpdateControl;
+import org.jetbrains.annotations.NotNull;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.stream.Stream;
+
+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.utils.bluegreen.BlueGreenUtils.instantStrToMillis;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/** {@link FlinkBlueGreenDeploymentController} tests. */
+@EnableKubernetesMockClient(crud = true)
+public class FlinkBlueGreenDeploymentControllerTest {
+
+ 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 String CUSTOM_CONFIG_FIELD = "custom-configuration-field";
+ private static final int DEFAULT_DELETION_DELAY_VALUE = 500;
+ private static final int ALT_DELETION_DELAY_VALUE = 1000;
+ private static final int MINIMUM_ABORT_GRACE_PERIOD = 1000;
+ private static final String TEST_CHECKPOINT_PATH = "/tmp/checkpoints";
+ private static final String TEST_INITIAL_SAVEPOINT_PATH = "/tmp/savepoints";
+ private final FlinkConfigManager configManager = new FlinkConfigManager(new Configuration());
+ private TestingFlinkService flinkService;
+ private Context context;
+ private TestingFlinkBlueGreenDeploymentController testController;
+
+ private KubernetesClient kubernetesClient;
+
+ @BeforeEach
+ public void setup() {
+ flinkService = new TestingFlinkService(kubernetesClient);
+ context = flinkService.getContext();
+ testController = new TestingFlinkBlueGreenDeploymentController(configManager, flinkService);
+ }
+
+ @ParameterizedTest
+ @MethodSource("flinkVersionsAndSavepointPaths")
+ public void verifyBasicDeployment(FlinkVersion flinkVersion, String initialSavepointPath)
+ throws Exception {
+ var blueGreenDeployment =
+ buildSessionCluster(
+ TEST_DEPLOYMENT_NAME,
+ TEST_NAMESPACE,
+ flinkVersion,
+ initialSavepointPath,
+ UpgradeMode.STATELESS);
+ executeBasicDeployment(flinkVersion, blueGreenDeployment, true, initialSavepointPath);
+ }
+
+ @ParameterizedTest
+ @MethodSource("flinkVersionsAndSavepointPathsAndUpgradeModes")
+ public void verifyBasicTransition(
+ FlinkVersion flinkVersion, String initialSavepointPath, UpgradeMode upgradeMode)
+ throws Exception {
+ var blueGreenDeployment =
+ buildSessionCluster(
+ TEST_DEPLOYMENT_NAME, TEST_NAMESPACE, flinkVersion, null, upgradeMode);
+ var rs = executeBasicDeployment(flinkVersion, blueGreenDeployment, false, null);
+
+ // Simulate a change in the spec to trigger a Green deployment
+ String customValue = UUID.randomUUID().toString();
+ simulateChangeInSpec(
+ rs.deployment, customValue, ALT_DELETION_DELAY_VALUE, initialSavepointPath);
+
+ var expectedSavepointPath = initialSavepointPath;
+
+ if (upgradeMode != UpgradeMode.STATELESS) {
+ // In this case there will ALWAYS be a savepoint generated with this value,
+ // regardless of the initialSavepointPath
+ expectedSavepointPath = "savepoint_1";
+ rs = handleSavepoint(rs);
+ }
+
+ // Transitioning to the Green deployment
+
+ testTransitionToGreen(rs, customValue, expectedSavepointPath);
+ }
+
+ @NotNull
+ private TestingFlinkBlueGreenDeploymentController.BlueGreenReconciliationResult handleSavepoint(
+ TestingFlinkBlueGreenDeploymentController.BlueGreenReconciliationResult rs)
+ throws Exception {
+
+ var triggers = flinkService.getSavepointTriggers();
+ triggers.clear();
+
+ rs = reconcile(rs.deployment);
+
+ // Simulating a pending savepoint
+ triggers.put(rs.deployment.getStatus().getSavepointTriggerId(), false);
+
+ // Should be in SAVEPOINTING_BLUE state first
+ assertEquals(
+ FlinkBlueGreenDeploymentState.SAVEPOINTING_BLUE,
+ rs.reconciledStatus.getBlueGreenState());
+ assertTrue(rs.updateControl.isPatchStatus());
+ assertTrue(rs.updateControl.getScheduleDelay().isPresent());
+
+ // This next reconciliation should continue waiting on the pending savepoint
+ rs = reconcile(rs.deployment);
+
+ // NOTE: internally the above reconcile call invokes the fetchSavepointInfo on the trigger,
+ // the TestFlinkService automatically sets it to "true" (completed)
+
+ assertTrue(rs.updateControl.isNoUpdate());
+ assertTrue(rs.updateControl.getScheduleDelay().isPresent());
+
+ // Completing the savepoint
+ triggers.put(rs.deployment.getStatus().getSavepointTriggerId(), true);
+
+ // This next reconciliation should move on to the next state
+ rs = reconcile(rs.deployment);
+
+ assertEquals(
+ FlinkBlueGreenDeploymentState.ACTIVE_BLUE, rs.reconciledStatus.getBlueGreenState());
+ assertTrue(rs.updateControl.isPatchStatus());
+ assertTrue(rs.updateControl.getScheduleDelay().isPresent());
+ return rs;
+ }
+
+ @ParameterizedTest
+ @MethodSource("org.apache.flink.kubernetes.operator.TestUtils#flinkVersions")
+ public void verifyFailureBeforeTransition(FlinkVersion flinkVersion) throws Exception {
+ var blueGreenDeployment =
+ buildSessionCluster(
+ TEST_DEPLOYMENT_NAME,
+ TEST_NAMESPACE,
+ flinkVersion,
+ TEST_INITIAL_SAVEPOINT_PATH,
+ UpgradeMode.STATELESS);
+ var rs =
+ executeBasicDeployment(
+ flinkVersion, blueGreenDeployment, false, TEST_INITIAL_SAVEPOINT_PATH);
+
+ // Simulate a change in the spec to trigger a Blue deployment
+ simulateChangeInSpec(rs.deployment, UUID.randomUUID().toString(), 0, null);
+
+ // Simulate a failure in the running deployment
+ simulateJobFailure(getFlinkDeployments().get(0));
+
+ // Initiate the Green deployment
+ var minReconciliationTs = System.currentTimeMillis() - 1;
+ rs = reconcile(rs.deployment);
+
+ assertTrue(rs.updateControl.isPatchStatus());
+ assertTrue(
+ minReconciliationTs
+ < instantStrToMillis(rs.reconciledStatus.getLastReconciledTimestamp()));
+
+ assertFailingJobStatus(rs);
+
+ var flinkDeployments = getFlinkDeployments();
+ assertEquals(1, flinkDeployments.size());
+ assertEquals(
+ JobStatus.RECONCILING,
+ flinkDeployments.get(0).getStatus().getJobStatus().getState());
+ assertEquals(
+ ReconciliationState.UPGRADING,
+ flinkDeployments.get(0).getStatus().getReconciliationStatus().getState());
+
+ // No update
+ rs = reconcile(rs.deployment);
+ assertTrue(rs.updateControl.isNoUpdate());
+ }
+
+ @ParameterizedTest
+ @MethodSource({"org.apache.flink.kubernetes.operator.TestUtils#flinkVersions"})
+ public void verifyFailureDuringTransition(FlinkVersion flinkVersion) throws Exception {
+ var blueGreenDeployment =
+ buildSessionCluster(
+ TEST_DEPLOYMENT_NAME,
+ TEST_NAMESPACE,
+ flinkVersion,
+ null,
+ UpgradeMode.STATELESS);
+
+ // Overriding the maxNumRetries and Reschedule Interval
+ var abortGracePeriodMs = 1200;
+ var reconciliationReschedulingIntervalMs = 3000;
+ Map configuration = blueGreenDeployment.getSpec().getConfiguration();
+ configuration.put(ABORT_GRACE_PERIOD.key(), String.valueOf(abortGracePeriodMs));
+ configuration.put(
+ RECONCILIATION_RESCHEDULING_INTERVAL.key(),
+ String.valueOf(reconciliationReschedulingIntervalMs));
+
+ var rs =
+ executeBasicDeployment(
+ flinkVersion, blueGreenDeployment, false, TEST_INITIAL_SAVEPOINT_PATH);
+
+ // Simulate a change in the spec to trigger a Blue deployment
+ String customValue = UUID.randomUUID().toString();
+ simulateChangeInSpec(rs.deployment, customValue, 0, null);
+
+ // Initiate the Green deployment
+ rs = reconcile(rs.deployment);
+
+ // We should be TRANSITIONING_TO_GREEN at this point
+ assertEquals(
+ FlinkBlueGreenDeploymentState.TRANSITIONING_TO_GREEN,
+ rs.reconciledStatus.getBlueGreenState());
+ assertEquals(
+ customValue,
+ getFlinkConfigurationValue(
+ rs.deployment.getSpec().getTemplate().getSpec(), CUSTOM_CONFIG_FIELD));
+
+ // Simulating the Blue deployment doesn't start correctly (status will remain the same)
+ Long reschedDelayMs = 0L;
+ for (int i = 0; i < 2; i++) {
+ rs = reconcile(rs.deployment);
+ assertTrue(rs.updateControl.isPatchStatus());
+ assertFalse(rs.updateControl.isPatchResource());
+ assertTrue(rs.updateControl.getScheduleDelay().isPresent());
+ reschedDelayMs = rs.updateControl.getScheduleDelay().get();
+ assertTrue(
+ reschedDelayMs == reconciliationReschedulingIntervalMs && reschedDelayMs > 0);
+ assertTrue(
+ instantStrToMillis(rs.reconciledStatus.getAbortTimestamp())
+ > System.currentTimeMillis());
+ }
+
+ // Wait until the delay
+ Thread.sleep(reschedDelayMs);
+
+ // After the retries are exhausted
+ rs = reconcile(rs.deployment);
+
+ assertTrue(rs.updateControl.isPatchStatus());
+
+ // The first job should be RUNNING, the second should be SUSPENDED
+ assertFailingJobStatus(rs);
+ // No longer TRANSITIONING_TO_GREEN and rolled back to ACTIVE_BLUE
+ assertEquals(
+ FlinkBlueGreenDeploymentState.ACTIVE_BLUE, rs.reconciledStatus.getBlueGreenState());
+ var flinkDeployments = getFlinkDeployments();
+ assertEquals(2, flinkDeployments.size());
+ assertEquals(
+ JobStatus.RUNNING, flinkDeployments.get(0).getStatus().getJobStatus().getState());
+ assertEquals(
+ ReconciliationState.DEPLOYED,
+ flinkDeployments.get(0).getStatus().getReconciliationStatus().getState());
+ // The B/G controller changes the State = SUSPENDED, the actual suspension is done by the
+ // FlinkDeploymentController
+ assertEquals(JobState.SUSPENDED, flinkDeployments.get(1).getSpec().getJob().getState());
+ assertEquals(
+ ReconciliationState.UPGRADING,
+ flinkDeployments.get(1).getStatus().getReconciliationStatus().getState());
+ assertTrue(instantStrToMillis(rs.reconciledStatus.getAbortTimestamp()) > 0);
+
+ // Simulate another change in the spec to trigger a redeployment
+ customValue = UUID.randomUUID().toString();
+ simulateChangeInSpec(rs.deployment, customValue, ALT_DELETION_DELAY_VALUE, null);
+
+ // Initiate the redeployment
+ 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 {
+ var blueGreenDeployment =
+ buildSessionCluster(
+ TEST_DEPLOYMENT_NAME,
+ TEST_NAMESPACE,
+ flinkVersion,
+ TEST_INITIAL_SAVEPOINT_PATH,
+ UpgradeMode.STATELESS);
+
+ // Initiate the Blue deployment
+ var originalSpec = blueGreenDeployment.getSpec();
+ var rs = initialPhaseBasicDeployment(blueGreenDeployment, false);
+
+ // Job starting...
+ simulateSubmitAndSuccessfulJobStart(getFlinkDeployments().get(0));
+
+ // Simulate a spec change before the transition is complete
+ simulateChangeInSpec(rs.deployment, "MODIFIED_VALUE", 0, null);
+ var moddedSpec = rs.deployment.getSpec();
+ rs = reconcile(rs.deployment);
+
+ // The spec change should have been preserved
+ assertNotEquals(
+ SpecUtils.writeSpecAsJSON(originalSpec, "spec"),
+ SpecUtils.writeSpecAsJSON(rs.deployment.getSpec(), "spec"));
+
+ assertEquals(
+ SpecUtils.writeSpecAsJSON(moddedSpec, "spec"),
+ rs.deployment.getStatus().getLastReconciledSpec(),
+ "spec");
+ }
+
+ @ParameterizedTest
+ @MethodSource("org.apache.flink.kubernetes.operator.TestUtils#flinkVersions")
+ public void verifyFailureBeforeFirstDeployment(FlinkVersion flinkVersion) throws Exception {
+ var blueGreenDeployment =
+ buildSessionCluster(
+ TEST_DEPLOYMENT_NAME,
+ TEST_NAMESPACE,
+ flinkVersion,
+ TEST_INITIAL_SAVEPOINT_PATH,
+ UpgradeMode.STATELESS);
+
+ // Initiate the Blue deployment
+ var rs = initialPhaseBasicDeployment(blueGreenDeployment, false);
+
+ // Simulating the job did not start correctly before the AbortGracePeriodMs
+ Thread.sleep(MINIMUM_ABORT_GRACE_PERIOD);
+
+ rs = reconcile(rs.deployment);
+
+ assertFailingJobStatus(rs);
+
+ // No longer TRANSITIONING_TO_GREEN and rolled back to INITIALIZING_BLUE
+ assertEquals(
+ FlinkBlueGreenDeploymentState.INITIALIZING_BLUE,
+ rs.reconciledStatus.getBlueGreenState());
+ var flinkDeployments = getFlinkDeployments();
+ assertEquals(1, flinkDeployments.size());
+ // The B/G controller changes the State = SUSPENDED, the actual suspension is done by the
+ // FlinkDeploymentController
+ assertEquals(JobState.SUSPENDED, flinkDeployments.get(0).getSpec().getJob().getState());
+
+ // No-op if the spec remains the same
+ rs = reconcile(rs.deployment);
+ assertTrue(rs.updateControl.isNoUpdate());
+
+ simulateChangeInSpec(rs.deployment, "MODIFIED_VALUE", 0, null);
+
+ // Resubmitting should re-start the Initialization to Blue
+ rs = reconcile(rs.deployment);
+
+ // Any error should've been cleaned up
+ assertNull(rs.reconciledStatus.getError());
+ assertTrue(rs.updateControl.isPatchStatus());
+ assertTrue(
+ rs.updateControl.getScheduleDelay().isPresent()
+ && rs.updateControl.getScheduleDelay().get() > 0);
+ flinkDeployments = getFlinkDeployments();
+ assertEquals(1, flinkDeployments.size());
+ assertEquals(
+ FlinkBlueGreenDeploymentState.TRANSITIONING_TO_BLUE,
+ rs.reconciledStatus.getBlueGreenState());
+ }
+
+ @ParameterizedTest
+ @MethodSource("patchScenarioProvider")
+ public void verifyPatchScenario(FlinkVersion flinkVersion, PatchTestCase testCase)
+ throws Exception {
+ var rs = setupActiveBlueDeployment(flinkVersion);
+
+ testCase.applyChanges(rs.deployment, kubernetesClient);
+
+ // PatchTopLevelTestCase should now be ignored (return noUpdate)
+ if (testCase instanceof PatchTopLevelTestCase) {
+ var result = reconcileAndVerifyIgnoreBehavior(rs);
+ testCase.verifySpecificBehavior(result, getFlinkDeployments());
+ } else {
+ var result = reconcileAndVerifyPatchBehavior(rs);
+ testCase.verifySpecificBehavior(result, getFlinkDeployments());
+ assertFinalized(
+ result.minReconciliationTs,
+ result.rs,
+ FlinkBlueGreenDeploymentState.ACTIVE_BLUE);
+ }
+ }
+
+ @ParameterizedTest
+ @MethodSource("org.apache.flink.kubernetes.operator.TestUtils#flinkVersions")
+ public void verifySavepointFailureRecovery(FlinkVersion flinkVersion) throws Exception {
+ var blueGreenDeployment =
+ buildSessionCluster(
+ TEST_DEPLOYMENT_NAME,
+ TEST_NAMESPACE,
+ flinkVersion,
+ null,
+ UpgradeMode.LAST_STATE);
+
+ var rs = executeBasicDeployment(flinkVersion, blueGreenDeployment, false, null);
+
+ // First attempt: Configure service to throw exception
+ flinkService.setSavepointTriggerException(
+ new IllegalStateException("Job not in valid state for savepoint"));
+
+ String customValue = UUID.randomUUID().toString();
+ simulateSpecChange(rs.deployment, customValue);
+
+ // Should fail with savepoint error
+ rs = reconcile(rs.deployment);
+ assertFailingWithError(rs, "Job not in valid state for savepoint");
+
+ // Recovery: Clear the exception and try again with new spec change
+ flinkService.clearSavepointTriggerException();
+ customValue = UUID.randomUUID().toString() + "_recovery";
+ simulateChangeInSpec(rs.deployment, customValue, ALT_DELETION_DELAY_VALUE, null);
+
+ // Should now succeed and trigger savepoint properly
+ rs = handleSavepoint(rs);
+
+ // Continue with successful transition
+ testTransitionToGreen(rs, customValue, "savepoint_1");
+ }
+
+ @ParameterizedTest
+ @MethodSource("savepointExceptionProvider")
+ public void verifySavepointFailureWithDifferentExceptionTypes(
+ FlinkVersion flinkVersion, Exception savepointException) throws Exception {
+
+ String expectedErrorFragment = savepointException.getMessage();
+
+ var blueGreenDeployment =
+ buildSessionCluster(
+ TEST_DEPLOYMENT_NAME,
+ TEST_NAMESPACE,
+ flinkVersion,
+ null,
+ UpgradeMode.SAVEPOINT);
+ var rs = executeBasicDeployment(flinkVersion, blueGreenDeployment, false, null);
+
+ flinkService.setSavepointTriggerException(savepointException);
+ simulateChangeInSpec(rs.deployment, UUID.randomUUID().toString(), 0, null);
+
+ rs = reconcile(rs.deployment);
+
+ assertFailingJobStatus(rs);
+ assertTrue(rs.reconciledStatus.getError().contains("Could not trigger Savepoint"));
+ assertTrue(rs.reconciledStatus.getError().contains(expectedErrorFragment));
+
+ // Should remain in ACTIVE_BLUE state (no transition started)
+ assertEquals(
+ FlinkBlueGreenDeploymentState.ACTIVE_BLUE, rs.reconciledStatus.getBlueGreenState());
+
+ // Verify only Blue deployment exists (Green was never created)
+ var flinkDeployments = getFlinkDeployments();
+ assertEquals(1, flinkDeployments.size());
+ }
+
+ @ParameterizedTest
+ @MethodSource("org.apache.flink.kubernetes.operator.TestUtils#flinkVersions")
+ public void verifySavepointFetchFailureRecovery(FlinkVersion flinkVersion) throws Exception {
+ String error = "Savepoint corrupted or not found";
+
+ var blueGreenDeployment =
+ buildSessionCluster(
+ TEST_DEPLOYMENT_NAME,
+ TEST_NAMESPACE,
+ flinkVersion,
+ null,
+ UpgradeMode.SAVEPOINT);
+
+ var rs = executeBasicDeployment(flinkVersion, blueGreenDeployment, false, null);
+
+ String customValue = UUID.randomUUID().toString();
+ simulateSpecChange(rs.deployment, customValue);
+
+ // Trigger savepoint successfully and go through savepointing flow
+ rs = handleSavepoint(rs);
+
+ // Configure service to return fetch error
+ flinkService.setSavepointFetchError(error);
+
+ // The next reconciliation will fail in configureInitialSavepoint due to fetch error
+ rs = reconcile(rs.deployment);
+ assertFailingWithError(rs, "Could not start Transition", error);
+
+ // Recovery: Clear the fetch error and try again with new spec change
+ flinkService.clearSavepointFetchError();
+ customValue = UUID.randomUUID().toString() + "_recovery";
+ simulateChangeInSpec(rs.deployment, customValue, ALT_DELETION_DELAY_VALUE, null);
+
+ // Should now succeed and complete transition properly
+ rs = handleSavepoint(rs);
+
+ // Continue with successful transition - second savepoint will be "savepoint_2"
+ testTransitionToGreen(rs, customValue, "savepoint_2");
+ }
+
+ @ParameterizedTest
+ @MethodSource("savepointErrorProvider")
+ public void verifySavepointFetchFailureWithDifferentErrors(
+ FlinkVersion flinkVersion, String errorMessage, boolean isFetchError) throws Exception {
+
+ var blueGreenDeployment =
+ buildSessionCluster(
+ TEST_DEPLOYMENT_NAME,
+ TEST_NAMESPACE,
+ flinkVersion,
+ null,
+ UpgradeMode.SAVEPOINT);
+ var rs = executeBasicDeployment(flinkVersion, blueGreenDeployment, false, null);
+
+ simulateSpecChange(rs.deployment, UUID.randomUUID().toString());
+
+ if (isFetchError) {
+ // Trigger savepoint successfully and go through savepointing flow
+ rs = handleSavepoint(rs);
+
+ // Configure service to return fetch error
+ flinkService.setSavepointFetchError(errorMessage);
+
+ // The next reconciliation will fail in configureInitialSavepoint due to fetch error
+ rs = reconcile(rs.deployment);
+ assertFailingWithError(rs, "Could not start Transition", errorMessage);
+ } else {
+ // Configure service to throw trigger exception
+ flinkService.setSavepointTriggerException(new RuntimeException(errorMessage));
+
+ rs = reconcile(rs.deployment);
+ assertFailingWithError(rs, "Could not trigger Savepoint", errorMessage);
+ }
+
+ // Should remain in ACTIVE_BLUE state after failure
+ assertEquals(
+ FlinkBlueGreenDeploymentState.ACTIVE_BLUE, rs.reconciledStatus.getBlueGreenState());
+
+ // Only Blue deployment should exist (Green transition never started)
+ var flinkDeployments = getFlinkDeployments();
+ assertEquals(1, flinkDeployments.size());
+ }
+
+ // ==================== Parameterized Test Inputs ====================
+
+ static Stream savepointErrorProvider() {
+ return TestUtils.flinkVersions()
+ .flatMap(
+ flinkVersionArgs -> {
+ FlinkVersion version = (FlinkVersion) flinkVersionArgs.get()[0];
+ return Stream.of(
+ // Fetch errors
+ Arguments.of(version, "Savepoint file corrupted", true),
+ Arguments.of(version, "Storage system unavailable", true),
+ Arguments.of(
+ version, "Access denied to savepoint location", true),
+ Arguments.of(version, "Savepoint metadata missing", true),
+ // Trigger exceptions
+ Arguments.of(version, "Network timeout", false),
+ Arguments.of(version, "Job not running", false),
+ Arguments.of(version, "Service unavailable", false),
+ Arguments.of(version, "Generic error", false));
+ });
+ }
+
+ static Stream savepointExceptionProvider() {
+ return TestUtils.flinkVersions()
+ .flatMap(
+ flinkVersionArgs -> {
+ FlinkVersion version = (FlinkVersion) flinkVersionArgs.get()[0];
+ return Stream.of(
+ Arguments.of(version, new IOException("Network timeout")),
+ Arguments.of(
+ version, new IllegalStateException("Job not running")),
+ Arguments.of(
+ version, new RuntimeException("Service unavailable")),
+ Arguments.of(version, new Exception("Generic error")));
+ });
+ }
+
+ static Stream patchScenarioProvider() {
+ // Extract FlinkVersions from TestUtils and combine with PatchTypes
+ return TestUtils.flinkVersions()
+ .flatMap(
+ args -> {
+ FlinkVersion version = (FlinkVersion) args.get()[0];
+ return Stream.of(
+ Arguments.of(version, new PatchChildTestCase()),
+ Arguments.of(version, new PatchTopLevelTestCase()),
+ Arguments.of(version, new PatchBothTestCase()));
+ });
+ }
+
+ static Stream flinkVersionsAndSavepointPaths() {
+ return TestUtils.flinkVersions()
+ .flatMap(
+ args -> {
+ FlinkVersion version = (FlinkVersion) args.get()[0];
+ return Stream.of(
+ Arguments.of(version, null),
+ Arguments.of(version, TEST_INITIAL_SAVEPOINT_PATH));
+ });
+ }
+
+ static Stream flinkVersionsAndSavepointPathsAndUpgradeModes() {
+ return TestUtils.flinkVersions()
+ .flatMap(
+ args -> {
+ FlinkVersion version = (FlinkVersion) args.get()[0];
+ return Stream.of(
+ Arguments.of(version, null, UpgradeMode.SAVEPOINT),
+ Arguments.of(version, null, UpgradeMode.LAST_STATE),
+ Arguments.of(version, null, UpgradeMode.STATELESS),
+ Arguments.of(
+ version,
+ TEST_INITIAL_SAVEPOINT_PATH,
+ UpgradeMode.SAVEPOINT),
+ Arguments.of(
+ version,
+ TEST_INITIAL_SAVEPOINT_PATH,
+ UpgradeMode.LAST_STATE),
+ Arguments.of(
+ version,
+ TEST_INITIAL_SAVEPOINT_PATH,
+ UpgradeMode.STATELESS));
+ });
+ }
+
+ // ==================== Test Case Interfaces and Implementations ====================
+
+ interface PatchTestCase {
+ void applyChanges(FlinkBlueGreenDeployment deployment, KubernetesClient client);
+
+ void verifySpecificBehavior(ReconcileResult result, List deployments);
+ }
+
+ static class PatchChildTestCase implements PatchTestCase {
+ @Override
+ public void applyChanges(FlinkBlueGreenDeployment deployment, KubernetesClient client) {
+ FlinkDeploymentSpec spec = deployment.getSpec().getTemplate().getSpec();
+
+ // Add a configuration change that ReflectiveDiffBuilder considers ignorable
+ spec.getFlinkConfiguration()
+ .put("kubernetes.operator.reconcile.interval", "100 SECONDS");
+
+ deployment.getSpec().getTemplate().setSpec(spec);
+ client.resource(deployment).createOrReplace();
+ }
+
+ @Override
+ public void verifySpecificBehavior(
+ ReconcileResult result, List deployments) {
+ assertEquals(1, deployments.size());
+ assertEquals(
+ "100 SECONDS",
+ getFlinkConfigurationValue(
+ deployments.get(0).getSpec(),
+ "kubernetes.operator.reconcile.interval"));
+ }
+ }
+
+ static class PatchTopLevelTestCase implements PatchTestCase {
+ @Override
+ public void applyChanges(FlinkBlueGreenDeployment deployment, KubernetesClient client) {
+ FlinkBlueGreenDeploymentSpec bgSpec = deployment.getSpec();
+ FlinkDeploymentTemplateSpec template = bgSpec.getTemplate();
+ Map configuration = new HashMap<>(bgSpec.getConfiguration());
+ configuration.put("custom.top.level", "custom-top-level-value");
+ bgSpec.setConfiguration(configuration);
+ bgSpec.setTemplate(template);
+ client.resource(deployment).createOrReplace();
+ }
+
+ @Override
+ public void verifySpecificBehavior(
+ ReconcileResult result, List deployments) {
+ assertEquals(1, deployments.size());
+ var existingDeployment = result.existingFlinkDeployment;
+ var currentDeployment = deployments.get(0);
+
+ // FlinkDeployment should remain unchanged for top-level only changes
+ assertEquals(existingDeployment, currentDeployment);
+ }
+ }
+
+ static class PatchBothTestCase implements PatchTestCase {
+ @Override
+ public void applyChanges(FlinkBlueGreenDeployment deployment, KubernetesClient client) {
+ FlinkBlueGreenDeploymentSpec bgSpec = deployment.getSpec();
+ FlinkDeploymentTemplateSpec template = bgSpec.getTemplate();
+
+ // 1. Add top-level configuration change
+ Map configuration = new HashMap<>(bgSpec.getConfiguration());
+ configuration.put("custom.both.level", "custom-both-level-value");
+ bgSpec.setConfiguration(configuration);
+
+ // 2. Add nested spec change
+ FlinkDeploymentSpec spec = template.getSpec();
+ spec.getFlinkConfiguration()
+ .put("kubernetes.operator.reconcile.interval", "100 SECONDS");
+ template.setSpec(spec);
+
+ bgSpec.setTemplate(template);
+ client.resource(deployment).createOrReplace();
+ }
+
+ @Override
+ public void verifySpecificBehavior(
+ ReconcileResult result, List deployments) {
+ assertEquals(1, deployments.size());
+
+ // Child spec change should be applied to FlinkDeployment
+ assertEquals(
+ "100 SECONDS",
+ getFlinkConfigurationValue(
+ deployments.get(0).getSpec(),
+ "kubernetes.operator.reconcile.interval"));
+
+ // Top-level changes should be preserved in reconciled spec
+ assertNotNull(result.rs.reconciledStatus.getLastReconciledSpec());
+ assertEquals(
+ SpecUtils.writeSpecAsJSON(result.rs.deployment.getSpec(), "spec"),
+ result.rs.reconciledStatus.getLastReconciledSpec());
+ }
+ }
+
+ // ==================== Helper Classes ====================
+
+ static class ReconcileResult {
+ final long minReconciliationTs;
+ final TestingFlinkBlueGreenDeploymentController.BlueGreenReconciliationResult rs;
+ final FlinkDeployment existingFlinkDeployment;
+
+ ReconcileResult(
+ long minReconciliationTs,
+ TestingFlinkBlueGreenDeploymentController.BlueGreenReconciliationResult rs,
+ FlinkDeployment existingFlinkDeployment) {
+ this.minReconciliationTs = minReconciliationTs;
+ this.rs = rs;
+ this.existingFlinkDeployment = existingFlinkDeployment;
+ }
+ }
+
+ // ==================== Common Test Helper Methods ====================
+
+ private TestingFlinkBlueGreenDeploymentController.BlueGreenReconciliationResult
+ setupActiveBlueDeployment(FlinkVersion flinkVersion) throws Exception {
+ var blueGreenDeployment =
+ buildSessionCluster(
+ TEST_DEPLOYMENT_NAME,
+ TEST_NAMESPACE,
+ flinkVersion,
+ TEST_INITIAL_SAVEPOINT_PATH,
+ UpgradeMode.STATELESS);
+ return executeBasicDeployment(
+ flinkVersion, blueGreenDeployment, false, TEST_INITIAL_SAVEPOINT_PATH);
+ }
+
+ private ReconcileResult reconcileAndVerifyPatchBehavior(
+ TestingFlinkBlueGreenDeploymentController.BlueGreenReconciliationResult rs)
+ throws Exception {
+
+ // Initiating a patch operation
+ var minReconciliationTs = System.currentTimeMillis() - 1;
+ rs = reconcile(rs.deployment);
+
+ assertPatchOperationTriggered(rs, minReconciliationTs);
+ assertTransitioningState(rs);
+
+ var flinkDeployments = getFlinkDeployments();
+ assertEquals(1, flinkDeployments.size());
+
+ // The patch operation reinitialized the deployment, simulating startup
+ simulateSuccessfulJobStart(flinkDeployments.get(0));
+
+ minReconciliationTs = System.currentTimeMillis() - 1;
+ rs = reconcile(rs.deployment);
+
+ return new ReconcileResult(minReconciliationTs, rs, null);
+ }
+
+ private ReconcileResult reconcileAndVerifyIgnoreBehavior(
+ TestingFlinkBlueGreenDeploymentController.BlueGreenReconciliationResult rs)
+ throws Exception {
+
+ var flinkDeployments = getFlinkDeployments();
+ assertEquals(1, flinkDeployments.size());
+ var existingFlinkDeployment = flinkDeployments.get(0);
+
+ var minReconciliationTs = System.currentTimeMillis() - 1;
+ rs = reconcile(rs.deployment);
+
+ assertIgnoreOperationTriggered(rs);
+
+ return new ReconcileResult(minReconciliationTs, rs, existingFlinkDeployment);
+ }
+
+ private void assertIgnoreOperationTriggered(
+ TestingFlinkBlueGreenDeploymentController.BlueGreenReconciliationResult rs) {
+ // For IGNORE behavior, we expect noUpdate (no patch status, no reschedule)
+ assertFalse(rs.updateControl.isPatchStatus());
+ assertFalse(rs.updateControl.isPatchResource());
+ assertFalse(rs.updateControl.getScheduleDelay().isPresent());
+ }
+
+ private void assertPatchOperationTriggered(
+ TestingFlinkBlueGreenDeploymentController.BlueGreenReconciliationResult rs,
+ long minReconciliationTs) {
+ assertTrue(rs.updateControl.isPatchStatus());
+ assertTrue(rs.updateControl.getScheduleDelay().isPresent());
+ assertTrue(rs.updateControl.getScheduleDelay().get() > 0);
+ assertTrue(
+ minReconciliationTs
+ < instantStrToMillis(rs.reconciledStatus.getLastReconciledTimestamp()));
+ }
+
+ private void assertTransitioningState(
+ TestingFlinkBlueGreenDeploymentController.BlueGreenReconciliationResult rs) {
+ assertEquals(
+ FlinkBlueGreenDeploymentState.TRANSITIONING_TO_BLUE,
+ rs.reconciledStatus.getBlueGreenState());
+ assertEquals(JobStatus.RECONCILING, rs.reconciledStatus.getJobStatus().getState());
+ }
+
+ private static void assertFailingJobStatus(
+ TestingFlinkBlueGreenDeploymentController.BlueGreenReconciliationResult rs) {
+ assertEquals(JobStatus.FAILING, rs.reconciledStatus.getJobStatus().getState());
+ assertNotNull(rs.reconciledStatus.getError());
+ }
+
+ private static void assertFailingWithError(
+ TestingFlinkBlueGreenDeploymentController.BlueGreenReconciliationResult rs,
+ String... expectedErrorFragments) {
+ assertFailingJobStatus(rs);
+ for (String fragment : expectedErrorFragments) {
+ assertTrue(rs.reconciledStatus.getError().contains(fragment));
+ }
+ }
+
+ private void assertFinalized(
+ long minReconciliationTs,
+ TestingFlinkBlueGreenDeploymentController.BlueGreenReconciliationResult rs,
+ FlinkBlueGreenDeploymentState expectedBGDeploymentState)
+ throws Exception {
+ assertTrue(rs.updateControl.isPatchStatus());
+ assertTrue(
+ minReconciliationTs
+ < instantStrToMillis(rs.reconciledStatus.getLastReconciledTimestamp()));
+ assertNotNull(rs.reconciledStatus.getLastReconciledSpec());
+ assertEquals(
+ SpecUtils.writeSpecAsJSON(rs.deployment.getSpec(), "spec"),
+ rs.reconciledStatus.getLastReconciledSpec());
+ assertEquals(expectedBGDeploymentState, rs.reconciledStatus.getBlueGreenState());
+ assertEquals(JobStatus.RUNNING, rs.reconciledStatus.getJobStatus().getState());
+ assertEquals(0, instantStrToMillis(rs.reconciledStatus.getDeploymentReadyTimestamp()));
+ assertEquals(0, instantStrToMillis(rs.reconciledStatus.getAbortTimestamp()));
+
+ // Subsequent reconciliation calls after finalization = NO-OP
+ rs = reconcile(rs.deployment);
+ assertTrue(rs.updateControl.isNoUpdate());
+ }
+
+ private TestingFlinkBlueGreenDeploymentController.BlueGreenReconciliationResult
+ executeBasicDeployment(
+ FlinkVersion flinkVersion,
+ FlinkBlueGreenDeployment blueGreenDeployment,
+ boolean execAssertions,
+ String expectedInitialSavepointPath)
+ throws Exception {
+
+ // 1. Initiate the Blue deployment
+ var rs = initialPhaseBasicDeployment(blueGreenDeployment, execAssertions);
+
+ var flinkDeployments = getFlinkDeployments();
+ var deploymentA = flinkDeployments.get(0);
+
+ if (execAssertions) {
+ assertEquals(1, flinkDeployments.size());
+ verifyOwnerReferences(rs.deployment, deploymentA);
+ assertEquals(
+ expectedInitialSavepointPath,
+ deploymentA.getSpec().getJob().getInitialSavepointPath());
+ }
+
+ simulateSubmitAndSuccessfulJobStart(deploymentA);
+
+ // 2. Mark the Blue deployment ready and finalize it
+ var minReconciliationTs = System.currentTimeMillis() - 1;
+ rs = reconcile(rs.deployment);
+
+ if (execAssertions) {
+ assertFinalized(minReconciliationTs, rs, FlinkBlueGreenDeploymentState.ACTIVE_BLUE);
+ }
+
+ return rs;
+ }
+
+ @NotNull
+ private TestingFlinkBlueGreenDeploymentController.BlueGreenReconciliationResult
+ initialPhaseBasicDeployment(
+ FlinkBlueGreenDeployment blueGreenDeployment, boolean execAssertions)
+ throws Exception {
+ Long minReconciliationTs = System.currentTimeMillis() - 1;
+
+ // 1a. Initializing deploymentStatus with this call
+ var rs = reconcile(blueGreenDeployment);
+
+ if (execAssertions) {
+ assertTrue(rs.updateControl.getScheduleDelay().isPresent());
+ assertEquals(0, (long) rs.updateControl.getScheduleDelay().get());
+ assertEquals(
+ FlinkBlueGreenDeploymentState.INITIALIZING_BLUE,
+ rs.reconciledStatus.getBlueGreenState());
+ }
+
+ // 1b. Executing the actual deployment
+ rs = reconcile(rs.deployment);
+
+ if (execAssertions) {
+ assertTrue(rs.updateControl.isPatchStatus());
+ assertTrue(
+ minReconciliationTs
+ < instantStrToMillis(rs.reconciledStatus.getLastReconciledTimestamp()));
+
+ // check the status (reconciled spec, reconciled ts, a/b state)
+ assertEquals(
+ FlinkBlueGreenDeploymentState.TRANSITIONING_TO_BLUE,
+ rs.reconciledStatus.getBlueGreenState());
+ assertNotNull(rs.reconciledStatus.getLastReconciledSpec());
+ assertEquals(JobStatus.RECONCILING, rs.reconciledStatus.getJobStatus().getState());
+ assertNull(rs.reconciledStatus.getDeploymentReadyTimestamp());
+ }
+
+ return rs;
+ }
+
+ private void assertDeploymentDeleted(
+ TestingFlinkBlueGreenDeploymentController.BlueGreenReconciliationResult rs,
+ long expectedDeletionDelay,
+ FlinkBlueGreenDeploymentSpec bgSpecBefore)
+ throws Exception {
+ var deletionDelay = rs.updateControl.getScheduleDelay().get();
+
+ assertTrue(rs.updateControl.isPatchStatus());
+ assertEquals(expectedDeletionDelay, deletionDelay);
+ assertTrue(instantStrToMillis(rs.reconciledStatus.getDeploymentReadyTimestamp()) > 0);
+ assertEquals(
+ SpecUtils.writeSpecAsJSON(bgSpecBefore, "spec"),
+ rs.reconciledStatus.getLastReconciledSpec());
+
+ // A reconciliation before the deletion delay has expired should result in no-op
+ rs = reconcile(rs.deployment);
+ var remainingDeletionDelay = rs.updateControl.getScheduleDelay().get();
+ assertTrue(remainingDeletionDelay <= expectedDeletionDelay);
+ assertTrue(rs.updateControl.isNoUpdate());
+
+ Thread.sleep(remainingDeletionDelay);
+ }
+
+ private void testTransitionToGreen(
+ TestingFlinkBlueGreenDeploymentController.BlueGreenReconciliationResult rs,
+ String customValue,
+ String expectedSavepointPath)
+ throws Exception {
+
+ // Initiate the Green deployment
+ Long minReconciliationTs = System.currentTimeMillis() - 1;
+ var bgSpecBefore = rs.deployment.getSpec();
+ rs = reconcile(rs.deployment);
+
+ var flinkDeployments = getFlinkDeployments();
+ var greenDeploymentName = flinkDeployments.get(1).getMetadata().getName();
+
+ // Any error should've been cleaned up
+ assertNull(rs.reconciledStatus.getError());
+ assertTrue(rs.updateControl.isPatchStatus());
+ assertTrue(
+ minReconciliationTs
+ < instantStrToMillis(rs.reconciledStatus.getLastReconciledTimestamp()));
+ assertEquals(2, flinkDeployments.size());
+ assertNull(flinkDeployments.get(0).getSpec().getJob().getInitialSavepointPath());
+ assertEquals(
+ expectedSavepointPath,
+ flinkDeployments.get(1).getSpec().getJob().getInitialSavepointPath());
+
+ assertEquals(
+ FlinkBlueGreenDeploymentState.TRANSITIONING_TO_GREEN,
+ rs.reconciledStatus.getBlueGreenState());
+ assertNotNull(rs.reconciledStatus.getLastReconciledSpec());
+ assertEquals(0, instantStrToMillis(rs.reconciledStatus.getDeploymentReadyTimestamp()));
+ assertEquals(
+ customValue,
+ getFlinkConfigurationValue(
+ rs.deployment.getSpec().getTemplate().getSpec(), CUSTOM_CONFIG_FIELD));
+
+ // Initiate and mark the Green deployment ready
+ simulateSuccessfulJobStart(getFlinkDeployments().get(1));
+ rs = reconcile(rs.deployment);
+
+ // Logic for the deployment to get deleted
+ assertDeploymentDeleted(rs, ALT_DELETION_DELAY_VALUE, bgSpecBefore);
+
+ // Calling the rescheduled reconciliation (will delete the deployment)
+ reconcile(rs.deployment);
+
+ // Old Blue deployment deleted, Green is the active one
+ flinkDeployments = getFlinkDeployments();
+ assertEquals(1, flinkDeployments.size());
+ assertEquals(greenDeploymentName, flinkDeployments.get(0).getMetadata().getName());
+
+ minReconciliationTs = System.currentTimeMillis() - 1;
+ rs = reconcile(rs.deployment);
+
+ assertTrue(rs.updateControl.isPatchStatus());
+ assertFinalized(minReconciliationTs, rs, FlinkBlueGreenDeploymentState.ACTIVE_GREEN);
+ }
+
+ private void simulateChangeInSpec(
+ FlinkBlueGreenDeployment blueGreenDeployment,
+ String customFieldValue,
+ int customDeletionDelayMs,
+ String initialSavepointPath) {
+ FlinkBlueGreenDeploymentSpec bgSpec = blueGreenDeployment.getSpec();
+ FlinkDeploymentTemplateSpec template = bgSpec.getTemplate();
+
+ if (customDeletionDelayMs > 0) {
+ bgSpec.getConfiguration()
+ .put(DEPLOYMENT_DELETION_DELAY.key(), String.valueOf(customDeletionDelayMs));
+ }
+
+ FlinkDeploymentSpec spec = template.getSpec();
+ spec.getFlinkConfiguration().put(CUSTOM_CONFIG_FIELD, customFieldValue);
+
+ if (initialSavepointPath != null) {
+ spec.getJob().setInitialSavepointPath(initialSavepointPath);
+ }
+
+ template.setSpec(spec);
+ kubernetesClient.resource(blueGreenDeployment).createOrReplace();
+ }
+
+ private void simulateSpecChange(FlinkBlueGreenDeployment deployment, String customValue) {
+ simulateChangeInSpec(deployment, customValue, 0, null);
+ }
+
+ /*
+ Convenience function to reconcile and get the frequently used `BlueGreenReconciliationResult`
+ */
+ private TestingFlinkBlueGreenDeploymentController.BlueGreenReconciliationResult reconcile(
+ FlinkBlueGreenDeployment blueGreenDeployment) throws Exception {
+ UpdateControl updateControl =
+ testController.reconcile(blueGreenDeployment, context);
+
+ return new TestingFlinkBlueGreenDeploymentController.BlueGreenReconciliationResult(
+ updateControl,
+ updateControl.isNoUpdate()
+ ? blueGreenDeployment
+ : updateControl.getResource().get(),
+ updateControl.isNoUpdate()
+ ? blueGreenDeployment.getStatus()
+ : updateControl.getResource().get().getStatus());
+ }
+
+ private void simulateSubmitAndSuccessfulJobStart(FlinkDeployment deployment) throws Exception {
+ // TODO: is this correct? Doing this to give the TestingFlinkService awareness of the job
+ JobSpec jobSpec = deployment.getSpec().getJob();
+ Configuration conf = new Configuration();
+ conf.set(SavepointConfigOptions.SAVEPOINT_PATH, TEST_CHECKPOINT_PATH);
+ flinkService.submitApplicationCluster(jobSpec, conf, false);
+ var jobId = flinkService.listJobs().get(0).f1.getJobId().toString();
+ deployment.getStatus().getJobStatus().setJobId(jobId);
+ simulateSuccessfulJobStart(deployment);
+ }
+
+ private void simulateSuccessfulJobStart(FlinkDeployment deployment) {
+ deployment.getStatus().getJobStatus().setState(JobStatus.RUNNING);
+ deployment.getStatus().getReconciliationStatus().setState(ReconciliationState.DEPLOYED);
+ deployment
+ .getStatus()
+ .getReconciliationStatus()
+ .serializeAndSetLastReconciledSpec(deployment.getSpec(), deployment);
+ deployment.getStatus().getReconciliationStatus().markReconciledSpecAsStable();
+ kubernetesClient.resource(deployment).update();
+ }
+
+ private void simulateJobFailure(FlinkDeployment deployment) {
+ deployment.getStatus().getJobStatus().setState(JobStatus.RECONCILING);
+ deployment.getStatus().getReconciliationStatus().setState(ReconciliationState.UPGRADING);
+ kubernetesClient.resource(deployment).update();
+ }
+
+ private static void verifyOwnerReferences(
+ FlinkBlueGreenDeployment parent, FlinkDeployment child) {
+ var ownerReferences = child.getMetadata().getOwnerReferences();
+ assertEquals(1, ownerReferences.size());
+ var ownerRef = ownerReferences.get(0);
+ assertEquals(parent.getMetadata().getName(), ownerRef.getName());
+ assertEquals(parent.getKind(), ownerRef.getKind());
+ assertEquals(parent.getApiVersion(), ownerRef.getApiVersion());
+ }
+
+ private List getFlinkDeployments() {
+ return kubernetesClient
+ .resources(FlinkDeployment.class)
+ .inNamespace(TEST_NAMESPACE)
+ .list()
+ .getItems();
+ }
+
+ private static FlinkBlueGreenDeployment buildSessionCluster(
+ 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());
+
+ 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));
+
+ 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/TestingFlinkBlueGreenDeploymentController.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/TestingFlinkBlueGreenDeploymentController.java
new file mode 100644
index 0000000000..6e8e058ce2
--- /dev/null
+++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/TestingFlinkBlueGreenDeploymentController.java
@@ -0,0 +1,82 @@
+/*
+ * 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.Configuration;
+import org.apache.flink.kubernetes.operator.TestUtils;
+import org.apache.flink.kubernetes.operator.TestingFlinkResourceContextFactory;
+import org.apache.flink.kubernetes.operator.TestingFlinkService;
+import org.apache.flink.kubernetes.operator.api.FlinkBlueGreenDeployment;
+import org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentStatus;
+import org.apache.flink.kubernetes.operator.config.FlinkConfigManager;
+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;
+import lombok.Getter;
+
+/** A wrapper around {@link FlinkBlueGreenDeploymentController} used by unit tests. */
+public class TestingFlinkBlueGreenDeploymentController
+ implements Reconciler {
+
+ @Getter private TestingFlinkResourceContextFactory contextFactory;
+
+ public final FlinkBlueGreenDeploymentController flinkBlueGreenDeploymentController;
+
+ public TestingFlinkBlueGreenDeploymentController(
+ FlinkConfigManager configManager, TestingFlinkService flinkService) {
+ contextFactory =
+ new TestingFlinkResourceContextFactory(
+ configManager,
+ TestUtils.createTestMetricGroup(new Configuration()),
+ flinkService,
+ null);
+
+ flinkBlueGreenDeploymentController = new FlinkBlueGreenDeploymentController(contextFactory);
+ }
+
+ @Override
+ public UpdateControl reconcile(
+ FlinkBlueGreenDeployment flinkBlueGreenDeployment,
+ Context context)
+ throws Exception {
+ var cloned = ReconciliationUtils.clone(flinkBlueGreenDeployment);
+ 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 {
+ public UpdateControl updateControl;
+
+ public FlinkBlueGreenDeployment deployment;
+
+ public FlinkBlueGreenDeploymentStatus reconciledStatus;
+ }
+}
diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/diff/FlinkBlueGreenDeploymentSpecDiffTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/diff/FlinkBlueGreenDeploymentSpecDiffTest.java
new file mode 100644
index 0000000000..377868339f
--- /dev/null
+++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/diff/FlinkBlueGreenDeploymentSpecDiffTest.java
@@ -0,0 +1,326 @@
+/*
+ * 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.reconciler.diff;
+
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.kubernetes.operator.api.bluegreen.BlueGreenDiffType;
+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.KubernetesDeploymentMode;
+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.ObjectMeta;
+import org.junit.jupiter.api.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+/** Tests for FlinkBlueGreenDeploymentSpecDiff. */
+public class FlinkBlueGreenDeploymentSpecDiffTest {
+
+ private static final KubernetesDeploymentMode DEPLOYMENT_MODE = KubernetesDeploymentMode.NATIVE;
+
+ @Test
+ public void testNullValidation() {
+ FlinkBlueGreenDeploymentSpec validSpec = createBasicSpec();
+
+ // Test null left spec
+ assertThrows(
+ NullPointerException.class,
+ () -> new FlinkBlueGreenDeploymentSpecDiff(DEPLOYMENT_MODE, null, validSpec));
+
+ // Test null right spec
+ assertThrows(
+ NullPointerException.class,
+ () -> new FlinkBlueGreenDeploymentSpecDiff(DEPLOYMENT_MODE, validSpec, null));
+
+ // Test null template in left spec
+ FlinkBlueGreenDeploymentSpec specWithNullTemplate = new FlinkBlueGreenDeploymentSpec();
+ specWithNullTemplate.setTemplate(null);
+
+ assertThrows(
+ IllegalArgumentException.class,
+ () ->
+ new FlinkBlueGreenDeploymentSpecDiff(
+ DEPLOYMENT_MODE, specWithNullTemplate, validSpec));
+
+ // Test null template in right spec
+ assertThrows(
+ IllegalArgumentException.class,
+ () ->
+ new FlinkBlueGreenDeploymentSpecDiff(
+ DEPLOYMENT_MODE, validSpec, specWithNullTemplate));
+
+ // Test null nested spec in template
+ FlinkBlueGreenDeploymentSpec specWithNullNestedSpec = new FlinkBlueGreenDeploymentSpec();
+ FlinkDeploymentTemplateSpec templateWithNullSpec = new FlinkDeploymentTemplateSpec();
+ templateWithNullSpec.setSpec(null);
+ specWithNullNestedSpec.setTemplate(templateWithNullSpec);
+
+ assertThrows(
+ IllegalArgumentException.class,
+ () ->
+ new FlinkBlueGreenDeploymentSpecDiff(
+ DEPLOYMENT_MODE, specWithNullNestedSpec, validSpec));
+ }
+
+ @Test
+ public void testIgnoreForIdenticalSpecs() {
+ FlinkBlueGreenDeploymentSpec spec1 = createBasicSpec();
+ FlinkBlueGreenDeploymentSpec spec2 = createBasicSpec();
+
+ FlinkBlueGreenDeploymentSpecDiff diff =
+ new FlinkBlueGreenDeploymentSpecDiff(DEPLOYMENT_MODE, spec1, spec2);
+
+ assertEquals(BlueGreenDiffType.IGNORE, diff.compare());
+ }
+
+ @Test
+ public void testIgnoreForMetadataDifference() {
+ FlinkBlueGreenDeploymentSpec spec1 = createBasicSpec();
+ FlinkBlueGreenDeploymentSpec spec2 = createBasicSpec();
+
+ // Change metadata in spec2 - should be ignored since only FlinkDeploymentSpec matters
+ ObjectMeta metadata = new ObjectMeta();
+ metadata.setName("different-name");
+ spec2.getTemplate().setMetadata(metadata);
+
+ FlinkBlueGreenDeploymentSpecDiff diff =
+ new FlinkBlueGreenDeploymentSpecDiff(DEPLOYMENT_MODE, spec1, spec2);
+
+ assertEquals(BlueGreenDiffType.IGNORE, diff.compare());
+ }
+
+ @Test
+ public void testIgnoreForConfigurationDifference() {
+ FlinkBlueGreenDeploymentSpec spec1 = createBasicSpec();
+ FlinkBlueGreenDeploymentSpec spec2 = createBasicSpec();
+
+ // Change configuration in spec2 - should be ignored since only FlinkDeploymentSpec matters
+ Map config = new HashMap<>();
+ config.put("custom.config", "different-value");
+ spec2.setConfiguration(config);
+
+ FlinkBlueGreenDeploymentSpecDiff diff =
+ new FlinkBlueGreenDeploymentSpecDiff(DEPLOYMENT_MODE, spec1, spec2);
+
+ assertEquals(BlueGreenDiffType.IGNORE, diff.compare());
+ }
+
+ @Test
+ public void testTransitionForNestedSpecDifference() {
+ FlinkBlueGreenDeploymentSpec spec1 = createBasicSpec();
+ FlinkBlueGreenDeploymentSpec spec2 = createBasicSpec();
+
+ // Change nested spec property - setSavepointRedeployNonce triggers TRANSITION
+ spec2.getTemplate().getSpec().getJob().setSavepointRedeployNonce(12345L);
+
+ FlinkBlueGreenDeploymentSpecDiff diff =
+ new FlinkBlueGreenDeploymentSpecDiff(DEPLOYMENT_MODE, spec1, spec2);
+
+ assertEquals(BlueGreenDiffType.TRANSITION, diff.compare());
+ }
+
+ @Test
+ public void testPatchChildForIgnorableNestedChanges() {
+ FlinkBlueGreenDeploymentSpec spec1 = createBasicSpec();
+ FlinkBlueGreenDeploymentSpec spec2 = createBasicSpec();
+
+ // Make changes that ReflectiveDiffBuilder considers IGNORE but specs are different
+ // Based on SpecDiffTest, these changes result in DiffType.IGNORE
+ spec2.getTemplate().getSpec().getJob().setUpgradeMode(UpgradeMode.LAST_STATE);
+ spec2.getTemplate().getSpec().getJob().setAllowNonRestoredState(true);
+ spec2.getTemplate().getSpec().getJob().setInitialSavepointPath("local:///tmp");
+ spec2.getTemplate().getSpec().getJob().setSavepointTriggerNonce(123L);
+
+ FlinkBlueGreenDeploymentSpecDiff diff =
+ new FlinkBlueGreenDeploymentSpecDiff(DEPLOYMENT_MODE, spec1, spec2);
+
+ assertEquals(BlueGreenDiffType.PATCH_CHILD, diff.compare());
+ }
+
+ @Test
+ public void testPatchChildForIgnorableConfigChanges() {
+ FlinkBlueGreenDeploymentSpec spec1 = createBasicSpec();
+ FlinkBlueGreenDeploymentSpec spec2 = createBasicSpec();
+
+ spec2.getTemplate().getSpec().setFlinkConfiguration(new HashMap<>());
+
+ // Add configuration changes that ReflectiveDiffBuilder considers ignorable
+ spec2.getTemplate()
+ .getSpec()
+ .getFlinkConfiguration()
+ .put("kubernetes.operator.reconcile.interval", "100 SECONDS");
+
+ FlinkBlueGreenDeploymentSpecDiff diff =
+ new FlinkBlueGreenDeploymentSpecDiff(DEPLOYMENT_MODE, spec1, spec2);
+
+ assertEquals(BlueGreenDiffType.PATCH_CHILD, diff.compare());
+ }
+
+ @Test
+ public void testPatchChildWithTopLevelAndIgnorableNestedChanges() {
+ FlinkBlueGreenDeploymentSpec spec1 = createBasicSpec();
+ FlinkBlueGreenDeploymentSpec spec2 = createBasicSpec();
+
+ // Change both top-level (which should be ignored) and nested spec (ignorable changes)
+ Map config = new HashMap<>();
+ config.put("custom.config", "different-value");
+ spec2.setConfiguration(config);
+
+ // Add nested changes that are ignorable by ReflectiveDiffBuilder
+ spec2.getTemplate().getSpec().getJob().setUpgradeMode(UpgradeMode.SAVEPOINT);
+ spec2.getTemplate().getSpec().getJob().setInitialSavepointPath("local:///tmp/savepoints");
+
+ FlinkBlueGreenDeploymentSpecDiff diff =
+ new FlinkBlueGreenDeploymentSpecDiff(DEPLOYMENT_MODE, spec1, spec2);
+
+ // Should return PATCH_CHILD since only FlinkDeploymentSpec matters and it has ignorable
+ // changes
+ assertEquals(BlueGreenDiffType.PATCH_CHILD, diff.compare());
+ }
+
+ // Note: Case 2 (ReflectiveDiffBuilder returns IGNORE for different FlinkDeploymentSpecs)
+ // may not occur in practice with current ReflectiveDiffBuilder implementation.
+ // If FlinkDeploymentSpecs are different, ReflectiveDiffBuilder typically returns
+ // a meaningful diff type (SCALE, UPGRADE, etc.) rather than IGNORE.
+ // The logic is preserved in the implementation for completeness.
+
+ @Test
+ public void testTransitionForTopLevelAndNestedDifferences() {
+ FlinkBlueGreenDeploymentSpec spec1 = createBasicSpec();
+ FlinkBlueGreenDeploymentSpec spec2 = createBasicSpec();
+
+ // Change both top-level (configuration) and nested spec
+ // With new logic, only nested spec changes matter - setSavepointRedeployNonce triggers
+ // TRANSITION
+ Map config = new HashMap<>();
+ config.put("custom.config", "different-value");
+ spec2.setConfiguration(config);
+ spec2.getTemplate().getSpec().getJob().setSavepointRedeployNonce(12345L);
+
+ FlinkBlueGreenDeploymentSpecDiff diff =
+ new FlinkBlueGreenDeploymentSpecDiff(DEPLOYMENT_MODE, spec1, spec2);
+
+ assertEquals(BlueGreenDiffType.TRANSITION, diff.compare());
+ }
+
+ @Test
+ public void testTransitionForScaleDifference() {
+ FlinkBlueGreenDeploymentSpec spec1 = createBasicSpec();
+ FlinkBlueGreenDeploymentSpec spec2 = createBasicSpec();
+
+ // Change parallelism - this should trigger SCALE in ReflectiveDiffBuilder
+ spec2.getTemplate().getSpec().getJob().setParallelism(10);
+
+ FlinkBlueGreenDeploymentSpecDiff diff =
+ new FlinkBlueGreenDeploymentSpecDiff(DEPLOYMENT_MODE, spec1, spec2);
+
+ assertEquals(BlueGreenDiffType.TRANSITION, diff.compare());
+ }
+
+ @Test
+ public void testTransitionForUpgradeDifference() {
+ FlinkBlueGreenDeploymentSpec spec1 = createBasicSpec();
+ FlinkBlueGreenDeploymentSpec spec2 = createBasicSpec();
+
+ // Change Flink version - this should trigger UPGRADE in ReflectiveDiffBuilder
+ spec2.getTemplate().getSpec().setFlinkVersion(FlinkVersion.v1_17);
+
+ FlinkBlueGreenDeploymentSpecDiff diff =
+ new FlinkBlueGreenDeploymentSpecDiff(DEPLOYMENT_MODE, spec1, spec2);
+
+ assertEquals(BlueGreenDiffType.TRANSITION, diff.compare());
+ }
+
+ @Test
+ public void testTransitionOverridesPatchChild() {
+ FlinkBlueGreenDeploymentSpec spec1 = createBasicSpec();
+ FlinkBlueGreenDeploymentSpec spec2 = createBasicSpec();
+
+ // Change both top-level and nested spec, but nested change should trigger TRANSITION
+ // With new logic, only the nested FlinkDeploymentSpec change matters
+ Map config = new HashMap<>();
+ config.put("custom.config", "different-value");
+ spec2.setConfiguration(config);
+ spec2.getTemplate().getSpec().getJob().setParallelism(10); // This triggers SCALE
+
+ FlinkBlueGreenDeploymentSpecDiff diff =
+ new FlinkBlueGreenDeploymentSpecDiff(DEPLOYMENT_MODE, spec1, spec2);
+
+ // Should return TRANSITION since parallelism change triggers SCALE
+ assertEquals(BlueGreenDiffType.TRANSITION, diff.compare());
+ }
+
+ private FlinkBlueGreenDeploymentSpec createBasicSpec() {
+ // Create a basic FlinkDeploymentSpec
+ FlinkDeploymentSpec deploymentSpec =
+ FlinkDeploymentSpec.builder()
+ .image("flink:1.16")
+ .flinkVersion(FlinkVersion.v1_16)
+ .serviceAccount("flink")
+ .flinkConfiguration(new ConfigObjectNode())
+ .jobManager(
+ JobManagerSpec.builder()
+ .resource(
+ new Resource(
+ 1.0,
+ MemorySize.parse("1024m").toString(),
+ null))
+ .replicas(1)
+ .build())
+ .taskManager(
+ TaskManagerSpec.builder()
+ .resource(
+ new Resource(
+ 1.0,
+ MemorySize.parse("1024m").toString(),
+ null))
+ .build())
+ .job(
+ JobSpec.builder()
+ .jarURI(
+ "local:///opt/flink/examples/streaming/StateMachineExample.jar")
+ .parallelism(2)
+ .upgradeMode(UpgradeMode.STATELESS)
+ .state(JobState.RUNNING)
+ .build())
+ .build();
+
+ // Create template
+ FlinkDeploymentTemplateSpec template =
+ FlinkDeploymentTemplateSpec.builder().spec(deploymentSpec).build();
+
+ // Create Blue/Green spec
+ FlinkBlueGreenDeploymentSpec blueGreenSpec = new FlinkBlueGreenDeploymentSpec();
+ blueGreenSpec.setTemplate(template);
+
+ return blueGreenSpec;
+ }
+}
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
new file mode 100644
index 0000000000..d31d14aa24
--- /dev/null
+++ b/helm/flink-kubernetes-operator/crds/flinkbluegreendeployments.flink.apache.org-v1.yml
@@ -0,0 +1,10922 @@
+# Generated by Fabric8 CRDGenerator, manual edits might get overwritten!
+apiVersion: "apiextensions.k8s.io/v1"
+kind: "CustomResourceDefinition"
+metadata:
+ name: "flinkbluegreendeployments.flink.apache.org"
+spec:
+ group: "flink.apache.org"
+ names:
+ kind: "FlinkBlueGreenDeployment"
+ plural: "flinkbluegreendeployments"
+ shortNames:
+ - "flinkbgdep"
+ singular: "flinkbluegreendeployment"
+ scope: "Namespaced"
+ versions:
+ - additionalPrinterColumns:
+ - jsonPath: ".status.jobStatus.state"
+ name: "Job Status"
+ priority: 0
+ type: "string"
+ name: "v1beta1"
+ schema:
+ openAPIV3Schema:
+ properties:
+ spec:
+ properties:
+ configuration:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ template:
+ properties:
+ metadata:
+ properties:
+ annotations:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ creationTimestamp:
+ type: "string"
+ deletionGracePeriodSeconds:
+ type: "integer"
+ deletionTimestamp:
+ type: "string"
+ finalizers:
+ items:
+ type: "string"
+ type: "array"
+ generateName:
+ type: "string"
+ generation:
+ type: "integer"
+ labels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ managedFields:
+ items:
+ properties:
+ apiVersion:
+ type: "string"
+ fieldsType:
+ type: "string"
+ fieldsV1:
+ type: "object"
+ manager:
+ type: "string"
+ operation:
+ type: "string"
+ subresource:
+ type: "string"
+ time:
+ type: "string"
+ type: "object"
+ type: "array"
+ name:
+ type: "string"
+ namespace:
+ type: "string"
+ ownerReferences:
+ items:
+ properties:
+ apiVersion:
+ type: "string"
+ blockOwnerDeletion:
+ type: "boolean"
+ controller:
+ type: "boolean"
+ kind:
+ type: "string"
+ name:
+ type: "string"
+ uid:
+ type: "string"
+ type: "object"
+ type: "array"
+ resourceVersion:
+ type: "string"
+ selfLink:
+ type: "string"
+ uid:
+ type: "string"
+ type: "object"
+ spec:
+ properties:
+ flinkConfiguration:
+ 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"
+ image:
+ type: "string"
+ imagePullPolicy:
+ type: "string"
+ ingress:
+ properties:
+ annotations:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ className:
+ type: "string"
+ labels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ template:
+ type: "string"
+ tls:
+ items:
+ properties:
+ hosts:
+ items:
+ type: "string"
+ type: "array"
+ secretName:
+ type: "string"
+ type: "object"
+ type: "array"
+ type: "object"
+ job:
+ properties:
+ allowNonRestoredState:
+ type: "boolean"
+ args:
+ items:
+ type: "string"
+ type: "array"
+ autoscalerResetNonce:
+ type: "integer"
+ checkpointTriggerNonce:
+ type: "integer"
+ entryClass:
+ type: "string"
+ initialSavepointPath:
+ type: "string"
+ jarURI:
+ type: "string"
+ parallelism:
+ type: "integer"
+ savepointRedeployNonce:
+ type: "integer"
+ savepointTriggerNonce:
+ type: "integer"
+ state:
+ enum:
+ - "running"
+ - "suspended"
+ type: "string"
+ upgradeMode:
+ enum:
+ - "last-state"
+ - "savepoint"
+ - "stateless"
+ type: "string"
+ type: "object"
+ jobManager:
+ properties:
+ podTemplate:
+ properties:
+ apiVersion:
+ type: "string"
+ kind:
+ type: "string"
+ metadata:
+ properties:
+ annotations:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ creationTimestamp:
+ type: "string"
+ deletionGracePeriodSeconds:
+ type: "integer"
+ deletionTimestamp:
+ type: "string"
+ finalizers:
+ items:
+ type: "string"
+ type: "array"
+ generateName:
+ type: "string"
+ generation:
+ type: "integer"
+ labels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ managedFields:
+ items:
+ properties:
+ apiVersion:
+ type: "string"
+ fieldsType:
+ type: "string"
+ fieldsV1:
+ type: "object"
+ manager:
+ type: "string"
+ operation:
+ type: "string"
+ subresource:
+ type: "string"
+ time:
+ type: "string"
+ type: "object"
+ type: "array"
+ name:
+ type: "string"
+ namespace:
+ type: "string"
+ ownerReferences:
+ items:
+ properties:
+ apiVersion:
+ type: "string"
+ blockOwnerDeletion:
+ type: "boolean"
+ controller:
+ type: "boolean"
+ kind:
+ type: "string"
+ name:
+ type: "string"
+ uid:
+ type: "string"
+ type: "object"
+ type: "array"
+ resourceVersion:
+ type: "string"
+ selfLink:
+ type: "string"
+ uid:
+ type: "string"
+ type: "object"
+ spec:
+ properties:
+ activeDeadlineSeconds:
+ type: "integer"
+ affinity:
+ properties:
+ nodeAffinity:
+ properties:
+ preferredDuringSchedulingIgnoredDuringExecution:
+ items:
+ properties:
+ preference:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchFields:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ type: "object"
+ weight:
+ type: "integer"
+ type: "object"
+ type: "array"
+ requiredDuringSchedulingIgnoredDuringExecution:
+ properties:
+ nodeSelectorTerms:
+ items:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchFields:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ type: "object"
+ type: "array"
+ type: "object"
+ type: "object"
+ podAffinity:
+ properties:
+ preferredDuringSchedulingIgnoredDuringExecution:
+ items:
+ properties:
+ podAffinityTerm:
+ properties:
+ labelSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ matchLabelKeys:
+ items:
+ type: "string"
+ type: "array"
+ mismatchLabelKeys:
+ items:
+ type: "string"
+ type: "array"
+ namespaceSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ namespaces:
+ items:
+ type: "string"
+ type: "array"
+ topologyKey:
+ type: "string"
+ type: "object"
+ weight:
+ type: "integer"
+ type: "object"
+ type: "array"
+ requiredDuringSchedulingIgnoredDuringExecution:
+ items:
+ properties:
+ labelSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ matchLabelKeys:
+ items:
+ type: "string"
+ type: "array"
+ mismatchLabelKeys:
+ items:
+ type: "string"
+ type: "array"
+ namespaceSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ namespaces:
+ items:
+ type: "string"
+ type: "array"
+ topologyKey:
+ type: "string"
+ type: "object"
+ type: "array"
+ type: "object"
+ podAntiAffinity:
+ properties:
+ preferredDuringSchedulingIgnoredDuringExecution:
+ items:
+ properties:
+ podAffinityTerm:
+ properties:
+ labelSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ matchLabelKeys:
+ items:
+ type: "string"
+ type: "array"
+ mismatchLabelKeys:
+ items:
+ type: "string"
+ type: "array"
+ namespaceSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ namespaces:
+ items:
+ type: "string"
+ type: "array"
+ topologyKey:
+ type: "string"
+ type: "object"
+ weight:
+ type: "integer"
+ type: "object"
+ type: "array"
+ requiredDuringSchedulingIgnoredDuringExecution:
+ items:
+ properties:
+ labelSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ matchLabelKeys:
+ items:
+ type: "string"
+ type: "array"
+ mismatchLabelKeys:
+ items:
+ type: "string"
+ type: "array"
+ namespaceSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ namespaces:
+ items:
+ type: "string"
+ type: "array"
+ topologyKey:
+ type: "string"
+ type: "object"
+ type: "array"
+ type: "object"
+ type: "object"
+ automountServiceAccountToken:
+ type: "boolean"
+ containers:
+ items:
+ properties:
+ args:
+ items:
+ type: "string"
+ type: "array"
+ command:
+ items:
+ type: "string"
+ type: "array"
+ env:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ valueFrom:
+ properties:
+ configMapKeyRef:
+ properties:
+ key:
+ type: "string"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ fieldRef:
+ properties:
+ apiVersion:
+ type: "string"
+ fieldPath:
+ type: "string"
+ type: "object"
+ resourceFieldRef:
+ properties:
+ containerName:
+ type: "string"
+ divisor:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ resource:
+ type: "string"
+ type: "object"
+ secretKeyRef:
+ properties:
+ key:
+ type: "string"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ type: "object"
+ type: "object"
+ type: "array"
+ envFrom:
+ items:
+ properties:
+ configMapRef:
+ properties:
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ prefix:
+ type: "string"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ type: "object"
+ type: "array"
+ image:
+ type: "string"
+ imagePullPolicy:
+ type: "string"
+ lifecycle:
+ properties:
+ postStart:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ sleep:
+ properties:
+ seconds:
+ type: "integer"
+ type: "object"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ preStop:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ sleep:
+ properties:
+ seconds:
+ type: "integer"
+ type: "object"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ stopSignal:
+ type: "string"
+ type: "object"
+ livenessProbe:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ failureThreshold:
+ type: "integer"
+ grpc:
+ properties:
+ port:
+ type: "integer"
+ service:
+ type: "string"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ initialDelaySeconds:
+ type: "integer"
+ periodSeconds:
+ type: "integer"
+ successThreshold:
+ type: "integer"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ timeoutSeconds:
+ type: "integer"
+ type: "object"
+ name:
+ type: "string"
+ ports:
+ items:
+ properties:
+ containerPort:
+ type: "integer"
+ hostIP:
+ type: "string"
+ hostPort:
+ type: "integer"
+ name:
+ type: "string"
+ protocol:
+ type: "string"
+ type: "object"
+ type: "array"
+ readinessProbe:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ failureThreshold:
+ type: "integer"
+ grpc:
+ properties:
+ port:
+ type: "integer"
+ service:
+ type: "string"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ initialDelaySeconds:
+ type: "integer"
+ periodSeconds:
+ type: "integer"
+ successThreshold:
+ type: "integer"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ timeoutSeconds:
+ type: "integer"
+ type: "object"
+ resizePolicy:
+ items:
+ properties:
+ resourceName:
+ type: "string"
+ restartPolicy:
+ type: "string"
+ type: "object"
+ type: "array"
+ resources:
+ properties:
+ claims:
+ items:
+ properties:
+ name:
+ type: "string"
+ request:
+ type: "string"
+ type: "object"
+ type: "array"
+ limits:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ requests:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ restartPolicy:
+ type: "string"
+ securityContext:
+ properties:
+ allowPrivilegeEscalation:
+ type: "boolean"
+ appArmorProfile:
+ properties:
+ localhostProfile:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ capabilities:
+ properties:
+ add:
+ items:
+ type: "string"
+ type: "array"
+ drop:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ privileged:
+ type: "boolean"
+ procMount:
+ type: "string"
+ readOnlyRootFilesystem:
+ type: "boolean"
+ runAsGroup:
+ type: "integer"
+ runAsNonRoot:
+ type: "boolean"
+ runAsUser:
+ type: "integer"
+ seLinuxOptions:
+ properties:
+ level:
+ type: "string"
+ role:
+ type: "string"
+ type:
+ type: "string"
+ user:
+ type: "string"
+ type: "object"
+ seccompProfile:
+ properties:
+ localhostProfile:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ windowsOptions:
+ properties:
+ gmsaCredentialSpec:
+ type: "string"
+ gmsaCredentialSpecName:
+ type: "string"
+ hostProcess:
+ type: "boolean"
+ runAsUserName:
+ type: "string"
+ type: "object"
+ type: "object"
+ startupProbe:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ failureThreshold:
+ type: "integer"
+ grpc:
+ properties:
+ port:
+ type: "integer"
+ service:
+ type: "string"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ initialDelaySeconds:
+ type: "integer"
+ periodSeconds:
+ type: "integer"
+ successThreshold:
+ type: "integer"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ timeoutSeconds:
+ type: "integer"
+ type: "object"
+ stdin:
+ type: "boolean"
+ stdinOnce:
+ type: "boolean"
+ terminationMessagePath:
+ type: "string"
+ terminationMessagePolicy:
+ type: "string"
+ tty:
+ type: "boolean"
+ volumeDevices:
+ items:
+ properties:
+ devicePath:
+ type: "string"
+ name:
+ type: "string"
+ type: "object"
+ type: "array"
+ volumeMounts:
+ items:
+ properties:
+ mountPath:
+ type: "string"
+ mountPropagation:
+ type: "string"
+ name:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ recursiveReadOnly:
+ type: "string"
+ subPath:
+ type: "string"
+ subPathExpr:
+ type: "string"
+ type: "object"
+ type: "array"
+ workingDir:
+ type: "string"
+ type: "object"
+ type: "array"
+ dnsConfig:
+ properties:
+ nameservers:
+ items:
+ type: "string"
+ type: "array"
+ options:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ searches:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ dnsPolicy:
+ type: "string"
+ enableServiceLinks:
+ type: "boolean"
+ ephemeralContainers:
+ items:
+ properties:
+ args:
+ items:
+ type: "string"
+ type: "array"
+ command:
+ items:
+ type: "string"
+ type: "array"
+ env:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ valueFrom:
+ properties:
+ configMapKeyRef:
+ properties:
+ key:
+ type: "string"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ fieldRef:
+ properties:
+ apiVersion:
+ type: "string"
+ fieldPath:
+ type: "string"
+ type: "object"
+ resourceFieldRef:
+ properties:
+ containerName:
+ type: "string"
+ divisor:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ resource:
+ type: "string"
+ type: "object"
+ secretKeyRef:
+ properties:
+ key:
+ type: "string"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ type: "object"
+ type: "object"
+ type: "array"
+ envFrom:
+ items:
+ properties:
+ configMapRef:
+ properties:
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ prefix:
+ type: "string"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ type: "object"
+ type: "array"
+ image:
+ type: "string"
+ imagePullPolicy:
+ type: "string"
+ lifecycle:
+ properties:
+ postStart:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ sleep:
+ properties:
+ seconds:
+ type: "integer"
+ type: "object"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ preStop:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ sleep:
+ properties:
+ seconds:
+ type: "integer"
+ type: "object"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ stopSignal:
+ type: "string"
+ type: "object"
+ livenessProbe:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ failureThreshold:
+ type: "integer"
+ grpc:
+ properties:
+ port:
+ type: "integer"
+ service:
+ type: "string"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ initialDelaySeconds:
+ type: "integer"
+ periodSeconds:
+ type: "integer"
+ successThreshold:
+ type: "integer"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ timeoutSeconds:
+ type: "integer"
+ type: "object"
+ name:
+ type: "string"
+ ports:
+ items:
+ properties:
+ containerPort:
+ type: "integer"
+ hostIP:
+ type: "string"
+ hostPort:
+ type: "integer"
+ name:
+ type: "string"
+ protocol:
+ type: "string"
+ type: "object"
+ type: "array"
+ readinessProbe:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ failureThreshold:
+ type: "integer"
+ grpc:
+ properties:
+ port:
+ type: "integer"
+ service:
+ type: "string"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ initialDelaySeconds:
+ type: "integer"
+ periodSeconds:
+ type: "integer"
+ successThreshold:
+ type: "integer"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ timeoutSeconds:
+ type: "integer"
+ type: "object"
+ resizePolicy:
+ items:
+ properties:
+ resourceName:
+ type: "string"
+ restartPolicy:
+ type: "string"
+ type: "object"
+ type: "array"
+ resources:
+ properties:
+ claims:
+ items:
+ properties:
+ name:
+ type: "string"
+ request:
+ type: "string"
+ type: "object"
+ type: "array"
+ limits:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ requests:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ restartPolicy:
+ type: "string"
+ securityContext:
+ properties:
+ allowPrivilegeEscalation:
+ type: "boolean"
+ appArmorProfile:
+ properties:
+ localhostProfile:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ capabilities:
+ properties:
+ add:
+ items:
+ type: "string"
+ type: "array"
+ drop:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ privileged:
+ type: "boolean"
+ procMount:
+ type: "string"
+ readOnlyRootFilesystem:
+ type: "boolean"
+ runAsGroup:
+ type: "integer"
+ runAsNonRoot:
+ type: "boolean"
+ runAsUser:
+ type: "integer"
+ seLinuxOptions:
+ properties:
+ level:
+ type: "string"
+ role:
+ type: "string"
+ type:
+ type: "string"
+ user:
+ type: "string"
+ type: "object"
+ seccompProfile:
+ properties:
+ localhostProfile:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ windowsOptions:
+ properties:
+ gmsaCredentialSpec:
+ type: "string"
+ gmsaCredentialSpecName:
+ type: "string"
+ hostProcess:
+ type: "boolean"
+ runAsUserName:
+ type: "string"
+ type: "object"
+ type: "object"
+ startupProbe:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ failureThreshold:
+ type: "integer"
+ grpc:
+ properties:
+ port:
+ type: "integer"
+ service:
+ type: "string"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ initialDelaySeconds:
+ type: "integer"
+ periodSeconds:
+ type: "integer"
+ successThreshold:
+ type: "integer"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ timeoutSeconds:
+ type: "integer"
+ type: "object"
+ stdin:
+ type: "boolean"
+ stdinOnce:
+ type: "boolean"
+ targetContainerName:
+ type: "string"
+ terminationMessagePath:
+ type: "string"
+ terminationMessagePolicy:
+ type: "string"
+ tty:
+ type: "boolean"
+ volumeDevices:
+ items:
+ properties:
+ devicePath:
+ type: "string"
+ name:
+ type: "string"
+ type: "object"
+ type: "array"
+ volumeMounts:
+ items:
+ properties:
+ mountPath:
+ type: "string"
+ mountPropagation:
+ type: "string"
+ name:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ recursiveReadOnly:
+ type: "string"
+ subPath:
+ type: "string"
+ subPathExpr:
+ type: "string"
+ type: "object"
+ type: "array"
+ workingDir:
+ type: "string"
+ type: "object"
+ type: "array"
+ hostAliases:
+ items:
+ properties:
+ hostnames:
+ items:
+ type: "string"
+ type: "array"
+ ip:
+ type: "string"
+ type: "object"
+ type: "array"
+ hostIPC:
+ type: "boolean"
+ hostNetwork:
+ type: "boolean"
+ hostPID:
+ type: "boolean"
+ hostUsers:
+ type: "boolean"
+ hostname:
+ type: "string"
+ imagePullSecrets:
+ items:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ type: "array"
+ initContainers:
+ items:
+ properties:
+ args:
+ items:
+ type: "string"
+ type: "array"
+ command:
+ items:
+ type: "string"
+ type: "array"
+ env:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ valueFrom:
+ properties:
+ configMapKeyRef:
+ properties:
+ key:
+ type: "string"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ fieldRef:
+ properties:
+ apiVersion:
+ type: "string"
+ fieldPath:
+ type: "string"
+ type: "object"
+ resourceFieldRef:
+ properties:
+ containerName:
+ type: "string"
+ divisor:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ resource:
+ type: "string"
+ type: "object"
+ secretKeyRef:
+ properties:
+ key:
+ type: "string"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ type: "object"
+ type: "object"
+ type: "array"
+ envFrom:
+ items:
+ properties:
+ configMapRef:
+ properties:
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ prefix:
+ type: "string"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ type: "object"
+ type: "array"
+ image:
+ type: "string"
+ imagePullPolicy:
+ type: "string"
+ lifecycle:
+ properties:
+ postStart:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ sleep:
+ properties:
+ seconds:
+ type: "integer"
+ type: "object"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ preStop:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ sleep:
+ properties:
+ seconds:
+ type: "integer"
+ type: "object"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ stopSignal:
+ type: "string"
+ type: "object"
+ livenessProbe:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ failureThreshold:
+ type: "integer"
+ grpc:
+ properties:
+ port:
+ type: "integer"
+ service:
+ type: "string"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ initialDelaySeconds:
+ type: "integer"
+ periodSeconds:
+ type: "integer"
+ successThreshold:
+ type: "integer"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ timeoutSeconds:
+ type: "integer"
+ type: "object"
+ name:
+ type: "string"
+ ports:
+ items:
+ properties:
+ containerPort:
+ type: "integer"
+ hostIP:
+ type: "string"
+ hostPort:
+ type: "integer"
+ name:
+ type: "string"
+ protocol:
+ type: "string"
+ type: "object"
+ type: "array"
+ readinessProbe:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ failureThreshold:
+ type: "integer"
+ grpc:
+ properties:
+ port:
+ type: "integer"
+ service:
+ type: "string"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ initialDelaySeconds:
+ type: "integer"
+ periodSeconds:
+ type: "integer"
+ successThreshold:
+ type: "integer"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ timeoutSeconds:
+ type: "integer"
+ type: "object"
+ resizePolicy:
+ items:
+ properties:
+ resourceName:
+ type: "string"
+ restartPolicy:
+ type: "string"
+ type: "object"
+ type: "array"
+ resources:
+ properties:
+ claims:
+ items:
+ properties:
+ name:
+ type: "string"
+ request:
+ type: "string"
+ type: "object"
+ type: "array"
+ limits:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ requests:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ restartPolicy:
+ type: "string"
+ securityContext:
+ properties:
+ allowPrivilegeEscalation:
+ type: "boolean"
+ appArmorProfile:
+ properties:
+ localhostProfile:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ capabilities:
+ properties:
+ add:
+ items:
+ type: "string"
+ type: "array"
+ drop:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ privileged:
+ type: "boolean"
+ procMount:
+ type: "string"
+ readOnlyRootFilesystem:
+ type: "boolean"
+ runAsGroup:
+ type: "integer"
+ runAsNonRoot:
+ type: "boolean"
+ runAsUser:
+ type: "integer"
+ seLinuxOptions:
+ properties:
+ level:
+ type: "string"
+ role:
+ type: "string"
+ type:
+ type: "string"
+ user:
+ type: "string"
+ type: "object"
+ seccompProfile:
+ properties:
+ localhostProfile:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ windowsOptions:
+ properties:
+ gmsaCredentialSpec:
+ type: "string"
+ gmsaCredentialSpecName:
+ type: "string"
+ hostProcess:
+ type: "boolean"
+ runAsUserName:
+ type: "string"
+ type: "object"
+ type: "object"
+ startupProbe:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ failureThreshold:
+ type: "integer"
+ grpc:
+ properties:
+ port:
+ type: "integer"
+ service:
+ type: "string"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ initialDelaySeconds:
+ type: "integer"
+ periodSeconds:
+ type: "integer"
+ successThreshold:
+ type: "integer"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ timeoutSeconds:
+ type: "integer"
+ type: "object"
+ stdin:
+ type: "boolean"
+ stdinOnce:
+ type: "boolean"
+ terminationMessagePath:
+ type: "string"
+ terminationMessagePolicy:
+ type: "string"
+ tty:
+ type: "boolean"
+ volumeDevices:
+ items:
+ properties:
+ devicePath:
+ type: "string"
+ name:
+ type: "string"
+ type: "object"
+ type: "array"
+ volumeMounts:
+ items:
+ properties:
+ mountPath:
+ type: "string"
+ mountPropagation:
+ type: "string"
+ name:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ recursiveReadOnly:
+ type: "string"
+ subPath:
+ type: "string"
+ subPathExpr:
+ type: "string"
+ type: "object"
+ type: "array"
+ workingDir:
+ type: "string"
+ type: "object"
+ type: "array"
+ nodeName:
+ type: "string"
+ nodeSelector:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ os:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ overhead:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ preemptionPolicy:
+ type: "string"
+ priority:
+ type: "integer"
+ priorityClassName:
+ type: "string"
+ readinessGates:
+ items:
+ properties:
+ conditionType:
+ type: "string"
+ type: "object"
+ type: "array"
+ resourceClaims:
+ items:
+ properties:
+ name:
+ type: "string"
+ resourceClaimName:
+ type: "string"
+ resourceClaimTemplateName:
+ type: "string"
+ type: "object"
+ type: "array"
+ resources:
+ properties:
+ claims:
+ items:
+ properties:
+ name:
+ type: "string"
+ request:
+ type: "string"
+ type: "object"
+ type: "array"
+ limits:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ requests:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ restartPolicy:
+ type: "string"
+ runtimeClassName:
+ type: "string"
+ schedulerName:
+ type: "string"
+ schedulingGates:
+ items:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ type: "array"
+ securityContext:
+ properties:
+ appArmorProfile:
+ properties:
+ localhostProfile:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ fsGroup:
+ type: "integer"
+ fsGroupChangePolicy:
+ type: "string"
+ runAsGroup:
+ type: "integer"
+ runAsNonRoot:
+ type: "boolean"
+ runAsUser:
+ type: "integer"
+ seLinuxChangePolicy:
+ type: "string"
+ seLinuxOptions:
+ properties:
+ level:
+ type: "string"
+ role:
+ type: "string"
+ type:
+ type: "string"
+ user:
+ type: "string"
+ type: "object"
+ seccompProfile:
+ properties:
+ localhostProfile:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ supplementalGroups:
+ items:
+ type: "integer"
+ type: "array"
+ supplementalGroupsPolicy:
+ type: "string"
+ sysctls:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ windowsOptions:
+ properties:
+ gmsaCredentialSpec:
+ type: "string"
+ gmsaCredentialSpecName:
+ type: "string"
+ hostProcess:
+ type: "boolean"
+ runAsUserName:
+ type: "string"
+ type: "object"
+ type: "object"
+ serviceAccount:
+ type: "string"
+ serviceAccountName:
+ type: "string"
+ setHostnameAsFQDN:
+ type: "boolean"
+ shareProcessNamespace:
+ type: "boolean"
+ subdomain:
+ type: "string"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ tolerations:
+ items:
+ properties:
+ effect:
+ type: "string"
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ tolerationSeconds:
+ type: "integer"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ topologySpreadConstraints:
+ items:
+ properties:
+ labelSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ matchLabelKeys:
+ items:
+ type: "string"
+ type: "array"
+ maxSkew:
+ type: "integer"
+ minDomains:
+ type: "integer"
+ nodeAffinityPolicy:
+ type: "string"
+ nodeTaintsPolicy:
+ type: "string"
+ topologyKey:
+ type: "string"
+ whenUnsatisfiable:
+ type: "string"
+ type: "object"
+ type: "array"
+ volumes:
+ items:
+ properties:
+ awsElasticBlockStore:
+ properties:
+ fsType:
+ type: "string"
+ partition:
+ type: "integer"
+ readOnly:
+ type: "boolean"
+ volumeID:
+ type: "string"
+ type: "object"
+ azureDisk:
+ properties:
+ cachingMode:
+ type: "string"
+ diskName:
+ type: "string"
+ diskURI:
+ type: "string"
+ fsType:
+ type: "string"
+ kind:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ type: "object"
+ azureFile:
+ properties:
+ readOnly:
+ type: "boolean"
+ secretName:
+ type: "string"
+ shareName:
+ type: "string"
+ type: "object"
+ cephfs:
+ properties:
+ monitors:
+ items:
+ type: "string"
+ type: "array"
+ path:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ secretFile:
+ type: "string"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ user:
+ type: "string"
+ type: "object"
+ cinder:
+ properties:
+ fsType:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ volumeID:
+ type: "string"
+ type: "object"
+ configMap:
+ properties:
+ defaultMode:
+ type: "integer"
+ items:
+ items:
+ properties:
+ key:
+ type: "string"
+ mode:
+ type: "integer"
+ path:
+ type: "string"
+ type: "object"
+ type: "array"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ csi:
+ properties:
+ driver:
+ type: "string"
+ fsType:
+ type: "string"
+ nodePublishSecretRef:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ readOnly:
+ type: "boolean"
+ volumeAttributes:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ downwardAPI:
+ properties:
+ defaultMode:
+ type: "integer"
+ items:
+ items:
+ properties:
+ fieldRef:
+ properties:
+ apiVersion:
+ type: "string"
+ fieldPath:
+ type: "string"
+ type: "object"
+ mode:
+ type: "integer"
+ path:
+ type: "string"
+ resourceFieldRef:
+ properties:
+ containerName:
+ type: "string"
+ divisor:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ resource:
+ type: "string"
+ type: "object"
+ type: "object"
+ type: "array"
+ type: "object"
+ emptyDir:
+ properties:
+ medium:
+ type: "string"
+ sizeLimit:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ ephemeral:
+ properties:
+ volumeClaimTemplate:
+ properties:
+ metadata:
+ properties:
+ annotations:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ creationTimestamp:
+ type: "string"
+ deletionGracePeriodSeconds:
+ type: "integer"
+ deletionTimestamp:
+ type: "string"
+ finalizers:
+ items:
+ type: "string"
+ type: "array"
+ generateName:
+ type: "string"
+ generation:
+ type: "integer"
+ labels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ managedFields:
+ items:
+ properties:
+ apiVersion:
+ type: "string"
+ fieldsType:
+ type: "string"
+ fieldsV1:
+ type: "object"
+ manager:
+ type: "string"
+ operation:
+ type: "string"
+ subresource:
+ type: "string"
+ time:
+ type: "string"
+ type: "object"
+ type: "array"
+ name:
+ type: "string"
+ namespace:
+ type: "string"
+ ownerReferences:
+ items:
+ properties:
+ apiVersion:
+ type: "string"
+ blockOwnerDeletion:
+ type: "boolean"
+ controller:
+ type: "boolean"
+ kind:
+ type: "string"
+ name:
+ type: "string"
+ uid:
+ type: "string"
+ type: "object"
+ type: "array"
+ resourceVersion:
+ type: "string"
+ selfLink:
+ type: "string"
+ uid:
+ type: "string"
+ type: "object"
+ spec:
+ properties:
+ accessModes:
+ items:
+ type: "string"
+ type: "array"
+ dataSource:
+ properties:
+ apiGroup:
+ type: "string"
+ kind:
+ type: "string"
+ name:
+ type: "string"
+ type: "object"
+ dataSourceRef:
+ properties:
+ apiGroup:
+ type: "string"
+ kind:
+ type: "string"
+ name:
+ type: "string"
+ namespace:
+ type: "string"
+ type: "object"
+ resources:
+ properties:
+ limits:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ requests:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ selector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ storageClassName:
+ type: "string"
+ volumeAttributesClassName:
+ type: "string"
+ volumeMode:
+ type: "string"
+ volumeName:
+ type: "string"
+ type: "object"
+ type: "object"
+ type: "object"
+ fc:
+ properties:
+ fsType:
+ type: "string"
+ lun:
+ type: "integer"
+ readOnly:
+ type: "boolean"
+ targetWWNs:
+ items:
+ type: "string"
+ type: "array"
+ wwids:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ flexVolume:
+ properties:
+ driver:
+ type: "string"
+ fsType:
+ type: "string"
+ options:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ readOnly:
+ type: "boolean"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ type: "object"
+ flocker:
+ properties:
+ datasetName:
+ type: "string"
+ datasetUUID:
+ type: "string"
+ type: "object"
+ gcePersistentDisk:
+ properties:
+ fsType:
+ type: "string"
+ partition:
+ type: "integer"
+ pdName:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ type: "object"
+ gitRepo:
+ properties:
+ directory:
+ type: "string"
+ repository:
+ type: "string"
+ revision:
+ type: "string"
+ type: "object"
+ glusterfs:
+ properties:
+ endpoints:
+ type: "string"
+ path:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ type: "object"
+ hostPath:
+ properties:
+ path:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ image:
+ properties:
+ pullPolicy:
+ type: "string"
+ reference:
+ type: "string"
+ type: "object"
+ iscsi:
+ properties:
+ chapAuthDiscovery:
+ type: "boolean"
+ chapAuthSession:
+ type: "boolean"
+ fsType:
+ type: "string"
+ initiatorName:
+ type: "string"
+ iqn:
+ type: "string"
+ iscsiInterface:
+ type: "string"
+ lun:
+ type: "integer"
+ portals:
+ items:
+ type: "string"
+ type: "array"
+ readOnly:
+ type: "boolean"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ targetPortal:
+ type: "string"
+ type: "object"
+ name:
+ type: "string"
+ nfs:
+ properties:
+ path:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ server:
+ type: "string"
+ type: "object"
+ persistentVolumeClaim:
+ properties:
+ claimName:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ type: "object"
+ photonPersistentDisk:
+ properties:
+ fsType:
+ type: "string"
+ pdID:
+ type: "string"
+ type: "object"
+ portworxVolume:
+ properties:
+ fsType:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ volumeID:
+ type: "string"
+ type: "object"
+ projected:
+ properties:
+ defaultMode:
+ type: "integer"
+ sources:
+ items:
+ properties:
+ clusterTrustBundle:
+ properties:
+ labelSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ path:
+ type: "string"
+ signerName:
+ type: "string"
+ type: "object"
+ configMap:
+ properties:
+ items:
+ items:
+ properties:
+ key:
+ type: "string"
+ mode:
+ type: "integer"
+ path:
+ type: "string"
+ type: "object"
+ type: "array"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ downwardAPI:
+ properties:
+ items:
+ items:
+ properties:
+ fieldRef:
+ properties:
+ apiVersion:
+ type: "string"
+ fieldPath:
+ type: "string"
+ type: "object"
+ mode:
+ type: "integer"
+ path:
+ type: "string"
+ resourceFieldRef:
+ properties:
+ containerName:
+ type: "string"
+ divisor:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ resource:
+ type: "string"
+ type: "object"
+ type: "object"
+ type: "array"
+ type: "object"
+ secret:
+ properties:
+ items:
+ items:
+ properties:
+ key:
+ type: "string"
+ mode:
+ type: "integer"
+ path:
+ type: "string"
+ type: "object"
+ type: "array"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ serviceAccountToken:
+ properties:
+ audience:
+ type: "string"
+ expirationSeconds:
+ type: "integer"
+ path:
+ type: "string"
+ type: "object"
+ type: "object"
+ type: "array"
+ type: "object"
+ quobyte:
+ properties:
+ group:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ registry:
+ type: "string"
+ tenant:
+ type: "string"
+ user:
+ type: "string"
+ volume:
+ type: "string"
+ type: "object"
+ rbd:
+ properties:
+ fsType:
+ type: "string"
+ image:
+ type: "string"
+ keyring:
+ type: "string"
+ monitors:
+ items:
+ type: "string"
+ type: "array"
+ pool:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ user:
+ type: "string"
+ type: "object"
+ scaleIO:
+ properties:
+ fsType:
+ type: "string"
+ gateway:
+ type: "string"
+ protectionDomain:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ sslEnabled:
+ type: "boolean"
+ storageMode:
+ type: "string"
+ storagePool:
+ type: "string"
+ system:
+ type: "string"
+ volumeName:
+ type: "string"
+ type: "object"
+ secret:
+ properties:
+ defaultMode:
+ type: "integer"
+ items:
+ items:
+ properties:
+ key:
+ type: "string"
+ mode:
+ type: "integer"
+ path:
+ type: "string"
+ type: "object"
+ type: "array"
+ optional:
+ type: "boolean"
+ secretName:
+ type: "string"
+ type: "object"
+ storageos:
+ properties:
+ fsType:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ volumeName:
+ type: "string"
+ volumeNamespace:
+ type: "string"
+ type: "object"
+ vsphereVolume:
+ properties:
+ fsType:
+ type: "string"
+ storagePolicyID:
+ type: "string"
+ storagePolicyName:
+ type: "string"
+ volumePath:
+ type: "string"
+ type: "object"
+ type: "object"
+ type: "array"
+ type: "object"
+ status:
+ properties:
+ conditions:
+ items:
+ properties:
+ lastProbeTime:
+ type: "string"
+ lastTransitionTime:
+ type: "string"
+ message:
+ type: "string"
+ observedGeneration:
+ type: "integer"
+ reason:
+ type: "string"
+ status:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ type: "array"
+ containerStatuses:
+ items:
+ properties:
+ allocatedResources:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ allocatedResourcesStatus:
+ items:
+ properties:
+ name:
+ type: "string"
+ resources:
+ items:
+ properties:
+ health:
+ type: "string"
+ resourceID:
+ type: "string"
+ type: "object"
+ type: "array"
+ type: "object"
+ type: "array"
+ containerID:
+ type: "string"
+ image:
+ type: "string"
+ imageID:
+ type: "string"
+ lastState:
+ properties:
+ running:
+ properties:
+ startedAt:
+ type: "string"
+ type: "object"
+ terminated:
+ properties:
+ containerID:
+ type: "string"
+ exitCode:
+ type: "integer"
+ finishedAt:
+ type: "string"
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ signal:
+ type: "integer"
+ startedAt:
+ type: "string"
+ type: "object"
+ waiting:
+ properties:
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ type: "object"
+ type: "object"
+ name:
+ type: "string"
+ ready:
+ type: "boolean"
+ resources:
+ properties:
+ claims:
+ items:
+ properties:
+ name:
+ type: "string"
+ request:
+ type: "string"
+ type: "object"
+ type: "array"
+ limits:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ requests:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ restartCount:
+ type: "integer"
+ started:
+ type: "boolean"
+ state:
+ properties:
+ running:
+ properties:
+ startedAt:
+ type: "string"
+ type: "object"
+ terminated:
+ properties:
+ containerID:
+ type: "string"
+ exitCode:
+ type: "integer"
+ finishedAt:
+ type: "string"
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ signal:
+ type: "integer"
+ startedAt:
+ type: "string"
+ type: "object"
+ waiting:
+ properties:
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ type: "object"
+ type: "object"
+ stopSignal:
+ type: "string"
+ user:
+ properties:
+ linux:
+ properties:
+ gid:
+ type: "integer"
+ supplementalGroups:
+ items:
+ type: "integer"
+ type: "array"
+ uid:
+ type: "integer"
+ type: "object"
+ type: "object"
+ volumeMounts:
+ items:
+ properties:
+ mountPath:
+ type: "string"
+ name:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ recursiveReadOnly:
+ type: "string"
+ type: "object"
+ type: "array"
+ type: "object"
+ type: "array"
+ ephemeralContainerStatuses:
+ items:
+ properties:
+ allocatedResources:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ allocatedResourcesStatus:
+ items:
+ properties:
+ name:
+ type: "string"
+ resources:
+ items:
+ properties:
+ health:
+ type: "string"
+ resourceID:
+ type: "string"
+ type: "object"
+ type: "array"
+ type: "object"
+ type: "array"
+ containerID:
+ type: "string"
+ image:
+ type: "string"
+ imageID:
+ type: "string"
+ lastState:
+ properties:
+ running:
+ properties:
+ startedAt:
+ type: "string"
+ type: "object"
+ terminated:
+ properties:
+ containerID:
+ type: "string"
+ exitCode:
+ type: "integer"
+ finishedAt:
+ type: "string"
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ signal:
+ type: "integer"
+ startedAt:
+ type: "string"
+ type: "object"
+ waiting:
+ properties:
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ type: "object"
+ type: "object"
+ name:
+ type: "string"
+ ready:
+ type: "boolean"
+ resources:
+ properties:
+ claims:
+ items:
+ properties:
+ name:
+ type: "string"
+ request:
+ type: "string"
+ type: "object"
+ type: "array"
+ limits:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ requests:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ restartCount:
+ type: "integer"
+ started:
+ type: "boolean"
+ state:
+ properties:
+ running:
+ properties:
+ startedAt:
+ type: "string"
+ type: "object"
+ terminated:
+ properties:
+ containerID:
+ type: "string"
+ exitCode:
+ type: "integer"
+ finishedAt:
+ type: "string"
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ signal:
+ type: "integer"
+ startedAt:
+ type: "string"
+ type: "object"
+ waiting:
+ properties:
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ type: "object"
+ type: "object"
+ stopSignal:
+ type: "string"
+ user:
+ properties:
+ linux:
+ properties:
+ gid:
+ type: "integer"
+ supplementalGroups:
+ items:
+ type: "integer"
+ type: "array"
+ uid:
+ type: "integer"
+ type: "object"
+ type: "object"
+ volumeMounts:
+ items:
+ properties:
+ mountPath:
+ type: "string"
+ name:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ recursiveReadOnly:
+ type: "string"
+ type: "object"
+ type: "array"
+ type: "object"
+ type: "array"
+ hostIP:
+ type: "string"
+ hostIPs:
+ items:
+ properties:
+ ip:
+ type: "string"
+ type: "object"
+ type: "array"
+ initContainerStatuses:
+ items:
+ properties:
+ allocatedResources:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ allocatedResourcesStatus:
+ items:
+ properties:
+ name:
+ type: "string"
+ resources:
+ items:
+ properties:
+ health:
+ type: "string"
+ resourceID:
+ type: "string"
+ type: "object"
+ type: "array"
+ type: "object"
+ type: "array"
+ containerID:
+ type: "string"
+ image:
+ type: "string"
+ imageID:
+ type: "string"
+ lastState:
+ properties:
+ running:
+ properties:
+ startedAt:
+ type: "string"
+ type: "object"
+ terminated:
+ properties:
+ containerID:
+ type: "string"
+ exitCode:
+ type: "integer"
+ finishedAt:
+ type: "string"
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ signal:
+ type: "integer"
+ startedAt:
+ type: "string"
+ type: "object"
+ waiting:
+ properties:
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ type: "object"
+ type: "object"
+ name:
+ type: "string"
+ ready:
+ type: "boolean"
+ resources:
+ properties:
+ claims:
+ items:
+ properties:
+ name:
+ type: "string"
+ request:
+ type: "string"
+ type: "object"
+ type: "array"
+ limits:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ requests:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ restartCount:
+ type: "integer"
+ started:
+ type: "boolean"
+ state:
+ properties:
+ running:
+ properties:
+ startedAt:
+ type: "string"
+ type: "object"
+ terminated:
+ properties:
+ containerID:
+ type: "string"
+ exitCode:
+ type: "integer"
+ finishedAt:
+ type: "string"
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ signal:
+ type: "integer"
+ startedAt:
+ type: "string"
+ type: "object"
+ waiting:
+ properties:
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ type: "object"
+ type: "object"
+ stopSignal:
+ type: "string"
+ user:
+ properties:
+ linux:
+ properties:
+ gid:
+ type: "integer"
+ supplementalGroups:
+ items:
+ type: "integer"
+ type: "array"
+ uid:
+ type: "integer"
+ type: "object"
+ type: "object"
+ volumeMounts:
+ items:
+ properties:
+ mountPath:
+ type: "string"
+ name:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ recursiveReadOnly:
+ type: "string"
+ type: "object"
+ type: "array"
+ type: "object"
+ type: "array"
+ message:
+ type: "string"
+ nominatedNodeName:
+ type: "string"
+ observedGeneration:
+ type: "integer"
+ phase:
+ type: "string"
+ podIP:
+ type: "string"
+ podIPs:
+ items:
+ properties:
+ ip:
+ type: "string"
+ type: "object"
+ type: "array"
+ qosClass:
+ type: "string"
+ reason:
+ type: "string"
+ resize:
+ type: "string"
+ resourceClaimStatuses:
+ items:
+ properties:
+ name:
+ type: "string"
+ resourceClaimName:
+ type: "string"
+ type: "object"
+ type: "array"
+ startTime:
+ type: "string"
+ type: "object"
+ type: "object"
+ replicas:
+ type: "integer"
+ resource:
+ properties:
+ cpu:
+ type: "number"
+ ephemeralStorage:
+ type: "string"
+ memory:
+ type: "string"
+ type: "object"
+ type: "object"
+ logConfiguration:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ mode:
+ enum:
+ - "native"
+ - "standalone"
+ type: "string"
+ podTemplate:
+ properties:
+ apiVersion:
+ type: "string"
+ kind:
+ type: "string"
+ metadata:
+ properties:
+ annotations:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ creationTimestamp:
+ type: "string"
+ deletionGracePeriodSeconds:
+ type: "integer"
+ deletionTimestamp:
+ type: "string"
+ finalizers:
+ items:
+ type: "string"
+ type: "array"
+ generateName:
+ type: "string"
+ generation:
+ type: "integer"
+ labels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ managedFields:
+ items:
+ properties:
+ apiVersion:
+ type: "string"
+ fieldsType:
+ type: "string"
+ fieldsV1:
+ type: "object"
+ manager:
+ type: "string"
+ operation:
+ type: "string"
+ subresource:
+ type: "string"
+ time:
+ type: "string"
+ type: "object"
+ type: "array"
+ name:
+ type: "string"
+ namespace:
+ type: "string"
+ ownerReferences:
+ items:
+ properties:
+ apiVersion:
+ type: "string"
+ blockOwnerDeletion:
+ type: "boolean"
+ controller:
+ type: "boolean"
+ kind:
+ type: "string"
+ name:
+ type: "string"
+ uid:
+ type: "string"
+ type: "object"
+ type: "array"
+ resourceVersion:
+ type: "string"
+ selfLink:
+ type: "string"
+ uid:
+ type: "string"
+ type: "object"
+ spec:
+ properties:
+ activeDeadlineSeconds:
+ type: "integer"
+ affinity:
+ properties:
+ nodeAffinity:
+ properties:
+ preferredDuringSchedulingIgnoredDuringExecution:
+ items:
+ properties:
+ preference:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchFields:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ type: "object"
+ weight:
+ type: "integer"
+ type: "object"
+ type: "array"
+ requiredDuringSchedulingIgnoredDuringExecution:
+ properties:
+ nodeSelectorTerms:
+ items:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchFields:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ type: "object"
+ type: "array"
+ type: "object"
+ type: "object"
+ podAffinity:
+ properties:
+ preferredDuringSchedulingIgnoredDuringExecution:
+ items:
+ properties:
+ podAffinityTerm:
+ properties:
+ labelSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ matchLabelKeys:
+ items:
+ type: "string"
+ type: "array"
+ mismatchLabelKeys:
+ items:
+ type: "string"
+ type: "array"
+ namespaceSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ namespaces:
+ items:
+ type: "string"
+ type: "array"
+ topologyKey:
+ type: "string"
+ type: "object"
+ weight:
+ type: "integer"
+ type: "object"
+ type: "array"
+ requiredDuringSchedulingIgnoredDuringExecution:
+ items:
+ properties:
+ labelSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ matchLabelKeys:
+ items:
+ type: "string"
+ type: "array"
+ mismatchLabelKeys:
+ items:
+ type: "string"
+ type: "array"
+ namespaceSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ namespaces:
+ items:
+ type: "string"
+ type: "array"
+ topologyKey:
+ type: "string"
+ type: "object"
+ type: "array"
+ type: "object"
+ podAntiAffinity:
+ properties:
+ preferredDuringSchedulingIgnoredDuringExecution:
+ items:
+ properties:
+ podAffinityTerm:
+ properties:
+ labelSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ matchLabelKeys:
+ items:
+ type: "string"
+ type: "array"
+ mismatchLabelKeys:
+ items:
+ type: "string"
+ type: "array"
+ namespaceSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ namespaces:
+ items:
+ type: "string"
+ type: "array"
+ topologyKey:
+ type: "string"
+ type: "object"
+ weight:
+ type: "integer"
+ type: "object"
+ type: "array"
+ requiredDuringSchedulingIgnoredDuringExecution:
+ items:
+ properties:
+ labelSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ matchLabelKeys:
+ items:
+ type: "string"
+ type: "array"
+ mismatchLabelKeys:
+ items:
+ type: "string"
+ type: "array"
+ namespaceSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ namespaces:
+ items:
+ type: "string"
+ type: "array"
+ topologyKey:
+ type: "string"
+ type: "object"
+ type: "array"
+ type: "object"
+ type: "object"
+ automountServiceAccountToken:
+ type: "boolean"
+ containers:
+ items:
+ properties:
+ args:
+ items:
+ type: "string"
+ type: "array"
+ command:
+ items:
+ type: "string"
+ type: "array"
+ env:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ valueFrom:
+ properties:
+ configMapKeyRef:
+ properties:
+ key:
+ type: "string"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ fieldRef:
+ properties:
+ apiVersion:
+ type: "string"
+ fieldPath:
+ type: "string"
+ type: "object"
+ resourceFieldRef:
+ properties:
+ containerName:
+ type: "string"
+ divisor:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ resource:
+ type: "string"
+ type: "object"
+ secretKeyRef:
+ properties:
+ key:
+ type: "string"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ type: "object"
+ type: "object"
+ type: "array"
+ envFrom:
+ items:
+ properties:
+ configMapRef:
+ properties:
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ prefix:
+ type: "string"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ type: "object"
+ type: "array"
+ image:
+ type: "string"
+ imagePullPolicy:
+ type: "string"
+ lifecycle:
+ properties:
+ postStart:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ sleep:
+ properties:
+ seconds:
+ type: "integer"
+ type: "object"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ preStop:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ sleep:
+ properties:
+ seconds:
+ type: "integer"
+ type: "object"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ stopSignal:
+ type: "string"
+ type: "object"
+ livenessProbe:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ failureThreshold:
+ type: "integer"
+ grpc:
+ properties:
+ port:
+ type: "integer"
+ service:
+ type: "string"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ initialDelaySeconds:
+ type: "integer"
+ periodSeconds:
+ type: "integer"
+ successThreshold:
+ type: "integer"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ timeoutSeconds:
+ type: "integer"
+ type: "object"
+ name:
+ type: "string"
+ ports:
+ items:
+ properties:
+ containerPort:
+ type: "integer"
+ hostIP:
+ type: "string"
+ hostPort:
+ type: "integer"
+ name:
+ type: "string"
+ protocol:
+ type: "string"
+ type: "object"
+ type: "array"
+ readinessProbe:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ failureThreshold:
+ type: "integer"
+ grpc:
+ properties:
+ port:
+ type: "integer"
+ service:
+ type: "string"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ initialDelaySeconds:
+ type: "integer"
+ periodSeconds:
+ type: "integer"
+ successThreshold:
+ type: "integer"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ timeoutSeconds:
+ type: "integer"
+ type: "object"
+ resizePolicy:
+ items:
+ properties:
+ resourceName:
+ type: "string"
+ restartPolicy:
+ type: "string"
+ type: "object"
+ type: "array"
+ resources:
+ properties:
+ claims:
+ items:
+ properties:
+ name:
+ type: "string"
+ request:
+ type: "string"
+ type: "object"
+ type: "array"
+ limits:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ requests:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ restartPolicy:
+ type: "string"
+ securityContext:
+ properties:
+ allowPrivilegeEscalation:
+ type: "boolean"
+ appArmorProfile:
+ properties:
+ localhostProfile:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ capabilities:
+ properties:
+ add:
+ items:
+ type: "string"
+ type: "array"
+ drop:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ privileged:
+ type: "boolean"
+ procMount:
+ type: "string"
+ readOnlyRootFilesystem:
+ type: "boolean"
+ runAsGroup:
+ type: "integer"
+ runAsNonRoot:
+ type: "boolean"
+ runAsUser:
+ type: "integer"
+ seLinuxOptions:
+ properties:
+ level:
+ type: "string"
+ role:
+ type: "string"
+ type:
+ type: "string"
+ user:
+ type: "string"
+ type: "object"
+ seccompProfile:
+ properties:
+ localhostProfile:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ windowsOptions:
+ properties:
+ gmsaCredentialSpec:
+ type: "string"
+ gmsaCredentialSpecName:
+ type: "string"
+ hostProcess:
+ type: "boolean"
+ runAsUserName:
+ type: "string"
+ type: "object"
+ type: "object"
+ startupProbe:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ failureThreshold:
+ type: "integer"
+ grpc:
+ properties:
+ port:
+ type: "integer"
+ service:
+ type: "string"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ initialDelaySeconds:
+ type: "integer"
+ periodSeconds:
+ type: "integer"
+ successThreshold:
+ type: "integer"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ timeoutSeconds:
+ type: "integer"
+ type: "object"
+ stdin:
+ type: "boolean"
+ stdinOnce:
+ type: "boolean"
+ terminationMessagePath:
+ type: "string"
+ terminationMessagePolicy:
+ type: "string"
+ tty:
+ type: "boolean"
+ volumeDevices:
+ items:
+ properties:
+ devicePath:
+ type: "string"
+ name:
+ type: "string"
+ type: "object"
+ type: "array"
+ volumeMounts:
+ items:
+ properties:
+ mountPath:
+ type: "string"
+ mountPropagation:
+ type: "string"
+ name:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ recursiveReadOnly:
+ type: "string"
+ subPath:
+ type: "string"
+ subPathExpr:
+ type: "string"
+ type: "object"
+ type: "array"
+ workingDir:
+ type: "string"
+ type: "object"
+ type: "array"
+ dnsConfig:
+ properties:
+ nameservers:
+ items:
+ type: "string"
+ type: "array"
+ options:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ searches:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ dnsPolicy:
+ type: "string"
+ enableServiceLinks:
+ type: "boolean"
+ ephemeralContainers:
+ items:
+ properties:
+ args:
+ items:
+ type: "string"
+ type: "array"
+ command:
+ items:
+ type: "string"
+ type: "array"
+ env:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ valueFrom:
+ properties:
+ configMapKeyRef:
+ properties:
+ key:
+ type: "string"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ fieldRef:
+ properties:
+ apiVersion:
+ type: "string"
+ fieldPath:
+ type: "string"
+ type: "object"
+ resourceFieldRef:
+ properties:
+ containerName:
+ type: "string"
+ divisor:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ resource:
+ type: "string"
+ type: "object"
+ secretKeyRef:
+ properties:
+ key:
+ type: "string"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ type: "object"
+ type: "object"
+ type: "array"
+ envFrom:
+ items:
+ properties:
+ configMapRef:
+ properties:
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ prefix:
+ type: "string"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ type: "object"
+ type: "array"
+ image:
+ type: "string"
+ imagePullPolicy:
+ type: "string"
+ lifecycle:
+ properties:
+ postStart:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ sleep:
+ properties:
+ seconds:
+ type: "integer"
+ type: "object"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ preStop:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ sleep:
+ properties:
+ seconds:
+ type: "integer"
+ type: "object"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ stopSignal:
+ type: "string"
+ type: "object"
+ livenessProbe:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ failureThreshold:
+ type: "integer"
+ grpc:
+ properties:
+ port:
+ type: "integer"
+ service:
+ type: "string"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ initialDelaySeconds:
+ type: "integer"
+ periodSeconds:
+ type: "integer"
+ successThreshold:
+ type: "integer"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ timeoutSeconds:
+ type: "integer"
+ type: "object"
+ name:
+ type: "string"
+ ports:
+ items:
+ properties:
+ containerPort:
+ type: "integer"
+ hostIP:
+ type: "string"
+ hostPort:
+ type: "integer"
+ name:
+ type: "string"
+ protocol:
+ type: "string"
+ type: "object"
+ type: "array"
+ readinessProbe:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ failureThreshold:
+ type: "integer"
+ grpc:
+ properties:
+ port:
+ type: "integer"
+ service:
+ type: "string"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ initialDelaySeconds:
+ type: "integer"
+ periodSeconds:
+ type: "integer"
+ successThreshold:
+ type: "integer"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ timeoutSeconds:
+ type: "integer"
+ type: "object"
+ resizePolicy:
+ items:
+ properties:
+ resourceName:
+ type: "string"
+ restartPolicy:
+ type: "string"
+ type: "object"
+ type: "array"
+ resources:
+ properties:
+ claims:
+ items:
+ properties:
+ name:
+ type: "string"
+ request:
+ type: "string"
+ type: "object"
+ type: "array"
+ limits:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ requests:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ restartPolicy:
+ type: "string"
+ securityContext:
+ properties:
+ allowPrivilegeEscalation:
+ type: "boolean"
+ appArmorProfile:
+ properties:
+ localhostProfile:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ capabilities:
+ properties:
+ add:
+ items:
+ type: "string"
+ type: "array"
+ drop:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ privileged:
+ type: "boolean"
+ procMount:
+ type: "string"
+ readOnlyRootFilesystem:
+ type: "boolean"
+ runAsGroup:
+ type: "integer"
+ runAsNonRoot:
+ type: "boolean"
+ runAsUser:
+ type: "integer"
+ seLinuxOptions:
+ properties:
+ level:
+ type: "string"
+ role:
+ type: "string"
+ type:
+ type: "string"
+ user:
+ type: "string"
+ type: "object"
+ seccompProfile:
+ properties:
+ localhostProfile:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ windowsOptions:
+ properties:
+ gmsaCredentialSpec:
+ type: "string"
+ gmsaCredentialSpecName:
+ type: "string"
+ hostProcess:
+ type: "boolean"
+ runAsUserName:
+ type: "string"
+ type: "object"
+ type: "object"
+ startupProbe:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ failureThreshold:
+ type: "integer"
+ grpc:
+ properties:
+ port:
+ type: "integer"
+ service:
+ type: "string"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ initialDelaySeconds:
+ type: "integer"
+ periodSeconds:
+ type: "integer"
+ successThreshold:
+ type: "integer"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ timeoutSeconds:
+ type: "integer"
+ type: "object"
+ stdin:
+ type: "boolean"
+ stdinOnce:
+ type: "boolean"
+ targetContainerName:
+ type: "string"
+ terminationMessagePath:
+ type: "string"
+ terminationMessagePolicy:
+ type: "string"
+ tty:
+ type: "boolean"
+ volumeDevices:
+ items:
+ properties:
+ devicePath:
+ type: "string"
+ name:
+ type: "string"
+ type: "object"
+ type: "array"
+ volumeMounts:
+ items:
+ properties:
+ mountPath:
+ type: "string"
+ mountPropagation:
+ type: "string"
+ name:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ recursiveReadOnly:
+ type: "string"
+ subPath:
+ type: "string"
+ subPathExpr:
+ type: "string"
+ type: "object"
+ type: "array"
+ workingDir:
+ type: "string"
+ type: "object"
+ type: "array"
+ hostAliases:
+ items:
+ properties:
+ hostnames:
+ items:
+ type: "string"
+ type: "array"
+ ip:
+ type: "string"
+ type: "object"
+ type: "array"
+ hostIPC:
+ type: "boolean"
+ hostNetwork:
+ type: "boolean"
+ hostPID:
+ type: "boolean"
+ hostUsers:
+ type: "boolean"
+ hostname:
+ type: "string"
+ imagePullSecrets:
+ items:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ type: "array"
+ initContainers:
+ items:
+ properties:
+ args:
+ items:
+ type: "string"
+ type: "array"
+ command:
+ items:
+ type: "string"
+ type: "array"
+ env:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ valueFrom:
+ properties:
+ configMapKeyRef:
+ properties:
+ key:
+ type: "string"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ fieldRef:
+ properties:
+ apiVersion:
+ type: "string"
+ fieldPath:
+ type: "string"
+ type: "object"
+ resourceFieldRef:
+ properties:
+ containerName:
+ type: "string"
+ divisor:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ resource:
+ type: "string"
+ type: "object"
+ secretKeyRef:
+ properties:
+ key:
+ type: "string"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ type: "object"
+ type: "object"
+ type: "array"
+ envFrom:
+ items:
+ properties:
+ configMapRef:
+ properties:
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ prefix:
+ type: "string"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ type: "object"
+ type: "array"
+ image:
+ type: "string"
+ imagePullPolicy:
+ type: "string"
+ lifecycle:
+ properties:
+ postStart:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ sleep:
+ properties:
+ seconds:
+ type: "integer"
+ type: "object"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ preStop:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ sleep:
+ properties:
+ seconds:
+ type: "integer"
+ type: "object"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ stopSignal:
+ type: "string"
+ type: "object"
+ livenessProbe:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ failureThreshold:
+ type: "integer"
+ grpc:
+ properties:
+ port:
+ type: "integer"
+ service:
+ type: "string"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ initialDelaySeconds:
+ type: "integer"
+ periodSeconds:
+ type: "integer"
+ successThreshold:
+ type: "integer"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ timeoutSeconds:
+ type: "integer"
+ type: "object"
+ name:
+ type: "string"
+ ports:
+ items:
+ properties:
+ containerPort:
+ type: "integer"
+ hostIP:
+ type: "string"
+ hostPort:
+ type: "integer"
+ name:
+ type: "string"
+ protocol:
+ type: "string"
+ type: "object"
+ type: "array"
+ readinessProbe:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ failureThreshold:
+ type: "integer"
+ grpc:
+ properties:
+ port:
+ type: "integer"
+ service:
+ type: "string"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ initialDelaySeconds:
+ type: "integer"
+ periodSeconds:
+ type: "integer"
+ successThreshold:
+ type: "integer"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ timeoutSeconds:
+ type: "integer"
+ type: "object"
+ resizePolicy:
+ items:
+ properties:
+ resourceName:
+ type: "string"
+ restartPolicy:
+ type: "string"
+ type: "object"
+ type: "array"
+ resources:
+ properties:
+ claims:
+ items:
+ properties:
+ name:
+ type: "string"
+ request:
+ type: "string"
+ type: "object"
+ type: "array"
+ limits:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ requests:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ restartPolicy:
+ type: "string"
+ securityContext:
+ properties:
+ allowPrivilegeEscalation:
+ type: "boolean"
+ appArmorProfile:
+ properties:
+ localhostProfile:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ capabilities:
+ properties:
+ add:
+ items:
+ type: "string"
+ type: "array"
+ drop:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ privileged:
+ type: "boolean"
+ procMount:
+ type: "string"
+ readOnlyRootFilesystem:
+ type: "boolean"
+ runAsGroup:
+ type: "integer"
+ runAsNonRoot:
+ type: "boolean"
+ runAsUser:
+ type: "integer"
+ seLinuxOptions:
+ properties:
+ level:
+ type: "string"
+ role:
+ type: "string"
+ type:
+ type: "string"
+ user:
+ type: "string"
+ type: "object"
+ seccompProfile:
+ properties:
+ localhostProfile:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ windowsOptions:
+ properties:
+ gmsaCredentialSpec:
+ type: "string"
+ gmsaCredentialSpecName:
+ type: "string"
+ hostProcess:
+ type: "boolean"
+ runAsUserName:
+ type: "string"
+ type: "object"
+ type: "object"
+ startupProbe:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ failureThreshold:
+ type: "integer"
+ grpc:
+ properties:
+ port:
+ type: "integer"
+ service:
+ type: "string"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ initialDelaySeconds:
+ type: "integer"
+ periodSeconds:
+ type: "integer"
+ successThreshold:
+ type: "integer"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ timeoutSeconds:
+ type: "integer"
+ type: "object"
+ stdin:
+ type: "boolean"
+ stdinOnce:
+ type: "boolean"
+ terminationMessagePath:
+ type: "string"
+ terminationMessagePolicy:
+ type: "string"
+ tty:
+ type: "boolean"
+ volumeDevices:
+ items:
+ properties:
+ devicePath:
+ type: "string"
+ name:
+ type: "string"
+ type: "object"
+ type: "array"
+ volumeMounts:
+ items:
+ properties:
+ mountPath:
+ type: "string"
+ mountPropagation:
+ type: "string"
+ name:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ recursiveReadOnly:
+ type: "string"
+ subPath:
+ type: "string"
+ subPathExpr:
+ type: "string"
+ type: "object"
+ type: "array"
+ workingDir:
+ type: "string"
+ type: "object"
+ type: "array"
+ nodeName:
+ type: "string"
+ nodeSelector:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ os:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ overhead:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ preemptionPolicy:
+ type: "string"
+ priority:
+ type: "integer"
+ priorityClassName:
+ type: "string"
+ readinessGates:
+ items:
+ properties:
+ conditionType:
+ type: "string"
+ type: "object"
+ type: "array"
+ resourceClaims:
+ items:
+ properties:
+ name:
+ type: "string"
+ resourceClaimName:
+ type: "string"
+ resourceClaimTemplateName:
+ type: "string"
+ type: "object"
+ type: "array"
+ resources:
+ properties:
+ claims:
+ items:
+ properties:
+ name:
+ type: "string"
+ request:
+ type: "string"
+ type: "object"
+ type: "array"
+ limits:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ requests:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ restartPolicy:
+ type: "string"
+ runtimeClassName:
+ type: "string"
+ schedulerName:
+ type: "string"
+ schedulingGates:
+ items:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ type: "array"
+ securityContext:
+ properties:
+ appArmorProfile:
+ properties:
+ localhostProfile:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ fsGroup:
+ type: "integer"
+ fsGroupChangePolicy:
+ type: "string"
+ runAsGroup:
+ type: "integer"
+ runAsNonRoot:
+ type: "boolean"
+ runAsUser:
+ type: "integer"
+ seLinuxChangePolicy:
+ type: "string"
+ seLinuxOptions:
+ properties:
+ level:
+ type: "string"
+ role:
+ type: "string"
+ type:
+ type: "string"
+ user:
+ type: "string"
+ type: "object"
+ seccompProfile:
+ properties:
+ localhostProfile:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ supplementalGroups:
+ items:
+ type: "integer"
+ type: "array"
+ supplementalGroupsPolicy:
+ type: "string"
+ sysctls:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ windowsOptions:
+ properties:
+ gmsaCredentialSpec:
+ type: "string"
+ gmsaCredentialSpecName:
+ type: "string"
+ hostProcess:
+ type: "boolean"
+ runAsUserName:
+ type: "string"
+ type: "object"
+ type: "object"
+ serviceAccount:
+ type: "string"
+ serviceAccountName:
+ type: "string"
+ setHostnameAsFQDN:
+ type: "boolean"
+ shareProcessNamespace:
+ type: "boolean"
+ subdomain:
+ type: "string"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ tolerations:
+ items:
+ properties:
+ effect:
+ type: "string"
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ tolerationSeconds:
+ type: "integer"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ topologySpreadConstraints:
+ items:
+ properties:
+ labelSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ matchLabelKeys:
+ items:
+ type: "string"
+ type: "array"
+ maxSkew:
+ type: "integer"
+ minDomains:
+ type: "integer"
+ nodeAffinityPolicy:
+ type: "string"
+ nodeTaintsPolicy:
+ type: "string"
+ topologyKey:
+ type: "string"
+ whenUnsatisfiable:
+ type: "string"
+ type: "object"
+ type: "array"
+ volumes:
+ items:
+ properties:
+ awsElasticBlockStore:
+ properties:
+ fsType:
+ type: "string"
+ partition:
+ type: "integer"
+ readOnly:
+ type: "boolean"
+ volumeID:
+ type: "string"
+ type: "object"
+ azureDisk:
+ properties:
+ cachingMode:
+ type: "string"
+ diskName:
+ type: "string"
+ diskURI:
+ type: "string"
+ fsType:
+ type: "string"
+ kind:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ type: "object"
+ azureFile:
+ properties:
+ readOnly:
+ type: "boolean"
+ secretName:
+ type: "string"
+ shareName:
+ type: "string"
+ type: "object"
+ cephfs:
+ properties:
+ monitors:
+ items:
+ type: "string"
+ type: "array"
+ path:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ secretFile:
+ type: "string"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ user:
+ type: "string"
+ type: "object"
+ cinder:
+ properties:
+ fsType:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ volumeID:
+ type: "string"
+ type: "object"
+ configMap:
+ properties:
+ defaultMode:
+ type: "integer"
+ items:
+ items:
+ properties:
+ key:
+ type: "string"
+ mode:
+ type: "integer"
+ path:
+ type: "string"
+ type: "object"
+ type: "array"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ csi:
+ properties:
+ driver:
+ type: "string"
+ fsType:
+ type: "string"
+ nodePublishSecretRef:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ readOnly:
+ type: "boolean"
+ volumeAttributes:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ downwardAPI:
+ properties:
+ defaultMode:
+ type: "integer"
+ items:
+ items:
+ properties:
+ fieldRef:
+ properties:
+ apiVersion:
+ type: "string"
+ fieldPath:
+ type: "string"
+ type: "object"
+ mode:
+ type: "integer"
+ path:
+ type: "string"
+ resourceFieldRef:
+ properties:
+ containerName:
+ type: "string"
+ divisor:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ resource:
+ type: "string"
+ type: "object"
+ type: "object"
+ type: "array"
+ type: "object"
+ emptyDir:
+ properties:
+ medium:
+ type: "string"
+ sizeLimit:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ ephemeral:
+ properties:
+ volumeClaimTemplate:
+ properties:
+ metadata:
+ properties:
+ annotations:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ creationTimestamp:
+ type: "string"
+ deletionGracePeriodSeconds:
+ type: "integer"
+ deletionTimestamp:
+ type: "string"
+ finalizers:
+ items:
+ type: "string"
+ type: "array"
+ generateName:
+ type: "string"
+ generation:
+ type: "integer"
+ labels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ managedFields:
+ items:
+ properties:
+ apiVersion:
+ type: "string"
+ fieldsType:
+ type: "string"
+ fieldsV1:
+ type: "object"
+ manager:
+ type: "string"
+ operation:
+ type: "string"
+ subresource:
+ type: "string"
+ time:
+ type: "string"
+ type: "object"
+ type: "array"
+ name:
+ type: "string"
+ namespace:
+ type: "string"
+ ownerReferences:
+ items:
+ properties:
+ apiVersion:
+ type: "string"
+ blockOwnerDeletion:
+ type: "boolean"
+ controller:
+ type: "boolean"
+ kind:
+ type: "string"
+ name:
+ type: "string"
+ uid:
+ type: "string"
+ type: "object"
+ type: "array"
+ resourceVersion:
+ type: "string"
+ selfLink:
+ type: "string"
+ uid:
+ type: "string"
+ type: "object"
+ spec:
+ properties:
+ accessModes:
+ items:
+ type: "string"
+ type: "array"
+ dataSource:
+ properties:
+ apiGroup:
+ type: "string"
+ kind:
+ type: "string"
+ name:
+ type: "string"
+ type: "object"
+ dataSourceRef:
+ properties:
+ apiGroup:
+ type: "string"
+ kind:
+ type: "string"
+ name:
+ type: "string"
+ namespace:
+ type: "string"
+ type: "object"
+ resources:
+ properties:
+ limits:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ requests:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ selector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ storageClassName:
+ type: "string"
+ volumeAttributesClassName:
+ type: "string"
+ volumeMode:
+ type: "string"
+ volumeName:
+ type: "string"
+ type: "object"
+ type: "object"
+ type: "object"
+ fc:
+ properties:
+ fsType:
+ type: "string"
+ lun:
+ type: "integer"
+ readOnly:
+ type: "boolean"
+ targetWWNs:
+ items:
+ type: "string"
+ type: "array"
+ wwids:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ flexVolume:
+ properties:
+ driver:
+ type: "string"
+ fsType:
+ type: "string"
+ options:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ readOnly:
+ type: "boolean"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ type: "object"
+ flocker:
+ properties:
+ datasetName:
+ type: "string"
+ datasetUUID:
+ type: "string"
+ type: "object"
+ gcePersistentDisk:
+ properties:
+ fsType:
+ type: "string"
+ partition:
+ type: "integer"
+ pdName:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ type: "object"
+ gitRepo:
+ properties:
+ directory:
+ type: "string"
+ repository:
+ type: "string"
+ revision:
+ type: "string"
+ type: "object"
+ glusterfs:
+ properties:
+ endpoints:
+ type: "string"
+ path:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ type: "object"
+ hostPath:
+ properties:
+ path:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ image:
+ properties:
+ pullPolicy:
+ type: "string"
+ reference:
+ type: "string"
+ type: "object"
+ iscsi:
+ properties:
+ chapAuthDiscovery:
+ type: "boolean"
+ chapAuthSession:
+ type: "boolean"
+ fsType:
+ type: "string"
+ initiatorName:
+ type: "string"
+ iqn:
+ type: "string"
+ iscsiInterface:
+ type: "string"
+ lun:
+ type: "integer"
+ portals:
+ items:
+ type: "string"
+ type: "array"
+ readOnly:
+ type: "boolean"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ targetPortal:
+ type: "string"
+ type: "object"
+ name:
+ type: "string"
+ nfs:
+ properties:
+ path:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ server:
+ type: "string"
+ type: "object"
+ persistentVolumeClaim:
+ properties:
+ claimName:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ type: "object"
+ photonPersistentDisk:
+ properties:
+ fsType:
+ type: "string"
+ pdID:
+ type: "string"
+ type: "object"
+ portworxVolume:
+ properties:
+ fsType:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ volumeID:
+ type: "string"
+ type: "object"
+ projected:
+ properties:
+ defaultMode:
+ type: "integer"
+ sources:
+ items:
+ properties:
+ clusterTrustBundle:
+ properties:
+ labelSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ path:
+ type: "string"
+ signerName:
+ type: "string"
+ type: "object"
+ configMap:
+ properties:
+ items:
+ items:
+ properties:
+ key:
+ type: "string"
+ mode:
+ type: "integer"
+ path:
+ type: "string"
+ type: "object"
+ type: "array"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ downwardAPI:
+ properties:
+ items:
+ items:
+ properties:
+ fieldRef:
+ properties:
+ apiVersion:
+ type: "string"
+ fieldPath:
+ type: "string"
+ type: "object"
+ mode:
+ type: "integer"
+ path:
+ type: "string"
+ resourceFieldRef:
+ properties:
+ containerName:
+ type: "string"
+ divisor:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ resource:
+ type: "string"
+ type: "object"
+ type: "object"
+ type: "array"
+ type: "object"
+ secret:
+ properties:
+ items:
+ items:
+ properties:
+ key:
+ type: "string"
+ mode:
+ type: "integer"
+ path:
+ type: "string"
+ type: "object"
+ type: "array"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ serviceAccountToken:
+ properties:
+ audience:
+ type: "string"
+ expirationSeconds:
+ type: "integer"
+ path:
+ type: "string"
+ type: "object"
+ type: "object"
+ type: "array"
+ type: "object"
+ quobyte:
+ properties:
+ group:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ registry:
+ type: "string"
+ tenant:
+ type: "string"
+ user:
+ type: "string"
+ volume:
+ type: "string"
+ type: "object"
+ rbd:
+ properties:
+ fsType:
+ type: "string"
+ image:
+ type: "string"
+ keyring:
+ type: "string"
+ monitors:
+ items:
+ type: "string"
+ type: "array"
+ pool:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ user:
+ type: "string"
+ type: "object"
+ scaleIO:
+ properties:
+ fsType:
+ type: "string"
+ gateway:
+ type: "string"
+ protectionDomain:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ sslEnabled:
+ type: "boolean"
+ storageMode:
+ type: "string"
+ storagePool:
+ type: "string"
+ system:
+ type: "string"
+ volumeName:
+ type: "string"
+ type: "object"
+ secret:
+ properties:
+ defaultMode:
+ type: "integer"
+ items:
+ items:
+ properties:
+ key:
+ type: "string"
+ mode:
+ type: "integer"
+ path:
+ type: "string"
+ type: "object"
+ type: "array"
+ optional:
+ type: "boolean"
+ secretName:
+ type: "string"
+ type: "object"
+ storageos:
+ properties:
+ fsType:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ volumeName:
+ type: "string"
+ volumeNamespace:
+ type: "string"
+ type: "object"
+ vsphereVolume:
+ properties:
+ fsType:
+ type: "string"
+ storagePolicyID:
+ type: "string"
+ storagePolicyName:
+ type: "string"
+ volumePath:
+ type: "string"
+ type: "object"
+ type: "object"
+ type: "array"
+ type: "object"
+ status:
+ properties:
+ conditions:
+ items:
+ properties:
+ lastProbeTime:
+ type: "string"
+ lastTransitionTime:
+ type: "string"
+ message:
+ type: "string"
+ observedGeneration:
+ type: "integer"
+ reason:
+ type: "string"
+ status:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ type: "array"
+ containerStatuses:
+ items:
+ properties:
+ allocatedResources:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ allocatedResourcesStatus:
+ items:
+ properties:
+ name:
+ type: "string"
+ resources:
+ items:
+ properties:
+ health:
+ type: "string"
+ resourceID:
+ type: "string"
+ type: "object"
+ type: "array"
+ type: "object"
+ type: "array"
+ containerID:
+ type: "string"
+ image:
+ type: "string"
+ imageID:
+ type: "string"
+ lastState:
+ properties:
+ running:
+ properties:
+ startedAt:
+ type: "string"
+ type: "object"
+ terminated:
+ properties:
+ containerID:
+ type: "string"
+ exitCode:
+ type: "integer"
+ finishedAt:
+ type: "string"
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ signal:
+ type: "integer"
+ startedAt:
+ type: "string"
+ type: "object"
+ waiting:
+ properties:
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ type: "object"
+ type: "object"
+ name:
+ type: "string"
+ ready:
+ type: "boolean"
+ resources:
+ properties:
+ claims:
+ items:
+ properties:
+ name:
+ type: "string"
+ request:
+ type: "string"
+ type: "object"
+ type: "array"
+ limits:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ requests:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ restartCount:
+ type: "integer"
+ started:
+ type: "boolean"
+ state:
+ properties:
+ running:
+ properties:
+ startedAt:
+ type: "string"
+ type: "object"
+ terminated:
+ properties:
+ containerID:
+ type: "string"
+ exitCode:
+ type: "integer"
+ finishedAt:
+ type: "string"
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ signal:
+ type: "integer"
+ startedAt:
+ type: "string"
+ type: "object"
+ waiting:
+ properties:
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ type: "object"
+ type: "object"
+ stopSignal:
+ type: "string"
+ user:
+ properties:
+ linux:
+ properties:
+ gid:
+ type: "integer"
+ supplementalGroups:
+ items:
+ type: "integer"
+ type: "array"
+ uid:
+ type: "integer"
+ type: "object"
+ type: "object"
+ volumeMounts:
+ items:
+ properties:
+ mountPath:
+ type: "string"
+ name:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ recursiveReadOnly:
+ type: "string"
+ type: "object"
+ type: "array"
+ type: "object"
+ type: "array"
+ ephemeralContainerStatuses:
+ items:
+ properties:
+ allocatedResources:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ allocatedResourcesStatus:
+ items:
+ properties:
+ name:
+ type: "string"
+ resources:
+ items:
+ properties:
+ health:
+ type: "string"
+ resourceID:
+ type: "string"
+ type: "object"
+ type: "array"
+ type: "object"
+ type: "array"
+ containerID:
+ type: "string"
+ image:
+ type: "string"
+ imageID:
+ type: "string"
+ lastState:
+ properties:
+ running:
+ properties:
+ startedAt:
+ type: "string"
+ type: "object"
+ terminated:
+ properties:
+ containerID:
+ type: "string"
+ exitCode:
+ type: "integer"
+ finishedAt:
+ type: "string"
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ signal:
+ type: "integer"
+ startedAt:
+ type: "string"
+ type: "object"
+ waiting:
+ properties:
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ type: "object"
+ type: "object"
+ name:
+ type: "string"
+ ready:
+ type: "boolean"
+ resources:
+ properties:
+ claims:
+ items:
+ properties:
+ name:
+ type: "string"
+ request:
+ type: "string"
+ type: "object"
+ type: "array"
+ limits:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ requests:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ restartCount:
+ type: "integer"
+ started:
+ type: "boolean"
+ state:
+ properties:
+ running:
+ properties:
+ startedAt:
+ type: "string"
+ type: "object"
+ terminated:
+ properties:
+ containerID:
+ type: "string"
+ exitCode:
+ type: "integer"
+ finishedAt:
+ type: "string"
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ signal:
+ type: "integer"
+ startedAt:
+ type: "string"
+ type: "object"
+ waiting:
+ properties:
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ type: "object"
+ type: "object"
+ stopSignal:
+ type: "string"
+ user:
+ properties:
+ linux:
+ properties:
+ gid:
+ type: "integer"
+ supplementalGroups:
+ items:
+ type: "integer"
+ type: "array"
+ uid:
+ type: "integer"
+ type: "object"
+ type: "object"
+ volumeMounts:
+ items:
+ properties:
+ mountPath:
+ type: "string"
+ name:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ recursiveReadOnly:
+ type: "string"
+ type: "object"
+ type: "array"
+ type: "object"
+ type: "array"
+ hostIP:
+ type: "string"
+ hostIPs:
+ items:
+ properties:
+ ip:
+ type: "string"
+ type: "object"
+ type: "array"
+ initContainerStatuses:
+ items:
+ properties:
+ allocatedResources:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ allocatedResourcesStatus:
+ items:
+ properties:
+ name:
+ type: "string"
+ resources:
+ items:
+ properties:
+ health:
+ type: "string"
+ resourceID:
+ type: "string"
+ type: "object"
+ type: "array"
+ type: "object"
+ type: "array"
+ containerID:
+ type: "string"
+ image:
+ type: "string"
+ imageID:
+ type: "string"
+ lastState:
+ properties:
+ running:
+ properties:
+ startedAt:
+ type: "string"
+ type: "object"
+ terminated:
+ properties:
+ containerID:
+ type: "string"
+ exitCode:
+ type: "integer"
+ finishedAt:
+ type: "string"
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ signal:
+ type: "integer"
+ startedAt:
+ type: "string"
+ type: "object"
+ waiting:
+ properties:
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ type: "object"
+ type: "object"
+ name:
+ type: "string"
+ ready:
+ type: "boolean"
+ resources:
+ properties:
+ claims:
+ items:
+ properties:
+ name:
+ type: "string"
+ request:
+ type: "string"
+ type: "object"
+ type: "array"
+ limits:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ requests:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ restartCount:
+ type: "integer"
+ started:
+ type: "boolean"
+ state:
+ properties:
+ running:
+ properties:
+ startedAt:
+ type: "string"
+ type: "object"
+ terminated:
+ properties:
+ containerID:
+ type: "string"
+ exitCode:
+ type: "integer"
+ finishedAt:
+ type: "string"
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ signal:
+ type: "integer"
+ startedAt:
+ type: "string"
+ type: "object"
+ waiting:
+ properties:
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ type: "object"
+ type: "object"
+ stopSignal:
+ type: "string"
+ user:
+ properties:
+ linux:
+ properties:
+ gid:
+ type: "integer"
+ supplementalGroups:
+ items:
+ type: "integer"
+ type: "array"
+ uid:
+ type: "integer"
+ type: "object"
+ type: "object"
+ volumeMounts:
+ items:
+ properties:
+ mountPath:
+ type: "string"
+ name:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ recursiveReadOnly:
+ type: "string"
+ type: "object"
+ type: "array"
+ type: "object"
+ type: "array"
+ message:
+ type: "string"
+ nominatedNodeName:
+ type: "string"
+ observedGeneration:
+ type: "integer"
+ phase:
+ type: "string"
+ podIP:
+ type: "string"
+ podIPs:
+ items:
+ properties:
+ ip:
+ type: "string"
+ type: "object"
+ type: "array"
+ qosClass:
+ type: "string"
+ reason:
+ type: "string"
+ resize:
+ type: "string"
+ resourceClaimStatuses:
+ items:
+ properties:
+ name:
+ type: "string"
+ resourceClaimName:
+ type: "string"
+ type: "object"
+ type: "array"
+ startTime:
+ type: "string"
+ type: "object"
+ type: "object"
+ restartNonce:
+ type: "integer"
+ serviceAccount:
+ type: "string"
+ taskManager:
+ properties:
+ podTemplate:
+ properties:
+ apiVersion:
+ type: "string"
+ kind:
+ type: "string"
+ metadata:
+ properties:
+ annotations:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ creationTimestamp:
+ type: "string"
+ deletionGracePeriodSeconds:
+ type: "integer"
+ deletionTimestamp:
+ type: "string"
+ finalizers:
+ items:
+ type: "string"
+ type: "array"
+ generateName:
+ type: "string"
+ generation:
+ type: "integer"
+ labels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ managedFields:
+ items:
+ properties:
+ apiVersion:
+ type: "string"
+ fieldsType:
+ type: "string"
+ fieldsV1:
+ type: "object"
+ manager:
+ type: "string"
+ operation:
+ type: "string"
+ subresource:
+ type: "string"
+ time:
+ type: "string"
+ type: "object"
+ type: "array"
+ name:
+ type: "string"
+ namespace:
+ type: "string"
+ ownerReferences:
+ items:
+ properties:
+ apiVersion:
+ type: "string"
+ blockOwnerDeletion:
+ type: "boolean"
+ controller:
+ type: "boolean"
+ kind:
+ type: "string"
+ name:
+ type: "string"
+ uid:
+ type: "string"
+ type: "object"
+ type: "array"
+ resourceVersion:
+ type: "string"
+ selfLink:
+ type: "string"
+ uid:
+ type: "string"
+ type: "object"
+ spec:
+ properties:
+ activeDeadlineSeconds:
+ type: "integer"
+ affinity:
+ properties:
+ nodeAffinity:
+ properties:
+ preferredDuringSchedulingIgnoredDuringExecution:
+ items:
+ properties:
+ preference:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchFields:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ type: "object"
+ weight:
+ type: "integer"
+ type: "object"
+ type: "array"
+ requiredDuringSchedulingIgnoredDuringExecution:
+ properties:
+ nodeSelectorTerms:
+ items:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchFields:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ type: "object"
+ type: "array"
+ type: "object"
+ type: "object"
+ podAffinity:
+ properties:
+ preferredDuringSchedulingIgnoredDuringExecution:
+ items:
+ properties:
+ podAffinityTerm:
+ properties:
+ labelSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ matchLabelKeys:
+ items:
+ type: "string"
+ type: "array"
+ mismatchLabelKeys:
+ items:
+ type: "string"
+ type: "array"
+ namespaceSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ namespaces:
+ items:
+ type: "string"
+ type: "array"
+ topologyKey:
+ type: "string"
+ type: "object"
+ weight:
+ type: "integer"
+ type: "object"
+ type: "array"
+ requiredDuringSchedulingIgnoredDuringExecution:
+ items:
+ properties:
+ labelSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ matchLabelKeys:
+ items:
+ type: "string"
+ type: "array"
+ mismatchLabelKeys:
+ items:
+ type: "string"
+ type: "array"
+ namespaceSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ namespaces:
+ items:
+ type: "string"
+ type: "array"
+ topologyKey:
+ type: "string"
+ type: "object"
+ type: "array"
+ type: "object"
+ podAntiAffinity:
+ properties:
+ preferredDuringSchedulingIgnoredDuringExecution:
+ items:
+ properties:
+ podAffinityTerm:
+ properties:
+ labelSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ matchLabelKeys:
+ items:
+ type: "string"
+ type: "array"
+ mismatchLabelKeys:
+ items:
+ type: "string"
+ type: "array"
+ namespaceSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ namespaces:
+ items:
+ type: "string"
+ type: "array"
+ topologyKey:
+ type: "string"
+ type: "object"
+ weight:
+ type: "integer"
+ type: "object"
+ type: "array"
+ requiredDuringSchedulingIgnoredDuringExecution:
+ items:
+ properties:
+ labelSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ matchLabelKeys:
+ items:
+ type: "string"
+ type: "array"
+ mismatchLabelKeys:
+ items:
+ type: "string"
+ type: "array"
+ namespaceSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ namespaces:
+ items:
+ type: "string"
+ type: "array"
+ topologyKey:
+ type: "string"
+ type: "object"
+ type: "array"
+ type: "object"
+ type: "object"
+ automountServiceAccountToken:
+ type: "boolean"
+ containers:
+ items:
+ properties:
+ args:
+ items:
+ type: "string"
+ type: "array"
+ command:
+ items:
+ type: "string"
+ type: "array"
+ env:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ valueFrom:
+ properties:
+ configMapKeyRef:
+ properties:
+ key:
+ type: "string"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ fieldRef:
+ properties:
+ apiVersion:
+ type: "string"
+ fieldPath:
+ type: "string"
+ type: "object"
+ resourceFieldRef:
+ properties:
+ containerName:
+ type: "string"
+ divisor:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ resource:
+ type: "string"
+ type: "object"
+ secretKeyRef:
+ properties:
+ key:
+ type: "string"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ type: "object"
+ type: "object"
+ type: "array"
+ envFrom:
+ items:
+ properties:
+ configMapRef:
+ properties:
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ prefix:
+ type: "string"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ type: "object"
+ type: "array"
+ image:
+ type: "string"
+ imagePullPolicy:
+ type: "string"
+ lifecycle:
+ properties:
+ postStart:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ sleep:
+ properties:
+ seconds:
+ type: "integer"
+ type: "object"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ preStop:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ sleep:
+ properties:
+ seconds:
+ type: "integer"
+ type: "object"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ stopSignal:
+ type: "string"
+ type: "object"
+ livenessProbe:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ failureThreshold:
+ type: "integer"
+ grpc:
+ properties:
+ port:
+ type: "integer"
+ service:
+ type: "string"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ initialDelaySeconds:
+ type: "integer"
+ periodSeconds:
+ type: "integer"
+ successThreshold:
+ type: "integer"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ timeoutSeconds:
+ type: "integer"
+ type: "object"
+ name:
+ type: "string"
+ ports:
+ items:
+ properties:
+ containerPort:
+ type: "integer"
+ hostIP:
+ type: "string"
+ hostPort:
+ type: "integer"
+ name:
+ type: "string"
+ protocol:
+ type: "string"
+ type: "object"
+ type: "array"
+ readinessProbe:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ failureThreshold:
+ type: "integer"
+ grpc:
+ properties:
+ port:
+ type: "integer"
+ service:
+ type: "string"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ initialDelaySeconds:
+ type: "integer"
+ periodSeconds:
+ type: "integer"
+ successThreshold:
+ type: "integer"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ timeoutSeconds:
+ type: "integer"
+ type: "object"
+ resizePolicy:
+ items:
+ properties:
+ resourceName:
+ type: "string"
+ restartPolicy:
+ type: "string"
+ type: "object"
+ type: "array"
+ resources:
+ properties:
+ claims:
+ items:
+ properties:
+ name:
+ type: "string"
+ request:
+ type: "string"
+ type: "object"
+ type: "array"
+ limits:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ requests:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ restartPolicy:
+ type: "string"
+ securityContext:
+ properties:
+ allowPrivilegeEscalation:
+ type: "boolean"
+ appArmorProfile:
+ properties:
+ localhostProfile:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ capabilities:
+ properties:
+ add:
+ items:
+ type: "string"
+ type: "array"
+ drop:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ privileged:
+ type: "boolean"
+ procMount:
+ type: "string"
+ readOnlyRootFilesystem:
+ type: "boolean"
+ runAsGroup:
+ type: "integer"
+ runAsNonRoot:
+ type: "boolean"
+ runAsUser:
+ type: "integer"
+ seLinuxOptions:
+ properties:
+ level:
+ type: "string"
+ role:
+ type: "string"
+ type:
+ type: "string"
+ user:
+ type: "string"
+ type: "object"
+ seccompProfile:
+ properties:
+ localhostProfile:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ windowsOptions:
+ properties:
+ gmsaCredentialSpec:
+ type: "string"
+ gmsaCredentialSpecName:
+ type: "string"
+ hostProcess:
+ type: "boolean"
+ runAsUserName:
+ type: "string"
+ type: "object"
+ type: "object"
+ startupProbe:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ failureThreshold:
+ type: "integer"
+ grpc:
+ properties:
+ port:
+ type: "integer"
+ service:
+ type: "string"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ initialDelaySeconds:
+ type: "integer"
+ periodSeconds:
+ type: "integer"
+ successThreshold:
+ type: "integer"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ timeoutSeconds:
+ type: "integer"
+ type: "object"
+ stdin:
+ type: "boolean"
+ stdinOnce:
+ type: "boolean"
+ terminationMessagePath:
+ type: "string"
+ terminationMessagePolicy:
+ type: "string"
+ tty:
+ type: "boolean"
+ volumeDevices:
+ items:
+ properties:
+ devicePath:
+ type: "string"
+ name:
+ type: "string"
+ type: "object"
+ type: "array"
+ volumeMounts:
+ items:
+ properties:
+ mountPath:
+ type: "string"
+ mountPropagation:
+ type: "string"
+ name:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ recursiveReadOnly:
+ type: "string"
+ subPath:
+ type: "string"
+ subPathExpr:
+ type: "string"
+ type: "object"
+ type: "array"
+ workingDir:
+ type: "string"
+ type: "object"
+ type: "array"
+ dnsConfig:
+ properties:
+ nameservers:
+ items:
+ type: "string"
+ type: "array"
+ options:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ searches:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ dnsPolicy:
+ type: "string"
+ enableServiceLinks:
+ type: "boolean"
+ ephemeralContainers:
+ items:
+ properties:
+ args:
+ items:
+ type: "string"
+ type: "array"
+ command:
+ items:
+ type: "string"
+ type: "array"
+ env:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ valueFrom:
+ properties:
+ configMapKeyRef:
+ properties:
+ key:
+ type: "string"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ fieldRef:
+ properties:
+ apiVersion:
+ type: "string"
+ fieldPath:
+ type: "string"
+ type: "object"
+ resourceFieldRef:
+ properties:
+ containerName:
+ type: "string"
+ divisor:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ resource:
+ type: "string"
+ type: "object"
+ secretKeyRef:
+ properties:
+ key:
+ type: "string"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ type: "object"
+ type: "object"
+ type: "array"
+ envFrom:
+ items:
+ properties:
+ configMapRef:
+ properties:
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ prefix:
+ type: "string"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ type: "object"
+ type: "array"
+ image:
+ type: "string"
+ imagePullPolicy:
+ type: "string"
+ lifecycle:
+ properties:
+ postStart:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ sleep:
+ properties:
+ seconds:
+ type: "integer"
+ type: "object"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ preStop:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ sleep:
+ properties:
+ seconds:
+ type: "integer"
+ type: "object"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ stopSignal:
+ type: "string"
+ type: "object"
+ livenessProbe:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ failureThreshold:
+ type: "integer"
+ grpc:
+ properties:
+ port:
+ type: "integer"
+ service:
+ type: "string"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ initialDelaySeconds:
+ type: "integer"
+ periodSeconds:
+ type: "integer"
+ successThreshold:
+ type: "integer"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ timeoutSeconds:
+ type: "integer"
+ type: "object"
+ name:
+ type: "string"
+ ports:
+ items:
+ properties:
+ containerPort:
+ type: "integer"
+ hostIP:
+ type: "string"
+ hostPort:
+ type: "integer"
+ name:
+ type: "string"
+ protocol:
+ type: "string"
+ type: "object"
+ type: "array"
+ readinessProbe:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ failureThreshold:
+ type: "integer"
+ grpc:
+ properties:
+ port:
+ type: "integer"
+ service:
+ type: "string"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ initialDelaySeconds:
+ type: "integer"
+ periodSeconds:
+ type: "integer"
+ successThreshold:
+ type: "integer"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ timeoutSeconds:
+ type: "integer"
+ type: "object"
+ resizePolicy:
+ items:
+ properties:
+ resourceName:
+ type: "string"
+ restartPolicy:
+ type: "string"
+ type: "object"
+ type: "array"
+ resources:
+ properties:
+ claims:
+ items:
+ properties:
+ name:
+ type: "string"
+ request:
+ type: "string"
+ type: "object"
+ type: "array"
+ limits:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ requests:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ restartPolicy:
+ type: "string"
+ securityContext:
+ properties:
+ allowPrivilegeEscalation:
+ type: "boolean"
+ appArmorProfile:
+ properties:
+ localhostProfile:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ capabilities:
+ properties:
+ add:
+ items:
+ type: "string"
+ type: "array"
+ drop:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ privileged:
+ type: "boolean"
+ procMount:
+ type: "string"
+ readOnlyRootFilesystem:
+ type: "boolean"
+ runAsGroup:
+ type: "integer"
+ runAsNonRoot:
+ type: "boolean"
+ runAsUser:
+ type: "integer"
+ seLinuxOptions:
+ properties:
+ level:
+ type: "string"
+ role:
+ type: "string"
+ type:
+ type: "string"
+ user:
+ type: "string"
+ type: "object"
+ seccompProfile:
+ properties:
+ localhostProfile:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ windowsOptions:
+ properties:
+ gmsaCredentialSpec:
+ type: "string"
+ gmsaCredentialSpecName:
+ type: "string"
+ hostProcess:
+ type: "boolean"
+ runAsUserName:
+ type: "string"
+ type: "object"
+ type: "object"
+ startupProbe:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ failureThreshold:
+ type: "integer"
+ grpc:
+ properties:
+ port:
+ type: "integer"
+ service:
+ type: "string"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ initialDelaySeconds:
+ type: "integer"
+ periodSeconds:
+ type: "integer"
+ successThreshold:
+ type: "integer"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ timeoutSeconds:
+ type: "integer"
+ type: "object"
+ stdin:
+ type: "boolean"
+ stdinOnce:
+ type: "boolean"
+ targetContainerName:
+ type: "string"
+ terminationMessagePath:
+ type: "string"
+ terminationMessagePolicy:
+ type: "string"
+ tty:
+ type: "boolean"
+ volumeDevices:
+ items:
+ properties:
+ devicePath:
+ type: "string"
+ name:
+ type: "string"
+ type: "object"
+ type: "array"
+ volumeMounts:
+ items:
+ properties:
+ mountPath:
+ type: "string"
+ mountPropagation:
+ type: "string"
+ name:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ recursiveReadOnly:
+ type: "string"
+ subPath:
+ type: "string"
+ subPathExpr:
+ type: "string"
+ type: "object"
+ type: "array"
+ workingDir:
+ type: "string"
+ type: "object"
+ type: "array"
+ hostAliases:
+ items:
+ properties:
+ hostnames:
+ items:
+ type: "string"
+ type: "array"
+ ip:
+ type: "string"
+ type: "object"
+ type: "array"
+ hostIPC:
+ type: "boolean"
+ hostNetwork:
+ type: "boolean"
+ hostPID:
+ type: "boolean"
+ hostUsers:
+ type: "boolean"
+ hostname:
+ type: "string"
+ imagePullSecrets:
+ items:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ type: "array"
+ initContainers:
+ items:
+ properties:
+ args:
+ items:
+ type: "string"
+ type: "array"
+ command:
+ items:
+ type: "string"
+ type: "array"
+ env:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ valueFrom:
+ properties:
+ configMapKeyRef:
+ properties:
+ key:
+ type: "string"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ fieldRef:
+ properties:
+ apiVersion:
+ type: "string"
+ fieldPath:
+ type: "string"
+ type: "object"
+ resourceFieldRef:
+ properties:
+ containerName:
+ type: "string"
+ divisor:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ resource:
+ type: "string"
+ type: "object"
+ secretKeyRef:
+ properties:
+ key:
+ type: "string"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ type: "object"
+ type: "object"
+ type: "array"
+ envFrom:
+ items:
+ properties:
+ configMapRef:
+ properties:
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ prefix:
+ type: "string"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ type: "object"
+ type: "array"
+ image:
+ type: "string"
+ imagePullPolicy:
+ type: "string"
+ lifecycle:
+ properties:
+ postStart:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ sleep:
+ properties:
+ seconds:
+ type: "integer"
+ type: "object"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ preStop:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ sleep:
+ properties:
+ seconds:
+ type: "integer"
+ type: "object"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ stopSignal:
+ type: "string"
+ type: "object"
+ livenessProbe:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ failureThreshold:
+ type: "integer"
+ grpc:
+ properties:
+ port:
+ type: "integer"
+ service:
+ type: "string"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ initialDelaySeconds:
+ type: "integer"
+ periodSeconds:
+ type: "integer"
+ successThreshold:
+ type: "integer"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ timeoutSeconds:
+ type: "integer"
+ type: "object"
+ name:
+ type: "string"
+ ports:
+ items:
+ properties:
+ containerPort:
+ type: "integer"
+ hostIP:
+ type: "string"
+ hostPort:
+ type: "integer"
+ name:
+ type: "string"
+ protocol:
+ type: "string"
+ type: "object"
+ type: "array"
+ readinessProbe:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ failureThreshold:
+ type: "integer"
+ grpc:
+ properties:
+ port:
+ type: "integer"
+ service:
+ type: "string"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ initialDelaySeconds:
+ type: "integer"
+ periodSeconds:
+ type: "integer"
+ successThreshold:
+ type: "integer"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ timeoutSeconds:
+ type: "integer"
+ type: "object"
+ resizePolicy:
+ items:
+ properties:
+ resourceName:
+ type: "string"
+ restartPolicy:
+ type: "string"
+ type: "object"
+ type: "array"
+ resources:
+ properties:
+ claims:
+ items:
+ properties:
+ name:
+ type: "string"
+ request:
+ type: "string"
+ type: "object"
+ type: "array"
+ limits:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ requests:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ restartPolicy:
+ type: "string"
+ securityContext:
+ properties:
+ allowPrivilegeEscalation:
+ type: "boolean"
+ appArmorProfile:
+ properties:
+ localhostProfile:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ capabilities:
+ properties:
+ add:
+ items:
+ type: "string"
+ type: "array"
+ drop:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ privileged:
+ type: "boolean"
+ procMount:
+ type: "string"
+ readOnlyRootFilesystem:
+ type: "boolean"
+ runAsGroup:
+ type: "integer"
+ runAsNonRoot:
+ type: "boolean"
+ runAsUser:
+ type: "integer"
+ seLinuxOptions:
+ properties:
+ level:
+ type: "string"
+ role:
+ type: "string"
+ type:
+ type: "string"
+ user:
+ type: "string"
+ type: "object"
+ seccompProfile:
+ properties:
+ localhostProfile:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ windowsOptions:
+ properties:
+ gmsaCredentialSpec:
+ type: "string"
+ gmsaCredentialSpecName:
+ type: "string"
+ hostProcess:
+ type: "boolean"
+ runAsUserName:
+ type: "string"
+ type: "object"
+ type: "object"
+ startupProbe:
+ properties:
+ exec:
+ properties:
+ command:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ failureThreshold:
+ type: "integer"
+ grpc:
+ properties:
+ port:
+ type: "integer"
+ service:
+ type: "string"
+ type: "object"
+ httpGet:
+ properties:
+ host:
+ type: "string"
+ httpHeaders:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ path:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ scheme:
+ type: "string"
+ type: "object"
+ initialDelaySeconds:
+ type: "integer"
+ periodSeconds:
+ type: "integer"
+ successThreshold:
+ type: "integer"
+ tcpSocket:
+ properties:
+ host:
+ type: "string"
+ port:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ timeoutSeconds:
+ type: "integer"
+ type: "object"
+ stdin:
+ type: "boolean"
+ stdinOnce:
+ type: "boolean"
+ terminationMessagePath:
+ type: "string"
+ terminationMessagePolicy:
+ type: "string"
+ tty:
+ type: "boolean"
+ volumeDevices:
+ items:
+ properties:
+ devicePath:
+ type: "string"
+ name:
+ type: "string"
+ type: "object"
+ type: "array"
+ volumeMounts:
+ items:
+ properties:
+ mountPath:
+ type: "string"
+ mountPropagation:
+ type: "string"
+ name:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ recursiveReadOnly:
+ type: "string"
+ subPath:
+ type: "string"
+ subPathExpr:
+ type: "string"
+ type: "object"
+ type: "array"
+ workingDir:
+ type: "string"
+ type: "object"
+ type: "array"
+ nodeName:
+ type: "string"
+ nodeSelector:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ os:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ overhead:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ preemptionPolicy:
+ type: "string"
+ priority:
+ type: "integer"
+ priorityClassName:
+ type: "string"
+ readinessGates:
+ items:
+ properties:
+ conditionType:
+ type: "string"
+ type: "object"
+ type: "array"
+ resourceClaims:
+ items:
+ properties:
+ name:
+ type: "string"
+ resourceClaimName:
+ type: "string"
+ resourceClaimTemplateName:
+ type: "string"
+ type: "object"
+ type: "array"
+ resources:
+ properties:
+ claims:
+ items:
+ properties:
+ name:
+ type: "string"
+ request:
+ type: "string"
+ type: "object"
+ type: "array"
+ limits:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ requests:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ restartPolicy:
+ type: "string"
+ runtimeClassName:
+ type: "string"
+ schedulerName:
+ type: "string"
+ schedulingGates:
+ items:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ type: "array"
+ securityContext:
+ properties:
+ appArmorProfile:
+ properties:
+ localhostProfile:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ fsGroup:
+ type: "integer"
+ fsGroupChangePolicy:
+ type: "string"
+ runAsGroup:
+ type: "integer"
+ runAsNonRoot:
+ type: "boolean"
+ runAsUser:
+ type: "integer"
+ seLinuxChangePolicy:
+ type: "string"
+ seLinuxOptions:
+ properties:
+ level:
+ type: "string"
+ role:
+ type: "string"
+ type:
+ type: "string"
+ user:
+ type: "string"
+ type: "object"
+ seccompProfile:
+ properties:
+ localhostProfile:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ supplementalGroups:
+ items:
+ type: "integer"
+ type: "array"
+ supplementalGroupsPolicy:
+ type: "string"
+ sysctls:
+ items:
+ properties:
+ name:
+ type: "string"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ windowsOptions:
+ properties:
+ gmsaCredentialSpec:
+ type: "string"
+ gmsaCredentialSpecName:
+ type: "string"
+ hostProcess:
+ type: "boolean"
+ runAsUserName:
+ type: "string"
+ type: "object"
+ type: "object"
+ serviceAccount:
+ type: "string"
+ serviceAccountName:
+ type: "string"
+ setHostnameAsFQDN:
+ type: "boolean"
+ shareProcessNamespace:
+ type: "boolean"
+ subdomain:
+ type: "string"
+ terminationGracePeriodSeconds:
+ type: "integer"
+ tolerations:
+ items:
+ properties:
+ effect:
+ type: "string"
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ tolerationSeconds:
+ type: "integer"
+ value:
+ type: "string"
+ type: "object"
+ type: "array"
+ topologySpreadConstraints:
+ items:
+ properties:
+ labelSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ matchLabelKeys:
+ items:
+ type: "string"
+ type: "array"
+ maxSkew:
+ type: "integer"
+ minDomains:
+ type: "integer"
+ nodeAffinityPolicy:
+ type: "string"
+ nodeTaintsPolicy:
+ type: "string"
+ topologyKey:
+ type: "string"
+ whenUnsatisfiable:
+ type: "string"
+ type: "object"
+ type: "array"
+ volumes:
+ items:
+ properties:
+ awsElasticBlockStore:
+ properties:
+ fsType:
+ type: "string"
+ partition:
+ type: "integer"
+ readOnly:
+ type: "boolean"
+ volumeID:
+ type: "string"
+ type: "object"
+ azureDisk:
+ properties:
+ cachingMode:
+ type: "string"
+ diskName:
+ type: "string"
+ diskURI:
+ type: "string"
+ fsType:
+ type: "string"
+ kind:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ type: "object"
+ azureFile:
+ properties:
+ readOnly:
+ type: "boolean"
+ secretName:
+ type: "string"
+ shareName:
+ type: "string"
+ type: "object"
+ cephfs:
+ properties:
+ monitors:
+ items:
+ type: "string"
+ type: "array"
+ path:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ secretFile:
+ type: "string"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ user:
+ type: "string"
+ type: "object"
+ cinder:
+ properties:
+ fsType:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ volumeID:
+ type: "string"
+ type: "object"
+ configMap:
+ properties:
+ defaultMode:
+ type: "integer"
+ items:
+ items:
+ properties:
+ key:
+ type: "string"
+ mode:
+ type: "integer"
+ path:
+ type: "string"
+ type: "object"
+ type: "array"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ csi:
+ properties:
+ driver:
+ type: "string"
+ fsType:
+ type: "string"
+ nodePublishSecretRef:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ readOnly:
+ type: "boolean"
+ volumeAttributes:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ downwardAPI:
+ properties:
+ defaultMode:
+ type: "integer"
+ items:
+ items:
+ properties:
+ fieldRef:
+ properties:
+ apiVersion:
+ type: "string"
+ fieldPath:
+ type: "string"
+ type: "object"
+ mode:
+ type: "integer"
+ path:
+ type: "string"
+ resourceFieldRef:
+ properties:
+ containerName:
+ type: "string"
+ divisor:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ resource:
+ type: "string"
+ type: "object"
+ type: "object"
+ type: "array"
+ type: "object"
+ emptyDir:
+ properties:
+ medium:
+ type: "string"
+ sizeLimit:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ ephemeral:
+ properties:
+ volumeClaimTemplate:
+ properties:
+ metadata:
+ properties:
+ annotations:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ creationTimestamp:
+ type: "string"
+ deletionGracePeriodSeconds:
+ type: "integer"
+ deletionTimestamp:
+ type: "string"
+ finalizers:
+ items:
+ type: "string"
+ type: "array"
+ generateName:
+ type: "string"
+ generation:
+ type: "integer"
+ labels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ managedFields:
+ items:
+ properties:
+ apiVersion:
+ type: "string"
+ fieldsType:
+ type: "string"
+ fieldsV1:
+ type: "object"
+ manager:
+ type: "string"
+ operation:
+ type: "string"
+ subresource:
+ type: "string"
+ time:
+ type: "string"
+ type: "object"
+ type: "array"
+ name:
+ type: "string"
+ namespace:
+ type: "string"
+ ownerReferences:
+ items:
+ properties:
+ apiVersion:
+ type: "string"
+ blockOwnerDeletion:
+ type: "boolean"
+ controller:
+ type: "boolean"
+ kind:
+ type: "string"
+ name:
+ type: "string"
+ uid:
+ type: "string"
+ type: "object"
+ type: "array"
+ resourceVersion:
+ type: "string"
+ selfLink:
+ type: "string"
+ uid:
+ type: "string"
+ type: "object"
+ spec:
+ properties:
+ accessModes:
+ items:
+ type: "string"
+ type: "array"
+ dataSource:
+ properties:
+ apiGroup:
+ type: "string"
+ kind:
+ type: "string"
+ name:
+ type: "string"
+ type: "object"
+ dataSourceRef:
+ properties:
+ apiGroup:
+ type: "string"
+ kind:
+ type: "string"
+ name:
+ type: "string"
+ namespace:
+ type: "string"
+ type: "object"
+ resources:
+ properties:
+ limits:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ requests:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ selector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ storageClassName:
+ type: "string"
+ volumeAttributesClassName:
+ type: "string"
+ volumeMode:
+ type: "string"
+ volumeName:
+ type: "string"
+ type: "object"
+ type: "object"
+ type: "object"
+ fc:
+ properties:
+ fsType:
+ type: "string"
+ lun:
+ type: "integer"
+ readOnly:
+ type: "boolean"
+ targetWWNs:
+ items:
+ type: "string"
+ type: "array"
+ wwids:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ flexVolume:
+ properties:
+ driver:
+ type: "string"
+ fsType:
+ type: "string"
+ options:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ readOnly:
+ type: "boolean"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ type: "object"
+ flocker:
+ properties:
+ datasetName:
+ type: "string"
+ datasetUUID:
+ type: "string"
+ type: "object"
+ gcePersistentDisk:
+ properties:
+ fsType:
+ type: "string"
+ partition:
+ type: "integer"
+ pdName:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ type: "object"
+ gitRepo:
+ properties:
+ directory:
+ type: "string"
+ repository:
+ type: "string"
+ revision:
+ type: "string"
+ type: "object"
+ glusterfs:
+ properties:
+ endpoints:
+ type: "string"
+ path:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ type: "object"
+ hostPath:
+ properties:
+ path:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ image:
+ properties:
+ pullPolicy:
+ type: "string"
+ reference:
+ type: "string"
+ type: "object"
+ iscsi:
+ properties:
+ chapAuthDiscovery:
+ type: "boolean"
+ chapAuthSession:
+ type: "boolean"
+ fsType:
+ type: "string"
+ initiatorName:
+ type: "string"
+ iqn:
+ type: "string"
+ iscsiInterface:
+ type: "string"
+ lun:
+ type: "integer"
+ portals:
+ items:
+ type: "string"
+ type: "array"
+ readOnly:
+ type: "boolean"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ targetPortal:
+ type: "string"
+ type: "object"
+ name:
+ type: "string"
+ nfs:
+ properties:
+ path:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ server:
+ type: "string"
+ type: "object"
+ persistentVolumeClaim:
+ properties:
+ claimName:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ type: "object"
+ photonPersistentDisk:
+ properties:
+ fsType:
+ type: "string"
+ pdID:
+ type: "string"
+ type: "object"
+ portworxVolume:
+ properties:
+ fsType:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ volumeID:
+ type: "string"
+ type: "object"
+ projected:
+ properties:
+ defaultMode:
+ type: "integer"
+ sources:
+ items:
+ properties:
+ clusterTrustBundle:
+ properties:
+ labelSelector:
+ properties:
+ matchExpressions:
+ items:
+ properties:
+ key:
+ type: "string"
+ operator:
+ type: "string"
+ values:
+ items:
+ type: "string"
+ type: "array"
+ type: "object"
+ type: "array"
+ matchLabels:
+ additionalProperties:
+ type: "string"
+ type: "object"
+ type: "object"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ path:
+ type: "string"
+ signerName:
+ type: "string"
+ type: "object"
+ configMap:
+ properties:
+ items:
+ items:
+ properties:
+ key:
+ type: "string"
+ mode:
+ type: "integer"
+ path:
+ type: "string"
+ type: "object"
+ type: "array"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ downwardAPI:
+ properties:
+ items:
+ items:
+ properties:
+ fieldRef:
+ properties:
+ apiVersion:
+ type: "string"
+ fieldPath:
+ type: "string"
+ type: "object"
+ mode:
+ type: "integer"
+ path:
+ type: "string"
+ resourceFieldRef:
+ properties:
+ containerName:
+ type: "string"
+ divisor:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ resource:
+ type: "string"
+ type: "object"
+ type: "object"
+ type: "array"
+ type: "object"
+ secret:
+ properties:
+ items:
+ items:
+ properties:
+ key:
+ type: "string"
+ mode:
+ type: "integer"
+ path:
+ type: "string"
+ type: "object"
+ type: "array"
+ name:
+ type: "string"
+ optional:
+ type: "boolean"
+ type: "object"
+ serviceAccountToken:
+ properties:
+ audience:
+ type: "string"
+ expirationSeconds:
+ type: "integer"
+ path:
+ type: "string"
+ type: "object"
+ type: "object"
+ type: "array"
+ type: "object"
+ quobyte:
+ properties:
+ group:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ registry:
+ type: "string"
+ tenant:
+ type: "string"
+ user:
+ type: "string"
+ volume:
+ type: "string"
+ type: "object"
+ rbd:
+ properties:
+ fsType:
+ type: "string"
+ image:
+ type: "string"
+ keyring:
+ type: "string"
+ monitors:
+ items:
+ type: "string"
+ type: "array"
+ pool:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ user:
+ type: "string"
+ type: "object"
+ scaleIO:
+ properties:
+ fsType:
+ type: "string"
+ gateway:
+ type: "string"
+ protectionDomain:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ sslEnabled:
+ type: "boolean"
+ storageMode:
+ type: "string"
+ storagePool:
+ type: "string"
+ system:
+ type: "string"
+ volumeName:
+ type: "string"
+ type: "object"
+ secret:
+ properties:
+ defaultMode:
+ type: "integer"
+ items:
+ items:
+ properties:
+ key:
+ type: "string"
+ mode:
+ type: "integer"
+ path:
+ type: "string"
+ type: "object"
+ type: "array"
+ optional:
+ type: "boolean"
+ secretName:
+ type: "string"
+ type: "object"
+ storageos:
+ properties:
+ fsType:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ secretRef:
+ properties:
+ name:
+ type: "string"
+ type: "object"
+ volumeName:
+ type: "string"
+ volumeNamespace:
+ type: "string"
+ type: "object"
+ vsphereVolume:
+ properties:
+ fsType:
+ type: "string"
+ storagePolicyID:
+ type: "string"
+ storagePolicyName:
+ type: "string"
+ volumePath:
+ type: "string"
+ type: "object"
+ type: "object"
+ type: "array"
+ type: "object"
+ status:
+ properties:
+ conditions:
+ items:
+ properties:
+ lastProbeTime:
+ type: "string"
+ lastTransitionTime:
+ type: "string"
+ message:
+ type: "string"
+ observedGeneration:
+ type: "integer"
+ reason:
+ type: "string"
+ status:
+ type: "string"
+ type:
+ type: "string"
+ type: "object"
+ type: "array"
+ containerStatuses:
+ items:
+ properties:
+ allocatedResources:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ allocatedResourcesStatus:
+ items:
+ properties:
+ name:
+ type: "string"
+ resources:
+ items:
+ properties:
+ health:
+ type: "string"
+ resourceID:
+ type: "string"
+ type: "object"
+ type: "array"
+ type: "object"
+ type: "array"
+ containerID:
+ type: "string"
+ image:
+ type: "string"
+ imageID:
+ type: "string"
+ lastState:
+ properties:
+ running:
+ properties:
+ startedAt:
+ type: "string"
+ type: "object"
+ terminated:
+ properties:
+ containerID:
+ type: "string"
+ exitCode:
+ type: "integer"
+ finishedAt:
+ type: "string"
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ signal:
+ type: "integer"
+ startedAt:
+ type: "string"
+ type: "object"
+ waiting:
+ properties:
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ type: "object"
+ type: "object"
+ name:
+ type: "string"
+ ready:
+ type: "boolean"
+ resources:
+ properties:
+ claims:
+ items:
+ properties:
+ name:
+ type: "string"
+ request:
+ type: "string"
+ type: "object"
+ type: "array"
+ limits:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ requests:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ restartCount:
+ type: "integer"
+ started:
+ type: "boolean"
+ state:
+ properties:
+ running:
+ properties:
+ startedAt:
+ type: "string"
+ type: "object"
+ terminated:
+ properties:
+ containerID:
+ type: "string"
+ exitCode:
+ type: "integer"
+ finishedAt:
+ type: "string"
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ signal:
+ type: "integer"
+ startedAt:
+ type: "string"
+ type: "object"
+ waiting:
+ properties:
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ type: "object"
+ type: "object"
+ stopSignal:
+ type: "string"
+ user:
+ properties:
+ linux:
+ properties:
+ gid:
+ type: "integer"
+ supplementalGroups:
+ items:
+ type: "integer"
+ type: "array"
+ uid:
+ type: "integer"
+ type: "object"
+ type: "object"
+ volumeMounts:
+ items:
+ properties:
+ mountPath:
+ type: "string"
+ name:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ recursiveReadOnly:
+ type: "string"
+ type: "object"
+ type: "array"
+ type: "object"
+ type: "array"
+ ephemeralContainerStatuses:
+ items:
+ properties:
+ allocatedResources:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ allocatedResourcesStatus:
+ items:
+ properties:
+ name:
+ type: "string"
+ resources:
+ items:
+ properties:
+ health:
+ type: "string"
+ resourceID:
+ type: "string"
+ type: "object"
+ type: "array"
+ type: "object"
+ type: "array"
+ containerID:
+ type: "string"
+ image:
+ type: "string"
+ imageID:
+ type: "string"
+ lastState:
+ properties:
+ running:
+ properties:
+ startedAt:
+ type: "string"
+ type: "object"
+ terminated:
+ properties:
+ containerID:
+ type: "string"
+ exitCode:
+ type: "integer"
+ finishedAt:
+ type: "string"
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ signal:
+ type: "integer"
+ startedAt:
+ type: "string"
+ type: "object"
+ waiting:
+ properties:
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ type: "object"
+ type: "object"
+ name:
+ type: "string"
+ ready:
+ type: "boolean"
+ resources:
+ properties:
+ claims:
+ items:
+ properties:
+ name:
+ type: "string"
+ request:
+ type: "string"
+ type: "object"
+ type: "array"
+ limits:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ requests:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ restartCount:
+ type: "integer"
+ started:
+ type: "boolean"
+ state:
+ properties:
+ running:
+ properties:
+ startedAt:
+ type: "string"
+ type: "object"
+ terminated:
+ properties:
+ containerID:
+ type: "string"
+ exitCode:
+ type: "integer"
+ finishedAt:
+ type: "string"
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ signal:
+ type: "integer"
+ startedAt:
+ type: "string"
+ type: "object"
+ waiting:
+ properties:
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ type: "object"
+ type: "object"
+ stopSignal:
+ type: "string"
+ user:
+ properties:
+ linux:
+ properties:
+ gid:
+ type: "integer"
+ supplementalGroups:
+ items:
+ type: "integer"
+ type: "array"
+ uid:
+ type: "integer"
+ type: "object"
+ type: "object"
+ volumeMounts:
+ items:
+ properties:
+ mountPath:
+ type: "string"
+ name:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ recursiveReadOnly:
+ type: "string"
+ type: "object"
+ type: "array"
+ type: "object"
+ type: "array"
+ hostIP:
+ type: "string"
+ hostIPs:
+ items:
+ properties:
+ ip:
+ type: "string"
+ type: "object"
+ type: "array"
+ initContainerStatuses:
+ items:
+ properties:
+ allocatedResources:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ allocatedResourcesStatus:
+ items:
+ properties:
+ name:
+ type: "string"
+ resources:
+ items:
+ properties:
+ health:
+ type: "string"
+ resourceID:
+ type: "string"
+ type: "object"
+ type: "array"
+ type: "object"
+ type: "array"
+ containerID:
+ type: "string"
+ image:
+ type: "string"
+ imageID:
+ type: "string"
+ lastState:
+ properties:
+ running:
+ properties:
+ startedAt:
+ type: "string"
+ type: "object"
+ terminated:
+ properties:
+ containerID:
+ type: "string"
+ exitCode:
+ type: "integer"
+ finishedAt:
+ type: "string"
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ signal:
+ type: "integer"
+ startedAt:
+ type: "string"
+ type: "object"
+ waiting:
+ properties:
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ type: "object"
+ type: "object"
+ name:
+ type: "string"
+ ready:
+ type: "boolean"
+ resources:
+ properties:
+ claims:
+ items:
+ properties:
+ name:
+ type: "string"
+ request:
+ type: "string"
+ type: "object"
+ type: "array"
+ limits:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ requests:
+ additionalProperties:
+ anyOf:
+ - type: "integer"
+ - type: "string"
+ x-kubernetes-int-or-string: true
+ type: "object"
+ type: "object"
+ restartCount:
+ type: "integer"
+ started:
+ type: "boolean"
+ state:
+ properties:
+ running:
+ properties:
+ startedAt:
+ type: "string"
+ type: "object"
+ terminated:
+ properties:
+ containerID:
+ type: "string"
+ exitCode:
+ type: "integer"
+ finishedAt:
+ type: "string"
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ signal:
+ type: "integer"
+ startedAt:
+ type: "string"
+ type: "object"
+ waiting:
+ properties:
+ message:
+ type: "string"
+ reason:
+ type: "string"
+ type: "object"
+ type: "object"
+ stopSignal:
+ type: "string"
+ user:
+ properties:
+ linux:
+ properties:
+ gid:
+ type: "integer"
+ supplementalGroups:
+ items:
+ type: "integer"
+ type: "array"
+ uid:
+ type: "integer"
+ type: "object"
+ type: "object"
+ volumeMounts:
+ items:
+ properties:
+ mountPath:
+ type: "string"
+ name:
+ type: "string"
+ readOnly:
+ type: "boolean"
+ recursiveReadOnly:
+ type: "string"
+ type: "object"
+ type: "array"
+ type: "object"
+ type: "array"
+ message:
+ type: "string"
+ nominatedNodeName:
+ type: "string"
+ observedGeneration:
+ type: "integer"
+ phase:
+ type: "string"
+ podIP:
+ type: "string"
+ podIPs:
+ items:
+ properties:
+ ip:
+ type: "string"
+ type: "object"
+ type: "array"
+ qosClass:
+ type: "string"
+ reason:
+ type: "string"
+ resize:
+ type: "string"
+ resourceClaimStatuses:
+ items:
+ properties:
+ name:
+ type: "string"
+ resourceClaimName:
+ type: "string"
+ type: "object"
+ type: "array"
+ startTime:
+ type: "string"
+ type: "object"
+ type: "object"
+ replicas:
+ type: "integer"
+ resource:
+ properties:
+ cpu:
+ type: "number"
+ ephemeralStorage:
+ type: "string"
+ memory:
+ type: "string"
+ type: "object"
+ type: "object"
+ type: "object"
+ type: "object"
+ type: "object"
+ status:
+ properties:
+ abortTimestamp:
+ type: "string"
+ blueGreenState:
+ enum:
+ - "ACTIVE_BLUE"
+ - "ACTIVE_GREEN"
+ - "INITIALIZING_BLUE"
+ - "SAVEPOINTING_BLUE"
+ - "SAVEPOINTING_GREEN"
+ - "TRANSITIONING_TO_BLUE"
+ - "TRANSITIONING_TO_GREEN"
+ type: "string"
+ deploymentReadyTimestamp:
+ type: "string"
+ error:
+ type: "string"
+ jobStatus:
+ properties:
+ checkpointInfo:
+ properties:
+ formatType:
+ enum:
+ - "FULL"
+ - "INCREMENTAL"
+ - "UNKNOWN"
+ type: "string"
+ lastCheckpoint:
+ properties:
+ formatType:
+ enum:
+ - "FULL"
+ - "INCREMENTAL"
+ - "UNKNOWN"
+ type: "string"
+ timeStamp:
+ type: "integer"
+ triggerNonce:
+ type: "integer"
+ triggerType:
+ enum:
+ - "MANUAL"
+ - "PERIODIC"
+ - "UNKNOWN"
+ - "UPGRADE"
+ type: "string"
+ type: "object"
+ lastPeriodicCheckpointTimestamp:
+ type: "integer"
+ triggerId:
+ type: "string"
+ triggerTimestamp:
+ type: "integer"
+ triggerType:
+ enum:
+ - "MANUAL"
+ - "PERIODIC"
+ - "UNKNOWN"
+ - "UPGRADE"
+ type: "string"
+ type: "object"
+ jobId:
+ type: "string"
+ jobName:
+ type: "string"
+ savepointInfo:
+ properties:
+ formatType:
+ enum:
+ - "CANONICAL"
+ - "NATIVE"
+ - "UNKNOWN"
+ type: "string"
+ lastPeriodicSavepointTimestamp:
+ type: "integer"
+ lastSavepoint:
+ properties:
+ formatType:
+ enum:
+ - "CANONICAL"
+ - "NATIVE"
+ - "UNKNOWN"
+ type: "string"
+ location:
+ type: "string"
+ timeStamp:
+ type: "integer"
+ triggerNonce:
+ type: "integer"
+ triggerType:
+ enum:
+ - "MANUAL"
+ - "PERIODIC"
+ - "UNKNOWN"
+ - "UPGRADE"
+ type: "string"
+ type: "object"
+ savepointHistory:
+ items:
+ properties:
+ formatType:
+ enum:
+ - "CANONICAL"
+ - "NATIVE"
+ - "UNKNOWN"
+ type: "string"
+ location:
+ type: "string"
+ timeStamp:
+ type: "integer"
+ triggerNonce:
+ type: "integer"
+ triggerType:
+ enum:
+ - "MANUAL"
+ - "PERIODIC"
+ - "UNKNOWN"
+ - "UPGRADE"
+ type: "string"
+ type: "object"
+ type: "array"
+ triggerId:
+ type: "string"
+ triggerTimestamp:
+ type: "integer"
+ triggerType:
+ enum:
+ - "MANUAL"
+ - "PERIODIC"
+ - "UNKNOWN"
+ - "UPGRADE"
+ type: "string"
+ type: "object"
+ startTime:
+ type: "string"
+ state:
+ enum:
+ - "CANCELED"
+ - "CANCELLING"
+ - "CREATED"
+ - "FAILED"
+ - "FAILING"
+ - "FINISHED"
+ - "INITIALIZING"
+ - "RECONCILING"
+ - "RESTARTING"
+ - "RUNNING"
+ - "SUSPENDED"
+ type: "string"
+ updateTime:
+ type: "string"
+ upgradeSavepointPath:
+ type: "string"
+ type: "object"
+ lastReconciledSpec:
+ type: "string"
+ lastReconciledTimestamp:
+ type: "string"
+ savepointTriggerId:
+ type: "string"
+ type: "object"
+ type: "object"
+ served: true
+ storage: true
+ subresources:
+ status: {}
diff --git a/helm/flink-kubernetes-operator/templates/rbac/_helpers.tpl b/helm/flink-kubernetes-operator/templates/rbac/_helpers.tpl
index 44d1d7ef3a..3d398f35e2 100644
--- a/helm/flink-kubernetes-operator/templates/rbac/_helpers.tpl
+++ b/helm/flink-kubernetes-operator/templates/rbac/_helpers.tpl
@@ -128,6 +128,8 @@ rules:
- apiGroups:
- flink.apache.org
resources:
+ - flinkbluegreendeployments
+ - flinkbluegreendeployments/finalizers
- flinkdeployments
- flinkdeployments/finalizers
- flinksessionjobs
@@ -145,6 +147,7 @@ rules:
- apiGroups:
- flink.apache.org
resources:
+ - flinkbluegreendeployments/status
- flinkdeployments/status
- flinksessionjobs/status
- flinkstatesnapshots/status
diff --git a/pom.xml b/pom.xml
index 506a2efe96..11c071fa04 100644
--- a/pom.xml
+++ b/pom.xml
@@ -411,6 +411,7 @@ under the License.
**/.idea/**
+ helm/flink-kubernetes-operator/crds/flinkbluegreendeployments.flink.apache.org-v1.yml
helm/flink-kubernetes-operator/crds/flinkdeployments.flink.apache.org-v1.yml
helm/flink-kubernetes-operator/crds/flinksessionjobs.flink.apache.org-v1.yml
helm/flink-kubernetes-operator/crds/flinkstatesnapshots.flink.apache.org-v1.yml