diff --git a/docs/content/docs/custom-resource/reference.md b/docs/content/docs/custom-resource/reference.md index 909343c103..19cad011e7 100644 --- a/docs/content/docs/custom-resource/reference.md +++ b/docs/content/docs/custom-resource/reference.md @@ -57,6 +57,23 @@ This serves as a full reference for FlinkDeployment and FlinkSessionJob custom r | Parameter | Type | Docs | | ----------| ---- | ---- | +### FlinkBlueGreenDeploymentConfigOptions +**Class**: org.apache.flink.kubernetes.operator.api.spec.FlinkBlueGreenDeploymentConfigOptions + +**Description**: Configuration options to be used by the Flink Blue/Green Deployments. + +| Parameter | Type | Docs | +| ----------| ---- | ---- | + +### FlinkBlueGreenDeploymentSpec +**Class**: org.apache.flink.kubernetes.operator.api.spec.FlinkBlueGreenDeploymentSpec + +**Description**: Spec that describes a Flink application with blue/green deployment capabilities. + +| Parameter | Type | Docs | +| ----------| ---- | ---- | +| template | org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentTemplateSpec | | + ### FlinkDeploymentSpec **Class**: org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec @@ -78,6 +95,17 @@ This serves as a full reference for FlinkDeployment and FlinkSessionJob custom r | logConfiguration | java.util.Map | Log configuration overrides for the Flink deployment. Format logConfigFileName -> configContent. | | mode | org.apache.flink.kubernetes.operator.api.spec.KubernetesDeploymentMode | Deployment mode of the Flink cluster, native or standalone. | +### FlinkDeploymentTemplateSpec +**Class**: org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentTemplateSpec + +**Description**: Template Spec that describes a Flink application managed by the blue/green controller. + +| Parameter | Type | Docs | +| ----------| ---- | ---- | +| metadata | io.fabric8.kubernetes.api.model.ObjectMeta | | +| configuration | java.util.Map | | +| spec | org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec | | + ### FlinkSessionJobSpec **Class**: org.apache.flink.kubernetes.operator.api.spec.FlinkSessionJobSpec @@ -290,6 +318,33 @@ This serves as a full reference for FlinkDeployment and FlinkSessionJob custom r | UNKNOWN | Checkpoint format unknown, if the checkpoint was not triggered by the operator. | | description | org.apache.flink.configuration.description.InlineElement | | +### FlinkBlueGreenDeploymentState +**Class**: org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentState + +**Description**: Enumeration of the possible states of the blue/green transition. + +| Value | Docs | +| ----- | ---- | +| INITIALIZING_BLUE | We use this state while initializing for the first time, always with a "Blue" deployment type. | +| ACTIVE_BLUE | Identifies the system is running normally with a "Blue" deployment type. | +| ACTIVE_GREEN | Identifies the system is running normally with a "Green" deployment type. | +| TRANSITIONING_TO_BLUE | Identifies the system is transitioning from "Green" to "Blue". | +| TRANSITIONING_TO_GREEN | Identifies the system is transitioning from "Blue" to "Green". | + +### FlinkBlueGreenDeploymentStatus +**Class**: org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentStatus + +**Description**: Last observed status of the Flink Blue/Green deployment. + +| Parameter | Type | Docs | +| ----------| ---- | ---- | +| jobStatus | org.apache.flink.kubernetes.operator.api.status.JobStatus | | +| blueGreenState | org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentState | The state of the blue/green transition. | +| lastReconciledSpec | java.lang.String | Last reconciled (serialized) deployment spec. | +| lastReconciledTimestamp | java.lang.String | Timestamp of last reconciliation. | +| abortTimestamp | java.lang.String | Computed from abortGracePeriodMs, timestamp after which the deployment should be aborted. | +| deploymentReadyTimestamp | java.lang.String | Timestamp when the deployment became READY/STABLE. Used to determine when to delete it. | + ### FlinkDeploymentReconciliationStatus **Class**: org.apache.flink.kubernetes.operator.api.status.FlinkDeploymentReconciliationStatus diff --git a/e2e-tests/data/bluegreen.yaml b/e2e-tests/data/bluegreen.yaml new file mode 100644 index 0000000000..52391827c1 --- /dev/null +++ b/e2e-tests/data/bluegreen.yaml @@ -0,0 +1,62 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +apiVersion: flink.apache.org/v1beta1 +kind: FlinkBlueGreenDeployment +metadata: + name: basic-bluegreen-example +spec: + template: + configuration: + kubernetes.operator.bluegreen.deployment-deletion.delay: "1s" + spec: + image: flink:1.20 + flinkVersion: v1_20 + flinkConfiguration: + rest.port: "8081" + execution.checkpointing.interval: "1 min" + execution.checkpointing.storage: "filesystem" + state.backend.incremental: "true" + state.checkpoints.dir: "file:///flink-data/checkpoints" + state.checkpoints.num-retained: "50" + taskmanager.numberOfTaskSlots: "1" + serviceAccount: flink + jobManager: + resource: + memory: "2048m" + cpu: 1 + podTemplate: + spec: + containers: + - name: flink-main-container + volumeMounts: + - mountPath: /flink-data + name: flink-volume + volumes: + - name: flink-volume + hostPath: + path: /tmp/flink + type: Directory + taskManager: + resource: + memory: "2048m" + cpu: 1 + job: + jarURI: local:///opt/flink/examples/streaming/StateMachineExample.jar + parallelism: 1 + upgradeMode: stateless diff --git a/e2e-tests/test_bluegreen.sh b/e2e-tests/test_bluegreen.sh new file mode 100644 index 0000000000..192531e46b --- /dev/null +++ b/e2e-tests/test_bluegreen.sh @@ -0,0 +1,74 @@ +#!/usr/bin/env bash +################################################################################ +# 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. +################################################################################ + +# This script tests the Flink Blue/Green Deployments support as follows: +# - Create a FlinkBlueGreenDeployment which automatically starts a "Blue" FlinkDeployment +# - Once this setup is stable, we trigger a transition which will create the "Green" FlinkDeployment +# - Once it's stable, verify the "Blue" FlinkDeployment is torn down +# - Perform additional validation(s) before exiting + +SCRIPT_DIR=$(dirname "$(readlink -f "$0")") +source "${SCRIPT_DIR}/utils.sh" + +CLUSTER_ID="basic-bluegreen-example" +BG_CLUSTER_ID=$CLUSTER_ID +BLUE_CLUSTER_ID="basic-bluegreen-example-blue" +GREEN_CLUSTER_ID="basic-bluegreen-example-green" + +APPLICATION_YAML="${SCRIPT_DIR}/data/bluegreen.yaml" +APPLICATION_IDENTIFIER="flinkbgdep/$CLUSTER_ID" +BLUE_APPLICATION_IDENTIFIER="flinkdep/$BLUE_CLUSTER_ID" +GREEN_APPLICATION_IDENTIFIER="flinkdep/$GREEN_CLUSTER_ID" +TIMEOUT=300 + +#echo "BG_CLUSTER_ID " $BG_CLUSTER_ID +#echo "BLUE_CLUSTER_ID " $BLUE_CLUSTER_ID +#echo "APPLICATION_IDENTIFIER " $APPLICATION_IDENTIFIER +#echo "BLUE_APPLICATION_IDENTIFIER " $BLUE_APPLICATION_IDENTIFIER + +on_exit cleanup_and_exit "$APPLICATION_YAML" $TIMEOUT $BG_CLUSTER_ID + +retry_times 5 30 "kubectl apply -f $APPLICATION_YAML" || exit 1 + +wait_for_jobmanager_running $BLUE_CLUSTER_ID $TIMEOUT +wait_for_status $BLUE_APPLICATION_IDENTIFIER '.status.lifecycleState' STABLE ${TIMEOUT} || exit 1 +wait_for_status $APPLICATION_IDENTIFIER '.status.jobStatus.state' RUNNING ${TIMEOUT} || exit 1 +wait_for_status $APPLICATION_IDENTIFIER '.status.blueGreenState' ACTIVE_BLUE ${TIMEOUT} || exit 1 + +blue_job_id=$(kubectl get -oyaml flinkdep/basic-bluegreen-example-blue | yq '.status.jobStatus.jobId') + +echo "Giving a chance for checkpoints to be generated..." +sleep 5 +kubectl patch flinkbgdep ${BG_CLUSTER_ID} --type merge --patch '{"spec":{"template":{"spec":{"flinkConfiguration":{"rest.port":"8082","state.checkpoints.num-retained":"51"}}}}}' + +wait_for_status $GREEN_APPLICATION_IDENTIFIER '.status.lifecycleState' STABLE ${TIMEOUT} || exit 1 +kubectl wait --for=delete deployment --timeout=${TIMEOUT}s --selector="app=${BLUE_CLUSTER_ID}" +wait_for_status $APPLICATION_IDENTIFIER '.status.jobStatus.state' RUNNING ${TIMEOUT} || exit 1 +wait_for_status $APPLICATION_IDENTIFIER '.status.blueGreenState' ACTIVE_GREEN ${TIMEOUT} || exit 1 + +green_initialSavepointPath=$(kubectl get -oyaml $GREEN_APPLICATION_IDENTIFIER | yq '.spec.job.initialSavepointPath') + +if [[ $green_initialSavepointPath == '/flink-data/checkpoints/'$blue_job_id* ]]; then + echo 'Green deployment started from the expected initialSavepointPath: ' $green_initialSavepointPath +else + echo 'Unexpected initialSavepointPath: ' $green_initialSavepointPath + exit 1 +fi; + +echo "Successfully run the Flink Blue/Green Deployments test" \ No newline at end of file diff --git a/flink-kubernetes-operator-api/pom.xml b/flink-kubernetes-operator-api/pom.xml index 71b59abcc7..1601eddbec 100644 --- a/flink-kubernetes-operator-api/pom.xml +++ b/flink-kubernetes-operator-api/pom.xml @@ -212,6 +212,7 @@ under the License. flinkdeployments.flink.apache.org-v1.yml flinksessionjobs.flink.apache.org-v1.yml flinkstatesnapshots.flink.apache.org-v1.yml + flinkbluegreendeployments.flink.apache.org-v1.yml false @@ -236,6 +237,8 @@ under the License. + + @@ -253,6 +256,24 @@ under the License. + + + + + + + + flinkbgdeployments-remove-scale-subresource + package + + run + + + + + + diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/FlinkBlueGreenDeployment.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/FlinkBlueGreenDeployment.java new file mode 100644 index 0000000000..61451243f0 --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/FlinkBlueGreenDeployment.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.kubernetes.operator.api; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.kubernetes.operator.api.spec.FlinkBlueGreenDeploymentSpec; +import org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentStatus; + +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import io.fabric8.kubernetes.api.model.Namespaced; +import io.fabric8.kubernetes.client.CustomResource; +import io.fabric8.kubernetes.model.annotation.Group; +import io.fabric8.kubernetes.model.annotation.ShortNames; +import io.fabric8.kubernetes.model.annotation.Version; + +/** Custom resource definition that represents a deployments with Blue/Green rollout capability. */ +@Experimental +@JsonInclude(JsonInclude.Include.NON_NULL) +@JsonDeserialize() +@Group(CrdConstants.API_GROUP) +@Version(CrdConstants.API_VERSION) +@ShortNames({"flinkbgdep"}) +public class FlinkBlueGreenDeployment + extends CustomResource + implements Namespaced {} diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/BlueGreenDiffType.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/BlueGreenDiffType.java new file mode 100644 index 0000000000..81fe89b6b1 --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/BlueGreenDiffType.java @@ -0,0 +1,38 @@ +/* + * 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 found in Blue/Green deployment specifications. + */ +public enum BlueGreenDiffType { + /** No differences found between specifications. */ + IGNORE, + + /** Only top-level properties (metadata, configuration) have differences. */ + PATCH_TOP_LEVEL, + + /** Only the nested FlinkDeploymentSpec has differences. */ + PATCH_CHILD, + + /** Both top-level and nested specifications have differences. */ + PATCH_BOTH, + + /** The nested spec has SCALE or UPGRADE differences, requiring a blue/green transition. */ + TRANSITION +} diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/DeploymentType.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/DeploymentType.java new file mode 100644 index 0000000000..b4826b4d8d --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/DeploymentType.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.kubernetes.operator.api.bluegreen; + +import org.apache.flink.kubernetes.operator.api.FlinkDeployment; + +/** + * Enumeration of the two possible Flink Blue/Green deployment types. Only one of each type will be + * present at all times for a particular job. + */ +public enum DeploymentType { + /** Identifier for the first or "Blue" deployment type. */ + BLUE, + + /** Identifier for the second or "Green" deployment type. */ + GREEN; + + public static final String LABEL_KEY = "flink/blue-green-deployment-type"; + + public static DeploymentType fromDeployment(FlinkDeployment flinkDeployment) { + String typeAnnotation = flinkDeployment.getMetadata().getLabels().get(LABEL_KEY); + return DeploymentType.valueOf(typeAnnotation); + } +} diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/FlinkBlueGreenDeploymentConfigOptions.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/FlinkBlueGreenDeploymentConfigOptions.java new file mode 100644 index 0000000000..ae574ed727 --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/FlinkBlueGreenDeploymentConfigOptions.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.kubernetes.operator.api.spec; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +import java.time.Duration; + +/** Configuration options to be used by the Flink Blue/Green Deployments. */ +public class FlinkBlueGreenDeploymentConfigOptions { + + public static final String K8S_OP_CONF_PREFIX = "kubernetes.operator."; + + public static final String BLUE_GREEN_CONF_PREFIX = K8S_OP_CONF_PREFIX + "bluegreen."; + + public static ConfigOptions.OptionBuilder operatorConfig(String key) { + return ConfigOptions.key(BLUE_GREEN_CONF_PREFIX + key); + } + + /** + * NOTE: The string durations need to be in format "{length value}{time unit label}", e.g. + * "123ms", "321 s". If no time unit label is specified, it will be considered as milliseconds. + * There is no fall back to parse ISO-8601 duration format, until Flink 2.x + * + *

Supported time unit labels are: + * + *

    + *
  • DAYS: "d", "day" + *
  • HOURS: "h", "hour" + *
  • MINUTES: "m", "min", "minute" + *
  • SECONDS: "s", "sec", "second" + *
  • MILLISECONDS: "ms", "milli", "millisecond" + *
  • MICROSECONDS: "µs", "micro", "microsecond" + *
  • NANOSECONDS: "ns", "nano", "nanosecond" + *
+ */ + public static final ConfigOption ABORT_GRACE_PERIOD = + operatorConfig("abort.grace-period") + .durationType() + .defaultValue(Duration.ofMinutes(10)) + .withDescription( + "The max time to wait in milliseconds for a deployment to become ready before aborting it. Cannot be smaller than 10 minutes."); + + public static final ConfigOption RECONCILIATION_RESCHEDULING_INTERVAL = + operatorConfig("reconciliation.reschedule-interval") + .durationType() + .defaultValue(Duration.ofSeconds(15)) + .withDescription( + "Configurable delay in milliseconds to use when the operator reschedules a reconciliation."); + + public static final ConfigOption DEPLOYMENT_DELETION_DELAY = + operatorConfig("deployment-deletion.delay") + .durationType() + .defaultValue(Duration.ofMillis(0)) + .withDescription( + "Configurable delay in milliseconds before deleting a deployment after being marked done."); +} diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/FlinkBlueGreenDeploymentSpec.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/FlinkBlueGreenDeploymentSpec.java new file mode 100644 index 0000000000..bfaf99b40e --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/FlinkBlueGreenDeploymentSpec.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.kubernetes.operator.api.spec; + +import org.apache.flink.annotation.Experimental; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; + +/** Spec that describes a Flink application with blue/green deployment capabilities. */ +@Experimental +@Data +@NoArgsConstructor +@AllArgsConstructor +@JsonIgnoreProperties(ignoreUnknown = true) +public class FlinkBlueGreenDeploymentSpec { + + private FlinkDeploymentTemplateSpec template; +} diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/FlinkDeploymentTemplateSpec.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/FlinkDeploymentTemplateSpec.java new file mode 100644 index 0000000000..2c2b49970a --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/FlinkDeploymentTemplateSpec.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.kubernetes.operator.api.spec; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.fabric8.kubernetes.api.model.ObjectMeta; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; +import lombok.experimental.SuperBuilder; + +import java.util.LinkedHashMap; +import java.util.Map; + +/** Template Spec that describes a Flink application managed by the blue/green controller. */ +@AllArgsConstructor +@NoArgsConstructor +@Data +@SuperBuilder +public class FlinkDeploymentTemplateSpec { + + @JsonProperty("metadata") + private ObjectMeta metadata; + + @JsonProperty("configuration") + private Map configuration; + + @JsonProperty("spec") + private FlinkDeploymentSpec spec; + + @JsonIgnore + private Map additionalProperties = new LinkedHashMap(); +} diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/status/FlinkBlueGreenDeploymentState.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/status/FlinkBlueGreenDeploymentState.java new file mode 100644 index 0000000000..a8a8ba1d64 --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/status/FlinkBlueGreenDeploymentState.java @@ -0,0 +1,46 @@ +/* + * 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.status; + +/** Enumeration of the possible states of the blue/green transition. */ +public enum FlinkBlueGreenDeploymentState { + + /** + * We use this state while initializing for the first time, always with a "Blue" deployment + * type. + */ + INITIALIZING_BLUE, + + /** Identifies the system is running normally with a "Blue" deployment type. */ + ACTIVE_BLUE, + + /** Identifies the system is running normally with a "Green" deployment type. */ + ACTIVE_GREEN, + + /** Identifies the system is transitioning from "Green" to "Blue". */ + TRANSITIONING_TO_BLUE, + + /** Identifies the system is transitioning from "Blue" to "Green". */ + TRANSITIONING_TO_GREEN, + + /** Identifies the system is savepointing "Blue" before it transitions to "Green". */ + SAVEPOINTING_BLUE, + + /** Identifies the system is savepointing "Green" before it transitions to "Blue". */ + SAVEPOINTING_GREEN, +} diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/status/FlinkBlueGreenDeploymentStatus.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/status/FlinkBlueGreenDeploymentStatus.java new file mode 100644 index 0000000000..04119b7d7b --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/status/FlinkBlueGreenDeploymentStatus.java @@ -0,0 +1,58 @@ +/* + * 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.status; + +import org.apache.flink.annotation.Experimental; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; +import lombok.ToString; +import lombok.experimental.SuperBuilder; + +/** Last observed status of the Flink Blue/Green deployment. */ +@Experimental +@Data +@AllArgsConstructor +@NoArgsConstructor +@ToString(callSuper = true) +@SuperBuilder +@JsonIgnoreProperties(ignoreUnknown = true) +public class FlinkBlueGreenDeploymentStatus { + + private JobStatus jobStatus = new JobStatus(); + + /** The state of the blue/green transition. */ + private FlinkBlueGreenDeploymentState blueGreenState; + + /** Last reconciled (serialized) deployment spec. */ + private String lastReconciledSpec; + + /** Timestamp of last reconciliation. */ + private String lastReconciledTimestamp; + + /** Computed from abortGracePeriodMs, timestamp after which the deployment should be aborted. */ + private String abortTimestamp; + + /** Timestamp when the deployment became READY/STABLE. Used to determine when to delete it. */ + private String deploymentReadyTimestamp; + + /** Persisted triggerId to track transition with savepoint. Only used with UpgradeMode.SAVEPOINT */ + private String savepointTriggerId; +} diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/RemoveScaleSubResource.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/RemoveScaleSubResource.java new file mode 100644 index 0000000000..f30d51977e --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/RemoveScaleSubResource.java @@ -0,0 +1,41 @@ +/* + * 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.utils; + +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.stream.Collectors; + +/** + * Utility to remove scale subresource from CRD. Required by the {@link + * org.apache.flink.kubernetes.operator.api.FlinkBlueGreenDeployment}. + */ +public class RemoveScaleSubResource { + + public static void main(String[] args) throws Exception { + var path = Paths.get(args[0]); + var filtered = + Files.readAllLines(path).stream() + .filter(line -> !line.trim().equals("scale:")) + .filter(line -> !line.trim().startsWith("specReplicasPath:")) + .collect(Collectors.toList()); + + // Write back (overwrites file) + Files.write(path, filtered); + } +} diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java index 458dd69173..66b8ce35f8 100644 --- a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/utils/SpecUtils.java @@ -99,6 +99,27 @@ public static String writeSpecWithMeta( } } + public static String writeSpecAsJSON(Object object, String wrapperKey) { + ObjectNode wrapper = objectMapper.createObjectNode(); + wrapper.set(wrapperKey, objectMapper.valueToTree(checkNotNull(object))); + + try { + return objectMapper.writeValueAsString(wrapper); + } catch (JsonProcessingException e) { + throw new RuntimeException( + "Could not serialize " + wrapperKey + ", this indicates a bug...", e); + } + } + + public static T readSpecFromJSON(String serialized, String wrapperKey, Class valueType) { + try { + ObjectNode wrapper = (ObjectNode) objectMapper.readTree(serialized); + return objectMapper.treeToValue(wrapper.get(wrapperKey), valueType); + } catch (JsonProcessingException e) { + throw new RuntimeException("Could not deserialize spec, this indicates a bug...", e); + } + } + // We do not have access to Flink's Preconditions from here private static T checkNotNull(T object) { if (object == null) { 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 06c361aea3..c876f5fff7 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; @@ -244,6 +245,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(); @@ -264,6 +271,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..b46f865537 --- /dev/null +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/BlueGreenDiffType.java @@ -0,0 +1,37 @@ +/* + * 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, + + // TODO: the PATCH_TOP_LEVEL and PATCH_BOTH values are redundant, eliminate + /** Changes that only affect the top-level configuration. */ + PATCH_TOP_LEVEL, + + /** Changes that affect both top-level and child specifications. */ + PATCH_BOTH +} 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..ee28ea8175 --- /dev/null +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentController.java @@ -0,0 +1,145 @@ +/* + * 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.spec.FlinkBlueGreenDeploymentConfigOptions.ABORT_GRACE_PERIOD; +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 static long minimumAbortGracePeriodMs = ABORT_GRACE_PERIOD.defaultValue().toMillis(); + + 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) + .rescheduleAfter(100); + } 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..42a3cefbec --- /dev/null +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeployments.java @@ -0,0 +1,80 @@ +/* + * 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.DeploymentType; + +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 = DeploymentType.fromDeployment(dependentDeployment); + + if (flinkBlueGreenDeploymentType == DeploymentType.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..8682d847a9 --- /dev/null +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/BlueGreenContext.java @@ -0,0 +1,42 @@ +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.DeploymentType; +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(DeploymentType type) { + return type == DeploymentType.BLUE ? getBlueDeployment() : getGreenDeployment(); + } +} 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..dc3247db2a --- /dev/null +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/BlueGreenDeploymentService.java @@ -0,0 +1,537 @@ +/* + * 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.BlueGreenDiffType; +import org.apache.flink.kubernetes.operator.api.bluegreen.DeploymentType; +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.BlueGreenSpecUtils; +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.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.controller.bluegreen.BlueGreenKubernetesService.updateFlinkDeployment; +import static org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenSpecUtils.fetchSavepointInfo; +import static org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenSpecUtils.getLastCheckpoint; +import static org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenSpecUtils.isSavepointRequired; +import static org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenSpecUtils.lookForCheckpoint; +import static org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenSpecUtils.prepareFlinkDeployment; +import static org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenSpecUtils.triggerSavepoint; +import static org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenUtils.getReconciliationReschedInterval; +import static org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenUtils.instantStrToMillis; +import static org.apache.flink.kubernetes.operator.utils.bluegreen.BlueGreenUtils.millisToInstantStr; + +/** 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 nextDeploymentType 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, + DeploymentType nextDeploymentType, + FlinkBlueGreenDeploymentState nextState, + Savepoint lastCheckpoint, + boolean isFirstDeployment) { + ObjectMeta bgMeta = context.getBgDeployment().getMetadata(); + + FlinkDeployment flinkDeployment = + prepareFlinkDeployment( + context.getBgDeployment(), + nextDeploymentType, + lastCheckpoint, + isFirstDeployment, + bgMeta); + + deployCluster(context, flinkDeployment); + + BlueGreenUtils.setAbortTimestamp(context); + + return patchStatusUpdateControl(context, nextState, JobStatus.RECONCILING) + .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 currentDeploymentType the current deployment type + * @return UpdateControl for the deployment + */ + public UpdateControl checkAndInitiateDeployment( + BlueGreenContext context, DeploymentType currentDeploymentType) { + BlueGreenDiffType specDiff = BlueGreenSpecUtils.getSpecDiff(context); + + if (specDiff != BlueGreenDiffType.IGNORE) { + FlinkDeployment currentFlinkDeployment = + context.getDeploymentByType(currentDeploymentType); + + if (isFlinkDeploymentReady(currentFlinkDeployment)) { + if (specDiff == BlueGreenDiffType.TRANSITION) { + if (handleSavepoint(context, currentFlinkDeployment)) { + // This is the only portion where the last reconciled spec is not set, + // so we can reprocess TRANSITION after the savepoint is done + var savepointingState = calculateSavepointingState(currentDeploymentType); + return patchStatusUpdateControl(context, savepointingState, null) + .rescheduleAfter(getReconciliationReschedInterval(context)); + } + + BlueGreenSpecUtils.setLastReconciledSpec(context); + return startTransition(context, currentDeploymentType, currentFlinkDeployment); + } else { + BlueGreenSpecUtils.setLastReconciledSpec(context); + return patchFlinkDeployment(context, currentDeploymentType, specDiff); + } + } else { + if (context.getDeploymentStatus().getJobStatus().getState() != JobStatus.FAILING) { + BlueGreenSpecUtils.setLastReconciledSpec(context); + return patchStatusUpdateControl(context, null, JobStatus.FAILING); + } + } + } + + return UpdateControl.noUpdate(); + } + + private UpdateControl patchFlinkDeployment( + BlueGreenContext context, + DeploymentType currentDeploymentType, + BlueGreenDiffType specDiff) { + + if (specDiff == BlueGreenDiffType.PATCH_BOTH || specDiff == BlueGreenDiffType.PATCH_CHILD) { + FlinkDeployment nextFlinkDeployment = + context.getDeploymentByType(currentDeploymentType); + + nextFlinkDeployment.setSpec( + context.getBgDeployment().getSpec().getTemplate().getSpec()); + + updateFlinkDeployment(nextFlinkDeployment, context); + } + + return patchStatusUpdateControl( + context, + calculatePatchingState(currentDeploymentType), + JobStatus.RECONCILING) + .rescheduleAfter(BlueGreenUtils.getReconciliationReschedInterval(context)); + } + + private UpdateControl startTransition( + BlueGreenContext context, + DeploymentType currentDeploymentType, + FlinkDeployment currentFlinkDeployment) { + DeploymentTransition transition = calculateTransition(currentDeploymentType); + + Savepoint lastCheckpoint = configureInitialSavepoint(context, currentFlinkDeployment); + + return initiateDeployment( + context, + transition.nextDeploymentType, + transition.nextState, + lastCheckpoint, + false); + } + + private DeploymentTransition calculateTransition(DeploymentType currentType) { + if (DeploymentType.BLUE == currentType) { + return new DeploymentTransition( + DeploymentType.GREEN, FlinkBlueGreenDeploymentState.TRANSITIONING_TO_GREEN); + } else { + return new DeploymentTransition( + DeploymentType.BLUE, FlinkBlueGreenDeploymentState.TRANSITIONING_TO_BLUE); + } + } + + private FlinkBlueGreenDeploymentState calculatePatchingState(DeploymentType currentType) { + if (DeploymentType.BLUE == currentType) { + return FlinkBlueGreenDeploymentState.TRANSITIONING_TO_BLUE; + } else { + return FlinkBlueGreenDeploymentState.TRANSITIONING_TO_GREEN; + } + } + + // ==================== Savepointing Methods ==================== + + public boolean monitorSavepoint( + BlueGreenContext context, DeploymentType currentDeploymentType) { + + FlinkResourceContext ctx = + context.getCtxFactory().getResourceContext( + context.getDeploymentByType(currentDeploymentType), + context.getJosdkContext()); + + String savepointTriggerId = context.getDeploymentStatus().getSavepointTriggerId(); + var savepointFetchResult = fetchSavepointInfo(ctx, savepointTriggerId); + + return !savepointFetchResult.isPending(); + } + + private Savepoint configureInitialSavepoint( + BlueGreenContext context, + FlinkDeployment currentFlinkDeployment) { + + FlinkResourceContext ctx = + context.getCtxFactory() + .getResourceContext(currentFlinkDeployment, context.getJosdkContext()); + + // If a savepoint is required we fetch it, should be ready by this point + if (isSavepointRequired(context)) { + String savepointTriggerId = context.getDeploymentStatus().getSavepointTriggerId(); + var savepointFetchResult = fetchSavepointInfo(ctx, savepointTriggerId); + + org.apache.flink.core.execution.SavepointFormatType coreSavepointFormatType = + ctx.getObserveConfig().get(KubernetesOperatorConfigOptions.OPERATOR_SAVEPOINT_FORMAT_TYPE); + + var savepointFormatType = SavepointFormatType.valueOf(coreSavepointFormatType.toString()); + + return Savepoint.of(savepointFetchResult.getLocation(), SnapshotTriggerType.MANUAL, savepointFormatType); + } + + // Else we start looking for the last checkpoint if needed + + if (!lookForCheckpoint(context)) { + return null; + } + + return getLastCheckpoint(ctx); + } + + private boolean handleSavepoint( + BlueGreenContext context, + FlinkDeployment currentFlinkDeployment) { + + 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.debug("Savepoint previously requested (triggerId: {})", savepointTriggerId); + return false; + } + + private FlinkBlueGreenDeploymentState calculateSavepointingState(DeploymentType currentType) { + if (DeploymentType.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 currentDeploymentType the current deployment type being transitioned from + * @return UpdateControl for the transition + */ + public UpdateControl monitorTransition( + BlueGreenContext context, DeploymentType currentDeploymentType) { + + handleSpecChangesDuringTransition(context); + + TransitionState transitionState = determineTransitionState(context, currentDeploymentType); + + if (isFlinkDeploymentReady(transitionState.nextDeployment)) { + return shouldWeDelete( + context, + transitionState.currentDeployment, + transitionState.nextDeployment, + transitionState.nextState); + } else { + return shouldWeAbort( + context, transitionState.nextDeployment, transitionState.nextState); + } + } + + private void handleSpecChangesDuringTransition(BlueGreenContext context) { + if (BlueGreenSpecUtils.hasSpecChanged(context)) { + BlueGreenSpecUtils.revertToLastSpec(context); + LOG.warn( + "Blue/Green Spec change detected during transition, ignored and reverted to the last reconciled spec"); + } + } + + private TransitionState determineTransitionState( + BlueGreenContext context, DeploymentType currentDeploymentType) { + TransitionState transitionState; + + if (DeploymentType.BLUE == currentDeploymentType) { + 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: " + + currentDeploymentType); + + 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) + .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, abortTimestamp, deploymentName); + } + } + + private UpdateControl retryDeployment( + BlueGreenContext context, long abortTimestamp, String deploymentName) { + + long delay = abortTimestamp - System.currentTimeMillis(); + LOG.info( + "FlinkDeployment '{}' not ready yet, retrying in {} seconds.", + deploymentName, + delay / 1000); + + return patchStatusUpdateControl(context, 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); + + LOG.warn( + "Aborting deployment '{}', rolling B/G deployment back to {}", + deploymentName, + previousState); + + return patchStatusUpdateControl(context, null, JobStatus.FAILING); + } + + 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); + } + + // ==================== Common Utility Methods ==================== + + public static UpdateControl patchStatusUpdateControl( + BlueGreenContext context, + FlinkBlueGreenDeploymentState deploymentState, + JobStatus jobState) { + + var deploymentStatus = context.getDeploymentStatus(); + var flinkBlueGreenDeployment = context.getBgDeployment(); + + if (deploymentState != null) { + deploymentStatus.setBlueGreenState(deploymentState); + } + + if (jobState != null) { + deploymentStatus.getJobStatus().setState(jobState); + } + + deploymentStatus.setLastReconciledTimestamp(java.time.Instant.now().toString()); + flinkBlueGreenDeployment.setStatus(deploymentStatus); + return UpdateControl.patchStatus(flinkBlueGreenDeployment); + } + + // ==================== DTO/Result Classes ==================== + + @Getter + @AllArgsConstructor + private static class DeploymentTransition { + final DeploymentType nextDeploymentType; + 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..0396ed10a4 --- /dev/null +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/BlueGreenStateHandlerRegistry.java @@ -0,0 +1,78 @@ +/* + * 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) { + BlueGreenStateHandler handler = handlers.get(state); + if (handler == null) { + throw new IllegalStateException("No handler found for state: " + state); + } + return handler; + } +} 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..8a3e4f0a1f --- /dev/null +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/handlers/AbstractBlueGreenStateHandler.java @@ -0,0 +1,50 @@ +/* + * 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 static final Logger LOG = LoggerFactory.getLogger(AbstractBlueGreenStateHandler.class); + + private final FlinkBlueGreenDeploymentState supportedState; + + protected final BlueGreenDeploymentService deploymentService; + + protected AbstractBlueGreenStateHandler( + FlinkBlueGreenDeploymentState supportedState, + BlueGreenDeploymentService deploymentService) { + this.supportedState = supportedState; + this.deploymentService = deploymentService; + } + + @Override + public FlinkBlueGreenDeploymentState getSupportedState() { + return supportedState; + } + + protected Logger getLogger() { + return LOG; + } +} 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..dfbec34cdb --- /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.DeploymentType; +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) { + DeploymentType currentType = getCurrentDeploymentType(); + return deploymentService.checkAndInitiateDeployment(context, currentType); + } + + private DeploymentType getCurrentDeploymentType() { + return getSupportedState() == FlinkBlueGreenDeploymentState.ACTIVE_BLUE + ? DeploymentType.BLUE + : DeploymentType.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..5cb8f84a3d --- /dev/null +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/handlers/InitializingBlueStateHandler.java @@ -0,0 +1,47 @@ +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.DeploymentType; +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.utils.bluegreen.BlueGreenSpecUtils; + +import io.javaoperatorsdk.operator.api.reconciler.UpdateControl; + +/** 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(); + + // We only allow a deployment if it's indeed the first (null last spec) + // or if we're recovering (failing status) and the spec has changed + if (deploymentStatus.getLastReconciledSpec() == null + || (deploymentStatus.getJobStatus().getState().equals(JobStatus.FAILING) + && BlueGreenSpecUtils.hasSpecChanged(context))) { + + BlueGreenSpecUtils.setLastReconciledSpec(context); + return deploymentService.initiateDeployment( + context, + DeploymentType.BLUE, + FlinkBlueGreenDeploymentState.TRANSITIONING_TO_BLUE, + null, + true); + } else { + getLogger() + .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..e4e00ecfda --- /dev/null +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/bluegreen/handlers/SavepointingStateHandler.java @@ -0,0 +1,48 @@ +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.DeploymentType; +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) { + DeploymentType currentType = getCurrentDeploymentType(); + var isSavepointReady = deploymentService.monitorSavepoint(context, currentType); + + // Savepoint complete, continue with the transition + if (isSavepointReady) { + var nextState = getSupportedState() == FlinkBlueGreenDeploymentState.SAVEPOINTING_BLUE ? + FlinkBlueGreenDeploymentState.ACTIVE_BLUE : + FlinkBlueGreenDeploymentState.ACTIVE_GREEN; + return patchStatusUpdateControl(context, nextState, null) + .rescheduleAfter(500); + } + + return UpdateControl.noUpdate() + .rescheduleAfter(getReconciliationReschedInterval(context)); + } + + private DeploymentType getCurrentDeploymentType() { + return getSupportedState() == FlinkBlueGreenDeploymentState.SAVEPOINTING_BLUE + ? DeploymentType.BLUE + : DeploymentType.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..22df2cec6b --- /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.DeploymentType; +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) { + DeploymentType currentType = getCurrentDeploymentType(); + return deploymentService.monitorTransition(context, currentType); + } + + private DeploymentType getCurrentDeploymentType() { + return getSupportedState() == FlinkBlueGreenDeploymentState.TRANSITIONING_TO_BLUE + ? DeploymentType.GREEN // Transitioning FROM green TO blue + : DeploymentType.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..faf2247be0 --- /dev/null +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/diff/FlinkBlueGreenDeploymentSpecDiff.java @@ -0,0 +1,153 @@ +/* + * 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; + +import java.util.Objects; + +/** + * Diff class for comparing FlinkBlueGreenDeploymentSpec objects. Provides specialized comparison + * logic that delegates nested FlinkDeploymentSpec comparison to ReflectiveDiffBuilder while + * handling top-level differences. + */ +@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. + * + * @return BlueGreenDiffType indicating the type of difference found + */ + public BlueGreenDiffType compare() { + // Check if both specs are identical + if (left.equals(right)) { + return BlueGreenDiffType.IGNORE; + } + + boolean hasTopLevelDiff = hasTopLevelDifferences(); + BlueGreenDiffType childDiffType = getChildSpecDiffType(); + + // If nested spec has SCALE or UPGRADE differences, return TRANSITION regardless + if (childDiffType == BlueGreenDiffType.TRANSITION) { + return BlueGreenDiffType.TRANSITION; + } + + // Determine result based on where differences are found + boolean hasChildDiff = childDiffType != BlueGreenDiffType.IGNORE; + + if (hasTopLevelDiff && hasChildDiff) { + return BlueGreenDiffType.PATCH_BOTH; + } else if (hasTopLevelDiff) { + return BlueGreenDiffType.PATCH_TOP_LEVEL; + } else if (hasChildDiff) { + return BlueGreenDiffType.PATCH_CHILD; + } else { + return BlueGreenDiffType.IGNORE; + } + } + + /** + * 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"); + } + } + + /** + * Checks if there are differences in top-level properties (metadata, configuration). + * + * @return true if top-level differences exist, false otherwise + */ + private boolean hasTopLevelDifferences() { + // Compare template metadata + boolean metadataDifferent = + !Objects.equals( + left.getTemplate().getMetadata(), right.getTemplate().getMetadata()); + + // Compare template configuration + boolean configurationDifferent = + !Objects.equals( + left.getTemplate().getConfiguration(), + right.getTemplate().getConfiguration()); + + return metadataDifferent || configurationDifferent; + } + + /** + * Gets the diff type for the nested FlinkDeploymentSpec using ReflectiveDiffBuilder. + * + * @return BlueGreenDiffType representing the child spec difference + */ + private BlueGreenDiffType getChildSpecDiffType() { + FlinkDeploymentSpec leftSpec = left.getTemplate().getSpec(); + FlinkDeploymentSpec rightSpec = right.getTemplate().getSpec(); + + // Delegate to ReflectiveDiffBuilder for nested spec comparison + DiffResult diffResult = + new ReflectiveDiffBuilder<>(deploymentMode, leftSpec, rightSpec).build(); + + DiffType diffType = diffResult.getType(); + + // Map DiffType to BlueGreenDiffType + switch (diffType) { + case IGNORE: + return BlueGreenDiffType.IGNORE; + case SCALE: + case UPGRADE: + return BlueGreenDiffType.TRANSITION; + default: + return BlueGreenDiffType.PATCH_CHILD; + } + } +} diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/bluegreen/BlueGreenSpecUtils.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/bluegreen/BlueGreenSpecUtils.java new file mode 100644 index 0000000000..4477a52e70 --- /dev/null +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/bluegreen/BlueGreenSpecUtils.java @@ -0,0 +1,272 @@ +/* + * 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.BlueGreenDiffType; +import org.apache.flink.kubernetes.operator.api.bluegreen.DeploymentType; +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 lombok.SneakyThrows; +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.controller.bluegreen.BlueGreenKubernetesService.getDependentObjectMeta; +import static org.apache.flink.kubernetes.operator.controller.bluegreen.BlueGreenKubernetesService.replaceFlinkBlueGreenDeployment; + +/** Utility methods for handling Blue/Green deployment specifications. */ +public class BlueGreenSpecUtils { + + private static final Logger LOG = LoggerFactory.getLogger(BlueGreenSpecUtils.class); + + /** + * 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 boolean isSavepointRequired(BlueGreenContext context) { + return UpgradeMode.SAVEPOINT + == context.getBgDeployment() + .getSpec() + .getTemplate() + .getSpec() + .getJob() + .getUpgradeMode(); + } + + 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; + } + + /** + * 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().getTemplate().getConfiguration(); + + if (configuration == null) { + return option.defaultValue(); + } + + return Configuration.fromMap(configuration).get(option); + } + + public static FlinkDeployment prepareFlinkDeployment( + FlinkBlueGreenDeployment bgDeployment, + DeploymentType deploymentType, + Savepoint lastCheckpoint, + boolean isFirstDeployment, + ObjectMeta bgMeta) { + // Deployment + FlinkDeployment flinkDeployment = new FlinkDeployment(); + FlinkBlueGreenDeploymentSpec spec = bgDeployment.getSpec(); + + String childDeploymentName = + bgMeta.getName() + "-" + deploymentType.toString().toLowerCase(); + + FlinkBlueGreenDeploymentSpec adjustedSpec = + adjustNameReferences( + spec, + bgMeta.getName(), + childDeploymentName, + "spec", + FlinkBlueGreenDeploymentSpec.class); + + // The B/G initialSavepointPath is only used in 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 start up, no checkpoint/savepoint"); + } + } else if (lastCheckpoint != null) { + String location = lastCheckpoint.getLocation().replace("file:", ""); + LOG.info("Using B/G checkpoint: " + location); + adjustedSpec.getTemplate().getSpec().getJob().setInitialSavepointPath(location); + } + + flinkDeployment.setSpec(adjustedSpec.getTemplate().getSpec()); + + // Deployment metadata + ObjectMeta flinkDeploymentMeta = getDependentObjectMeta(bgDeployment); + flinkDeploymentMeta.setName(childDeploymentName); + flinkDeploymentMeta.setLabels(Map.of(DeploymentType.LABEL_KEY, deploymentType.toString())); + flinkDeployment.setMetadata(flinkDeploymentMeta); + return flinkDeployment; + } + + public static void setLastReconciledSpec(BlueGreenContext context) { + FlinkBlueGreenDeploymentStatus deploymentStatus = context.getDeploymentStatus(); + deploymentStatus.setLastReconciledSpec( + SpecUtils.writeSpecAsJSON(context.getBgDeployment().getSpec(), "spec")); + deploymentStatus.setLastReconciledTimestamp(Instant.now().toString()); + } + + @SneakyThrows + public static String triggerSavepoint(FlinkResourceContext ctx) { + + 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)); + + return ctx.getFlinkService() + .triggerSavepoint(jobId, savepointFormatType, savepointDirectory, conf); + } + + public static SavepointFetchResult fetchSavepointInfo( + FlinkResourceContext ctx, + String triggerId) { + return ctx.getFlinkService() + .fetchSavepointInfo( + triggerId, + ctx.getResource().getStatus().getJobStatus().getJobId(), + ctx.getObserveConfig()); + } + + public static Savepoint getLastCheckpoint( + FlinkResourceContext resourceContext) { + + Optional lastCheckpoint = + resourceContext + .getFlinkService() + .getLastCheckpoint( + JobID.fromHexString( + resourceContext + .getResource() + .getStatus() + .getJobStatus() + .getJobId()), + resourceContext.getObserveConfig()); + + // Alternative action if no checkpoint is available? + if (lastCheckpoint.isEmpty()) { + throw new IllegalStateException( + "Last Checkpoint for Job " + + resourceContext.getResource().getMetadata().getName() + + " not found!"); + } + + return lastCheckpoint.get(); + } + + public static void revertToLastSpec(BlueGreenContext context) { + context.getBgDeployment() + .setSpec( + SpecUtils.readSpecFromJSON( + context.getDeploymentStatus().getLastReconciledSpec(), + "spec", + FlinkBlueGreenDeploymentSpec.class)); + replaceFlinkBlueGreenDeployment(context); + } +} 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..643c5c49ae --- /dev/null +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/bluegreen/BlueGreenUtils.java @@ -0,0 +1,108 @@ +/* + * 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.kubernetes.operator.controller.FlinkBlueGreenDeploymentController; +import org.apache.flink.kubernetes.operator.controller.bluegreen.BlueGreenContext; + +import java.time.Instant; + +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; + +/** Utility methods for time calculations and general Blue/Green deployment operations. */ +public class BlueGreenUtils { + + /** + * 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( + BlueGreenSpecUtils.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( + BlueGreenSpecUtils.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 = + BlueGreenSpecUtils.getConfigOption(context.getBgDeployment(), ABORT_GRACE_PERIOD) + .toMillis(); + return Math.max( + abortGracePeriod, FlinkBlueGreenDeploymentController.minimumAbortGracePeriodMs); + } + + /** + * 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))); + } +} diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/TestUtils.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/TestUtils.java index b961953a53..f3cdb44724 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 bee769fb96..c9e3018aee 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 @@ -621,10 +621,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); 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..3bcea6d073 --- /dev/null +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentControllerTest.java @@ -0,0 +1,973 @@ +/* + * 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.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.Event; +import io.fabric8.kubernetes.api.model.EventBuilder; +import io.fabric8.kubernetes.api.model.ObjectMetaBuilder; +import io.fabric8.kubernetes.client.KubernetesClient; +import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient; +import io.fabric8.kubernetes.client.server.mock.KubernetesMockServer; +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.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.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 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 KubernetesMockServer mockServer; + private KubernetesClient kubernetesClient; + + Event mockedEvent = + new EventBuilder() + .withNewMetadata() + .withName("name") + .endMetadata() + .withType("type") + .withReason("reason") + .build(); + + @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 = + upgradeMode == UpgradeMode.LAST_STATE ? TEST_CHECKPOINT_PATH : initialSavepointPath; + + if (upgradeMode == UpgradeMode.SAVEPOINT) { + // 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 + var rs2 = reconcile(rs.deployment); + + assertTrue(rs2.updateControl.isNoUpdate()); + assertTrue(rs2.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())); + + // Assert job status/state is left the way it is and that the Blue job never got submitted + assertEquals(JobStatus.FAILING, rs.reconciledStatus.getJobStatus().getState()); + 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 = 5000; + Map configuration = + blueGreenDeployment.getSpec().getTemplate().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, + rs.deployment + .getSpec() + .getTemplate() + .getSpec() + .getFlinkConfiguration() + .get(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 < abortGracePeriodMs && 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 + assertEquals(JobStatus.FAILING, rs.reconciledStatus.getJobStatus().getState()); + // 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); + } + + @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); + rs = reconcile(rs.deployment); + + // The spec should have been reverted + assertEquals( + SpecUtils.writeSpecAsJSON(originalSpec, "spec"), + SpecUtils.writeSpecAsJSON(rs.deployment.getSpec(), "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(FlinkBlueGreenDeploymentController.minimumAbortGracePeriodMs); + + rs = reconcile(rs.deployment); + + assertEquals(JobStatus.FAILING, rs.reconciledStatus.getJobStatus().getState()); + // 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 + var rs2 = reconcile(rs.deployment); + assertTrue(rs2.updateControl.isNoUpdate()); + + simulateChangeInSpec(rs.deployment, "MODIFIED_VALUE", 0, null); + + // Resubmitting should re-start the Initialization to Blue + rs = reconcile(rs.deployment); + + 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); + + var result = reconcileAndVerifyPatchBehavior(rs); + + testCase.verifySpecificBehavior(result, getFlinkDeployments()); + + assertFinalized( + result.minReconciliationTs, result.rs, FlinkBlueGreenDeploymentState.ACTIVE_BLUE); + } + + // ==================== Parameterized Test Inputs ==================== + + 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(); + spec.getJob().setSavepointRedeployNonce(12345L); + deployment.getSpec().getTemplate().setSpec(spec); + client.resource(deployment).createOrReplace(); + } + + @Override + public void verifySpecificBehavior( + ReconcileResult result, List deployments) { + assertEquals(1, deployments.size()); + assertEquals( + 12345L, + (long) deployments.get(0).getSpec().getJob().getSavepointRedeployNonce()); + } + } + + static class PatchTopLevelTestCase implements PatchTestCase { + @Override + public void applyChanges(FlinkBlueGreenDeployment deployment, KubernetesClient client) { + FlinkDeploymentTemplateSpec template = deployment.getSpec().getTemplate(); + Map configuration = new HashMap<>(template.getConfiguration()); + configuration.put("custom.top.level", "custom-top-level-value"); + template.setConfiguration(configuration); + deployment.getSpec().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) { + FlinkDeploymentTemplateSpec template = deployment.getSpec().getTemplate(); + + // 1. Add top-level configuration change + Map configuration = new HashMap<>(template.getConfiguration()); + configuration.put("custom.both.level", "custom-both-level-value"); + template.setConfiguration(configuration); + + // 2. Add nested spec change + FlinkDeploymentSpec spec = template.getSpec(); + spec.getJob().setSavepointRedeployNonce(67890L); + template.setSpec(spec); + + deployment.getSpec().setTemplate(template); + client.resource(deployment).createOrReplace(); + } + + @Override + public void verifySpecificBehavior( + ReconcileResult result, List deployments) { + assertEquals(1, deployments.size()); + var updatedDeployment = deployments.get(0); + + // Child spec change should be applied to FlinkDeployment + assertEquals( + 67890L, + (long) updatedDeployment.getSpec().getJob().getSavepointRedeployNonce()); + + // 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 { + + var flinkDeployments = getFlinkDeployments(); + assertEquals(1, flinkDeployments.size()); + var existingFlinkDeployment = flinkDeployments.get(0); + + var minReconciliationTs = System.currentTimeMillis() - 1; + rs = reconcile(rs.deployment); + + assertPatchOperationTriggered(rs, minReconciliationTs); + assertTransitioningState(rs); + + minReconciliationTs = System.currentTimeMillis() - 1; + rs = reconcile(rs.deployment); + + return new ReconcileResult(minReconciliationTs, rs, existingFlinkDeployment); + } + + 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 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()); + assertTrue(rs.updateControl.getScheduleDelay().get() > 0); + 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 + var rs2 = reconcile(rs.deployment); + var remainingDeletionDelay = rs2.updateControl.getScheduleDelay().get(); + assertTrue(remainingDeletionDelay <= expectedDeletionDelay); + assertTrue(rs2.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(); + + 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, + rs.deployment + .getSpec() + .getTemplate() + .getSpec() + .getFlinkConfiguration() + .get(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) { + FlinkDeploymentTemplateSpec template = blueGreenDeployment.getSpec().getTemplate(); + + if (customDeletionDelayMs > 0) { + template.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(); + } + + /* + 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.getResource().orElse(null), + updateControl.isNoUpdate() ? null : 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(conf) + .jobManager(new JobManagerSpec(new Resource(1.0, "2048m", "2G"), 1, null)) + .taskManager( + new TaskManagerSpec(new Resource(1.0, "2048m", "2G"), null, null)) + .build(); + + 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() + .configuration(configuration) + .spec(flinkDeploymentSpec) + .build(); + + return new FlinkBlueGreenDeploymentSpec(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..3706fd1122 --- /dev/null +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/TestingFlinkBlueGreenDeploymentController.java @@ -0,0 +1,83 @@ +/* + * 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); + flinkBlueGreenDeploymentController.minimumAbortGracePeriodMs = 1000; + } + + @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..c7bc338330 --- /dev/null +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/diff/FlinkBlueGreenDeploymentSpecDiffTest.java @@ -0,0 +1,262 @@ +/* + * 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.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 testPatchTopLevelForMetadataDifference() { + FlinkBlueGreenDeploymentSpec spec1 = createBasicSpec(); + FlinkBlueGreenDeploymentSpec spec2 = createBasicSpec(); + + // Change metadata in spec2 + ObjectMeta metadata = new ObjectMeta(); + metadata.setName("different-name"); + spec2.getTemplate().setMetadata(metadata); + + FlinkBlueGreenDeploymentSpecDiff diff = + new FlinkBlueGreenDeploymentSpecDiff(DEPLOYMENT_MODE, spec1, spec2); + + assertEquals(BlueGreenDiffType.PATCH_TOP_LEVEL, diff.compare()); + } + + @Test + public void testPatchTopLevelForConfigurationDifference() { + FlinkBlueGreenDeploymentSpec spec1 = createBasicSpec(); + FlinkBlueGreenDeploymentSpec spec2 = createBasicSpec(); + + // Change configuration in spec2 + Map config = new HashMap<>(); + config.put("custom.config", "different-value"); + spec2.getTemplate().setConfiguration(config); + + FlinkBlueGreenDeploymentSpecDiff diff = + new FlinkBlueGreenDeploymentSpecDiff(DEPLOYMENT_MODE, spec1, spec2); + + assertEquals(BlueGreenDiffType.PATCH_TOP_LEVEL, diff.compare()); + } + + @Test + public void testPatchChildForNestedSpecDifference() { + FlinkBlueGreenDeploymentSpec spec1 = createBasicSpec(); + FlinkBlueGreenDeploymentSpec spec2 = createBasicSpec(); + + // Change nested spec property that doesn't trigger SCALE/UPGRADE + spec2.getTemplate() + .getSpec() + .getJob() + .setSavepointRedeployNonce(12345L); // .setImage("different-image:latest"); + + FlinkBlueGreenDeploymentSpecDiff diff = + new FlinkBlueGreenDeploymentSpecDiff(DEPLOYMENT_MODE, spec1, spec2); + + assertEquals(BlueGreenDiffType.PATCH_CHILD, diff.compare()); + } + + @Test + public void testPatchBothForTopLevelAndNestedDifferences() { + FlinkBlueGreenDeploymentSpec spec1 = createBasicSpec(); + FlinkBlueGreenDeploymentSpec spec2 = createBasicSpec(); + + // Change both top-level (configuration) and nested spec + Map config = new HashMap<>(); + config.put("custom.config", "different-value"); + spec2.getTemplate().setConfiguration(config); + spec2.getTemplate() + .getSpec() + .getJob() + .setSavepointRedeployNonce(12345L); // ("different-image:latest"); + + FlinkBlueGreenDeploymentSpecDiff diff = + new FlinkBlueGreenDeploymentSpecDiff(DEPLOYMENT_MODE, spec1, spec2); + + assertEquals(BlueGreenDiffType.PATCH_BOTH, 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 testTransitionOverridesPatchBoth() { + FlinkBlueGreenDeploymentSpec spec1 = createBasicSpec(); + FlinkBlueGreenDeploymentSpec spec2 = createBasicSpec(); + + // Change both top-level and nested spec, but nested change should trigger TRANSITION + Map config = new HashMap<>(); + config.put("custom.config", "different-value"); + spec2.getTemplate().setConfiguration(config); + spec2.getTemplate().getSpec().getJob().setParallelism(10); // This triggers SCALE + + FlinkBlueGreenDeploymentSpecDiff diff = + new FlinkBlueGreenDeploymentSpecDiff(DEPLOYMENT_MODE, spec1, spec2); + + // Should return TRANSITION, not PATCH_BOTH + 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") + .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..e8c8c248b2 --- /dev/null +++ b/helm/flink-kubernetes-operator/crds/flinkbluegreendeployments.flink.apache.org-v1.yml @@ -0,0 +1,10921 @@ +# 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: + - description: Last observed state of the job. + jsonPath: .status.jobStatus.state + name: Job Status + priority: 0 + type: string + name: v1beta1 + schema: + openAPIV3Schema: + properties: + spec: + properties: + template: + properties: + configuration: + additionalProperties: + type: string + type: object + 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: + additionalProperties: + type: string + type: object + flinkVersion: + enum: + - v1_13 + - v1_14 + - v1_15 + - v1_16 + - v1_17 + - v1_18 + - v1_19 + - v1_20 + - v2_0 + type: string + image: + type: string + imagePullPolicy: + type: string + ingress: + properties: + annotations: + additionalProperties: + type: string + type: object + className: + type: string + labels: + additionalProperties: + type: string + type: object + template: + type: string + tls: + items: + properties: + hosts: + items: + type: string + type: array + secretName: + type: string + type: object + type: array + type: object + job: + properties: + allowNonRestoredState: + type: boolean + args: + items: + type: string + type: array + autoscalerResetNonce: + type: integer + checkpointTriggerNonce: + type: integer + entryClass: + type: string + initialSavepointPath: + type: string + jarURI: + type: string + parallelism: + type: integer + savepointRedeployNonce: + type: integer + savepointTriggerNonce: + type: integer + state: + enum: + - running + - suspended + type: string + upgradeMode: + enum: + - last-state + - savepoint + - stateless + type: string + type: object + jobManager: + properties: + podTemplate: + properties: + apiVersion: + type: string + kind: + type: string + metadata: + properties: + annotations: + additionalProperties: + type: string + type: object + creationTimestamp: + type: string + deletionGracePeriodSeconds: + type: integer + deletionTimestamp: + type: string + finalizers: + items: + type: string + type: array + generateName: + type: string + generation: + type: integer + labels: + additionalProperties: + type: string + type: object + managedFields: + items: + properties: + apiVersion: + type: string + fieldsType: + type: string + fieldsV1: + type: object + manager: + type: string + operation: + type: string + subresource: + type: string + time: + type: string + type: object + type: array + name: + type: string + namespace: + type: string + ownerReferences: + items: + properties: + apiVersion: + type: string + blockOwnerDeletion: + type: boolean + controller: + type: boolean + kind: + type: string + name: + type: string + uid: + type: string + type: object + type: array + resourceVersion: + type: string + selfLink: + type: string + uid: + type: string + type: object + spec: + properties: + activeDeadlineSeconds: + type: integer + affinity: + properties: + nodeAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + preference: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + properties: + nodeSelectorTerms: + items: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + type: array + type: object + type: object + podAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + podAntiAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + type: object + automountServiceAccountToken: + type: boolean + containers: + items: + properties: + args: + items: + type: string + type: array + command: + items: + type: string + type: array + env: + items: + properties: + name: + type: string + value: + type: string + valueFrom: + properties: + configMapKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + secretKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + envFrom: + items: + properties: + configMapRef: + properties: + name: + type: string + optional: + type: boolean + type: object + prefix: + type: string + secretRef: + properties: + name: + type: string + optional: + type: boolean + type: object + type: object + type: array + image: + type: string + imagePullPolicy: + type: string + lifecycle: + properties: + postStart: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + preStop: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + stopSignal: + type: string + type: object + livenessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + name: + type: string + ports: + items: + properties: + containerPort: + type: integer + hostIP: + type: string + hostPort: + type: integer + name: + type: string + protocol: + type: string + type: object + type: array + readinessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + resizePolicy: + items: + properties: + resourceName: + type: string + restartPolicy: + type: string + type: object + type: array + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartPolicy: + type: string + securityContext: + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + startupProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + stdin: + type: boolean + stdinOnce: + type: boolean + terminationMessagePath: + type: string + terminationMessagePolicy: + type: string + tty: + type: boolean + volumeDevices: + items: + properties: + devicePath: + type: string + name: + type: string + type: object + type: array + volumeMounts: + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + workingDir: + type: string + type: object + type: array + dnsConfig: + properties: + nameservers: + items: + type: string + type: array + options: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + searches: + items: + type: string + type: array + type: object + dnsPolicy: + type: string + enableServiceLinks: + type: boolean + ephemeralContainers: + items: + properties: + args: + items: + type: string + type: array + command: + items: + type: string + type: array + env: + items: + properties: + name: + type: string + value: + type: string + valueFrom: + properties: + configMapKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + secretKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + envFrom: + items: + properties: + configMapRef: + properties: + name: + type: string + optional: + type: boolean + type: object + prefix: + type: string + secretRef: + properties: + name: + type: string + optional: + type: boolean + type: object + type: object + type: array + image: + type: string + imagePullPolicy: + type: string + lifecycle: + properties: + postStart: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + preStop: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + stopSignal: + type: string + type: object + livenessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + name: + type: string + ports: + items: + properties: + containerPort: + type: integer + hostIP: + type: string + hostPort: + type: integer + name: + type: string + protocol: + type: string + type: object + type: array + readinessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + resizePolicy: + items: + properties: + resourceName: + type: string + restartPolicy: + type: string + type: object + type: array + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartPolicy: + type: string + securityContext: + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + startupProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + stdin: + type: boolean + stdinOnce: + type: boolean + targetContainerName: + type: string + terminationMessagePath: + type: string + terminationMessagePolicy: + type: string + tty: + type: boolean + volumeDevices: + items: + properties: + devicePath: + type: string + name: + type: string + type: object + type: array + volumeMounts: + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + workingDir: + type: string + type: object + type: array + hostAliases: + items: + properties: + hostnames: + items: + type: string + type: array + ip: + type: string + type: object + type: array + hostIPC: + type: boolean + hostNetwork: + type: boolean + hostPID: + type: boolean + hostUsers: + type: boolean + hostname: + type: string + imagePullSecrets: + items: + properties: + name: + type: string + type: object + type: array + initContainers: + items: + properties: + args: + items: + type: string + type: array + command: + items: + type: string + type: array + env: + items: + properties: + name: + type: string + value: + type: string + valueFrom: + properties: + configMapKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + secretKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + envFrom: + items: + properties: + configMapRef: + properties: + name: + type: string + optional: + type: boolean + type: object + prefix: + type: string + secretRef: + properties: + name: + type: string + optional: + type: boolean + type: object + type: object + type: array + image: + type: string + imagePullPolicy: + type: string + lifecycle: + properties: + postStart: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + preStop: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + stopSignal: + type: string + type: object + livenessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + name: + type: string + ports: + items: + properties: + containerPort: + type: integer + hostIP: + type: string + hostPort: + type: integer + name: + type: string + protocol: + type: string + type: object + type: array + readinessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + resizePolicy: + items: + properties: + resourceName: + type: string + restartPolicy: + type: string + type: object + type: array + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartPolicy: + type: string + securityContext: + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + startupProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + stdin: + type: boolean + stdinOnce: + type: boolean + terminationMessagePath: + type: string + terminationMessagePolicy: + type: string + tty: + type: boolean + volumeDevices: + items: + properties: + devicePath: + type: string + name: + type: string + type: object + type: array + volumeMounts: + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + workingDir: + type: string + type: object + type: array + nodeName: + type: string + nodeSelector: + additionalProperties: + type: string + type: object + os: + properties: + name: + type: string + type: object + overhead: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + preemptionPolicy: + type: string + priority: + type: integer + priorityClassName: + type: string + readinessGates: + items: + properties: + conditionType: + type: string + type: object + type: array + resourceClaims: + items: + properties: + name: + type: string + resourceClaimName: + type: string + resourceClaimTemplateName: + type: string + type: object + type: array + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartPolicy: + type: string + runtimeClassName: + type: string + schedulerName: + type: string + schedulingGates: + items: + properties: + name: + type: string + type: object + type: array + securityContext: + properties: + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + fsGroup: + type: integer + fsGroupChangePolicy: + type: string + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxChangePolicy: + type: string + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + supplementalGroups: + items: + type: integer + type: array + supplementalGroupsPolicy: + type: string + sysctls: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + serviceAccount: + type: string + serviceAccountName: + type: string + setHostnameAsFQDN: + type: boolean + shareProcessNamespace: + type: boolean + subdomain: + type: string + terminationGracePeriodSeconds: + type: integer + tolerations: + items: + properties: + effect: + type: string + key: + type: string + operator: + type: string + tolerationSeconds: + type: integer + value: + type: string + type: object + type: array + topologySpreadConstraints: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + maxSkew: + type: integer + minDomains: + type: integer + nodeAffinityPolicy: + type: string + nodeTaintsPolicy: + type: string + topologyKey: + type: string + whenUnsatisfiable: + type: string + type: object + type: array + volumes: + items: + properties: + awsElasticBlockStore: + properties: + fsType: + type: string + partition: + type: integer + readOnly: + type: boolean + volumeID: + type: string + type: object + azureDisk: + properties: + cachingMode: + type: string + diskName: + type: string + diskURI: + type: string + fsType: + type: string + kind: + type: string + readOnly: + type: boolean + type: object + azureFile: + properties: + readOnly: + type: boolean + secretName: + type: string + shareName: + type: string + type: object + cephfs: + properties: + monitors: + items: + type: string + type: array + path: + type: string + readOnly: + type: boolean + secretFile: + type: string + secretRef: + properties: + name: + type: string + type: object + user: + type: string + type: object + cinder: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + volumeID: + type: string + type: object + configMap: + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + csi: + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: + properties: + name: + type: string + type: object + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string + type: object + type: object + downwardAPI: + properties: + defaultMode: + type: integer + items: + items: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + mode: + type: integer + path: + type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + type: object + type: array + type: object + emptyDir: + properties: + medium: + type: string + sizeLimit: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + ephemeral: + properties: + volumeClaimTemplate: + properties: + metadata: + properties: + annotations: + additionalProperties: + type: string + type: object + creationTimestamp: + type: string + deletionGracePeriodSeconds: + type: integer + deletionTimestamp: + type: string + finalizers: + items: + type: string + type: array + generateName: + type: string + generation: + type: integer + labels: + additionalProperties: + type: string + type: object + managedFields: + items: + properties: + apiVersion: + type: string + fieldsType: + type: string + fieldsV1: + type: object + manager: + type: string + operation: + type: string + subresource: + type: string + time: + type: string + type: object + type: array + name: + type: string + namespace: + type: string + ownerReferences: + items: + properties: + apiVersion: + type: string + blockOwnerDeletion: + type: boolean + controller: + type: boolean + kind: + type: string + name: + type: string + uid: + type: string + type: object + type: array + resourceVersion: + type: string + selfLink: + type: string + uid: + type: string + type: object + spec: + properties: + accessModes: + items: + type: string + type: array + dataSource: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + type: object + dataSourceRef: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + namespace: + type: string + type: object + resources: + properties: + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + selector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + storageClassName: + type: string + volumeAttributesClassName: + type: string + volumeMode: + type: string + volumeName: + type: string + type: object + type: object + type: object + fc: + properties: + fsType: + type: string + lun: + type: integer + readOnly: + type: boolean + targetWWNs: + items: + type: string + type: array + wwids: + items: + type: string + type: array + type: object + flexVolume: + properties: + driver: + type: string + fsType: + type: string + options: + additionalProperties: + type: string + type: object + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + type: object + flocker: + properties: + datasetName: + type: string + datasetUUID: + type: string + type: object + gcePersistentDisk: + properties: + fsType: + type: string + partition: + type: integer + pdName: + type: string + readOnly: + type: boolean + type: object + gitRepo: + properties: + directory: + type: string + repository: + type: string + revision: + type: string + type: object + glusterfs: + properties: + endpoints: + type: string + path: + type: string + readOnly: + type: boolean + type: object + hostPath: + properties: + path: + type: string + type: + type: string + type: object + image: + properties: + pullPolicy: + type: string + reference: + type: string + type: object + iscsi: + properties: + chapAuthDiscovery: + type: boolean + chapAuthSession: + type: boolean + fsType: + type: string + initiatorName: + type: string + iqn: + type: string + iscsiInterface: + type: string + lun: + type: integer + portals: + items: + type: string + type: array + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + targetPortal: + type: string + type: object + name: + type: string + nfs: + properties: + path: + type: string + readOnly: + type: boolean + server: + type: string + type: object + persistentVolumeClaim: + properties: + claimName: + type: string + readOnly: + type: boolean + type: object + photonPersistentDisk: + properties: + fsType: + type: string + pdID: + type: string + type: object + portworxVolume: + properties: + fsType: + type: string + readOnly: + type: boolean + volumeID: + type: string + type: object + projected: + properties: + defaultMode: + type: integer + sources: + items: + properties: + clusterTrustBundle: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + name: + type: string + optional: + type: boolean + path: + type: string + signerName: + type: string + type: object + configMap: + properties: + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + downwardAPI: + properties: + items: + items: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + mode: + type: integer + path: + type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + type: object + type: array + type: object + secret: + properties: + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + serviceAccountToken: + properties: + audience: + type: string + expirationSeconds: + type: integer + path: + type: string + type: object + type: object + type: array + type: object + quobyte: + properties: + group: + type: string + readOnly: + type: boolean + registry: + type: string + tenant: + type: string + user: + type: string + volume: + type: string + type: object + rbd: + properties: + fsType: + type: string + image: + type: string + keyring: + type: string + monitors: + items: + type: string + type: array + pool: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + user: + type: string + type: object + scaleIO: + properties: + fsType: + type: string + gateway: + type: string + protectionDomain: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + sslEnabled: + type: boolean + storageMode: + type: string + storagePool: + type: string + system: + type: string + volumeName: + type: string + type: object + secret: + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + optional: + type: boolean + secretName: + type: string + type: object + storageos: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + volumeName: + type: string + volumeNamespace: + type: string + type: object + vsphereVolume: + properties: + fsType: + type: string + storagePolicyID: + type: string + storagePolicyName: + type: string + volumePath: + type: string + type: object + type: object + type: array + type: object + status: + properties: + conditions: + items: + properties: + lastProbeTime: + type: string + lastTransitionTime: + type: string + message: + type: string + observedGeneration: + type: integer + reason: + type: string + status: + type: string + type: + type: string + type: object + type: array + containerStatuses: + items: + properties: + allocatedResources: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + allocatedResourcesStatus: + items: + properties: + name: + type: string + resources: + items: + properties: + health: + type: string + resourceID: + type: string + type: object + type: array + type: object + type: array + containerID: + type: string + image: + type: string + imageID: + type: string + lastState: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + name: + type: string + ready: + type: boolean + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartCount: + type: integer + started: + type: boolean + state: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + stopSignal: + type: string + user: + properties: + linux: + properties: + gid: + type: integer + supplementalGroups: + items: + type: integer + type: array + uid: + type: integer + type: object + type: object + volumeMounts: + items: + properties: + mountPath: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + type: object + type: array + type: object + type: array + ephemeralContainerStatuses: + items: + properties: + allocatedResources: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + allocatedResourcesStatus: + items: + properties: + name: + type: string + resources: + items: + properties: + health: + type: string + resourceID: + type: string + type: object + type: array + type: object + type: array + containerID: + type: string + image: + type: string + imageID: + type: string + lastState: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + name: + type: string + ready: + type: boolean + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartCount: + type: integer + started: + type: boolean + state: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + stopSignal: + type: string + user: + properties: + linux: + properties: + gid: + type: integer + supplementalGroups: + items: + type: integer + type: array + uid: + type: integer + type: object + type: object + volumeMounts: + items: + properties: + mountPath: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + type: object + type: array + type: object + type: array + hostIP: + type: string + hostIPs: + items: + properties: + ip: + type: string + type: object + type: array + initContainerStatuses: + items: + properties: + allocatedResources: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + allocatedResourcesStatus: + items: + properties: + name: + type: string + resources: + items: + properties: + health: + type: string + resourceID: + type: string + type: object + type: array + type: object + type: array + containerID: + type: string + image: + type: string + imageID: + type: string + lastState: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + name: + type: string + ready: + type: boolean + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartCount: + type: integer + started: + type: boolean + state: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + stopSignal: + type: string + user: + properties: + linux: + properties: + gid: + type: integer + supplementalGroups: + items: + type: integer + type: array + uid: + type: integer + type: object + type: object + volumeMounts: + items: + properties: + mountPath: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + type: object + type: array + type: object + type: array + message: + type: string + nominatedNodeName: + type: string + observedGeneration: + type: integer + phase: + type: string + podIP: + type: string + podIPs: + items: + properties: + ip: + type: string + type: object + type: array + qosClass: + type: string + reason: + type: string + resize: + type: string + resourceClaimStatuses: + items: + properties: + name: + type: string + resourceClaimName: + type: string + type: object + type: array + startTime: + type: string + type: object + type: object + replicas: + type: integer + resource: + properties: + cpu: + type: number + ephemeralStorage: + type: string + memory: + type: string + type: object + type: object + logConfiguration: + additionalProperties: + type: string + type: object + mode: + enum: + - native + - standalone + type: string + podTemplate: + properties: + apiVersion: + type: string + kind: + type: string + metadata: + properties: + annotations: + additionalProperties: + type: string + type: object + creationTimestamp: + type: string + deletionGracePeriodSeconds: + type: integer + deletionTimestamp: + type: string + finalizers: + items: + type: string + type: array + generateName: + type: string + generation: + type: integer + labels: + additionalProperties: + type: string + type: object + managedFields: + items: + properties: + apiVersion: + type: string + fieldsType: + type: string + fieldsV1: + type: object + manager: + type: string + operation: + type: string + subresource: + type: string + time: + type: string + type: object + type: array + name: + type: string + namespace: + type: string + ownerReferences: + items: + properties: + apiVersion: + type: string + blockOwnerDeletion: + type: boolean + controller: + type: boolean + kind: + type: string + name: + type: string + uid: + type: string + type: object + type: array + resourceVersion: + type: string + selfLink: + type: string + uid: + type: string + type: object + spec: + properties: + activeDeadlineSeconds: + type: integer + affinity: + properties: + nodeAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + preference: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + properties: + nodeSelectorTerms: + items: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + type: array + type: object + type: object + podAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + podAntiAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + type: object + automountServiceAccountToken: + type: boolean + containers: + items: + properties: + args: + items: + type: string + type: array + command: + items: + type: string + type: array + env: + items: + properties: + name: + type: string + value: + type: string + valueFrom: + properties: + configMapKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + secretKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + envFrom: + items: + properties: + configMapRef: + properties: + name: + type: string + optional: + type: boolean + type: object + prefix: + type: string + secretRef: + properties: + name: + type: string + optional: + type: boolean + type: object + type: object + type: array + image: + type: string + imagePullPolicy: + type: string + lifecycle: + properties: + postStart: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + preStop: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + stopSignal: + type: string + type: object + livenessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + name: + type: string + ports: + items: + properties: + containerPort: + type: integer + hostIP: + type: string + hostPort: + type: integer + name: + type: string + protocol: + type: string + type: object + type: array + readinessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + resizePolicy: + items: + properties: + resourceName: + type: string + restartPolicy: + type: string + type: object + type: array + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartPolicy: + type: string + securityContext: + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + startupProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + stdin: + type: boolean + stdinOnce: + type: boolean + terminationMessagePath: + type: string + terminationMessagePolicy: + type: string + tty: + type: boolean + volumeDevices: + items: + properties: + devicePath: + type: string + name: + type: string + type: object + type: array + volumeMounts: + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + workingDir: + type: string + type: object + type: array + dnsConfig: + properties: + nameservers: + items: + type: string + type: array + options: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + searches: + items: + type: string + type: array + type: object + dnsPolicy: + type: string + enableServiceLinks: + type: boolean + ephemeralContainers: + items: + properties: + args: + items: + type: string + type: array + command: + items: + type: string + type: array + env: + items: + properties: + name: + type: string + value: + type: string + valueFrom: + properties: + configMapKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + secretKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + envFrom: + items: + properties: + configMapRef: + properties: + name: + type: string + optional: + type: boolean + type: object + prefix: + type: string + secretRef: + properties: + name: + type: string + optional: + type: boolean + type: object + type: object + type: array + image: + type: string + imagePullPolicy: + type: string + lifecycle: + properties: + postStart: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + preStop: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + stopSignal: + type: string + type: object + livenessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + name: + type: string + ports: + items: + properties: + containerPort: + type: integer + hostIP: + type: string + hostPort: + type: integer + name: + type: string + protocol: + type: string + type: object + type: array + readinessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + resizePolicy: + items: + properties: + resourceName: + type: string + restartPolicy: + type: string + type: object + type: array + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartPolicy: + type: string + securityContext: + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + startupProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + stdin: + type: boolean + stdinOnce: + type: boolean + targetContainerName: + type: string + terminationMessagePath: + type: string + terminationMessagePolicy: + type: string + tty: + type: boolean + volumeDevices: + items: + properties: + devicePath: + type: string + name: + type: string + type: object + type: array + volumeMounts: + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + workingDir: + type: string + type: object + type: array + hostAliases: + items: + properties: + hostnames: + items: + type: string + type: array + ip: + type: string + type: object + type: array + hostIPC: + type: boolean + hostNetwork: + type: boolean + hostPID: + type: boolean + hostUsers: + type: boolean + hostname: + type: string + imagePullSecrets: + items: + properties: + name: + type: string + type: object + type: array + initContainers: + items: + properties: + args: + items: + type: string + type: array + command: + items: + type: string + type: array + env: + items: + properties: + name: + type: string + value: + type: string + valueFrom: + properties: + configMapKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + secretKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + envFrom: + items: + properties: + configMapRef: + properties: + name: + type: string + optional: + type: boolean + type: object + prefix: + type: string + secretRef: + properties: + name: + type: string + optional: + type: boolean + type: object + type: object + type: array + image: + type: string + imagePullPolicy: + type: string + lifecycle: + properties: + postStart: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + preStop: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + stopSignal: + type: string + type: object + livenessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + name: + type: string + ports: + items: + properties: + containerPort: + type: integer + hostIP: + type: string + hostPort: + type: integer + name: + type: string + protocol: + type: string + type: object + type: array + readinessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + resizePolicy: + items: + properties: + resourceName: + type: string + restartPolicy: + type: string + type: object + type: array + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartPolicy: + type: string + securityContext: + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + startupProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + stdin: + type: boolean + stdinOnce: + type: boolean + terminationMessagePath: + type: string + terminationMessagePolicy: + type: string + tty: + type: boolean + volumeDevices: + items: + properties: + devicePath: + type: string + name: + type: string + type: object + type: array + volumeMounts: + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + workingDir: + type: string + type: object + type: array + nodeName: + type: string + nodeSelector: + additionalProperties: + type: string + type: object + os: + properties: + name: + type: string + type: object + overhead: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + preemptionPolicy: + type: string + priority: + type: integer + priorityClassName: + type: string + readinessGates: + items: + properties: + conditionType: + type: string + type: object + type: array + resourceClaims: + items: + properties: + name: + type: string + resourceClaimName: + type: string + resourceClaimTemplateName: + type: string + type: object + type: array + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartPolicy: + type: string + runtimeClassName: + type: string + schedulerName: + type: string + schedulingGates: + items: + properties: + name: + type: string + type: object + type: array + securityContext: + properties: + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + fsGroup: + type: integer + fsGroupChangePolicy: + type: string + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxChangePolicy: + type: string + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + supplementalGroups: + items: + type: integer + type: array + supplementalGroupsPolicy: + type: string + sysctls: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + serviceAccount: + type: string + serviceAccountName: + type: string + setHostnameAsFQDN: + type: boolean + shareProcessNamespace: + type: boolean + subdomain: + type: string + terminationGracePeriodSeconds: + type: integer + tolerations: + items: + properties: + effect: + type: string + key: + type: string + operator: + type: string + tolerationSeconds: + type: integer + value: + type: string + type: object + type: array + topologySpreadConstraints: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + maxSkew: + type: integer + minDomains: + type: integer + nodeAffinityPolicy: + type: string + nodeTaintsPolicy: + type: string + topologyKey: + type: string + whenUnsatisfiable: + type: string + type: object + type: array + volumes: + items: + properties: + awsElasticBlockStore: + properties: + fsType: + type: string + partition: + type: integer + readOnly: + type: boolean + volumeID: + type: string + type: object + azureDisk: + properties: + cachingMode: + type: string + diskName: + type: string + diskURI: + type: string + fsType: + type: string + kind: + type: string + readOnly: + type: boolean + type: object + azureFile: + properties: + readOnly: + type: boolean + secretName: + type: string + shareName: + type: string + type: object + cephfs: + properties: + monitors: + items: + type: string + type: array + path: + type: string + readOnly: + type: boolean + secretFile: + type: string + secretRef: + properties: + name: + type: string + type: object + user: + type: string + type: object + cinder: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + volumeID: + type: string + type: object + configMap: + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + csi: + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: + properties: + name: + type: string + type: object + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string + type: object + type: object + downwardAPI: + properties: + defaultMode: + type: integer + items: + items: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + mode: + type: integer + path: + type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + type: object + type: array + type: object + emptyDir: + properties: + medium: + type: string + sizeLimit: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + ephemeral: + properties: + volumeClaimTemplate: + properties: + metadata: + properties: + annotations: + additionalProperties: + type: string + type: object + creationTimestamp: + type: string + deletionGracePeriodSeconds: + type: integer + deletionTimestamp: + type: string + finalizers: + items: + type: string + type: array + generateName: + type: string + generation: + type: integer + labels: + additionalProperties: + type: string + type: object + managedFields: + items: + properties: + apiVersion: + type: string + fieldsType: + type: string + fieldsV1: + type: object + manager: + type: string + operation: + type: string + subresource: + type: string + time: + type: string + type: object + type: array + name: + type: string + namespace: + type: string + ownerReferences: + items: + properties: + apiVersion: + type: string + blockOwnerDeletion: + type: boolean + controller: + type: boolean + kind: + type: string + name: + type: string + uid: + type: string + type: object + type: array + resourceVersion: + type: string + selfLink: + type: string + uid: + type: string + type: object + spec: + properties: + accessModes: + items: + type: string + type: array + dataSource: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + type: object + dataSourceRef: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + namespace: + type: string + type: object + resources: + properties: + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + selector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + storageClassName: + type: string + volumeAttributesClassName: + type: string + volumeMode: + type: string + volumeName: + type: string + type: object + type: object + type: object + fc: + properties: + fsType: + type: string + lun: + type: integer + readOnly: + type: boolean + targetWWNs: + items: + type: string + type: array + wwids: + items: + type: string + type: array + type: object + flexVolume: + properties: + driver: + type: string + fsType: + type: string + options: + additionalProperties: + type: string + type: object + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + type: object + flocker: + properties: + datasetName: + type: string + datasetUUID: + type: string + type: object + gcePersistentDisk: + properties: + fsType: + type: string + partition: + type: integer + pdName: + type: string + readOnly: + type: boolean + type: object + gitRepo: + properties: + directory: + type: string + repository: + type: string + revision: + type: string + type: object + glusterfs: + properties: + endpoints: + type: string + path: + type: string + readOnly: + type: boolean + type: object + hostPath: + properties: + path: + type: string + type: + type: string + type: object + image: + properties: + pullPolicy: + type: string + reference: + type: string + type: object + iscsi: + properties: + chapAuthDiscovery: + type: boolean + chapAuthSession: + type: boolean + fsType: + type: string + initiatorName: + type: string + iqn: + type: string + iscsiInterface: + type: string + lun: + type: integer + portals: + items: + type: string + type: array + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + targetPortal: + type: string + type: object + name: + type: string + nfs: + properties: + path: + type: string + readOnly: + type: boolean + server: + type: string + type: object + persistentVolumeClaim: + properties: + claimName: + type: string + readOnly: + type: boolean + type: object + photonPersistentDisk: + properties: + fsType: + type: string + pdID: + type: string + type: object + portworxVolume: + properties: + fsType: + type: string + readOnly: + type: boolean + volumeID: + type: string + type: object + projected: + properties: + defaultMode: + type: integer + sources: + items: + properties: + clusterTrustBundle: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + name: + type: string + optional: + type: boolean + path: + type: string + signerName: + type: string + type: object + configMap: + properties: + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + downwardAPI: + properties: + items: + items: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + mode: + type: integer + path: + type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + type: object + type: array + type: object + secret: + properties: + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + serviceAccountToken: + properties: + audience: + type: string + expirationSeconds: + type: integer + path: + type: string + type: object + type: object + type: array + type: object + quobyte: + properties: + group: + type: string + readOnly: + type: boolean + registry: + type: string + tenant: + type: string + user: + type: string + volume: + type: string + type: object + rbd: + properties: + fsType: + type: string + image: + type: string + keyring: + type: string + monitors: + items: + type: string + type: array + pool: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + user: + type: string + type: object + scaleIO: + properties: + fsType: + type: string + gateway: + type: string + protectionDomain: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + sslEnabled: + type: boolean + storageMode: + type: string + storagePool: + type: string + system: + type: string + volumeName: + type: string + type: object + secret: + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + optional: + type: boolean + secretName: + type: string + type: object + storageos: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + volumeName: + type: string + volumeNamespace: + type: string + type: object + vsphereVolume: + properties: + fsType: + type: string + storagePolicyID: + type: string + storagePolicyName: + type: string + volumePath: + type: string + type: object + type: object + type: array + type: object + status: + properties: + conditions: + items: + properties: + lastProbeTime: + type: string + lastTransitionTime: + type: string + message: + type: string + observedGeneration: + type: integer + reason: + type: string + status: + type: string + type: + type: string + type: object + type: array + containerStatuses: + items: + properties: + allocatedResources: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + allocatedResourcesStatus: + items: + properties: + name: + type: string + resources: + items: + properties: + health: + type: string + resourceID: + type: string + type: object + type: array + type: object + type: array + containerID: + type: string + image: + type: string + imageID: + type: string + lastState: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + name: + type: string + ready: + type: boolean + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartCount: + type: integer + started: + type: boolean + state: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + stopSignal: + type: string + user: + properties: + linux: + properties: + gid: + type: integer + supplementalGroups: + items: + type: integer + type: array + uid: + type: integer + type: object + type: object + volumeMounts: + items: + properties: + mountPath: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + type: object + type: array + type: object + type: array + ephemeralContainerStatuses: + items: + properties: + allocatedResources: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + allocatedResourcesStatus: + items: + properties: + name: + type: string + resources: + items: + properties: + health: + type: string + resourceID: + type: string + type: object + type: array + type: object + type: array + containerID: + type: string + image: + type: string + imageID: + type: string + lastState: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + name: + type: string + ready: + type: boolean + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartCount: + type: integer + started: + type: boolean + state: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + stopSignal: + type: string + user: + properties: + linux: + properties: + gid: + type: integer + supplementalGroups: + items: + type: integer + type: array + uid: + type: integer + type: object + type: object + volumeMounts: + items: + properties: + mountPath: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + type: object + type: array + type: object + type: array + hostIP: + type: string + hostIPs: + items: + properties: + ip: + type: string + type: object + type: array + initContainerStatuses: + items: + properties: + allocatedResources: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + allocatedResourcesStatus: + items: + properties: + name: + type: string + resources: + items: + properties: + health: + type: string + resourceID: + type: string + type: object + type: array + type: object + type: array + containerID: + type: string + image: + type: string + imageID: + type: string + lastState: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + name: + type: string + ready: + type: boolean + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartCount: + type: integer + started: + type: boolean + state: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + stopSignal: + type: string + user: + properties: + linux: + properties: + gid: + type: integer + supplementalGroups: + items: + type: integer + type: array + uid: + type: integer + type: object + type: object + volumeMounts: + items: + properties: + mountPath: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + type: object + type: array + type: object + type: array + message: + type: string + nominatedNodeName: + type: string + observedGeneration: + type: integer + phase: + type: string + podIP: + type: string + podIPs: + items: + properties: + ip: + type: string + type: object + type: array + qosClass: + type: string + reason: + type: string + resize: + type: string + resourceClaimStatuses: + items: + properties: + name: + type: string + resourceClaimName: + type: string + type: object + type: array + startTime: + type: string + type: object + type: object + restartNonce: + type: integer + serviceAccount: + type: string + taskManager: + properties: + podTemplate: + properties: + apiVersion: + type: string + kind: + type: string + metadata: + properties: + annotations: + additionalProperties: + type: string + type: object + creationTimestamp: + type: string + deletionGracePeriodSeconds: + type: integer + deletionTimestamp: + type: string + finalizers: + items: + type: string + type: array + generateName: + type: string + generation: + type: integer + labels: + additionalProperties: + type: string + type: object + managedFields: + items: + properties: + apiVersion: + type: string + fieldsType: + type: string + fieldsV1: + type: object + manager: + type: string + operation: + type: string + subresource: + type: string + time: + type: string + type: object + type: array + name: + type: string + namespace: + type: string + ownerReferences: + items: + properties: + apiVersion: + type: string + blockOwnerDeletion: + type: boolean + controller: + type: boolean + kind: + type: string + name: + type: string + uid: + type: string + type: object + type: array + resourceVersion: + type: string + selfLink: + type: string + uid: + type: string + type: object + spec: + properties: + activeDeadlineSeconds: + type: integer + affinity: + properties: + nodeAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + preference: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + properties: + nodeSelectorTerms: + items: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + type: array + type: object + type: object + podAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + podAntiAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + type: object + automountServiceAccountToken: + type: boolean + containers: + items: + properties: + args: + items: + type: string + type: array + command: + items: + type: string + type: array + env: + items: + properties: + name: + type: string + value: + type: string + valueFrom: + properties: + configMapKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + secretKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + envFrom: + items: + properties: + configMapRef: + properties: + name: + type: string + optional: + type: boolean + type: object + prefix: + type: string + secretRef: + properties: + name: + type: string + optional: + type: boolean + type: object + type: object + type: array + image: + type: string + imagePullPolicy: + type: string + lifecycle: + properties: + postStart: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + preStop: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + stopSignal: + type: string + type: object + livenessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + name: + type: string + ports: + items: + properties: + containerPort: + type: integer + hostIP: + type: string + hostPort: + type: integer + name: + type: string + protocol: + type: string + type: object + type: array + readinessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + resizePolicy: + items: + properties: + resourceName: + type: string + restartPolicy: + type: string + type: object + type: array + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartPolicy: + type: string + securityContext: + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + startupProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + stdin: + type: boolean + stdinOnce: + type: boolean + terminationMessagePath: + type: string + terminationMessagePolicy: + type: string + tty: + type: boolean + volumeDevices: + items: + properties: + devicePath: + type: string + name: + type: string + type: object + type: array + volumeMounts: + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + workingDir: + type: string + type: object + type: array + dnsConfig: + properties: + nameservers: + items: + type: string + type: array + options: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + searches: + items: + type: string + type: array + type: object + dnsPolicy: + type: string + enableServiceLinks: + type: boolean + ephemeralContainers: + items: + properties: + args: + items: + type: string + type: array + command: + items: + type: string + type: array + env: + items: + properties: + name: + type: string + value: + type: string + valueFrom: + properties: + configMapKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + secretKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + envFrom: + items: + properties: + configMapRef: + properties: + name: + type: string + optional: + type: boolean + type: object + prefix: + type: string + secretRef: + properties: + name: + type: string + optional: + type: boolean + type: object + type: object + type: array + image: + type: string + imagePullPolicy: + type: string + lifecycle: + properties: + postStart: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + preStop: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + stopSignal: + type: string + type: object + livenessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + name: + type: string + ports: + items: + properties: + containerPort: + type: integer + hostIP: + type: string + hostPort: + type: integer + name: + type: string + protocol: + type: string + type: object + type: array + readinessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + resizePolicy: + items: + properties: + resourceName: + type: string + restartPolicy: + type: string + type: object + type: array + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartPolicy: + type: string + securityContext: + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + startupProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + stdin: + type: boolean + stdinOnce: + type: boolean + targetContainerName: + type: string + terminationMessagePath: + type: string + terminationMessagePolicy: + type: string + tty: + type: boolean + volumeDevices: + items: + properties: + devicePath: + type: string + name: + type: string + type: object + type: array + volumeMounts: + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + workingDir: + type: string + type: object + type: array + hostAliases: + items: + properties: + hostnames: + items: + type: string + type: array + ip: + type: string + type: object + type: array + hostIPC: + type: boolean + hostNetwork: + type: boolean + hostPID: + type: boolean + hostUsers: + type: boolean + hostname: + type: string + imagePullSecrets: + items: + properties: + name: + type: string + type: object + type: array + initContainers: + items: + properties: + args: + items: + type: string + type: array + command: + items: + type: string + type: array + env: + items: + properties: + name: + type: string + value: + type: string + valueFrom: + properties: + configMapKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + secretKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + envFrom: + items: + properties: + configMapRef: + properties: + name: + type: string + optional: + type: boolean + type: object + prefix: + type: string + secretRef: + properties: + name: + type: string + optional: + type: boolean + type: object + type: object + type: array + image: + type: string + imagePullPolicy: + type: string + lifecycle: + properties: + postStart: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + preStop: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + sleep: + properties: + seconds: + type: integer + type: object + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + stopSignal: + type: string + type: object + livenessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + name: + type: string + ports: + items: + properties: + containerPort: + type: integer + hostIP: + type: string + hostPort: + type: integer + name: + type: string + protocol: + type: string + type: object + type: array + readinessProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + resizePolicy: + items: + properties: + resourceName: + type: string + restartPolicy: + type: string + type: object + type: array + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartPolicy: + type: string + securityContext: + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + startupProbe: + properties: + exec: + properties: + command: + items: + type: string + type: array + type: object + failureThreshold: + type: integer + grpc: + properties: + port: + type: integer + service: + type: string + type: object + httpGet: + properties: + host: + type: string + httpHeaders: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + path: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + scheme: + type: string + type: object + initialDelaySeconds: + type: integer + periodSeconds: + type: integer + successThreshold: + type: integer + tcpSocket: + properties: + host: + type: string + port: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + terminationGracePeriodSeconds: + type: integer + timeoutSeconds: + type: integer + type: object + stdin: + type: boolean + stdinOnce: + type: boolean + terminationMessagePath: + type: string + terminationMessagePolicy: + type: string + tty: + type: boolean + volumeDevices: + items: + properties: + devicePath: + type: string + name: + type: string + type: object + type: array + volumeMounts: + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + workingDir: + type: string + type: object + type: array + nodeName: + type: string + nodeSelector: + additionalProperties: + type: string + type: object + os: + properties: + name: + type: string + type: object + overhead: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + preemptionPolicy: + type: string + priority: + type: integer + priorityClassName: + type: string + readinessGates: + items: + properties: + conditionType: + type: string + type: object + type: array + resourceClaims: + items: + properties: + name: + type: string + resourceClaimName: + type: string + resourceClaimTemplateName: + type: string + type: object + type: array + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartPolicy: + type: string + runtimeClassName: + type: string + schedulerName: + type: string + schedulingGates: + items: + properties: + name: + type: string + type: object + type: array + securityContext: + properties: + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + fsGroup: + type: integer + fsGroupChangePolicy: + type: string + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxChangePolicy: + type: string + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + supplementalGroups: + items: + type: integer + type: array + supplementalGroupsPolicy: + type: string + sysctls: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + serviceAccount: + type: string + serviceAccountName: + type: string + setHostnameAsFQDN: + type: boolean + shareProcessNamespace: + type: boolean + subdomain: + type: string + terminationGracePeriodSeconds: + type: integer + tolerations: + items: + properties: + effect: + type: string + key: + type: string + operator: + type: string + tolerationSeconds: + type: integer + value: + type: string + type: object + type: array + topologySpreadConstraints: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + maxSkew: + type: integer + minDomains: + type: integer + nodeAffinityPolicy: + type: string + nodeTaintsPolicy: + type: string + topologyKey: + type: string + whenUnsatisfiable: + type: string + type: object + type: array + volumes: + items: + properties: + awsElasticBlockStore: + properties: + fsType: + type: string + partition: + type: integer + readOnly: + type: boolean + volumeID: + type: string + type: object + azureDisk: + properties: + cachingMode: + type: string + diskName: + type: string + diskURI: + type: string + fsType: + type: string + kind: + type: string + readOnly: + type: boolean + type: object + azureFile: + properties: + readOnly: + type: boolean + secretName: + type: string + shareName: + type: string + type: object + cephfs: + properties: + monitors: + items: + type: string + type: array + path: + type: string + readOnly: + type: boolean + secretFile: + type: string + secretRef: + properties: + name: + type: string + type: object + user: + type: string + type: object + cinder: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + volumeID: + type: string + type: object + configMap: + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + csi: + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: + properties: + name: + type: string + type: object + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string + type: object + type: object + downwardAPI: + properties: + defaultMode: + type: integer + items: + items: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + mode: + type: integer + path: + type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + type: object + type: array + type: object + emptyDir: + properties: + medium: + type: string + sizeLimit: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + ephemeral: + properties: + volumeClaimTemplate: + properties: + metadata: + properties: + annotations: + additionalProperties: + type: string + type: object + creationTimestamp: + type: string + deletionGracePeriodSeconds: + type: integer + deletionTimestamp: + type: string + finalizers: + items: + type: string + type: array + generateName: + type: string + generation: + type: integer + labels: + additionalProperties: + type: string + type: object + managedFields: + items: + properties: + apiVersion: + type: string + fieldsType: + type: string + fieldsV1: + type: object + manager: + type: string + operation: + type: string + subresource: + type: string + time: + type: string + type: object + type: array + name: + type: string + namespace: + type: string + ownerReferences: + items: + properties: + apiVersion: + type: string + blockOwnerDeletion: + type: boolean + controller: + type: boolean + kind: + type: string + name: + type: string + uid: + type: string + type: object + type: array + resourceVersion: + type: string + selfLink: + type: string + uid: + type: string + type: object + spec: + properties: + accessModes: + items: + type: string + type: array + dataSource: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + type: object + dataSourceRef: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + namespace: + type: string + type: object + resources: + properties: + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + selector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + storageClassName: + type: string + volumeAttributesClassName: + type: string + volumeMode: + type: string + volumeName: + type: string + type: object + type: object + type: object + fc: + properties: + fsType: + type: string + lun: + type: integer + readOnly: + type: boolean + targetWWNs: + items: + type: string + type: array + wwids: + items: + type: string + type: array + type: object + flexVolume: + properties: + driver: + type: string + fsType: + type: string + options: + additionalProperties: + type: string + type: object + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + type: object + flocker: + properties: + datasetName: + type: string + datasetUUID: + type: string + type: object + gcePersistentDisk: + properties: + fsType: + type: string + partition: + type: integer + pdName: + type: string + readOnly: + type: boolean + type: object + gitRepo: + properties: + directory: + type: string + repository: + type: string + revision: + type: string + type: object + glusterfs: + properties: + endpoints: + type: string + path: + type: string + readOnly: + type: boolean + type: object + hostPath: + properties: + path: + type: string + type: + type: string + type: object + image: + properties: + pullPolicy: + type: string + reference: + type: string + type: object + iscsi: + properties: + chapAuthDiscovery: + type: boolean + chapAuthSession: + type: boolean + fsType: + type: string + initiatorName: + type: string + iqn: + type: string + iscsiInterface: + type: string + lun: + type: integer + portals: + items: + type: string + type: array + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + targetPortal: + type: string + type: object + name: + type: string + nfs: + properties: + path: + type: string + readOnly: + type: boolean + server: + type: string + type: object + persistentVolumeClaim: + properties: + claimName: + type: string + readOnly: + type: boolean + type: object + photonPersistentDisk: + properties: + fsType: + type: string + pdID: + type: string + type: object + portworxVolume: + properties: + fsType: + type: string + readOnly: + type: boolean + volumeID: + type: string + type: object + projected: + properties: + defaultMode: + type: integer + sources: + items: + properties: + clusterTrustBundle: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + name: + type: string + optional: + type: boolean + path: + type: string + signerName: + type: string + type: object + configMap: + properties: + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + downwardAPI: + properties: + items: + items: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + type: object + mode: + type: integer + path: + type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + resource: + type: string + type: object + type: object + type: array + type: object + secret: + properties: + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + serviceAccountToken: + properties: + audience: + type: string + expirationSeconds: + type: integer + path: + type: string + type: object + type: object + type: array + type: object + quobyte: + properties: + group: + type: string + readOnly: + type: boolean + registry: + type: string + tenant: + type: string + user: + type: string + volume: + type: string + type: object + rbd: + properties: + fsType: + type: string + image: + type: string + keyring: + type: string + monitors: + items: + type: string + type: array + pool: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + user: + type: string + type: object + scaleIO: + properties: + fsType: + type: string + gateway: + type: string + protectionDomain: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + sslEnabled: + type: boolean + storageMode: + type: string + storagePool: + type: string + system: + type: string + volumeName: + type: string + type: object + secret: + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + optional: + type: boolean + secretName: + type: string + type: object + storageos: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + volumeName: + type: string + volumeNamespace: + type: string + type: object + vsphereVolume: + properties: + fsType: + type: string + storagePolicyID: + type: string + storagePolicyName: + type: string + volumePath: + type: string + type: object + type: object + type: array + type: object + status: + properties: + conditions: + items: + properties: + lastProbeTime: + type: string + lastTransitionTime: + type: string + message: + type: string + observedGeneration: + type: integer + reason: + type: string + status: + type: string + type: + type: string + type: object + type: array + containerStatuses: + items: + properties: + allocatedResources: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + allocatedResourcesStatus: + items: + properties: + name: + type: string + resources: + items: + properties: + health: + type: string + resourceID: + type: string + type: object + type: array + type: object + type: array + containerID: + type: string + image: + type: string + imageID: + type: string + lastState: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + name: + type: string + ready: + type: boolean + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartCount: + type: integer + started: + type: boolean + state: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + stopSignal: + type: string + user: + properties: + linux: + properties: + gid: + type: integer + supplementalGroups: + items: + type: integer + type: array + uid: + type: integer + type: object + type: object + volumeMounts: + items: + properties: + mountPath: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + type: object + type: array + type: object + type: array + ephemeralContainerStatuses: + items: + properties: + allocatedResources: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + allocatedResourcesStatus: + items: + properties: + name: + type: string + resources: + items: + properties: + health: + type: string + resourceID: + type: string + type: object + type: array + type: object + type: array + containerID: + type: string + image: + type: string + imageID: + type: string + lastState: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + name: + type: string + ready: + type: boolean + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartCount: + type: integer + started: + type: boolean + state: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + stopSignal: + type: string + user: + properties: + linux: + properties: + gid: + type: integer + supplementalGroups: + items: + type: integer + type: array + uid: + type: integer + type: object + type: object + volumeMounts: + items: + properties: + mountPath: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + type: object + type: array + type: object + type: array + hostIP: + type: string + hostIPs: + items: + properties: + ip: + type: string + type: object + type: array + initContainerStatuses: + items: + properties: + allocatedResources: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + allocatedResourcesStatus: + items: + properties: + name: + type: string + resources: + items: + properties: + health: + type: string + resourceID: + type: string + type: object + type: array + type: object + type: array + containerID: + type: string + image: + type: string + imageID: + type: string + lastState: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + name: + type: string + ready: + type: boolean + resources: + properties: + claims: + items: + properties: + name: + type: string + request: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + x-kubernetes-int-or-string: true + type: object + type: object + restartCount: + type: integer + started: + type: boolean + state: + properties: + running: + properties: + startedAt: + type: string + type: object + terminated: + properties: + containerID: + type: string + exitCode: + type: integer + finishedAt: + type: string + message: + type: string + reason: + type: string + signal: + type: integer + startedAt: + type: string + type: object + waiting: + properties: + message: + type: string + reason: + type: string + type: object + type: object + stopSignal: + type: string + user: + properties: + linux: + properties: + gid: + type: integer + supplementalGroups: + items: + type: integer + type: array + uid: + type: integer + type: object + type: object + volumeMounts: + items: + properties: + mountPath: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + type: object + type: array + type: object + type: array + message: + type: string + nominatedNodeName: + type: string + observedGeneration: + type: integer + phase: + type: string + podIP: + type: string + podIPs: + items: + properties: + ip: + type: string + type: object + type: array + qosClass: + type: string + reason: + type: string + resize: + type: string + resourceClaimStatuses: + items: + properties: + name: + type: string + resourceClaimName: + type: string + type: object + type: array + startTime: + type: string + type: object + type: object + replicas: + type: integer + resource: + properties: + cpu: + type: number + ephemeralStorage: + type: string + memory: + type: string + type: object + type: object + type: object + 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 + 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 9f129f31f5..ac7b4c0641 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