From 9b8c0da885dec3c6a5abe1f0980c41aecb339745 Mon Sep 17 00:00:00 2001 From: Gyula Fora Date: Fri, 21 Feb 2025 15:49:55 +0100 Subject: [PATCH 01/19] [release] Update version to 1.12-SNAPSHOT --- Dockerfile | 2 +- docs/config.toml | 4 ++-- examples/autoscaling/pom.xml | 2 +- examples/flink-beam-example/pom.xml | 2 +- examples/flink-sql-runner-example/pom.xml | 2 +- examples/kubernetes-client-examples/pom.xml | 4 ++-- flink-autoscaler-plugin-jdbc/pom.xml | 2 +- flink-autoscaler-standalone/pom.xml | 2 +- flink-autoscaler/pom.xml | 2 +- flink-kubernetes-docs/pom.xml | 2 +- flink-kubernetes-operator-api/pom.xml | 2 +- flink-kubernetes-operator/pom.xml | 2 +- flink-kubernetes-standalone/pom.xml | 2 +- flink-kubernetes-webhook/pom.xml | 2 +- helm/flink-kubernetes-operator/Chart.yaml | 4 ++-- pom.xml | 2 +- 16 files changed, 19 insertions(+), 19 deletions(-) diff --git a/Dockerfile b/Dockerfile index 6c4891f116..b9528af4f3 100644 --- a/Dockerfile +++ b/Dockerfile @@ -38,7 +38,7 @@ RUN cd /app/tools/license; mkdir jars; cd jars; \ FROM eclipse-temurin:${JAVA_VERSION}-jre-jammy ENV FLINK_HOME=/opt/flink ENV FLINK_PLUGINS_DIR=$FLINK_HOME/plugins -ENV OPERATOR_VERSION=1.11-SNAPSHOT +ENV OPERATOR_VERSION=1.12-SNAPSHOT ENV OPERATOR_JAR=flink-kubernetes-operator-$OPERATOR_VERSION-shaded.jar ENV WEBHOOK_JAR=flink-kubernetes-webhook-$OPERATOR_VERSION-shaded.jar ENV KUBERNETES_STANDALONE_JAR=flink-kubernetes-standalone-$OPERATOR_VERSION.jar diff --git a/docs/config.toml b/docs/config.toml index bc8eba590d..15aaacad83 100644 --- a/docs/config.toml +++ b/docs/config.toml @@ -34,11 +34,11 @@ pygmentsUseClasses = true # we change the version for the complete docs when forking of a release branch # etc. # The full version string as referenced in Maven (e.g. 1.2.1) - Version = "1.11-SNAPSHOT" + Version = "1.12-SNAPSHOT" # For stable releases, leave the bugfix version out (e.g. 1.2). For snapshot # release this should be the same as the regular version - VersionTitle = "1.11-SNAPSHOT" + VersionTitle = "1.12-SNAPSHOT" # The branch for this version of the Apache Flink Kubernetes Operator Branch = "main" diff --git a/examples/autoscaling/pom.xml b/examples/autoscaling/pom.xml index b14206ee7c..6da06a85e4 100644 --- a/examples/autoscaling/pom.xml +++ b/examples/autoscaling/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-kubernetes-operator-parent - 1.11-SNAPSHOT + 1.12-SNAPSHOT ../.. diff --git a/examples/flink-beam-example/pom.xml b/examples/flink-beam-example/pom.xml index e1379a2274..f3704c2409 100644 --- a/examples/flink-beam-example/pom.xml +++ b/examples/flink-beam-example/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-kubernetes-operator-parent - 1.11-SNAPSHOT + 1.12-SNAPSHOT ../.. diff --git a/examples/flink-sql-runner-example/pom.xml b/examples/flink-sql-runner-example/pom.xml index d2e34d463a..eb8431e960 100644 --- a/examples/flink-sql-runner-example/pom.xml +++ b/examples/flink-sql-runner-example/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-kubernetes-operator-parent - 1.11-SNAPSHOT + 1.12-SNAPSHOT ../.. diff --git a/examples/kubernetes-client-examples/pom.xml b/examples/kubernetes-client-examples/pom.xml index 5e99a1ee5a..c1ba5b2cf2 100644 --- a/examples/kubernetes-client-examples/pom.xml +++ b/examples/kubernetes-client-examples/pom.xml @@ -24,7 +24,7 @@ under the License. org.apache.flink flink-kubernetes-operator-parent - 1.11-SNAPSHOT + 1.12-SNAPSHOT ../.. @@ -40,7 +40,7 @@ under the License. org.apache.flink flink-kubernetes-operator - 1.11-SNAPSHOT + 1.12-SNAPSHOT compile diff --git a/flink-autoscaler-plugin-jdbc/pom.xml b/flink-autoscaler-plugin-jdbc/pom.xml index 7d7d95b79d..748e233092 100644 --- a/flink-autoscaler-plugin-jdbc/pom.xml +++ b/flink-autoscaler-plugin-jdbc/pom.xml @@ -23,7 +23,7 @@ under the License. flink-kubernetes-operator-parent org.apache.flink - 1.11-SNAPSHOT + 1.12-SNAPSHOT .. diff --git a/flink-autoscaler-standalone/pom.xml b/flink-autoscaler-standalone/pom.xml index 17fa7425a0..dc3337e3cd 100644 --- a/flink-autoscaler-standalone/pom.xml +++ b/flink-autoscaler-standalone/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-kubernetes-operator-parent - 1.11-SNAPSHOT + 1.12-SNAPSHOT .. diff --git a/flink-autoscaler/pom.xml b/flink-autoscaler/pom.xml index e733a64959..a7463d573b 100644 --- a/flink-autoscaler/pom.xml +++ b/flink-autoscaler/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-kubernetes-operator-parent - 1.11-SNAPSHOT + 1.12-SNAPSHOT .. diff --git a/flink-kubernetes-docs/pom.xml b/flink-kubernetes-docs/pom.xml index 46d37cfd64..6e44541b54 100644 --- a/flink-kubernetes-docs/pom.xml +++ b/flink-kubernetes-docs/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-kubernetes-operator-parent - 1.11-SNAPSHOT + 1.12-SNAPSHOT .. diff --git a/flink-kubernetes-operator-api/pom.xml b/flink-kubernetes-operator-api/pom.xml index 3dc3a27d5e..193f154bd2 100644 --- a/flink-kubernetes-operator-api/pom.xml +++ b/flink-kubernetes-operator-api/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-kubernetes-operator-parent - 1.11-SNAPSHOT + 1.12-SNAPSHOT .. diff --git a/flink-kubernetes-operator/pom.xml b/flink-kubernetes-operator/pom.xml index 07d6be88ae..b455fffa5a 100644 --- a/flink-kubernetes-operator/pom.xml +++ b/flink-kubernetes-operator/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-kubernetes-operator-parent - 1.11-SNAPSHOT + 1.12-SNAPSHOT .. diff --git a/flink-kubernetes-standalone/pom.xml b/flink-kubernetes-standalone/pom.xml index ec8284ff22..cc1a17d404 100644 --- a/flink-kubernetes-standalone/pom.xml +++ b/flink-kubernetes-standalone/pom.xml @@ -24,7 +24,7 @@ under the License. flink-kubernetes-operator-parent org.apache.flink - 1.11-SNAPSHOT + 1.12-SNAPSHOT .. diff --git a/flink-kubernetes-webhook/pom.xml b/flink-kubernetes-webhook/pom.xml index 307b4963c7..90b22ebb61 100644 --- a/flink-kubernetes-webhook/pom.xml +++ b/flink-kubernetes-webhook/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-kubernetes-operator-parent - 1.11-SNAPSHOT + 1.12-SNAPSHOT ../pom.xml diff --git a/helm/flink-kubernetes-operator/Chart.yaml b/helm/flink-kubernetes-operator/Chart.yaml index 39623432c4..6b0edda808 100644 --- a/helm/flink-kubernetes-operator/Chart.yaml +++ b/helm/flink-kubernetes-operator/Chart.yaml @@ -21,6 +21,6 @@ apiVersion: v2 name: flink-kubernetes-operator description: A Helm chart for the Apache Flink Kubernetes Operator type: application -version: 1.11-SNAPSHOT -appVersion: 1.11-SNAPSHOT +version: 1.12-SNAPSHOT +appVersion: 1.12-SNAPSHOT icon: https://flink.apache.org/img/logo/png/50/color_50.png diff --git a/pom.xml b/pom.xml index 660fae6246..004a3db10f 100644 --- a/pom.xml +++ b/pom.xml @@ -29,7 +29,7 @@ under the License. org.apache.flink flink-kubernetes-operator-parent - 1.11-SNAPSHOT + 1.12-SNAPSHOT Flink Kubernetes: pom From e636a0000c39ef2496ac5c27a3994e95e5e78ec7 Mon Sep 17 00:00:00 2001 From: Gyula Fora Date: Fri, 21 Feb 2025 15:58:08 +0100 Subject: [PATCH 02/19] [release] Docs, ci and config updates after release cut --- .asf.yaml | 1 + .github/workflows/docs.yaml | 6 +++--- docs/config.toml | 2 +- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/.asf.yaml b/.asf.yaml index 82c2ae19b0..42fbb1f707 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -21,6 +21,7 @@ github: release-1.8: {} release-1.9: {} release-1.10: {} + release-1.11: {} notifications: commits: commits@flink.apache.org diff --git a/.github/workflows/docs.yaml b/.github/workflows/docs.yaml index 936e31e8f7..1f776c97ee 100644 --- a/.github/workflows/docs.yaml +++ b/.github/workflows/docs.yaml @@ -28,8 +28,8 @@ jobs: matrix: branch: - main - - release-1.9 - release-1.10 + - release-1.11 steps: - uses: actions/checkout@v3 with: @@ -41,8 +41,8 @@ jobs: echo "flink_branch=${currentBranch}" echo "flink_branch=${currentBranch}" >> ${GITHUB_ENV} if [ "${currentBranch}" = "main" ]; then - echo "flink_alias=release-1.11" >> ${GITHUB_ENV} - elif [ "${currentBranch}" = "release-1.10" ]; then + echo "flink_alias=release-1.12" >> ${GITHUB_ENV} + elif [ "${currentBranch}" = "release-1.11" ]; then echo "flink_alias=stable" >> ${GITHUB_ENV} else echo "flink_alias=${currentBranch}" >> ${GITHUB_ENV} diff --git a/docs/config.toml b/docs/config.toml index 15aaacad83..dfdee584c3 100644 --- a/docs/config.toml +++ b/docs/config.toml @@ -63,8 +63,8 @@ pygmentsUseClasses = true ] PreviousDocs = [ + ["1.11", "https://nightlies.apache.org/flink/flink-kubernetes-operator-docs-release-1.11"], ["1.10", "https://nightlies.apache.org/flink/flink-kubernetes-operator-docs-release-1.10"], - ["1.9", "https://nightlies.apache.org/flink/flink-kubernetes-operator-docs-release-1.9"], ] [markup] From 042c27edc4f1b74b8d60868b4a8c788a453b315b Mon Sep 17 00:00:00 2001 From: Gyula Fora Date: Mon, 24 Feb 2025 09:47:10 +0100 Subject: [PATCH 03/19] [FLINK-37372] Fix infinite loop bug in savepoint error handling --- .../service/AbstractFlinkService.java | 2 +- .../operator/utils/ExceptionUtils.java | 60 +++++++++++++++++++ .../service/AbstractFlinkServiceTest.java | 51 +++++++++++++++- 3 files changed, 111 insertions(+), 2 deletions(-) create mode 100644 flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/ExceptionUtils.java diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkService.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkService.java index 4d7634ea4c..93cb1262fe 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkService.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkService.java @@ -51,6 +51,7 @@ import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; import org.apache.flink.kubernetes.operator.utils.EnvUtils; import org.apache.flink.kubernetes.operator.utils.EventRecorder; +import org.apache.flink.kubernetes.operator.utils.ExceptionUtils; import org.apache.flink.kubernetes.operator.utils.FlinkUtils; import org.apache.flink.runtime.client.JobStatusMessage; import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneClientHAServices; @@ -98,7 +99,6 @@ import org.apache.flink.runtime.webmonitor.handlers.JarUploadHeaders; import org.apache.flink.runtime.webmonitor.handlers.JarUploadResponseBody; import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; -import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FileUtils; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.Preconditions; diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/ExceptionUtils.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/ExceptionUtils.java new file mode 100644 index 0000000000..98a9393b4c --- /dev/null +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/ExceptionUtils.java @@ -0,0 +1,60 @@ +/* + * 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; + +import org.apache.flink.types.DeserializationException; +import org.apache.flink.util.SerializedThrowable; + +import java.util.Optional; + +/** Exception utils. * */ +public class ExceptionUtils { + + /** + * Based on the flink ExceptionUtils#findThrowableSerializedAware but fixes an infinite loop bug + * resulting from SerializedThrowable deserialization errors. + */ + public static Optional findThrowableSerializedAware( + Throwable throwable, Class searchType, ClassLoader classLoader) { + + if (throwable == null || searchType == null) { + return Optional.empty(); + } + + Throwable t = throwable; + while (t != null) { + if (searchType.isAssignableFrom(t.getClass())) { + return Optional.of(searchType.cast(t)); + } else if (t instanceof SerializedThrowable) { + var deserialized = ((SerializedThrowable) t).deserializeError(classLoader); + // This is the key part of the fix: + // The deserializeError method returns the same SerializedThrowable if it cannot + // deserialize it. Previously this is what caused the infinite loop. + t = + deserialized == t + ? new DeserializationException( + "Could not deserialize SerializedThrowable") + : deserialized; + } else { + t = t.getCause(); + } + } + + return Optional.empty(); + } +} diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkServiceTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkServiceTest.java index a26cbb461f..9082ff358a 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkServiceTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkServiceTest.java @@ -423,6 +423,52 @@ public void cancelJobWithSavepointUpgradeModeTest(boolean deleteAfterSavepoint) } } + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void savepointErrorTest(boolean deserializable) throws Exception { + var testingClusterClient = + new TestingClusterClient<>(configuration, TestUtils.TEST_DEPLOYMENT_NAME); + var savepointPath = "file:///path/of/svp-1"; + configuration.set(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointPath); + + var savepointErr = new SerializedThrowable(new Exception("sp test err")); + if (!deserializable) { + var cachedException = SerializedThrowable.class.getDeclaredField("cachedException"); + cachedException.setAccessible(true); + cachedException.set(savepointErr, null); + + var bytes = SerializedThrowable.class.getDeclaredField("serializedException"); + bytes.setAccessible(true); + bytes.set(savepointErr, new byte[] {1, 2, 3}); + } + + testingClusterClient.setStopWithSavepointFunction( + (jobID, advanceToEndOfEventTime, savepointDir) -> { + CompletableFuture result = new CompletableFuture<>(); + result.completeExceptionally(savepointErr); + return result; + }); + + var flinkService = new TestingService(testingClusterClient); + + FlinkDeployment deployment = TestUtils.buildApplicationCluster(); + deployment.getStatus().setJobManagerDeploymentStatus(JobManagerDeploymentStatus.READY); + JobStatus jobStatus = deployment.getStatus().getJobStatus(); + jobStatus.setJobId(JobID.generate().toHexString()); + jobStatus.setState(RUNNING); + ReconciliationUtils.updateStatusForDeployedSpec(deployment, new Configuration()); + + assertThrows( + UpgradeFailureException.class, + () -> + flinkService.cancelJob( + deployment, + SuspendMode.SAVEPOINT, + configManager.getObserveConfig(deployment), + true), + "sp test err"); + } + @ParameterizedTest @ValueSource(booleans = {true, false}) public void cancelJobWithDrainOnSavepointUpgradeModeTest(boolean drainOnSavepoint) @@ -737,11 +783,14 @@ private void runNativeSavepointFormatTest(boolean failAfterSavepointCompletes) testingClusterClient.setStopWithSavepointFormat( (id, formatType, savepointDir) -> { if (failAfterSavepointCompletes) { - stopWithSavepointFuture.completeExceptionally( + CompletableFuture result = new CompletableFuture<>(); + stopWithSavepointFuture.completeExceptionally(new Exception()); + result.completeExceptionally( new CompletionException( new SerializedThrowable( new StopWithSavepointStoppingException( savepointPath, jobID)))); + return result; } else { stopWithSavepointFuture.complete( new Tuple3<>(id, formatType, savepointDir)); From 9eb3c385b90a5a2f08376720f3204d1784981a0c Mon Sep 17 00:00:00 2001 From: Luca Castelli Date: Tue, 25 Feb 2025 02:27:50 -0500 Subject: [PATCH 04/19] [FLINK-37370] [Observer] Finished batch jobs throw ReconciliationException and never reach FINISHED in the CR --- .../operator/observer/SnapshotObserver.java | 1 - .../service/AbstractFlinkService.java | 6 ++++ .../operator/TestingFlinkService.java | 5 +++ .../deployment/ApplicationObserverTest.java | 31 +++++++++++++++++++ 4 files changed, 42 insertions(+), 1 deletion(-) diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/SnapshotObserver.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/SnapshotObserver.java index 4255b09a1b..8f079415a2 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/SnapshotObserver.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/SnapshotObserver.java @@ -441,7 +441,6 @@ private long getMaxCountForSnapshotType( } private void observeLatestCheckpoint(FlinkResourceContext ctx, String jobId) { - var status = ctx.getResource().getStatus(); var jobStatus = status.getJobStatus(); diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkService.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkService.java index 93cb1262fe..8728bc2ca4 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkService.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkService.java @@ -546,6 +546,12 @@ public Optional getLastCheckpoint(JobID jobId, Configuration conf) { try { latestCheckpointOpt = getCheckpointInfo(jobId, conf).f0; } catch (Exception e) { + if (e instanceof RestClientException + && e.getMessage() != null + && e.getMessage().contains("Checkpointing has not been enabled")) { + LOG.warn("Checkpointing not enabled for job {}", jobId, e); + return Optional.empty(); + } throw new ReconciliationException("Could not observe latest savepoint information", e); } 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 42e3ca243a..fec1dfa64e 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 @@ -138,6 +138,7 @@ public class TestingFlinkService extends AbstractFlinkService { @Getter private final Map savepointTriggers = new HashMap<>(); @Getter private final Map checkpointTriggers = new HashMap<>(); private final Map checkpointStats = new HashMap<>(); + @Setter private boolean throwCheckpointingDisabledError = false; @Getter private int desiredReplicas = 0; @Getter private int cancelJobCallCount = 0; @@ -593,6 +594,10 @@ public Optional getLastCheckpoint(JobID jobId, Configuration conf) { Optional, Optional> getCheckpointInfo(JobID jobId, Configuration conf) throws Exception { + if (throwCheckpointingDisabledError) { + throw new RestClientException( + "Checkpointing has not been enabled", HttpResponseStatus.BAD_REQUEST); + } if (checkpointInfo != null) { return checkpointInfo; diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/ApplicationObserverTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/ApplicationObserverTest.java index 83e7fad5fe..aa4898d827 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/ApplicationObserverTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/ApplicationObserverTest.java @@ -898,4 +898,35 @@ public void jobStatusNotOverwrittenWhenTerminal() throws Exception { org.apache.flink.api.common.JobStatus.FINISHED, deployment.getStatus().getJobStatus().getState()); } + + @Test + public void getLastCheckpointShouldHandleCheckpointingNotEnabled() throws Exception { + Configuration conf = + configManager.getDeployConfig(deployment.getMetadata(), deployment.getSpec()); + flinkService.submitApplicationCluster(deployment.getSpec().getJob(), conf, false); + bringToReadyStatus(deployment); + + deployment + .getStatus() + .getJobStatus() + .setState(org.apache.flink.api.common.JobStatus.FINISHED); + var jobs = flinkService.listJobs(); + var oldStatus = jobs.get(0).f1; + jobs.get(0).f1 = + new JobStatusMessage( + oldStatus.getJobId(), + oldStatus.getJobName(), + org.apache.flink.api.common.JobStatus.FINISHED, + oldStatus.getStartTime()); + + flinkService.setThrowCheckpointingDisabledError(true); + observer.observe(deployment, readyContext); + + assertEquals( + 0, + countErrorEvents( + EventRecorder.Reason.CheckpointError.name(), + deployment.getMetadata().getNamespace(), + "Checkpointing has not been enabled")); + } } From 48359a326f417081bcfad984cbdab578ef9c906d Mon Sep 17 00:00:00 2001 From: PB Date: Mon, 10 Mar 2025 22:57:03 +0530 Subject: [PATCH 05/19] [FLINK-33525] Move ImpulseSource to new Source API (#950) --- examples/autoscaling/pom.xml | 7 ++ .../autoscaling/LoadSimulationPipeline.java | 69 +++++++++++-------- 2 files changed, 49 insertions(+), 27 deletions(-) diff --git a/examples/autoscaling/pom.xml b/examples/autoscaling/pom.xml index 6da06a85e4..993b22dd45 100644 --- a/examples/autoscaling/pom.xml +++ b/examples/autoscaling/pom.xml @@ -45,6 +45,13 @@ under the License. ${flink.version} provided + + + org.apache.flink + flink-connector-datagen + ${flink.version} + + org.apache.flink flink-clients diff --git a/examples/autoscaling/src/main/java/autoscaling/LoadSimulationPipeline.java b/examples/autoscaling/src/main/java/autoscaling/LoadSimulationPipeline.java index 0ddb23023b..14865e1b91 100644 --- a/examples/autoscaling/src/main/java/autoscaling/LoadSimulationPipeline.java +++ b/examples/autoscaling/src/main/java/autoscaling/LoadSimulationPipeline.java @@ -18,12 +18,16 @@ package autoscaling; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.functions.RichFlatMapFunction; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.connector.datagen.source.DataGeneratorSource; +import org.apache.flink.connector.datagen.source.GeneratorFunction; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.DiscardingSink; -import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.util.Collector; import org.slf4j.Logger; @@ -60,6 +64,11 @@ public class LoadSimulationPipeline { private static final Logger LOG = LoggerFactory.getLogger(LoadSimulationPipeline.class); + // Number of impulses (records) emitted per sampling interval. + // This value determines how many records should be generated within each `samplingIntervalMs` + // period. + private static final int IMPULSES_PER_SAMPLING_INTERVAL = 10; + public static void main(String[] args) throws Exception { var env = StreamExecutionEnvironment.getExecutionEnvironment(); env.disableOperatorChaining(); @@ -74,8 +83,39 @@ public static void main(String[] args) throws Exception { for (String branch : maxLoadPerTask.split("\n")) { String[] taskLoads = branch.split(";"); + /* + * Creates an unbounded stream that continuously emits the constant value 42L. + * Flink's DataGeneratorSource with RateLimiterStrategy is used to control the emission rate. + * + * Emission Rate Logic: + * - The goal is to generate a fixed number of impulses per sampling interval. + * - `samplingIntervalMs` defines the duration of one sampling interval in milliseconds. + * - We define `IMPULSES_PER_SAMPLING_INTERVAL = 10`, meaning that for every sampling interval, + * exactly 10 impulses should be generated. + * + * To calculate the total number of records emitted per second: + * 1. Determine how many sampling intervals fit within one second: + * samplingIntervalsPerSecond = 1000 / samplingIntervalMs; + * 2. Multiply this by the number of impulses per interval to get the total rate: + * impulsesPerSecond = IMPULSES_PER_SAMPLING_INTERVAL * samplingIntervalsPerSecond; + * + * Example: + * - If `samplingIntervalMs = 500 ms` and `IMPULSES_PER_SAMPLING_INTERVAL = 10`: + * impulsesPerSecond = (1000 / 500) * 10 = 2 * 10 = 20 records per second. + */ DataStream stream = - env.addSource(new ImpulseSource(samplingIntervalMs)).name("ImpulseSource"); + env.fromSource( + new DataGeneratorSource<>( + (GeneratorFunction) + (index) -> 42L, // Emits constant value 42 + Long.MAX_VALUE, // Unbounded stream + RateLimiterStrategy.perSecond( + (1000.0 / samplingIntervalMs) + * IMPULSES_PER_SAMPLING_INTERVAL), // Controls + // rate + Types.LONG), + WatermarkStrategy.noWatermarks(), + "ImpulseSource"); for (String load : taskLoads) { double maxLoad = Double.parseDouble(load); @@ -97,31 +137,6 @@ public static void main(String[] args) throws Exception { + ")"); } - private static class ImpulseSource implements SourceFunction { - private final int maxSleepTimeMs; - volatile boolean canceled; - - public ImpulseSource(int samplingInterval) { - this.maxSleepTimeMs = samplingInterval / 10; - } - - @Override - public void run(SourceContext sourceContext) throws Exception { - while (!canceled) { - synchronized (sourceContext.getCheckpointLock()) { - sourceContext.collect(42L); - } - // Provide an impulse to keep the load simulation active - Thread.sleep(maxSleepTimeMs); - } - } - - @Override - public void cancel() { - canceled = true; - } - } - private static class LoadSimulationFn extends RichFlatMapFunction { private final double maxLoad; From a1c510a25d9f7c10ff04eb36bb8bbaae9a33e44a Mon Sep 17 00:00:00 2001 From: schongloo Date: Tue, 11 Mar 2025 08:21:49 -0700 Subject: [PATCH 06/19] First code batch for FLIP-503 --- .../api/FlinkBlueGreenDeployment.java | 41 + .../api/bluegreen/DeploymentType.java | 32 + .../spec/FlinkBlueGreenDeploymentSpec.java | 36 + .../api/spec/FlinkDeploymentTemplateSpec.java | 55 + .../status/FlinkBlueGreenDeploymentState.java | 26 + .../FlinkBlueGreenDeploymentStatus.java | 55 + .../operator/api/utils/SpecUtils.java | 22 + .../kubernetes/operator/FlinkOperator.java | 8 + .../FlinkBlueGreenDeploymentController.java | 589 + .../controller/FlinkBlueGreenDeployments.java | 69 + .../metrics/OperatorJosdkMetrics.java | 3 + .../flink/kubernetes/operator/TestUtils.java | 15 +- .../operator/TestingFlinkService.java | 11 +- ...linkBlueGreenDeploymentControllerTest.java | 453 + ...ingFlinkBlueGreenDeploymentController.java | 84 + ...uegreendeployments.flink.apache.org-v1.yml | 10471 ++++++++++++++++ pom.xml | 1 + 17 files changed, 11966 insertions(+), 5 deletions(-) create mode 100644 flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/FlinkBlueGreenDeployment.java create mode 100644 flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/DeploymentType.java create mode 100644 flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/FlinkBlueGreenDeploymentSpec.java create mode 100644 flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/FlinkDeploymentTemplateSpec.java create mode 100644 flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/status/FlinkBlueGreenDeploymentState.java create mode 100644 flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/status/FlinkBlueGreenDeploymentStatus.java create mode 100644 flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentController.java create mode 100644 flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeployments.java create mode 100644 flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentControllerTest.java create mode 100644 flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/TestingFlinkBlueGreenDeploymentController.java create mode 100644 helm/flink-kubernetes-operator/crds/flinkbluegreendeployments.flink.apache.org-v1.yml 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/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..87b7408a0e --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/DeploymentType.java @@ -0,0 +1,32 @@ +/* + * 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. */ +public enum DeploymentType { + BLUE, + GREEN; + + public static DeploymentType fromDeployment(FlinkDeployment flinkDeployment) { + String typeAnnotation = + flinkDeployment.getMetadata().getLabels().get(DeploymentType.class.getSimpleName()); + return DeploymentType.valueOf(typeAnnotation); + } +} 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..4a84955bcb --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/FlinkDeploymentTemplateSpec.java @@ -0,0 +1,55 @@ +/* + * 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("deploymentDeletionDelaySec") + private int deploymentDeletionDelaySec; + + @JsonProperty("maxNumRetries") + private int maxNumRetries; + + @JsonProperty("reconciliationReschedulingIntervalMs") + private int reconciliationReschedulingIntervalMs; + + @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..5681ee7697 --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/status/FlinkBlueGreenDeploymentState.java @@ -0,0 +1,26 @@ +/* + * 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 { + ACTIVE_BLUE, + ACTIVE_GREEN, + TRANSITIONING_TO_BLUE, + TRANSITIONING_TO_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..1cf75f02ee --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/status/FlinkBlueGreenDeploymentStatus.java @@ -0,0 +1,55 @@ +/* + * 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 for A. */ + private String lastReconciledSpec; + + /** Timestamp of last reconciliation (either A or B). */ + private Long lastReconciledTimestamp; + + /** Current number of retries. */ + private int numRetries; + + /** Information about the TaskManagers for the scale subresource. */ + private TaskManagerInfo taskManager; +} 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..4488854509 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,28 @@ public static String writeSpecWithMeta( } } + public static String serializeObject(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 deserializeObject(String serialized, String wrapperKey, Class valueType) + throws JsonProcessingException { + 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 4bd2836f7a..3d293e89dc 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; @@ -242,6 +243,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(); @@ -262,6 +269,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/controller/FlinkBlueGreenDeploymentController.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentController.java new file mode 100644 index 0000000000..486532aa75 --- /dev/null +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentController.java @@ -0,0 +1,589 @@ +/* + * 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.JobID; +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.DeploymentType; +import org.apache.flink.kubernetes.operator.api.lifecycle.ResourceLifecycleState; +import org.apache.flink.kubernetes.operator.api.spec.FlinkBlueGreenDeploymentSpec; +import org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentState; +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.service.FlinkResourceContextFactory; +import org.apache.flink.util.Preconditions; + +import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableSet; + +import com.fasterxml.jackson.core.JsonProcessingException; +import io.fabric8.kubernetes.api.model.Event; +import io.fabric8.kubernetes.api.model.ObjectMeta; +import io.fabric8.kubernetes.api.model.OwnerReference; +import io.fabric8.kubernetes.api.model.StatusDetails; +import io.fabric8.kubernetes.client.dsl.PodResource; +import io.fabric8.kubernetes.client.dsl.Resource; +import io.javaoperatorsdk.operator.api.config.informer.InformerConfiguration; +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.EventSourceInitializer; +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.time.Instant; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** Controller that runs the main reconcile loop for Flink Blue/Green deployments. */ +@ControllerConfiguration +public class FlinkBlueGreenDeploymentController + implements Reconciler, + EventSourceInitializer { + + private static final Logger LOG = LoggerFactory.getLogger(FlinkDeploymentController.class); + private static final int DEFAULT_MAX_NUM_RETRIES = 5; + private static final int DEFAULT_RECONCILIATION_RESCHEDULING_INTERVAL_MS = 15000; + + private final FlinkResourceContextFactory ctxFactory; + + public FlinkBlueGreenDeploymentController(FlinkResourceContextFactory ctxFactory) { + this.ctxFactory = ctxFactory; + } + + @Override + public Map prepareEventSources( + EventSourceContext eventSourceContext) { + InformerConfiguration flinkDeploymentInformerConfig = + InformerConfiguration.from(FlinkDeployment.class, eventSourceContext) + .withSecondaryToPrimaryMapper(Mappers.fromOwnerReference()) + .withNamespacesInheritedFromController(eventSourceContext) + .followNamespaceChanges(true) + .build(); + + return EventSourceInitializer.nameEventSources( + new InformerEventSource<>(flinkDeploymentInformerConfig, eventSourceContext)); + } + + @Override + public UpdateControl reconcile( + FlinkBlueGreenDeployment flinkBlueGreenDeployment, + Context josdkContext) + throws Exception { + + FlinkBlueGreenDeploymentStatus deploymentStatus = flinkBlueGreenDeployment.getStatus(); + + if (deploymentStatus == null) { + deploymentStatus = new FlinkBlueGreenDeploymentStatus(); + deploymentStatus.setLastReconciledSpec( + SpecUtils.serializeObject(flinkBlueGreenDeployment.getSpec(), "spec")); + return initiateDeployment( + flinkBlueGreenDeployment, + deploymentStatus, + DeploymentType.BLUE, + FlinkBlueGreenDeploymentState.TRANSITIONING_TO_BLUE, + null, + josdkContext, + true); + } else { + FlinkBlueGreenDeployments deployments = + FlinkBlueGreenDeployments.fromSecondaryResources(josdkContext); + + // TODO: if a new deployment request comes while in the middle of a transition it's + // currently ignored, but the new spec remains changed, should we roll it back? + // TODO: if we choose to leave a previously failed deployment 'running' for debug + // purposes, + // we should flag it somehow as 'ROLLED_BACK' to signal that it can be overriden by a + // new deployment attempt. + switch (deploymentStatus.getBlueGreenState()) { + case ACTIVE_BLUE: + return checkAndInitiateDeployment( + flinkBlueGreenDeployment, + deployments, + deploymentStatus, + DeploymentType.BLUE, + josdkContext); + case ACTIVE_GREEN: + return checkAndInitiateDeployment( + flinkBlueGreenDeployment, + deployments, + deploymentStatus, + DeploymentType.GREEN, + josdkContext); + case TRANSITIONING_TO_BLUE: + return monitorTransition( + flinkBlueGreenDeployment, + deployments, + deploymentStatus, + DeploymentType.GREEN, + josdkContext); + case TRANSITIONING_TO_GREEN: + return monitorTransition( + flinkBlueGreenDeployment, + deployments, + deploymentStatus, + DeploymentType.BLUE, + josdkContext); + default: + return UpdateControl.noUpdate(); + } + } + } + + private UpdateControl monitorTransition( + FlinkBlueGreenDeployment bgDeployment, + FlinkBlueGreenDeployments deployments, + FlinkBlueGreenDeploymentStatus deploymentStatus, + DeploymentType currentDeploymentType, + Context josdkContext) { + + var nextState = FlinkBlueGreenDeploymentState.ACTIVE_BLUE; + FlinkDeployment currentDeployment; + FlinkDeployment nextDeployment; + + if (DeploymentType.BLUE == currentDeploymentType) { + nextState = FlinkBlueGreenDeploymentState.ACTIVE_GREEN; + currentDeployment = deployments.getFlinkDeploymentBlue(); + nextDeployment = deployments.getFlinkDeploymentGreen(); + } else { + currentDeployment = deployments.getFlinkDeploymentGreen(); + nextDeployment = deployments.getFlinkDeploymentBlue(); + } + + Preconditions.checkNotNull( + nextDeployment, + "Target Dependent Deployment resource not found. Blue/Green deployment name: " + + bgDeployment.getMetadata().getName() + + ", current deployment type: " + + currentDeploymentType); + + if (isDeploymentReady(nextDeployment, josdkContext, deploymentStatus)) { + return deleteAndFinalize( + bgDeployment, + deploymentStatus, + currentDeploymentType, + josdkContext, + currentDeployment, + nextState); + } else { + // This phase requires rescheduling the reconciliation because the pod initialization + // could get stuck + // (e.g. waiting for resources) + // TODO: figure out the course of action for error/failure cases + // TODO: should we reschedule indefinitely? Should this case check the "should abort" + // condition? + + int maxNumRetries = bgDeployment.getSpec().getTemplate().getMaxNumRetries(); + if (maxNumRetries <= 0) { + maxNumRetries = DEFAULT_MAX_NUM_RETRIES; + } + + if (deploymentStatus.getNumRetries() >= maxNumRetries) { + // ABORT + return patchStatusUpdateControl(bgDeployment, deploymentStatus, null, null, false); + } else { + // RETRY + deploymentStatus.setNumRetries(deploymentStatus.getNumRetries() + 1); + + LOG.info("Deployment " + nextDeployment.getMetadata().getName() + " not ready yet"); + int reconciliationReschedInterval = + bgDeployment + .getSpec() + .getTemplate() + .getReconciliationReschedulingIntervalMs(); + if (reconciliationReschedInterval <= 0) { + reconciliationReschedInterval = DEFAULT_RECONCILIATION_RESCHEDULING_INTERVAL_MS; + } + + return patchStatusUpdateControl(bgDeployment, deploymentStatus, null, null, false) + .rescheduleAfter(reconciliationReschedInterval); + } + } + } + + private UpdateControl deleteAndFinalize( + FlinkBlueGreenDeployment bgDeployment, + FlinkBlueGreenDeploymentStatus deploymentStatus, + DeploymentType currentDeploymentType, + Context josdkContext, + FlinkDeployment currentDeployment, + FlinkBlueGreenDeploymentState nextState) { + + if (currentDeployment != null) { + deleteDeployment(currentDeployment, josdkContext); + return UpdateControl.noUpdate(); + } else { + deploymentStatus.setLastReconciledSpec( + SpecUtils.serializeObject(bgDeployment.getSpec(), "spec")); + + // TODO: set the last reconciled spec here + + return patchStatusUpdateControl( + bgDeployment, deploymentStatus, nextState, JobStatus.RUNNING, false); + } + } + + private UpdateControl checkAndInitiateDeployment( + FlinkBlueGreenDeployment flinkBlueGreenDeployment, + FlinkBlueGreenDeployments deployments, + FlinkBlueGreenDeploymentStatus deploymentStatus, + DeploymentType currentDeploymentType, + Context josdkContext) + throws Exception { + + if (hasSpecChanged( + flinkBlueGreenDeployment.getSpec(), deploymentStatus, currentDeploymentType)) { + + FlinkDeployment currentFlinkDeployment = + DeploymentType.BLUE == currentDeploymentType + ? deployments.getFlinkDeploymentBlue() + : deployments.getFlinkDeploymentGreen(); + + // TODO: if the current deployment is not STABLE/RUNNING we'll revert to the last stable + // spec, report the error and abort + if (isDeploymentReady(currentFlinkDeployment, josdkContext, deploymentStatus)) { + + DeploymentType nextDeploymentType = DeploymentType.BLUE; + FlinkBlueGreenDeploymentState nextState = + FlinkBlueGreenDeploymentState.TRANSITIONING_TO_BLUE; + FlinkResourceContext resourceContext = + ctxFactory.getResourceContext(currentFlinkDeployment, josdkContext); + + // TODO: this operation is already done by hasSpecChanged() above, dedup later + String serializedSpec = + SpecUtils.serializeObject(flinkBlueGreenDeployment.getSpec(), "spec"); + + // Updating status + if (DeploymentType.BLUE == currentDeploymentType) { + nextState = FlinkBlueGreenDeploymentState.TRANSITIONING_TO_GREEN; + nextDeploymentType = DeploymentType.GREEN; + } + + Savepoint lastCheckpoint = configureSavepoint(resourceContext); + + return initiateDeployment( + flinkBlueGreenDeployment, + deploymentStatus, + nextDeploymentType, + nextState, + lastCheckpoint, + josdkContext, + false); + } else { + // If the current running FlinkDeployment is not in RUNNING/STABLE, + // we flag this Blue/Green as FAILING + if (deploymentStatus.getJobStatus().getState() != JobStatus.FAILING) { + return patchStatusUpdateControl( + flinkBlueGreenDeployment, + deploymentStatus, + null, + JobStatus.FAILING, + false); + } + } + } + + return UpdateControl.noUpdate(); + } + + public void logPotentialWarnings( + FlinkDeployment flinkDeployment, + Context josdkContext, + long lastReconciliationTimestamp) { + // Event reason constants + // https://github.com/kubernetes/kubernetes/blob/master/pkg/kubelet/events/event.go + Set badEventPatterns = + ImmutableSet.of( + "FAIL", "EXCEPTION", "BACKOFF", "ERROR", "EVICTION", "KILL", "EXCEED"); + Set goodPodPhases = ImmutableSet.of("PENDING", "RUNNING"); + + Set podPhases = + getDeploymentPods(josdkContext, flinkDeployment) + .map(p -> p.get().getStatus().getPhase().toUpperCase()) + .collect(Collectors.toSet()); + + podPhases.removeAll(goodPodPhases); + + if (!podPhases.isEmpty()) { + LOG.warn("Deployment not healthy, some Pods have the following status: " + podPhases); + } + + List badEvents = + josdkContext + .getClient() + .v1() + .events() + .inNamespace(flinkDeployment.getMetadata().getNamespace()) + .resources() + .map(Resource::item) + .filter(e -> !e.getType().equalsIgnoreCase("NORMAL")) + .filter( + e -> + e.getInvolvedObject() + .getName() + .contains(flinkDeployment.getMetadata().getName())) + .filter( + e -> + Instant.parse(e.getLastTimestamp()).toEpochMilli() + > lastReconciliationTimestamp) + .filter( + e -> + badEventPatterns.stream() + .anyMatch( + p -> + e.getReason() + .toUpperCase() + .contains(p))) + .collect(Collectors.toList()); + + if (!badEvents.isEmpty()) { + LOG.warn("Bad events detected: " + badEvents); + } + } + + private static Savepoint configureSavepoint( + FlinkResourceContext resourceContext) throws Exception { + // TODO: if the user specified an initialSavepointPath, use it and skip this + Optional lastCheckpoint = + resourceContext + .getFlinkService() + .getLastCheckpoint( + JobID.fromHexString( + resourceContext + .getResource() + .getStatus() + .getJobStatus() + .getJobId()), + resourceContext.getObserveConfig()); + + // TODO 1: check the last CP age with the logic from + // AbstractJobReconciler.changeLastStateIfCheckpointTooOld + // TODO 2: if no checkpoint is available, take a savepoint + if (lastCheckpoint.isEmpty()) { + throw new IllegalStateException( + "Last Checkpoint for Job " + + resourceContext.getResource().getMetadata().getName() + + " not found!"); + } + return lastCheckpoint.get(); + } + + private UpdateControl initiateDeployment( + FlinkBlueGreenDeployment flinkBlueGreenDeployment, + FlinkBlueGreenDeploymentStatus deploymentStatus, + DeploymentType nextDeploymentType, + FlinkBlueGreenDeploymentState nextState, + Savepoint lastCheckpoint, + Context josdkContext, + boolean isFirstDeployment) + throws JsonProcessingException { + deploy( + flinkBlueGreenDeployment, + nextDeploymentType, + lastCheckpoint, + josdkContext, + isFirstDeployment); + return patchStatusUpdateControl( + flinkBlueGreenDeployment, deploymentStatus, nextState, null, isFirstDeployment); + } + + private boolean isDeploymentReady( + FlinkDeployment deployment, + Context josdkContext, + FlinkBlueGreenDeploymentStatus deploymentStatus) { + if (ResourceLifecycleState.STABLE == deployment.getStatus().getLifecycleState() + && JobStatus.RUNNING == deployment.getStatus().getJobStatus().getState()) { + // TODO: verify, e.g. will pods be "pending" after the FlinkDeployment is RUNNING and + // STABLE? + int notRunningPods = + (int) + getDeploymentPods(josdkContext, deployment) + .filter( + p -> + !p.get() + .getStatus() + .getPhase() + .equalsIgnoreCase("RUNNING")) + .count(); + + if (notRunningPods > 0) { + LOG.warn("Waiting for " + notRunningPods + " Pods to transition to RUNNING status"); + } + + return notRunningPods == 0; + } + + logPotentialWarnings( + deployment, josdkContext, deploymentStatus.getLastReconciledTimestamp()); + return false; + } + + private static Stream getDeploymentPods( + Context josdkContext, FlinkDeployment deployment) { + var namespace = deployment.getMetadata().getNamespace(); + var deploymentName = deployment.getMetadata().getName(); + + return josdkContext + .getClient() + .pods() + .inNamespace(namespace) + .withLabel("app", deploymentName) + .resources(); + } + + private boolean hasSpecChanged( + FlinkBlueGreenDeploymentSpec newSpec, + FlinkBlueGreenDeploymentStatus deploymentStatus, + DeploymentType deploymentType) { + + String lastReconciledSpec = deploymentStatus.getLastReconciledSpec(); + + return !lastReconciledSpec.equals(SpecUtils.serializeObject(newSpec, "spec")); + } + + private UpdateControl patchStatusUpdateControl( + FlinkBlueGreenDeployment flinkBlueGreenDeployment, + FlinkBlueGreenDeploymentStatus deploymentStatus, + FlinkBlueGreenDeploymentState deploymentState, + JobStatus jobState, + boolean isFirstDeployment) { + if (deploymentState != null) { + deploymentStatus.setBlueGreenState(deploymentState); + } + + if (jobState != null) { + deploymentStatus.getJobStatus().setState(jobState); + } + + deploymentStatus.setLastReconciledTimestamp(System.currentTimeMillis()); + flinkBlueGreenDeployment.setStatus(deploymentStatus); + return UpdateControl.patchStatus(flinkBlueGreenDeployment); + } + + private void deploy( + FlinkBlueGreenDeployment bgDeployment, + DeploymentType deploymentType, + Savepoint lastCheckpoint, + Context josdkContext, + boolean isFirstDeployment) + throws JsonProcessingException { + ObjectMeta bgMeta = bgDeployment.getMetadata(); + + // Deployment + FlinkDeployment flinkDeployment = new FlinkDeployment(); + flinkDeployment.setApiVersion("flink.apache.org/v1beta1"); + flinkDeployment.setKind("FlinkDeployment"); + FlinkBlueGreenDeploymentSpec spec = bgDeployment.getSpec(); + + String childDeploymentName = + bgMeta.getName() + "-" + deploymentType.toString().toLowerCase(); + + FlinkBlueGreenDeploymentSpec adjustedSpec = + adjustNameReferences( + spec, + bgMeta.getName(), + childDeploymentName, + "spec", + FlinkBlueGreenDeploymentSpec.class); + + if (lastCheckpoint != null) { + String location = lastCheckpoint.getLocation().replace("file:", ""); + LOG.info("Using 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.getClass().getSimpleName(), deploymentType.toString())); + flinkDeployment.setMetadata(flinkDeploymentMeta); + + // Deploy + josdkContext.getClient().resource(flinkDeployment).createOrReplace(); + } + + private static void deleteDeployment( + FlinkDeployment currentDeployment, Context josdkContext) { + // TODO: This gets called multiple times, check to see if it's already in a TERMINATING + // state + // (or only execute if RUNNING) + List deletedStatus = + josdkContext + .getClient() + .resources(FlinkDeployment.class) + .inNamespace(currentDeployment.getMetadata().getNamespace()) + .withName(currentDeployment.getMetadata().getName()) + .delete(); + + boolean deleted = + deletedStatus.size() == 1 + && deletedStatus.get(0).getKind().equals("FlinkDeployment"); + if (!deleted) { + LOG.info("Deployment not deleted, will retry"); + } else { + LOG.info("Deployment deleted!"); + } + } + + private 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; + } + + private static T adjustNameReferences( + T spec, + String deploymentName, + String childDeploymentName, + String wrapperKey, + Class valueType) + throws JsonProcessingException { + String serializedSpec = SpecUtils.serializeObject(spec, wrapperKey); + String replacedSerializedSpec = serializedSpec.replace(deploymentName, childDeploymentName); + return SpecUtils.deserializeObject(replacedSerializedSpec, wrapperKey, valueType); + } + + public static void logAndThrow(String message) { + LOG.error(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..69e9f084b7 --- /dev/null +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeployments.java @@ -0,0 +1,69 @@ +/* + * 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 +class FlinkBlueGreenDeployments { + private FlinkDeployment flinkDeploymentBlue; + private FlinkDeployment flinkDeploymentGreen; + + 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/metrics/OperatorJosdkMetrics.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/metrics/OperatorJosdkMetrics.java index 5350e3cd76..bb14cd986c 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; @@ -211,6 +212,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/test/java/org/apache/flink/kubernetes/operator/TestUtils.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/TestUtils.java index 6af375edb3..97eab38e59 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 @@ -74,6 +74,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; @@ -478,7 +479,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 fec1dfa64e..fa9a63d782 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 @@ -580,10 +580,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..1832df166b --- /dev/null +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentControllerTest.java @@ -0,0 +1,453 @@ +/* + * 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.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.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +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 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.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 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("org.apache.flink.kubernetes.operator.TestUtils#flinkVersions") + public void verifyBasicDeployment(FlinkVersion flinkVersion) throws Exception { + var blueGreenDeployment = + buildSessionCluster(TEST_DEPLOYMENT_NAME, TEST_NAMESPACE, flinkVersion); + + // 1. Initiate the Green deployment + var bgSpecBefore = blueGreenDeployment.getSpec(); + Long minReconciliationTs = System.currentTimeMillis() - 1; + var rs = reconcile(blueGreenDeployment); + + assertSpec(rs, minReconciliationTs); + + // check the status (reconciled spec, reconciled ts, a/b state) + assertEquals( + FlinkBlueGreenDeploymentState.TRANSITIONING_TO_BLUE, + rs.reconciledStatus.getBlueGreenState()); + assertNotNull(rs.reconciledStatus.getLastReconciledSpec()); + assertNull(rs.reconciledStatus.getJobStatus().getState()); + + var flinkDeploymentList = getFlinkDeployments(); + assertEquals(1, flinkDeploymentList.size()); + var deploymentA = flinkDeploymentList.get(0); + + verifyOwnerReferences(rs.deployment, deploymentA); + + simulateSubmitAndSuccessfulJobStart(deploymentA); + + // 2. Finalize the Green deployment + minReconciliationTs = System.currentTimeMillis() - 1; + rs = reconcile(rs.deployment); + + assertSpec(rs, minReconciliationTs); + + assertEquals( + SpecUtils.serializeObject(bgSpecBefore, "spec"), + rs.reconciledStatus.getLastReconciledSpec()); + assertEquals( + FlinkBlueGreenDeploymentState.ACTIVE_BLUE, rs.reconciledStatus.getBlueGreenState()); + assertEquals(JobStatus.RUNNING, rs.reconciledStatus.getJobStatus().getState()); + } + + @ParameterizedTest + @MethodSource("org.apache.flink.kubernetes.operator.TestUtils#flinkVersions") + public void verifyBasicTransition(FlinkVersion flinkVersion) throws Exception { + var blueGreenDeployment = + buildSessionCluster(TEST_DEPLOYMENT_NAME, TEST_NAMESPACE, flinkVersion); + + // 1. Initiate the Green deployment + var rs = reconcile(blueGreenDeployment); + + // 2. Finalize the Green deployment + simulateSubmitAndSuccessfulJobStart(getFlinkDeployments().get(0)); + rs = reconcile(rs.deployment); + + // Verify noUpdate if reconciliation is triggered without a spec change + var rs2 = reconcile(rs.deployment); + assertTrue(rs2.updateControl.isNoUpdate()); + + // 3. Simulate a change in the spec to trigger a Blue deployment + simulateChangeInSpec(rs.deployment); + + // 4. Initiate the Blue deployment + var bgUpdatedSpec = rs.deployment.getSpec(); + Long minReconciliationTs = System.currentTimeMillis() - 1; + rs = reconcile(rs.deployment); + var flinkDeployments = getFlinkDeployments(); + var blueDeploymentName = flinkDeployments.get(1).getMetadata().getName(); + + assertSpec(rs, minReconciliationTs); + + assertEquals(2, flinkDeployments.size()); + assertNull(flinkDeployments.get(0).getSpec().getJob().getInitialSavepointPath()); + assertNotNull(flinkDeployments.get(1).getSpec().getJob().getInitialSavepointPath()); + + assertEquals( + FlinkBlueGreenDeploymentState.TRANSITIONING_TO_GREEN, + rs.reconciledStatus.getBlueGreenState()); + assertNotNull(rs.reconciledStatus.getLastReconciledSpec()); + + // 5. New Blue deployment successfully started + simulateSuccessfulJobStart(getFlinkDeployments().get(1)); + rs2 = reconcile(rs.deployment); + assertTrue(rs2.updateControl.isNoUpdate()); + + flinkDeployments = getFlinkDeployments(); + assertEquals(1, flinkDeployments.size()); + assertEquals(blueDeploymentName, flinkDeployments.get(0).getMetadata().getName()); + + // 6. Old Green deployment deleted, Blue promoted to Green + minReconciliationTs = System.currentTimeMillis() - 1; + rs = reconcile(rs.deployment); + + // Spec should still be the new one + assertSpec(rs, minReconciliationTs); + + assertNotNull(rs.reconciledStatus.getLastReconciledSpec()); + assertEquals( + SpecUtils.serializeObject(bgUpdatedSpec, "spec"), + rs.reconciledStatus.getLastReconciledSpec()); + assertEquals( + FlinkBlueGreenDeploymentState.ACTIVE_GREEN, + rs.reconciledStatus.getBlueGreenState()); + assertEquals(JobStatus.RUNNING, rs.reconciledStatus.getJobStatus().getState()); + assertTrue(rs.updateControl.isPatchStatus()); + } + + @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); + var originalSpec = blueGreenDeployment.getSpec(); + + // 1. Initiate the Green deployment + var rs = reconcile(blueGreenDeployment); + + // 2. Finalize the Green deployment + simulateSubmitAndSuccessfulJobStart(getFlinkDeployments().get(0)); + rs = reconcile(rs.deployment); + + // 3. Simulate a change in the spec to trigger a Blue deployment + simulateChangeInSpec(rs.deployment); + + // TODO: simulate a failure in the running deployment + simulateJobFailure(getFlinkDeployments().get(0)); + + // 4. Initiate the Blue deployment + var minReconciliationTs = System.currentTimeMillis() - 1; + rs = reconcile(rs.deployment); + + assertSpec(rs, minReconciliationTs); + + // 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()); + + // 5. 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); + var originalSpec = blueGreenDeployment.getSpec(); + + // Overriding the maxNumRetries and Reschedule Interval + var maxNumRetries = 2; + var reconciliationReschedulingIntervalMs = 5000; + blueGreenDeployment.getSpec().getTemplate().setMaxNumRetries(maxNumRetries); + blueGreenDeployment + .getSpec() + .getTemplate() + .setReconciliationReschedulingIntervalMs(reconciliationReschedulingIntervalMs); + + // 1. Initiate the Green deployment + var rs = reconcile(blueGreenDeployment); + + // 2. Finalize the Green deployment + simulateSubmitAndSuccessfulJobStart(getFlinkDeployments().get(0)); + rs = reconcile(rs.deployment); + + // 3. Simulate a change in the spec to trigger a Blue deployment + simulateChangeInSpec(rs.deployment); + + // 4. Initiate the Blue deployment + rs = reconcile(rs.deployment); + + // 4a. Simulating the Blue deployment doesn't start correctly (status will remain the same) + // Asserting the status retry count is incremented by 1 + long lastTs = System.currentTimeMillis(); + for (int i = 1; i <= maxNumRetries; i++) { + Thread.sleep(1); + rs = reconcile(rs.deployment); + assertTrue(rs.updateControl.isPatchStatus()); + assertFalse(rs.updateControl.isUpdateResource()); + assertTrue(rs.updateControl.getScheduleDelay().isPresent()); + assertEquals( + reconciliationReschedulingIntervalMs, + rs.updateControl.getScheduleDelay().get()); + assertEquals(i, rs.reconciledStatus.getNumRetries()); + assertTrue(rs.reconciledStatus.getLastReconciledTimestamp() > lastTs); + lastTs = rs.reconciledStatus.getLastReconciledTimestamp(); + System.out.println(); + } + + // 4b. After the retries are exhausted + var minReconciliationTs = System.currentTimeMillis() - 1; + rs = reconcile(rs.deployment); + + assertSpec(rs, minReconciliationTs); + + // The first job should be running OK, the second should be left in its broken state + 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()); + assertNull(flinkDeployments.get(1).getStatus().getJobStatus().getState()); + assertEquals( + ReconciliationState.UPGRADING, + flinkDeployments.get(1).getStatus().getReconciliationStatus().getState()); + } + + private static void assertSpec( + TestingFlinkBlueGreenDeploymentController.BlueGreenReconciliationResult rs, + long minReconciliationTs) { + assertTrue(rs.updateControl.isPatchStatus()); + assertTrue(minReconciliationTs < rs.reconciledStatus.getLastReconciledTimestamp()); + } + + private void simulateChangeInSpec(FlinkBlueGreenDeployment blueGreenDeployment) { + FlinkDeploymentSpec spec = blueGreenDeployment.getSpec().getTemplate().getSpec(); + spec.getFlinkConfiguration().put(TaskManagerOptions.NUM_TASK_SLOTS.key(), "10"); + blueGreenDeployment.getSpec().getTemplate().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(), + updateControl.isNoUpdate() ? null : updateControl.getResource().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, "/tmp/savepoint"); + 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).createOrReplace(); + } + + private void simulateJobFailure(FlinkDeployment deployment) { + deployment.getStatus().getJobStatus().setState(JobStatus.RECONCILING); + deployment.getStatus().getReconciliationStatus().setState(ReconciliationState.UPGRADING); + kubernetesClient.resource(deployment).createOrReplace(); + } + + 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) { + 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.STATELESS) + .state(JobState.RUNNING) + .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(); + + var flinkDeploymentTemplateSpec = + FlinkDeploymentTemplateSpec.builder() + .deploymentDeletionDelaySec(1) + .maxNumRetries(1) + .reconciliationReschedulingIntervalMs(2000) + .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..86ae604f49 --- /dev/null +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/TestingFlinkBlueGreenDeploymentController.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.kubernetes.operator.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.ErrorStatusHandler; +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, + ErrorStatusHandler { + + @Getter private TestingFlinkResourceContextFactory contextFactory; + + private FlinkBlueGreenDeploymentController flinkBlueGreenDeploymentController; + + public TestingFlinkBlueGreenDeploymentController( + FlinkConfigManager configManager, TestingFlinkService flinkService) { + contextFactory = + new TestingFlinkResourceContextFactory( + configManager, + TestUtils.createTestMetricGroup(new Configuration()), + flinkService, + null); + + flinkBlueGreenDeploymentController = new FlinkBlueGreenDeploymentController(contextFactory); + } + + @Override + public UpdateControl reconcile( + FlinkBlueGreenDeployment flinkBlueGreenDeployment, + Context context) + throws Exception { + var cloned = ReconciliationUtils.clone(flinkBlueGreenDeployment); + return flinkBlueGreenDeploymentController.reconcile(cloned, context); + } + + @Override + public ErrorStatusUpdateControl updateErrorStatus( + FlinkBlueGreenDeployment flinkBlueGreenDeployment, + Context context, + Exception e) { + return null; + } + + /** A simple DTO to handle common reconciliation results for tests. */ + @AllArgsConstructor + public static class BlueGreenReconciliationResult { + public UpdateControl updateControl; + + public FlinkBlueGreenDeployment deployment; + + public FlinkBlueGreenDeploymentStatus reconciledStatus; + } +} diff --git a/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..8b723697b1 --- /dev/null +++ b/helm/flink-kubernetes-operator/crds/flinkbluegreendeployments.flink.apache.org-v1.yml @@ -0,0 +1,10471 @@ +# 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: + deploymentDeletionDelaySec: + type: integer + maxNumRetries: + type: integer + 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 + reconciliationReschedulingIntervalMs: + type: integer + 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 + 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 + 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 + 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 + 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 + 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 + 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 + source: + properties: + resourceClaimName: + type: string + resourceClaimTemplateName: + type: string + type: object + type: object + type: array + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + source: + properties: + resourceClaimName: + type: string + resourceClaimTemplateName: + type: string + type: object + type: object + type: array + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + source: + properties: + resourceClaimName: + type: string + resourceClaimTemplateName: + type: string + type: object + type: object + type: array + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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: + blueGreenState: + enum: + - ACTIVE_BLUE + - ACTIVE_GREEN + - TRANSITIONING_TO_BLUE + - TRANSITIONING_TO_GREEN + 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: integer + numRetries: + type: integer + taskManager: + properties: + labelSelector: + type: string + replicas: + type: integer + type: object + type: object + type: object + served: true + storage: true + subresources: + scale: + labelSelectorPath: .status.taskManager.labelSelector + specReplicasPath: .spec.template.spec.taskManager.replicas + statusReplicasPath: .status.taskManager.replicas + status: {} diff --git a/pom.xml b/pom.xml index 004a3db10f..94cf3f8d9f 100644 --- a/pom.xml +++ b/pom.xml @@ -407,6 +407,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 From 7a8043e8b6176e1fd7fa9a12d1eb50bf859833ef Mon Sep 17 00:00:00 2001 From: schongloo Date: Tue, 11 Mar 2025 22:43:42 -0700 Subject: [PATCH 07/19] First code batch for FLIP-503 --- .../api/FlinkBlueGreenDeployment.java | 41 + .../api/bluegreen/DeploymentType.java | 32 + .../spec/FlinkBlueGreenDeploymentSpec.java | 36 + .../api/spec/FlinkDeploymentTemplateSpec.java | 55 + .../status/FlinkBlueGreenDeploymentState.java | 26 + .../FlinkBlueGreenDeploymentStatus.java | 55 + .../operator/api/utils/SpecUtils.java | 22 + .../kubernetes/operator/FlinkOperator.java | 7 + .../FlinkBlueGreenDeploymentController.java | 589 + .../controller/FlinkBlueGreenDeployments.java | 69 + .../metrics/OperatorJosdkMetrics.java | 3 + .../flink/kubernetes/operator/TestUtils.java | 15 +- .../operator/TestingFlinkService.java | 11 +- ...linkBlueGreenDeploymentControllerTest.java | 453 + ...ingFlinkBlueGreenDeploymentController.java | 84 + ...uegreendeployments.flink.apache.org-v1.yml | 10471 ++++++++++++++++ pom.xml | 1 + 17 files changed, 11965 insertions(+), 5 deletions(-) create mode 100644 flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/FlinkBlueGreenDeployment.java create mode 100644 flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/DeploymentType.java create mode 100644 flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/FlinkBlueGreenDeploymentSpec.java create mode 100644 flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/FlinkDeploymentTemplateSpec.java create mode 100644 flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/status/FlinkBlueGreenDeploymentState.java create mode 100644 flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/status/FlinkBlueGreenDeploymentStatus.java create mode 100644 flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentController.java create mode 100644 flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeployments.java create mode 100644 flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentControllerTest.java create mode 100644 flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/TestingFlinkBlueGreenDeploymentController.java create mode 100644 helm/flink-kubernetes-operator/crds/flinkbluegreendeployments.flink.apache.org-v1.yml 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/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..87b7408a0e --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/bluegreen/DeploymentType.java @@ -0,0 +1,32 @@ +/* + * 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. */ +public enum DeploymentType { + BLUE, + GREEN; + + public static DeploymentType fromDeployment(FlinkDeployment flinkDeployment) { + String typeAnnotation = + flinkDeployment.getMetadata().getLabels().get(DeploymentType.class.getSimpleName()); + return DeploymentType.valueOf(typeAnnotation); + } +} 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..4a84955bcb --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/spec/FlinkDeploymentTemplateSpec.java @@ -0,0 +1,55 @@ +/* + * 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("deploymentDeletionDelaySec") + private int deploymentDeletionDelaySec; + + @JsonProperty("maxNumRetries") + private int maxNumRetries; + + @JsonProperty("reconciliationReschedulingIntervalMs") + private int reconciliationReschedulingIntervalMs; + + @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..5681ee7697 --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/status/FlinkBlueGreenDeploymentState.java @@ -0,0 +1,26 @@ +/* + * 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 { + ACTIVE_BLUE, + ACTIVE_GREEN, + TRANSITIONING_TO_BLUE, + TRANSITIONING_TO_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..1cf75f02ee --- /dev/null +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/status/FlinkBlueGreenDeploymentStatus.java @@ -0,0 +1,55 @@ +/* + * 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 for A. */ + private String lastReconciledSpec; + + /** Timestamp of last reconciliation (either A or B). */ + private Long lastReconciledTimestamp; + + /** Current number of retries. */ + private int numRetries; + + /** Information about the TaskManagers for the scale subresource. */ + private TaskManagerInfo taskManager; +} 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..4488854509 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,28 @@ public static String writeSpecWithMeta( } } + public static String serializeObject(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 deserializeObject(String serialized, String wrapperKey, Class valueType) + throws JsonProcessingException { + 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 4bd2836f7a..c2a5ea91ea 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; @@ -242,6 +243,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(); 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..486532aa75 --- /dev/null +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentController.java @@ -0,0 +1,589 @@ +/* + * 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.JobID; +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.DeploymentType; +import org.apache.flink.kubernetes.operator.api.lifecycle.ResourceLifecycleState; +import org.apache.flink.kubernetes.operator.api.spec.FlinkBlueGreenDeploymentSpec; +import org.apache.flink.kubernetes.operator.api.status.FlinkBlueGreenDeploymentState; +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.service.FlinkResourceContextFactory; +import org.apache.flink.util.Preconditions; + +import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableSet; + +import com.fasterxml.jackson.core.JsonProcessingException; +import io.fabric8.kubernetes.api.model.Event; +import io.fabric8.kubernetes.api.model.ObjectMeta; +import io.fabric8.kubernetes.api.model.OwnerReference; +import io.fabric8.kubernetes.api.model.StatusDetails; +import io.fabric8.kubernetes.client.dsl.PodResource; +import io.fabric8.kubernetes.client.dsl.Resource; +import io.javaoperatorsdk.operator.api.config.informer.InformerConfiguration; +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.EventSourceInitializer; +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.time.Instant; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** Controller that runs the main reconcile loop for Flink Blue/Green deployments. */ +@ControllerConfiguration +public class FlinkBlueGreenDeploymentController + implements Reconciler, + EventSourceInitializer { + + private static final Logger LOG = LoggerFactory.getLogger(FlinkDeploymentController.class); + private static final int DEFAULT_MAX_NUM_RETRIES = 5; + private static final int DEFAULT_RECONCILIATION_RESCHEDULING_INTERVAL_MS = 15000; + + private final FlinkResourceContextFactory ctxFactory; + + public FlinkBlueGreenDeploymentController(FlinkResourceContextFactory ctxFactory) { + this.ctxFactory = ctxFactory; + } + + @Override + public Map prepareEventSources( + EventSourceContext eventSourceContext) { + InformerConfiguration flinkDeploymentInformerConfig = + InformerConfiguration.from(FlinkDeployment.class, eventSourceContext) + .withSecondaryToPrimaryMapper(Mappers.fromOwnerReference()) + .withNamespacesInheritedFromController(eventSourceContext) + .followNamespaceChanges(true) + .build(); + + return EventSourceInitializer.nameEventSources( + new InformerEventSource<>(flinkDeploymentInformerConfig, eventSourceContext)); + } + + @Override + public UpdateControl reconcile( + FlinkBlueGreenDeployment flinkBlueGreenDeployment, + Context josdkContext) + throws Exception { + + FlinkBlueGreenDeploymentStatus deploymentStatus = flinkBlueGreenDeployment.getStatus(); + + if (deploymentStatus == null) { + deploymentStatus = new FlinkBlueGreenDeploymentStatus(); + deploymentStatus.setLastReconciledSpec( + SpecUtils.serializeObject(flinkBlueGreenDeployment.getSpec(), "spec")); + return initiateDeployment( + flinkBlueGreenDeployment, + deploymentStatus, + DeploymentType.BLUE, + FlinkBlueGreenDeploymentState.TRANSITIONING_TO_BLUE, + null, + josdkContext, + true); + } else { + FlinkBlueGreenDeployments deployments = + FlinkBlueGreenDeployments.fromSecondaryResources(josdkContext); + + // TODO: if a new deployment request comes while in the middle of a transition it's + // currently ignored, but the new spec remains changed, should we roll it back? + // TODO: if we choose to leave a previously failed deployment 'running' for debug + // purposes, + // we should flag it somehow as 'ROLLED_BACK' to signal that it can be overriden by a + // new deployment attempt. + switch (deploymentStatus.getBlueGreenState()) { + case ACTIVE_BLUE: + return checkAndInitiateDeployment( + flinkBlueGreenDeployment, + deployments, + deploymentStatus, + DeploymentType.BLUE, + josdkContext); + case ACTIVE_GREEN: + return checkAndInitiateDeployment( + flinkBlueGreenDeployment, + deployments, + deploymentStatus, + DeploymentType.GREEN, + josdkContext); + case TRANSITIONING_TO_BLUE: + return monitorTransition( + flinkBlueGreenDeployment, + deployments, + deploymentStatus, + DeploymentType.GREEN, + josdkContext); + case TRANSITIONING_TO_GREEN: + return monitorTransition( + flinkBlueGreenDeployment, + deployments, + deploymentStatus, + DeploymentType.BLUE, + josdkContext); + default: + return UpdateControl.noUpdate(); + } + } + } + + private UpdateControl monitorTransition( + FlinkBlueGreenDeployment bgDeployment, + FlinkBlueGreenDeployments deployments, + FlinkBlueGreenDeploymentStatus deploymentStatus, + DeploymentType currentDeploymentType, + Context josdkContext) { + + var nextState = FlinkBlueGreenDeploymentState.ACTIVE_BLUE; + FlinkDeployment currentDeployment; + FlinkDeployment nextDeployment; + + if (DeploymentType.BLUE == currentDeploymentType) { + nextState = FlinkBlueGreenDeploymentState.ACTIVE_GREEN; + currentDeployment = deployments.getFlinkDeploymentBlue(); + nextDeployment = deployments.getFlinkDeploymentGreen(); + } else { + currentDeployment = deployments.getFlinkDeploymentGreen(); + nextDeployment = deployments.getFlinkDeploymentBlue(); + } + + Preconditions.checkNotNull( + nextDeployment, + "Target Dependent Deployment resource not found. Blue/Green deployment name: " + + bgDeployment.getMetadata().getName() + + ", current deployment type: " + + currentDeploymentType); + + if (isDeploymentReady(nextDeployment, josdkContext, deploymentStatus)) { + return deleteAndFinalize( + bgDeployment, + deploymentStatus, + currentDeploymentType, + josdkContext, + currentDeployment, + nextState); + } else { + // This phase requires rescheduling the reconciliation because the pod initialization + // could get stuck + // (e.g. waiting for resources) + // TODO: figure out the course of action for error/failure cases + // TODO: should we reschedule indefinitely? Should this case check the "should abort" + // condition? + + int maxNumRetries = bgDeployment.getSpec().getTemplate().getMaxNumRetries(); + if (maxNumRetries <= 0) { + maxNumRetries = DEFAULT_MAX_NUM_RETRIES; + } + + if (deploymentStatus.getNumRetries() >= maxNumRetries) { + // ABORT + return patchStatusUpdateControl(bgDeployment, deploymentStatus, null, null, false); + } else { + // RETRY + deploymentStatus.setNumRetries(deploymentStatus.getNumRetries() + 1); + + LOG.info("Deployment " + nextDeployment.getMetadata().getName() + " not ready yet"); + int reconciliationReschedInterval = + bgDeployment + .getSpec() + .getTemplate() + .getReconciliationReschedulingIntervalMs(); + if (reconciliationReschedInterval <= 0) { + reconciliationReschedInterval = DEFAULT_RECONCILIATION_RESCHEDULING_INTERVAL_MS; + } + + return patchStatusUpdateControl(bgDeployment, deploymentStatus, null, null, false) + .rescheduleAfter(reconciliationReschedInterval); + } + } + } + + private UpdateControl deleteAndFinalize( + FlinkBlueGreenDeployment bgDeployment, + FlinkBlueGreenDeploymentStatus deploymentStatus, + DeploymentType currentDeploymentType, + Context josdkContext, + FlinkDeployment currentDeployment, + FlinkBlueGreenDeploymentState nextState) { + + if (currentDeployment != null) { + deleteDeployment(currentDeployment, josdkContext); + return UpdateControl.noUpdate(); + } else { + deploymentStatus.setLastReconciledSpec( + SpecUtils.serializeObject(bgDeployment.getSpec(), "spec")); + + // TODO: set the last reconciled spec here + + return patchStatusUpdateControl( + bgDeployment, deploymentStatus, nextState, JobStatus.RUNNING, false); + } + } + + private UpdateControl checkAndInitiateDeployment( + FlinkBlueGreenDeployment flinkBlueGreenDeployment, + FlinkBlueGreenDeployments deployments, + FlinkBlueGreenDeploymentStatus deploymentStatus, + DeploymentType currentDeploymentType, + Context josdkContext) + throws Exception { + + if (hasSpecChanged( + flinkBlueGreenDeployment.getSpec(), deploymentStatus, currentDeploymentType)) { + + FlinkDeployment currentFlinkDeployment = + DeploymentType.BLUE == currentDeploymentType + ? deployments.getFlinkDeploymentBlue() + : deployments.getFlinkDeploymentGreen(); + + // TODO: if the current deployment is not STABLE/RUNNING we'll revert to the last stable + // spec, report the error and abort + if (isDeploymentReady(currentFlinkDeployment, josdkContext, deploymentStatus)) { + + DeploymentType nextDeploymentType = DeploymentType.BLUE; + FlinkBlueGreenDeploymentState nextState = + FlinkBlueGreenDeploymentState.TRANSITIONING_TO_BLUE; + FlinkResourceContext resourceContext = + ctxFactory.getResourceContext(currentFlinkDeployment, josdkContext); + + // TODO: this operation is already done by hasSpecChanged() above, dedup later + String serializedSpec = + SpecUtils.serializeObject(flinkBlueGreenDeployment.getSpec(), "spec"); + + // Updating status + if (DeploymentType.BLUE == currentDeploymentType) { + nextState = FlinkBlueGreenDeploymentState.TRANSITIONING_TO_GREEN; + nextDeploymentType = DeploymentType.GREEN; + } + + Savepoint lastCheckpoint = configureSavepoint(resourceContext); + + return initiateDeployment( + flinkBlueGreenDeployment, + deploymentStatus, + nextDeploymentType, + nextState, + lastCheckpoint, + josdkContext, + false); + } else { + // If the current running FlinkDeployment is not in RUNNING/STABLE, + // we flag this Blue/Green as FAILING + if (deploymentStatus.getJobStatus().getState() != JobStatus.FAILING) { + return patchStatusUpdateControl( + flinkBlueGreenDeployment, + deploymentStatus, + null, + JobStatus.FAILING, + false); + } + } + } + + return UpdateControl.noUpdate(); + } + + public void logPotentialWarnings( + FlinkDeployment flinkDeployment, + Context josdkContext, + long lastReconciliationTimestamp) { + // Event reason constants + // https://github.com/kubernetes/kubernetes/blob/master/pkg/kubelet/events/event.go + Set badEventPatterns = + ImmutableSet.of( + "FAIL", "EXCEPTION", "BACKOFF", "ERROR", "EVICTION", "KILL", "EXCEED"); + Set goodPodPhases = ImmutableSet.of("PENDING", "RUNNING"); + + Set podPhases = + getDeploymentPods(josdkContext, flinkDeployment) + .map(p -> p.get().getStatus().getPhase().toUpperCase()) + .collect(Collectors.toSet()); + + podPhases.removeAll(goodPodPhases); + + if (!podPhases.isEmpty()) { + LOG.warn("Deployment not healthy, some Pods have the following status: " + podPhases); + } + + List badEvents = + josdkContext + .getClient() + .v1() + .events() + .inNamespace(flinkDeployment.getMetadata().getNamespace()) + .resources() + .map(Resource::item) + .filter(e -> !e.getType().equalsIgnoreCase("NORMAL")) + .filter( + e -> + e.getInvolvedObject() + .getName() + .contains(flinkDeployment.getMetadata().getName())) + .filter( + e -> + Instant.parse(e.getLastTimestamp()).toEpochMilli() + > lastReconciliationTimestamp) + .filter( + e -> + badEventPatterns.stream() + .anyMatch( + p -> + e.getReason() + .toUpperCase() + .contains(p))) + .collect(Collectors.toList()); + + if (!badEvents.isEmpty()) { + LOG.warn("Bad events detected: " + badEvents); + } + } + + private static Savepoint configureSavepoint( + FlinkResourceContext resourceContext) throws Exception { + // TODO: if the user specified an initialSavepointPath, use it and skip this + Optional lastCheckpoint = + resourceContext + .getFlinkService() + .getLastCheckpoint( + JobID.fromHexString( + resourceContext + .getResource() + .getStatus() + .getJobStatus() + .getJobId()), + resourceContext.getObserveConfig()); + + // TODO 1: check the last CP age with the logic from + // AbstractJobReconciler.changeLastStateIfCheckpointTooOld + // TODO 2: if no checkpoint is available, take a savepoint + if (lastCheckpoint.isEmpty()) { + throw new IllegalStateException( + "Last Checkpoint for Job " + + resourceContext.getResource().getMetadata().getName() + + " not found!"); + } + return lastCheckpoint.get(); + } + + private UpdateControl initiateDeployment( + FlinkBlueGreenDeployment flinkBlueGreenDeployment, + FlinkBlueGreenDeploymentStatus deploymentStatus, + DeploymentType nextDeploymentType, + FlinkBlueGreenDeploymentState nextState, + Savepoint lastCheckpoint, + Context josdkContext, + boolean isFirstDeployment) + throws JsonProcessingException { + deploy( + flinkBlueGreenDeployment, + nextDeploymentType, + lastCheckpoint, + josdkContext, + isFirstDeployment); + return patchStatusUpdateControl( + flinkBlueGreenDeployment, deploymentStatus, nextState, null, isFirstDeployment); + } + + private boolean isDeploymentReady( + FlinkDeployment deployment, + Context josdkContext, + FlinkBlueGreenDeploymentStatus deploymentStatus) { + if (ResourceLifecycleState.STABLE == deployment.getStatus().getLifecycleState() + && JobStatus.RUNNING == deployment.getStatus().getJobStatus().getState()) { + // TODO: verify, e.g. will pods be "pending" after the FlinkDeployment is RUNNING and + // STABLE? + int notRunningPods = + (int) + getDeploymentPods(josdkContext, deployment) + .filter( + p -> + !p.get() + .getStatus() + .getPhase() + .equalsIgnoreCase("RUNNING")) + .count(); + + if (notRunningPods > 0) { + LOG.warn("Waiting for " + notRunningPods + " Pods to transition to RUNNING status"); + } + + return notRunningPods == 0; + } + + logPotentialWarnings( + deployment, josdkContext, deploymentStatus.getLastReconciledTimestamp()); + return false; + } + + private static Stream getDeploymentPods( + Context josdkContext, FlinkDeployment deployment) { + var namespace = deployment.getMetadata().getNamespace(); + var deploymentName = deployment.getMetadata().getName(); + + return josdkContext + .getClient() + .pods() + .inNamespace(namespace) + .withLabel("app", deploymentName) + .resources(); + } + + private boolean hasSpecChanged( + FlinkBlueGreenDeploymentSpec newSpec, + FlinkBlueGreenDeploymentStatus deploymentStatus, + DeploymentType deploymentType) { + + String lastReconciledSpec = deploymentStatus.getLastReconciledSpec(); + + return !lastReconciledSpec.equals(SpecUtils.serializeObject(newSpec, "spec")); + } + + private UpdateControl patchStatusUpdateControl( + FlinkBlueGreenDeployment flinkBlueGreenDeployment, + FlinkBlueGreenDeploymentStatus deploymentStatus, + FlinkBlueGreenDeploymentState deploymentState, + JobStatus jobState, + boolean isFirstDeployment) { + if (deploymentState != null) { + deploymentStatus.setBlueGreenState(deploymentState); + } + + if (jobState != null) { + deploymentStatus.getJobStatus().setState(jobState); + } + + deploymentStatus.setLastReconciledTimestamp(System.currentTimeMillis()); + flinkBlueGreenDeployment.setStatus(deploymentStatus); + return UpdateControl.patchStatus(flinkBlueGreenDeployment); + } + + private void deploy( + FlinkBlueGreenDeployment bgDeployment, + DeploymentType deploymentType, + Savepoint lastCheckpoint, + Context josdkContext, + boolean isFirstDeployment) + throws JsonProcessingException { + ObjectMeta bgMeta = bgDeployment.getMetadata(); + + // Deployment + FlinkDeployment flinkDeployment = new FlinkDeployment(); + flinkDeployment.setApiVersion("flink.apache.org/v1beta1"); + flinkDeployment.setKind("FlinkDeployment"); + FlinkBlueGreenDeploymentSpec spec = bgDeployment.getSpec(); + + String childDeploymentName = + bgMeta.getName() + "-" + deploymentType.toString().toLowerCase(); + + FlinkBlueGreenDeploymentSpec adjustedSpec = + adjustNameReferences( + spec, + bgMeta.getName(), + childDeploymentName, + "spec", + FlinkBlueGreenDeploymentSpec.class); + + if (lastCheckpoint != null) { + String location = lastCheckpoint.getLocation().replace("file:", ""); + LOG.info("Using 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.getClass().getSimpleName(), deploymentType.toString())); + flinkDeployment.setMetadata(flinkDeploymentMeta); + + // Deploy + josdkContext.getClient().resource(flinkDeployment).createOrReplace(); + } + + private static void deleteDeployment( + FlinkDeployment currentDeployment, Context josdkContext) { + // TODO: This gets called multiple times, check to see if it's already in a TERMINATING + // state + // (or only execute if RUNNING) + List deletedStatus = + josdkContext + .getClient() + .resources(FlinkDeployment.class) + .inNamespace(currentDeployment.getMetadata().getNamespace()) + .withName(currentDeployment.getMetadata().getName()) + .delete(); + + boolean deleted = + deletedStatus.size() == 1 + && deletedStatus.get(0).getKind().equals("FlinkDeployment"); + if (!deleted) { + LOG.info("Deployment not deleted, will retry"); + } else { + LOG.info("Deployment deleted!"); + } + } + + private 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; + } + + private static T adjustNameReferences( + T spec, + String deploymentName, + String childDeploymentName, + String wrapperKey, + Class valueType) + throws JsonProcessingException { + String serializedSpec = SpecUtils.serializeObject(spec, wrapperKey); + String replacedSerializedSpec = serializedSpec.replace(deploymentName, childDeploymentName); + return SpecUtils.deserializeObject(replacedSerializedSpec, wrapperKey, valueType); + } + + public static void logAndThrow(String message) { + LOG.error(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..69e9f084b7 --- /dev/null +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeployments.java @@ -0,0 +1,69 @@ +/* + * 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 +class FlinkBlueGreenDeployments { + private FlinkDeployment flinkDeploymentBlue; + private FlinkDeployment flinkDeploymentGreen; + + 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/metrics/OperatorJosdkMetrics.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/metrics/OperatorJosdkMetrics.java index 5350e3cd76..bb14cd986c 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; @@ -211,6 +212,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/test/java/org/apache/flink/kubernetes/operator/TestUtils.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/TestUtils.java index 6af375edb3..97eab38e59 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 @@ -74,6 +74,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; @@ -478,7 +479,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 fec1dfa64e..fa9a63d782 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 @@ -580,10 +580,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..1832df166b --- /dev/null +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentControllerTest.java @@ -0,0 +1,453 @@ +/* + * 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.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.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +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 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.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 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("org.apache.flink.kubernetes.operator.TestUtils#flinkVersions") + public void verifyBasicDeployment(FlinkVersion flinkVersion) throws Exception { + var blueGreenDeployment = + buildSessionCluster(TEST_DEPLOYMENT_NAME, TEST_NAMESPACE, flinkVersion); + + // 1. Initiate the Green deployment + var bgSpecBefore = blueGreenDeployment.getSpec(); + Long minReconciliationTs = System.currentTimeMillis() - 1; + var rs = reconcile(blueGreenDeployment); + + assertSpec(rs, minReconciliationTs); + + // check the status (reconciled spec, reconciled ts, a/b state) + assertEquals( + FlinkBlueGreenDeploymentState.TRANSITIONING_TO_BLUE, + rs.reconciledStatus.getBlueGreenState()); + assertNotNull(rs.reconciledStatus.getLastReconciledSpec()); + assertNull(rs.reconciledStatus.getJobStatus().getState()); + + var flinkDeploymentList = getFlinkDeployments(); + assertEquals(1, flinkDeploymentList.size()); + var deploymentA = flinkDeploymentList.get(0); + + verifyOwnerReferences(rs.deployment, deploymentA); + + simulateSubmitAndSuccessfulJobStart(deploymentA); + + // 2. Finalize the Green deployment + minReconciliationTs = System.currentTimeMillis() - 1; + rs = reconcile(rs.deployment); + + assertSpec(rs, minReconciliationTs); + + assertEquals( + SpecUtils.serializeObject(bgSpecBefore, "spec"), + rs.reconciledStatus.getLastReconciledSpec()); + assertEquals( + FlinkBlueGreenDeploymentState.ACTIVE_BLUE, rs.reconciledStatus.getBlueGreenState()); + assertEquals(JobStatus.RUNNING, rs.reconciledStatus.getJobStatus().getState()); + } + + @ParameterizedTest + @MethodSource("org.apache.flink.kubernetes.operator.TestUtils#flinkVersions") + public void verifyBasicTransition(FlinkVersion flinkVersion) throws Exception { + var blueGreenDeployment = + buildSessionCluster(TEST_DEPLOYMENT_NAME, TEST_NAMESPACE, flinkVersion); + + // 1. Initiate the Green deployment + var rs = reconcile(blueGreenDeployment); + + // 2. Finalize the Green deployment + simulateSubmitAndSuccessfulJobStart(getFlinkDeployments().get(0)); + rs = reconcile(rs.deployment); + + // Verify noUpdate if reconciliation is triggered without a spec change + var rs2 = reconcile(rs.deployment); + assertTrue(rs2.updateControl.isNoUpdate()); + + // 3. Simulate a change in the spec to trigger a Blue deployment + simulateChangeInSpec(rs.deployment); + + // 4. Initiate the Blue deployment + var bgUpdatedSpec = rs.deployment.getSpec(); + Long minReconciliationTs = System.currentTimeMillis() - 1; + rs = reconcile(rs.deployment); + var flinkDeployments = getFlinkDeployments(); + var blueDeploymentName = flinkDeployments.get(1).getMetadata().getName(); + + assertSpec(rs, minReconciliationTs); + + assertEquals(2, flinkDeployments.size()); + assertNull(flinkDeployments.get(0).getSpec().getJob().getInitialSavepointPath()); + assertNotNull(flinkDeployments.get(1).getSpec().getJob().getInitialSavepointPath()); + + assertEquals( + FlinkBlueGreenDeploymentState.TRANSITIONING_TO_GREEN, + rs.reconciledStatus.getBlueGreenState()); + assertNotNull(rs.reconciledStatus.getLastReconciledSpec()); + + // 5. New Blue deployment successfully started + simulateSuccessfulJobStart(getFlinkDeployments().get(1)); + rs2 = reconcile(rs.deployment); + assertTrue(rs2.updateControl.isNoUpdate()); + + flinkDeployments = getFlinkDeployments(); + assertEquals(1, flinkDeployments.size()); + assertEquals(blueDeploymentName, flinkDeployments.get(0).getMetadata().getName()); + + // 6. Old Green deployment deleted, Blue promoted to Green + minReconciliationTs = System.currentTimeMillis() - 1; + rs = reconcile(rs.deployment); + + // Spec should still be the new one + assertSpec(rs, minReconciliationTs); + + assertNotNull(rs.reconciledStatus.getLastReconciledSpec()); + assertEquals( + SpecUtils.serializeObject(bgUpdatedSpec, "spec"), + rs.reconciledStatus.getLastReconciledSpec()); + assertEquals( + FlinkBlueGreenDeploymentState.ACTIVE_GREEN, + rs.reconciledStatus.getBlueGreenState()); + assertEquals(JobStatus.RUNNING, rs.reconciledStatus.getJobStatus().getState()); + assertTrue(rs.updateControl.isPatchStatus()); + } + + @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); + var originalSpec = blueGreenDeployment.getSpec(); + + // 1. Initiate the Green deployment + var rs = reconcile(blueGreenDeployment); + + // 2. Finalize the Green deployment + simulateSubmitAndSuccessfulJobStart(getFlinkDeployments().get(0)); + rs = reconcile(rs.deployment); + + // 3. Simulate a change in the spec to trigger a Blue deployment + simulateChangeInSpec(rs.deployment); + + // TODO: simulate a failure in the running deployment + simulateJobFailure(getFlinkDeployments().get(0)); + + // 4. Initiate the Blue deployment + var minReconciliationTs = System.currentTimeMillis() - 1; + rs = reconcile(rs.deployment); + + assertSpec(rs, minReconciliationTs); + + // 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()); + + // 5. 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); + var originalSpec = blueGreenDeployment.getSpec(); + + // Overriding the maxNumRetries and Reschedule Interval + var maxNumRetries = 2; + var reconciliationReschedulingIntervalMs = 5000; + blueGreenDeployment.getSpec().getTemplate().setMaxNumRetries(maxNumRetries); + blueGreenDeployment + .getSpec() + .getTemplate() + .setReconciliationReschedulingIntervalMs(reconciliationReschedulingIntervalMs); + + // 1. Initiate the Green deployment + var rs = reconcile(blueGreenDeployment); + + // 2. Finalize the Green deployment + simulateSubmitAndSuccessfulJobStart(getFlinkDeployments().get(0)); + rs = reconcile(rs.deployment); + + // 3. Simulate a change in the spec to trigger a Blue deployment + simulateChangeInSpec(rs.deployment); + + // 4. Initiate the Blue deployment + rs = reconcile(rs.deployment); + + // 4a. Simulating the Blue deployment doesn't start correctly (status will remain the same) + // Asserting the status retry count is incremented by 1 + long lastTs = System.currentTimeMillis(); + for (int i = 1; i <= maxNumRetries; i++) { + Thread.sleep(1); + rs = reconcile(rs.deployment); + assertTrue(rs.updateControl.isPatchStatus()); + assertFalse(rs.updateControl.isUpdateResource()); + assertTrue(rs.updateControl.getScheduleDelay().isPresent()); + assertEquals( + reconciliationReschedulingIntervalMs, + rs.updateControl.getScheduleDelay().get()); + assertEquals(i, rs.reconciledStatus.getNumRetries()); + assertTrue(rs.reconciledStatus.getLastReconciledTimestamp() > lastTs); + lastTs = rs.reconciledStatus.getLastReconciledTimestamp(); + System.out.println(); + } + + // 4b. After the retries are exhausted + var minReconciliationTs = System.currentTimeMillis() - 1; + rs = reconcile(rs.deployment); + + assertSpec(rs, minReconciliationTs); + + // The first job should be running OK, the second should be left in its broken state + 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()); + assertNull(flinkDeployments.get(1).getStatus().getJobStatus().getState()); + assertEquals( + ReconciliationState.UPGRADING, + flinkDeployments.get(1).getStatus().getReconciliationStatus().getState()); + } + + private static void assertSpec( + TestingFlinkBlueGreenDeploymentController.BlueGreenReconciliationResult rs, + long minReconciliationTs) { + assertTrue(rs.updateControl.isPatchStatus()); + assertTrue(minReconciliationTs < rs.reconciledStatus.getLastReconciledTimestamp()); + } + + private void simulateChangeInSpec(FlinkBlueGreenDeployment blueGreenDeployment) { + FlinkDeploymentSpec spec = blueGreenDeployment.getSpec().getTemplate().getSpec(); + spec.getFlinkConfiguration().put(TaskManagerOptions.NUM_TASK_SLOTS.key(), "10"); + blueGreenDeployment.getSpec().getTemplate().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(), + updateControl.isNoUpdate() ? null : updateControl.getResource().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, "/tmp/savepoint"); + 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).createOrReplace(); + } + + private void simulateJobFailure(FlinkDeployment deployment) { + deployment.getStatus().getJobStatus().setState(JobStatus.RECONCILING); + deployment.getStatus().getReconciliationStatus().setState(ReconciliationState.UPGRADING); + kubernetesClient.resource(deployment).createOrReplace(); + } + + 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) { + 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.STATELESS) + .state(JobState.RUNNING) + .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(); + + var flinkDeploymentTemplateSpec = + FlinkDeploymentTemplateSpec.builder() + .deploymentDeletionDelaySec(1) + .maxNumRetries(1) + .reconciliationReschedulingIntervalMs(2000) + .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..86ae604f49 --- /dev/null +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/TestingFlinkBlueGreenDeploymentController.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.kubernetes.operator.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.ErrorStatusHandler; +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, + ErrorStatusHandler { + + @Getter private TestingFlinkResourceContextFactory contextFactory; + + private FlinkBlueGreenDeploymentController flinkBlueGreenDeploymentController; + + public TestingFlinkBlueGreenDeploymentController( + FlinkConfigManager configManager, TestingFlinkService flinkService) { + contextFactory = + new TestingFlinkResourceContextFactory( + configManager, + TestUtils.createTestMetricGroup(new Configuration()), + flinkService, + null); + + flinkBlueGreenDeploymentController = new FlinkBlueGreenDeploymentController(contextFactory); + } + + @Override + public UpdateControl reconcile( + FlinkBlueGreenDeployment flinkBlueGreenDeployment, + Context context) + throws Exception { + var cloned = ReconciliationUtils.clone(flinkBlueGreenDeployment); + return flinkBlueGreenDeploymentController.reconcile(cloned, context); + } + + @Override + public ErrorStatusUpdateControl updateErrorStatus( + FlinkBlueGreenDeployment flinkBlueGreenDeployment, + Context context, + Exception e) { + return null; + } + + /** A simple DTO to handle common reconciliation results for tests. */ + @AllArgsConstructor + public static class BlueGreenReconciliationResult { + public UpdateControl updateControl; + + public FlinkBlueGreenDeployment deployment; + + public FlinkBlueGreenDeploymentStatus reconciledStatus; + } +} diff --git a/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..8b723697b1 --- /dev/null +++ b/helm/flink-kubernetes-operator/crds/flinkbluegreendeployments.flink.apache.org-v1.yml @@ -0,0 +1,10471 @@ +# 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: + deploymentDeletionDelaySec: + type: integer + maxNumRetries: + type: integer + 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 + reconciliationReschedulingIntervalMs: + type: integer + 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 + 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 + 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 + 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 + 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 + 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 + 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 + source: + properties: + resourceClaimName: + type: string + resourceClaimTemplateName: + type: string + type: object + type: object + type: array + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + source: + properties: + resourceClaimName: + type: string + resourceClaimTemplateName: + type: string + type: object + type: object + type: array + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + source: + properties: + resourceClaimName: + type: string + resourceClaimTemplateName: + type: string + type: object + type: object + type: array + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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: + blueGreenState: + enum: + - ACTIVE_BLUE + - ACTIVE_GREEN + - TRANSITIONING_TO_BLUE + - TRANSITIONING_TO_GREEN + 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: integer + numRetries: + type: integer + taskManager: + properties: + labelSelector: + type: string + replicas: + type: integer + type: object + type: object + type: object + served: true + storage: true + subresources: + scale: + labelSelectorPath: .status.taskManager.labelSelector + specReplicasPath: .spec.template.spec.taskManager.replicas + statusReplicasPath: .status.taskManager.replicas + status: {} diff --git a/pom.xml b/pom.xml index 660fae6246..d47a07a4f3 100644 --- a/pom.xml +++ b/pom.xml @@ -407,6 +407,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 From c4d460bb4fd3b67b4a23a840a87f811ea5463b70 Mon Sep 17 00:00:00 2001 From: Luca Castelli Date: Sun, 16 Mar 2025 14:25:32 -0400 Subject: [PATCH 08/19] [FLINK-37370] [Observer] Fix exception caught when handling checkpointing not enabled for batch jobs and add batch e2e test --- .github/workflows/ci.yml | 7 ++ .github/workflows/e2e.yaml | 2 + e2e-tests/data/flinkdep-batch-cr.yaml | 87 +++++++++++++++++++ e2e-tests/test_batch_job.sh | 62 +++++++++++++ .../service/AbstractFlinkService.java | 3 +- .../operator/TestingFlinkService.java | 6 +- 6 files changed, 164 insertions(+), 3 deletions(-) create mode 100644 e2e-tests/data/flinkdep-batch-cr.yaml create mode 100755 e2e-tests/test_batch_job.sh diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 9618a2b3b8..a2dd76da1b 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -161,6 +161,7 @@ jobs: - test_autoscaler.sh - test_flink_operator_ha.sh - test_snapshot.sh + - test_batch_job.sh exclude: - flink-version: v1_16 test: test_autoscaler.sh @@ -172,18 +173,24 @@ jobs: test: test_flink_operator_ha.sh - flink-version: v1_16 test: test_snapshot.sh + - flink-version: v1_16 + test: test_batch_job.sh - flink-version: v1_17 test: test_dynamic_config.sh - flink-version: v1_17 test: test_flink_operator_ha.sh - flink-version: v1_17 test: test_snapshot.sh + - flink-version: v1_17 + test: test_batch_job.sh - flink-version: v1_18 test: test_dynamic_config.sh - flink-version: v1_18 test: test_flink_operator_ha.sh - flink-version: v1_18 test: test_snapshot.sh + - flink-version: v1_18 + test: test_batch_job.sh - flink-version: v1_19 test: test_snapshot.sh uses: ./.github/workflows/e2e.yaml diff --git a/.github/workflows/e2e.yaml b/.github/workflows/e2e.yaml index 26f90a8895..ecfcd07fef 100644 --- a/.github/workflows/e2e.yaml +++ b/.github/workflows/e2e.yaml @@ -88,6 +88,8 @@ jobs: EXAMPLES_JAR="https://repo1.maven.org/maven2/org/apache/flink/flink-examples-streaming_2.12/1.14.4/flink-examples-streaming_2.12-1.14.4.jar" if [[ ${{ inputs.flink-version }} == v2* ]]; then EXAMPLES_JAR="https://repo1.maven.org/maven2/org/apache/flink/flink-examples-streaming/2.0-preview1/flink-examples-streaming-2.0-preview1.jar" + elif [[ "${{ inputs.test }}" == "test_batch_job.sh" ]]; then + EXAMPLES_JAR="https://repo1.maven.org/maven2/org/apache/flink/flink-examples-streaming/1.20.1/flink-examples-streaming-1.20.1.jar" fi ESCAPED_EXAMPLES_JAR=$(printf '%s\n' "$EXAMPLES_JAR" | sed -e 's/[\/&]/\\&/g') diff --git a/e2e-tests/data/flinkdep-batch-cr.yaml b/e2e-tests/data/flinkdep-batch-cr.yaml new file mode 100644 index 0000000000..159199ce4f --- /dev/null +++ b/e2e-tests/data/flinkdep-batch-cr.yaml @@ -0,0 +1,87 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +apiVersion: flink.apache.org/v1beta1 +kind: FlinkDeployment +metadata: + namespace: default + name: flink-example-wordcount-batch +spec: + image: flink:1.20 + flinkVersion: v1_20 + ingress: + template: "/{{namespace}}/{{name}}(/|$)(.*)" + className: "nginx" + annotations: + nginx.ingress.kubernetes.io/rewrite-target: "/$2" + flinkConfiguration: + taskmanager.numberOfTaskSlots: "2" + kubernetes.operator.snapshot.resource.enabled: "false" + serviceAccount: flink + podTemplate: + spec: + initContainers: + - name: artifacts-fetcher + image: busybox:1.35.0 + imagePullPolicy: IfNotPresent + # Use wget or other tools to get user jars from remote storage + command: [ 'wget', 'STREAMING_EXAMPLES_JAR_URL', '-O', '/flink-artifact/myjob.jar' ] + volumeMounts: + - mountPath: /flink-artifact + name: flink-artifact + containers: + # Do not change the main container name + - name: flink-main-container + resources: + requests: + ephemeral-storage: 2048Mi + limits: + ephemeral-storage: 2048Mi + volumeMounts: + - mountPath: /opt/flink/usrlib + name: flink-artifact + volumes: + - name: flink-artifact + emptyDir: { } + jobManager: + resource: + memory: "1024m" + cpu: 0.5 + taskManager: + resource: + memory: "1Gi" + cpu: 0.5 + job: + jarURI: local:///opt/flink/usrlib/myjob.jar + entryClass: org.apache.flink.streaming.examples.wordcount.WordCount + args: ["--execution-mode", "BATCH"] + parallelism: 2 + upgradeMode: stateless + mode: native + +--- +apiVersion: networking.k8s.io/v1 +kind: IngressClass +metadata: + annotations: + ingressclass.kubernetes.io/is-default-class: "true" + labels: + app.kubernetes.io/component: controller + name: nginx +spec: + controller: k8s.io/ingress-nginx diff --git a/e2e-tests/test_batch_job.sh b/e2e-tests/test_batch_job.sh new file mode 100755 index 0000000000..2cbf6a5d4d --- /dev/null +++ b/e2e-tests/test_batch_job.sh @@ -0,0 +1,62 @@ +#!/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 basic Flink batch job operations on Kubernetes: +# 1. Deploys a FlinkDeployment for a batch job. +# 2. Waits for the JobManager to become ready. +# 3. Verifies that the job reaches the FINISHED state. +# 4. Applies a no-op spec change and verifies the job remains in the FINISHED state. +# 5. Checks the operator logs for the expected job state transition message. +# 6. Checks the JobManager logs for successful application completion. +# 7. Applies a spec change and verifies the job re-runs successfully. +SCRIPT_DIR=$(dirname "$(readlink -f "$0")") +source "${SCRIPT_DIR}/utils.sh" + +CLUSTER_ID="flink-example-wordcount-batch" +APPLICATION_YAML="${SCRIPT_DIR}/data/flinkdep-batch-cr.yaml" +APPLICATION_IDENTIFIER="flinkdep/$CLUSTER_ID" +TIMEOUT=300 + +on_exit cleanup_and_exit "$APPLICATION_YAML" $TIMEOUT $CLUSTER_ID + +retry_times 5 30 "kubectl apply -f $APPLICATION_YAML" || exit 1 + +wait_for_jobmanager_running $CLUSTER_ID $TIMEOUT + +# Wait for the job to reach the FINISHED state. +wait_for_status $APPLICATION_IDENTIFIER '.status.jobStatus.state' FINISHED $TIMEOUT || exit 1 + +# Apply a no-op spec change; verify the job remains in the FINISHED state. +kubectl patch flinkdep ${CLUSTER_ID} --type merge --patch '{"spec":{"flinkConfiguration": {"kubernetes.operator.deployment.readiness.timeout": "6h" } } }' +wait_for_status $APPLICATION_IDENTIFIER '.status.jobStatus.state' FINISHED $TIMEOUT || exit 1 + +# Verify the job status change to FINISHED shows up in the operator logs. +operator_pod_name=$(get_operator_pod_name) +wait_for_operator_logs "$operator_pod_name" "Job status changed from .* to FINISHED" ${TIMEOUT} || exit 1 + +# Verify the job completed successfully in the job manager logs. +jm_pod_name=$(get_jm_pod_name $CLUSTER_ID) +wait_for_logs "$jm_pod_name" "Application completed SUCCESSFULLY" ${TIMEOUT} || exit 1 + +# Apply a spec change; verify the job re-runs and reaches the FINISHED state. +kubectl patch flinkdep ${CLUSTER_ID} --type merge --patch '{"spec":{"job": {"parallelism": 1 } } }' +wait_for_status $APPLICATION_IDENTIFIER '.status.jobStatus.state' RECONCILING $TIMEOUT || exit 1 +wait_for_status $APPLICATION_IDENTIFIER '.status.jobStatus.state' FINISHED $TIMEOUT || exit 1 + +echo "Successfully ran the batch job test" \ No newline at end of file diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkService.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkService.java index 8728bc2ca4..d88aca65be 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkService.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkService.java @@ -142,6 +142,7 @@ import java.util.Optional; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -546,7 +547,7 @@ public Optional getLastCheckpoint(JobID jobId, Configuration conf) { try { latestCheckpointOpt = getCheckpointInfo(jobId, conf).f0; } catch (Exception e) { - if (e instanceof RestClientException + if (e instanceof ExecutionException && e.getMessage() != null && e.getMessage().contains("Checkpointing has not been enabled")) { LOG.warn("Checkpointing not enabled for job {}", jobId, e); 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 fec1dfa64e..c18e8f34c5 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 @@ -98,6 +98,7 @@ import java.util.Random; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; import java.util.function.Consumer; import java.util.stream.Collectors; @@ -595,8 +596,9 @@ public Optional getLastCheckpoint(JobID jobId, Configuration conf) { Optional> getCheckpointInfo(JobID jobId, Configuration conf) throws Exception { if (throwCheckpointingDisabledError) { - throw new RestClientException( - "Checkpointing has not been enabled", HttpResponseStatus.BAD_REQUEST); + throw new ExecutionException( + new RestClientException( + "Checkpointing has not been enabled", HttpResponseStatus.BAD_REQUEST)); } if (checkpointInfo != null) { From 679b0337495693d84463584113bb50989e38a8bc Mon Sep 17 00:00:00 2001 From: Gyula Fora Date: Thu, 27 Feb 2025 11:20:21 +0100 Subject: [PATCH 09/19] [FLINK-37405] Validate config prefixes for Flink 2.0 --- .../decorators/FlinkConfMountDecorator.java | 24 +++++++++++++ .../FlinkConfMountDecoratorTest.java | 34 +++++++++++++++++++ 2 files changed, 58 insertions(+) diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkConfMountDecorator.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkConfMountDecorator.java index 8683996793..4615423ba8 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkConfMountDecorator.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkConfMountDecorator.java @@ -22,6 +22,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ConfigurationUtils; import org.apache.flink.configuration.DeploymentOptionsInternal; +import org.apache.flink.configuration.IllegalConfigurationException; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.RestOptions; import org.apache.flink.configuration.TaskManagerOptions; @@ -166,9 +167,32 @@ private List getClusterSideConfData(Configuration flinkConfig) { clusterSideConfig.removeConfig(TaskManagerOptions.BIND_HOST); clusterSideConfig.removeConfig(TaskManagerOptions.HOST); + validateConfigKeysForV2(clusterSideConfig); + return ConfigurationUtils.convertConfigToWritableLines(clusterSideConfig, false); } + private void validateConfigKeysForV2(Configuration clusterSideConfig) { + + // Only validate Flink 2.0 yaml configs + if (!useStandardYamlConfig()) { + return; + } + + var keys = clusterSideConfig.keySet(); + + for (var key1 : keys) { + for (var key2 : keys) { + if (key2.startsWith(key1 + ".")) { + throw new IllegalConfigurationException( + String.format( + "Overlapping key prefixes detected (%s -> %s), please replace with Flink v2 compatible, non-deprecated keys.", + key1, key2)); + } + } + } + } + @VisibleForTesting String getFlinkConfData(List confData) throws IOException { try (StringWriter sw = new StringWriter(); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkConfMountDecoratorTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkConfMountDecoratorTest.java index 3669176f3c..7b6a324659 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkConfMountDecoratorTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkConfMountDecoratorTest.java @@ -20,6 +20,7 @@ import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.IllegalConfigurationException; import org.apache.flink.configuration.YamlParserUtils; import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters; @@ -32,6 +33,7 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; +import org.junit.jupiter.params.provider.ValueSource; import java.io.IOException; import java.util.HashMap; @@ -95,6 +97,38 @@ void testConfigMap(FlinkVersion version, String expectedConfName, boolean standa } } + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testOverlappingKeyDetection(boolean standardYaml) throws IOException { + + flinkConfig.set( + FlinkConfigBuilder.FLINK_VERSION, + standardYaml ? FlinkVersion.v2_0 : FlinkVersion.v1_20); + flinkConfig.setString("k", "v"); + flinkConfig.setString("kv", "v2"); + + // Non overlapping keys + flinkConfMountDecorator.buildAccompanyingKubernetesResources(); + flinkConfig.setString("k.v", "v3"); + + IllegalConfigurationException err = null; + try { + var additionalResources = + flinkConfMountDecorator.buildAccompanyingKubernetesResources(); + assertThat(additionalResources).hasSize(1); + } catch (IllegalConfigurationException e) { + err = e; + } + + if (standardYaml) { + assertThat(err) + .isNotNull() + .hasMessageContaining("Overlapping key prefixes detected (k -> k.v)"); + } else { + assertThat(err).isNull(); + } + } + private static Stream testArgs() { return Stream.of( Arguments.arguments(FlinkVersion.v1_19, "flink-conf.yaml", false), From 68596830b890e5ffe68c0369d2566d76375936b7 Mon Sep 17 00:00:00 2001 From: Gyula Fora Date: Thu, 27 Mar 2025 08:25:28 +0100 Subject: [PATCH 10/19] [ci] PRs should not cancel each others or traget branches CI runs --- .github/workflows/ci.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index a2dd76da1b..724d23d526 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -28,7 +28,7 @@ on: - release-* pull_request: concurrency: - group: ${{ github.workflow }}-${{ github.event.workflow_run.head_branch }} + group: ${{ github.workflow }}-${{ github.ref_name }} cancel-in-progress: true jobs: From 7a65e024439df812eabf223c363f98265904a36a Mon Sep 17 00:00:00 2001 From: Rodrigo Date: Thu, 27 Mar 2025 03:32:57 -0700 Subject: [PATCH 11/19] [FLINK-37430] Operator hides the actual error on deployment issues --- .../operator/config/FlinkConfigManager.java | 2 + .../controller/FlinkDeploymentController.java | 3 +- .../controller/FlinkSessionJobController.java | 3 +- .../mutator/FlinkResourceMutator.java | 3 + .../AbstractFlinkResourceObserver.java | 1 + .../SnapshotTriggerTimestampStore.java | 2 + .../service/AbstractFlinkService.java | 10 ++- .../operator/utils/EventRecorder.java | 24 +++++-- .../operator/utils/ExceptionUtils.java | 60 +++++++++++++++++ .../operator/utils/SnapshotUtils.java | 1 + .../operator/utils/StatusRecorder.java | 1 + .../validation/FlinkResourceValidator.java | 1 + .../operator/TestingFlinkService.java | 8 +++ .../FlinkDeploymentControllerTest.java | 54 +++++++++++++++ .../FlinkSessionJobControllerTest.java | 26 ++++++++ .../operator/utils/ExceptionUtilsTest.java | 65 +++++++++++++++++++ 16 files changed, 255 insertions(+), 9 deletions(-) create mode 100644 flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/ExceptionUtilsTest.java diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManager.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManager.java index 730e16847c..8a2fd2651d 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManager.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManager.java @@ -238,6 +238,7 @@ public FlinkOperatorConfiguration getOperatorConfiguration( * * @param baseConfMap The configuration map that should be searched for relevant Flink version * prefixes. + * @param flinkVersion The FlinkVersion to be used * @return A list of relevant Flink version prefixes in order of ascending Flink version. */ protected static List getRelevantVersionPrefixes( @@ -381,6 +382,7 @@ private void applyConfigsFromCurrentSpec( * Get configuration for interacting with session jobs. Similar to the observe configuration for * FlinkDeployments. * + * @param name The name of the job * @param deployment FlinkDeployment for the session cluster * @param sessionJobSpec Session job spec * @return Session job config diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentController.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentController.java index ac0f7356e1..bae03cba2d 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentController.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentController.java @@ -33,6 +33,7 @@ import org.apache.flink.kubernetes.operator.service.FlinkResourceContextFactory; import org.apache.flink.kubernetes.operator.utils.EventRecorder; import org.apache.flink.kubernetes.operator.utils.EventSourceUtils; +import org.apache.flink.kubernetes.operator.utils.ExceptionUtils; import org.apache.flink.kubernetes.operator.utils.KubernetesClientUtils; import org.apache.flink.kubernetes.operator.utils.StatusRecorder; import org.apache.flink.kubernetes.operator.utils.ValidatorUtils; @@ -162,7 +163,7 @@ public UpdateControl reconcile(FlinkDeployment flinkApp, Contex flinkApp, EventRecorder.Type.Warning, "ClusterDeploymentException", - e.getMessage(), + ExceptionUtils.getExceptionMessage(e), EventRecorder.Component.JobManagerDeployment, josdkContext.getClient()); throw new ReconciliationException(e); diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobController.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobController.java index 1e818d6659..4838dea865 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobController.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobController.java @@ -30,6 +30,7 @@ import org.apache.flink.kubernetes.operator.service.FlinkResourceContextFactory; import org.apache.flink.kubernetes.operator.utils.EventRecorder; import org.apache.flink.kubernetes.operator.utils.EventSourceUtils; +import org.apache.flink.kubernetes.operator.utils.ExceptionUtils; import org.apache.flink.kubernetes.operator.utils.KubernetesClientUtils; import org.apache.flink.kubernetes.operator.utils.StatusRecorder; import org.apache.flink.kubernetes.operator.utils.ValidatorUtils; @@ -124,7 +125,7 @@ public UpdateControl reconcile( flinkSessionJob, EventRecorder.Type.Warning, "SessionJobException", - e.getMessage(), + ExceptionUtils.getExceptionMessage(e), EventRecorder.Component.Job, josdkContext.getClient()); throw new ReconciliationException(e); diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/mutator/FlinkResourceMutator.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/mutator/FlinkResourceMutator.java index 606d243b51..47e185c1ce 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/mutator/FlinkResourceMutator.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/mutator/FlinkResourceMutator.java @@ -31,6 +31,7 @@ public interface FlinkResourceMutator extends Plugin { * Mutate deployment and return the mutated Object. * * @param deployment A Flink application or session cluster deployment. + * @return the mutated Flink application or session cluster deployment. */ FlinkDeployment mutateDeployment(FlinkDeployment deployment); @@ -39,6 +40,7 @@ public interface FlinkResourceMutator extends Plugin { * * @param sessionJob the session job to be mutated. * @param session the target session cluster of the session job to be Mutated. + * @return the mutated session job. */ FlinkSessionJob mutateSessionJob(FlinkSessionJob sessionJob, Optional session); @@ -46,6 +48,7 @@ public interface FlinkResourceMutator extends Plugin { * Mutate snapshot and return the mutated Object. * * @param stateSnapshot the snapshot to be mutated. + * @return the mutated snapshot. */ FlinkStateSnapshot mutateStateSnapshot(FlinkStateSnapshot stateSnapshot); } diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/AbstractFlinkResourceObserver.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/AbstractFlinkResourceObserver.java index 9fd9f59687..1baf68f60a 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/AbstractFlinkResourceObserver.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/AbstractFlinkResourceObserver.java @@ -107,6 +107,7 @@ protected boolean isResourceReadyToBeObserved(FlinkResourceContext ctx) { * DEPLOYED state. * * @param ctx Context for resource. + * @return true if the resource was already upgraded, false otherwise. */ protected abstract boolean checkIfAlreadyUpgraded(FlinkResourceContext ctx); } diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/SnapshotTriggerTimestampStore.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/SnapshotTriggerTimestampStore.java index 21cad5e62b..b206840372 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/SnapshotTriggerTimestampStore.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/SnapshotTriggerTimestampStore.java @@ -53,6 +53,7 @@ public class SnapshotTriggerTimestampStore { * updated with this value. * * @param resource Flink resource + * @param snapshotType the snapshot type * @param snapshotsSupplier supplies related snapshot resources * @return instant of last trigger */ @@ -103,6 +104,7 @@ public Instant getLastPeriodicTriggerInstant( * Updates the time a periodic snapshot was last triggered for this resource. * * @param resource Kubernetes resource + * @param snapshotType the snapshot type * @param instant new timestamp */ public void updateLastPeriodicTriggerTimestamp( diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkService.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkService.java index d88aca65be..48a70b3a71 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkService.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkService.java @@ -954,7 +954,15 @@ private void deleteJar(Configuration conf, String jarId) { } } - /** Wait until Deployment is removed, return remaining timeout. */ + /** + * Wait until Deployment is removed, return remaining timeout. + * + * @param name name of the deployment + * @param deployment The deployment resource + * @param propagation DeletePropagation + * @param timeout Timeout to wait + * @return remaining timeout after deletion + */ @VisibleForTesting protected Duration deleteDeploymentBlocking( String name, diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/EventRecorder.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/EventRecorder.java index 02f73fcc17..4af3f1906e 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/EventRecorder.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/EventRecorder.java @@ -122,10 +122,16 @@ public boolean triggerEvent( } /** + * @param resource The resource + * @param type The type + * @param reason the reason + * @param message the message + * @param component the component + * @param messageKey the message key + * @param client the client * @param interval Interval for dedupe. Null mean no dedupe. - * @return */ - public boolean triggerEventWithInterval( + public void triggerEventWithInterval( AbstractFlinkResource resource, Type type, String reason, @@ -134,7 +140,7 @@ public boolean triggerEventWithInterval( String messageKey, KubernetesClient client, @Nullable Duration interval) { - return EventUtils.createOrUpdateEventWithInterval( + EventUtils.createOrUpdateEventWithInterval( client, resource, type, @@ -166,12 +172,18 @@ public boolean triggerEventOnce( } /** + * @param resource The resource + * @param type The type + * @param reason the reason + * @param message the message + * @param component the component + * @param messageKey the message key + * @param client the client * @param interval Interval for dedupe. Null mean no dedupe. * @param dedupePredicate Predicate for dedupe algorithm.. * @param labels Labels to store in meta data for dedupe. Do nothing if null. - * @return */ - public boolean triggerEventWithLabels( + public void triggerEventWithLabels( AbstractFlinkResource resource, Type type, String reason, @@ -182,7 +194,7 @@ public boolean triggerEventWithLabels( @Nullable Duration interval, @Nullable Predicate> dedupePredicate, @Nullable Map labels) { - return EventUtils.createOrUpdateEventWithLabels( + EventUtils.createOrUpdateEventWithLabels( client, resource, type, diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/ExceptionUtils.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/ExceptionUtils.java index 98a9393b4c..ad7bd6be92 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/ExceptionUtils.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/ExceptionUtils.java @@ -25,9 +25,17 @@ /** Exception utils. * */ public class ExceptionUtils { + private static final int EXCEPTION_LIMIT_FOR_EVENT_MESSAGE = 3; + /** * Based on the flink ExceptionUtils#findThrowableSerializedAware but fixes an infinite loop bug * resulting from SerializedThrowable deserialization errors. + * + * @param throwable the throwable to be processed + * @param searchType the type of the exception to search for + * @param classLoader the classloader to use for deserialization + * @param the exception type + * @return the found exception, or empty if it is not found. */ public static Optional findThrowableSerializedAware( Throwable throwable, Class searchType, ClassLoader classLoader) { @@ -57,4 +65,56 @@ public static Optional findThrowableSerializedAware( return Optional.empty(); } + + /** + * traverse the throwable and extract useful information for up to the first 3 possible + * exceptions in the hierarchy. + * + * @param throwable the throwable to be processed + * @return the exception message, which will have a format similar to "cause1 → cause2 + * → cause3" + */ + public static String getExceptionMessage(Throwable throwable) { + return getExceptionMessage(throwable, 0); + } + + /** + * Helper for recursion for `getExceptionMessage`. + * + * @param throwable the throwable to be processed + * @param level the level we are in. The caller will set this value to 0, and we will be + * incrementing it with each recursive call + * @return the exception message, which will have a format similar to "cause1 -> cause2 -> + * cause3" + */ + private static String getExceptionMessage(Throwable throwable, int level) { + if (throwable == null) { + return null; + } + + if (throwable instanceof SerializedThrowable) { + var deserialized = + ((SerializedThrowable) throwable) + .deserializeError(Thread.currentThread().getContextClassLoader()); + if (deserialized == throwable) { + return "Unknown Error (SerializedThrowable)"; + } else { + return getExceptionMessage(deserialized, level); + } + } + + var msg = + Optional.ofNullable(throwable.getMessage()) + .orElse(throwable.getClass().getSimpleName()); + + if (level >= EXCEPTION_LIMIT_FOR_EVENT_MESSAGE) { + return msg; + } + + if (throwable.getCause() == null) { + return msg; + } else { + return msg + " -> " + getExceptionMessage(throwable.getCause(), level + 1); + } + } } diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/SnapshotUtils.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/SnapshotUtils.java index f8ce07f44c..7ce3db3193 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/SnapshotUtils.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/SnapshotUtils.java @@ -134,6 +134,7 @@ public static SnapshotStatus getLastSnapshotStatus( * @param resource The resource to be snapshotted. * @param conf The observe configuration of the resource. * @param snapshotType The type of the snapshot. + * @param lastTrigger the last time the snapshot was triggered. * @return An optional {@link SnapshotTriggerType}. */ @VisibleForTesting diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/StatusRecorder.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/StatusRecorder.java index 8b4fc630d4..e06aa11193 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/StatusRecorder.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/StatusRecorder.java @@ -85,6 +85,7 @@ public void notifyListeners(CR resource, STATUS prevStatus) { * operator behavior. * * @param resource Resource for which status update should be performed + * @param client Kubernetes client to use for the update */ @SneakyThrows public void patchAndCacheStatus(CR resource, KubernetesClient client) { diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/validation/FlinkResourceValidator.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/validation/FlinkResourceValidator.java index 33f30d9965..8f2a37103e 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/validation/FlinkResourceValidator.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/validation/FlinkResourceValidator.java @@ -50,6 +50,7 @@ Optional validateSessionJob( * Validate and return optional error. * * @param savepoint the savepoint to be validated. + * @param target the target resource of the savepoint to be validated. * @return Optional error string, should be present iff validation resulted in an error */ Optional validateStateSnapshot( 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 c18e8f34c5..3dfc34a909 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 @@ -130,6 +130,7 @@ public class TestingFlinkService extends AbstractFlinkService { @Setter private boolean checkpointAvailable = true; @Setter private boolean jobManagerReady = true; @Setter private boolean deployFailure = false; + @Setter private Exception makeItFailWith; @Setter private boolean triggerSavepointFailure = false; @Setter private boolean disposeSavepointFailure = false; @Setter private Runnable sessionJobSubmittedCallback; @@ -212,6 +213,9 @@ public void submitApplicationCluster( } protected void deployApplicationCluster(JobSpec jobSpec, Configuration conf) throws Exception { + if (makeItFailWith != null) { + throw makeItFailWith; + } if (deployFailure) { throw new Exception("Deployment failure"); } @@ -270,6 +274,10 @@ public JobID submitJobToSessionCluster( @Nullable String savepoint) throws Exception { + if (makeItFailWith != null) { + throw makeItFailWith; + } + if (deployFailure) { throw new Exception("Deployment failure"); } diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentControllerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentControllerTest.java index 0354a1256e..18273c6372 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentControllerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentControllerTest.java @@ -41,6 +41,7 @@ import org.apache.flink.kubernetes.operator.utils.EventRecorder; import org.apache.flink.kubernetes.operator.utils.IngressUtils; import org.apache.flink.runtime.client.JobStatusMessage; +import org.apache.flink.util.SerializedThrowable; import io.fabric8.kubernetes.api.model.Event; import io.fabric8.kubernetes.api.model.EventBuilder; @@ -984,6 +985,33 @@ public void testEventOfNonDeploymentFailedException() throws Exception { assertEquals("Deployment failure", event.getMessage()); } + @Test + public void testEventOfNonDeploymentFailedChainedException() { + assertTrue(testController.flinkResourceEvents().isEmpty()); + var flinkDeployment = TestUtils.buildApplicationCluster(); + + flinkService.setMakeItFailWith( + new RuntimeException( + "Deployment Failure", + new IllegalStateException( + null, + new SerializedThrowable(new Exception("actual failure reason"))))); + try { + testController.reconcile(flinkDeployment, context); + fail(); + } catch (Exception expected) { + } + assertEquals(2, testController.flinkResourceEvents().size()); + + var event = testController.flinkResourceEvents().remove(); + assertEquals("Submit", event.getReason()); + event = testController.flinkResourceEvents().remove(); + assertEquals("ClusterDeploymentException", event.getReason()); + assertEquals( + "Deployment Failure -> IllegalStateException -> actual failure reason", + event.getMessage()); + } + @Test public void cleanUpNewDeployment() { FlinkDeployment flinkDeployment = TestUtils.buildApplicationCluster(); @@ -1064,6 +1092,32 @@ public void testInitialSavepointOnError() throws Exception { assertEquals("msp", flinkService.listJobs().get(0).f0); } + @Test + public void testErrorOnReconcileWithChainedExceptions() throws Exception { + FlinkDeployment flinkDeployment = TestUtils.buildApplicationCluster(); + flinkDeployment.getSpec().getJob().setInitialSavepointPath("msp"); + flinkService.setMakeItFailWith( + new RuntimeException( + "Deployment Failure", + new IllegalStateException( + null, + new SerializedThrowable(new Exception("actual failure reason"))))); + try { + testController.reconcile(flinkDeployment, context); + fail(); + } catch (Exception expected) { + } + assertEquals(2, testController.flinkResourceEvents().size()); + + var event = testController.flinkResourceEvents().remove(); + assertEquals("Submit", event.getReason()); + event = testController.flinkResourceEvents().remove(); + assertEquals("ClusterDeploymentException", event.getReason()); + assertEquals( + "Deployment Failure -> IllegalStateException -> actual failure reason", + event.getMessage()); + } + @Test public void testInitialHaError() throws Exception { var appCluster = TestUtils.buildApplicationCluster(FlinkVersion.v1_20); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobControllerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobControllerTest.java index 0a5490890f..b7ad6f1352 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobControllerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobControllerTest.java @@ -35,6 +35,7 @@ import org.apache.flink.kubernetes.operator.service.CheckpointHistoryWrapper; import org.apache.flink.kubernetes.operator.utils.EventRecorder; import org.apache.flink.runtime.client.JobStatusMessage; +import org.apache.flink.util.SerializedThrowable; import io.fabric8.kubernetes.client.KubernetesClient; import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient; @@ -615,6 +616,31 @@ public void testInitialSavepointOnError() throws Exception { assertEquals("msp", flinkService.listJobs().get(0).f0); } + @Test + public void testErrorOnReconcileWithChainedExceptions() throws Exception { + sessionJob.getSpec().getJob().setInitialSavepointPath("msp"); + flinkService.setMakeItFailWith( + new RuntimeException( + "Deployment Failure", + new IllegalStateException( + null, + new SerializedThrowable(new Exception("actual failure reason"))))); + try { + testController.reconcile(sessionJob, context); + fail(); + } catch (Exception expected) { + } + assertEquals(2, testController.events().size()); + + var event = testController.events().remove(); + assertEquals("Submit", event.getReason()); + event = testController.events().remove(); + assertEquals("SessionJobException", event.getReason()); + assertEquals( + "Deployment Failure -> IllegalStateException -> actual failure reason", + event.getMessage()); + } + @Test public void verifyCanaryHandling() throws Exception { var canary = TestUtils.createCanaryJob(); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/ExceptionUtilsTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/ExceptionUtilsTest.java new file mode 100644 index 0000000000..1bd8873d36 --- /dev/null +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/ExceptionUtilsTest.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.kubernetes.operator.utils; + +import org.apache.flink.util.SerializedThrowable; + +import org.junit.jupiter.api.Test; + +import java.io.IOException; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link ExceptionUtils}. */ +public class ExceptionUtilsTest { + + @Test + void testGetExceptionMessage_nullThrowable() { + assertThat(ExceptionUtils.getExceptionMessage(null)).isNull(); + } + + @Test + void testGetExceptionMessage_serializedThrowable() { + var serializedException = new SerializedThrowable(new Exception("Serialized Exception")); + assertThat(ExceptionUtils.getExceptionMessage(serializedException)) + .isEqualTo("Serialized Exception"); + } + + @Test + void testGetExceptionMessage_differentKindsOfExceptions() { + var ex4 = new RuntimeException("Cause 4"); + var ex3 = new RuntimeException("Cause 3", ex4); + var ex2 = new RuntimeException("Cause 2", new SerializedThrowable(ex3)); + var ex = new RuntimeException("Cause 1", ex2); + assertThat(ExceptionUtils.getExceptionMessage(ex)) + .isEqualTo("Cause 1 -> Cause 2 -> Cause 3 -> Cause 4"); + } + + @Test + void testSerializedThrowableError() { + var serializedException = new SerializedThrowable(new NonSerializableException()); + assertThat(ExceptionUtils.getExceptionMessage(serializedException)) + .isEqualTo("Unknown Error (SerializedThrowable)"); + } + + private static class NonSerializableException extends Exception { + private void writeObject(java.io.ObjectOutputStream stream) throws IOException { + throw new IOException(); + } + } +} From 93e68f2bc2a007dfcfe46a230a7ce49eaf3c7736 Mon Sep 17 00:00:00 2001 From: Gyula Fora Date: Thu, 27 Mar 2025 11:45:22 +0100 Subject: [PATCH 12/19] [FLINK-37571] Fix JobGraph removal for 2.0 last-state upgrades --- .../flink/kubernetes/operator/utils/FlinkUtils.java | 3 ++- .../kubernetes/operator/utils/FlinkUtilsTest.java | 10 ++++++---- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/FlinkUtils.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/FlinkUtils.java index 4c41aedf8f..dbd68ba4ed 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/FlinkUtils.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/FlinkUtils.java @@ -335,7 +335,8 @@ private static boolean checkpointExists(ConfigMap cm) { } private static boolean isJobGraphKey(Map.Entry entry) { - return entry.getKey().startsWith(Constants.JOB_GRAPH_STORE_KEY_PREFIX); + return entry.getKey().startsWith(Constants.JOB_GRAPH_STORE_KEY_PREFIX) + || entry.getKey().startsWith("executionPlan-"); } public static boolean isZookeeperHAActivated(Configuration configuration) { diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/FlinkUtilsTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/FlinkUtilsTest.java index 68ee3e3d95..003557f91e 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/FlinkUtilsTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/FlinkUtilsTest.java @@ -18,7 +18,6 @@ package org.apache.flink.kubernetes.operator.utils; -import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.HighAvailabilityOptions; @@ -51,6 +50,8 @@ import io.fabric8.kubernetes.client.server.mock.KubernetesMockServer; import io.javaoperatorsdk.operator.processing.event.ResourceID; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import java.net.HttpURLConnection; import java.util.Collections; @@ -125,12 +126,13 @@ public void testAddStartupProbe() { assertEquals(expectedProbe, pod.getSpec().getContainers().get(1).getStartupProbe()); } - @Test - public void testDeleteJobGraphInKubernetesHA() { + @ParameterizedTest + @ValueSource(strings = {"jobGraph-jobId", "executionPlan-jobId"}) + public void testDeleteJobGraphInKubernetesHA(String key) { final String name = "ha-configmap"; final String clusterId = "cluster-id"; final Map data = new HashMap<>(); - data.put(Constants.JOB_GRAPH_STORE_KEY_PREFIX + JobID.generate(), "job-graph-data"); + data.put(key, "job-graph-data"); data.put("leader", "localhost"); createHAConfigMapWithData(name, kubernetesClient.getNamespace(), clusterId, data); assertNotNull(kubernetesClient.configMaps().withName(name).get()); From 86ab948ed8d71a12ce2ef68f5a217e3683005bf5 Mon Sep 17 00:00:00 2001 From: Gyula Fora Date: Fri, 14 Mar 2025 08:36:15 +0100 Subject: [PATCH 13/19] [FLINK-37562] Do not check all JM replicas in observer --- .../deployment/AbstractFlinkDeploymentObserver.java | 7 +++---- .../org/apache/flink/kubernetes/operator/TestUtils.java | 4 ++-- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/deployment/AbstractFlinkDeploymentObserver.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/deployment/AbstractFlinkDeploymentObserver.java index 713e132ad2..cd817ec89d 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/deployment/AbstractFlinkDeploymentObserver.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/deployment/AbstractFlinkDeploymentObserver.java @@ -38,7 +38,6 @@ import io.fabric8.kubernetes.api.model.PodList; import io.fabric8.kubernetes.api.model.apps.Deployment; import io.fabric8.kubernetes.api.model.apps.DeploymentCondition; -import io.fabric8.kubernetes.api.model.apps.DeploymentSpec; import io.fabric8.kubernetes.api.model.apps.DeploymentStatus; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -117,11 +116,11 @@ protected void observeJmDeployment(FlinkResourceContext ctx) { ctx.getJosdkContext().getSecondaryResource(Deployment.class); if (deployment.isPresent()) { DeploymentStatus status = deployment.get().getStatus(); - DeploymentSpec spec = deployment.get().getSpec(); if (status != null && status.getAvailableReplicas() != null - && spec.getReplicas().intValue() == status.getReplicas() - && spec.getReplicas().intValue() == status.getAvailableReplicas() + // One available JM is enough to run the job correctly + && status.getReplicas() > 0 + && status.getAvailableReplicas() > 0 && ctx.getFlinkService().isJobManagerPortReady(ctx.getObserveConfig())) { // typically it takes a few seconds for the REST server to be ready 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 6af375edb3..d9492bfd60 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 @@ -117,7 +117,7 @@ public static Deployment createDeployment(boolean ready) { String nowTs = Instant.now().toString(); var status = new DeploymentStatus(); status.setAvailableReplicas(ready ? 1 : 0); - status.setReplicas(1); + status.setReplicas(2); var availableCondition = new DeploymentCondition(); availableCondition.setType("Available"); availableCondition.setStatus(ready ? "True" : "False"); @@ -125,7 +125,7 @@ public static Deployment createDeployment(boolean ready) { status.setConditions(List.of(availableCondition)); DeploymentSpec spec = new DeploymentSpec(); - spec.setReplicas(1); + spec.setReplicas(3); var meta = new ObjectMeta(); meta.setCreationTimestamp(nowTs); From b18bdb47fd18e6f51b79bc1321f7917d95021c04 Mon Sep 17 00:00:00 2001 From: Gyula Fora Date: Fri, 7 Mar 2025 14:40:05 +0100 Subject: [PATCH 14/19] [hotfix] Do not leak autoscaler configs to jobs --- .../operator/service/AbstractFlinkService.java | 4 +++- .../operator/service/AbstractFlinkServiceTest.java | 11 ++++++++--- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkService.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkService.java index 48a70b3a71..733ebd3ae2 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkService.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkService.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.autoscaler.config.AutoScalerOptions; import org.apache.flink.autoscaler.utils.JobStatusUtils; import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.CheckpointingOptions; @@ -984,7 +985,8 @@ protected static Configuration removeOperatorConfigs(Configuration config) { config.toMap() .forEach( (k, v) -> { - if (!k.startsWith(K8S_OP_CONF_PREFIX)) { + if (!k.startsWith(K8S_OP_CONF_PREFIX) + && !k.startsWith(AutoScalerOptions.AUTOSCALER_CONF_PREFIX)) { newConfig.setString(k, v); } }); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkServiceTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkServiceTest.java index 9082ff358a..a57d60a8a3 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkServiceTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkServiceTest.java @@ -987,10 +987,15 @@ public void fetchCheckpointInfoTest() throws Exception { @Test public void removeOperatorConfigTest() { - var key = "kubernetes.operator.meyKey"; - var deployConfig = Configuration.fromMap(Map.of("kubernetes.operator.meyKey", "v")); + var opKey1 = "kubernetes.operator.meyKey"; + var opKey2 = "job.autoscaler."; + var regularKey = "k"; + var deployConfig = + Configuration.fromMap(Map.of(opKey1, "v", opKey2, "v", regularKey, "v1")); var newConf = AbstractFlinkService.removeOperatorConfigs(deployConfig); - assertFalse(newConf.containsKey(key)); + assertFalse(newConf.containsKey(opKey1)); + assertFalse(newConf.containsKey(opKey2)); + assertTrue(newConf.containsKey(regularKey)); } @Test From cda493e52e97c09886a466cb9ecad8b679bedcb0 Mon Sep 17 00:00:00 2001 From: Gyula Fora Date: Fri, 28 Mar 2025 08:46:14 +0100 Subject: [PATCH 15/19] [FLINK-37455] Create error Event when job goes into FAILED state --- .../controller/FlinkDeploymentController.java | 45 +++++-------------- .../controller/FlinkSessionJobController.java | 18 +++++--- .../operator/observer/JobStatusObserver.java | 18 +++++--- .../operator/utils/EventRecorder.java | 3 +- .../operator/utils/ExceptionUtils.java | 9 ++-- .../operator/TestingFlinkService.java | 22 +++++++++ .../FlinkDeploymentControllerTest.java | 6 +-- .../FlinkSessionJobControllerTest.java | 4 +- .../observer/JobStatusObserverTest.java | 33 ++++++++++++++ .../operator/utils/ExceptionUtilsTest.java | 21 +++++++-- 10 files changed, 119 insertions(+), 60 deletions(-) diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentController.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentController.java index bae03cba2d..51235b8ebe 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentController.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentController.java @@ -155,17 +155,15 @@ public UpdateControl reconcile(FlinkDeployment flinkApp, Contex statusRecorder.patchAndCacheStatus(flinkApp, ctx.getKubernetesClient()); reconcilerFactory.getOrCreate(flinkApp).reconcile(ctx); } catch (UpgradeFailureException ufe) { - handleUpgradeFailure(ctx, ufe); + ReconciliationUtils.updateForReconciliationError(ctx, ufe); + triggerErrorEvent(ctx, ufe, ufe.getReason()); } catch (DeploymentFailedException dfe) { - handleDeploymentFailed(ctx, dfe); + flinkApp.getStatus().setJobManagerDeploymentStatus(JobManagerDeploymentStatus.ERROR); + flinkApp.getStatus().getJobStatus().setState(JobStatus.RECONCILING); + ReconciliationUtils.updateForReconciliationError(ctx, dfe); + triggerErrorEvent(ctx, dfe, dfe.getReason()); } catch (Exception e) { - eventRecorder.triggerEvent( - flinkApp, - EventRecorder.Type.Warning, - "ClusterDeploymentException", - ExceptionUtils.getExceptionMessage(e), - EventRecorder.Component.JobManagerDeployment, - josdkContext.getClient()); + triggerErrorEvent(ctx, e, EventRecorder.Reason.Error.name()); throw new ReconciliationException(e); } @@ -175,32 +173,13 @@ public UpdateControl reconcile(FlinkDeployment flinkApp, Contex ctx.getOperatorConfig(), flinkApp, previousDeployment, true); } - private void handleDeploymentFailed( - FlinkResourceContext ctx, DeploymentFailedException dfe) { - var flinkApp = ctx.getResource(); - LOG.error("Flink Deployment failed", dfe); - flinkApp.getStatus().setJobManagerDeploymentStatus(JobManagerDeploymentStatus.ERROR); - flinkApp.getStatus().getJobStatus().setState(JobStatus.RECONCILING); - ReconciliationUtils.updateForReconciliationError(ctx, dfe); + private void triggerErrorEvent( + FlinkResourceContext ctx, Exception e, String reason) { eventRecorder.triggerEvent( - flinkApp, - EventRecorder.Type.Warning, - dfe.getReason(), - dfe.getMessage(), - EventRecorder.Component.JobManagerDeployment, - ctx.getKubernetesClient()); - } - - private void handleUpgradeFailure( - FlinkResourceContext ctx, UpgradeFailureException ufe) { - LOG.error("Error while upgrading Flink Deployment", ufe); - var flinkApp = ctx.getResource(); - ReconciliationUtils.updateForReconciliationError(ctx, ufe); - eventRecorder.triggerEvent( - flinkApp, + ctx.getResource(), EventRecorder.Type.Warning, - ufe.getReason(), - ufe.getMessage(), + reason, + ExceptionUtils.getExceptionMessage(e), EventRecorder.Component.JobManagerDeployment, ctx.getKubernetesClient()); } diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobController.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobController.java index 4838dea865..7454864fe7 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobController.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobController.java @@ -121,13 +121,7 @@ public UpdateControl reconcile( statusRecorder.patchAndCacheStatus(flinkSessionJob, ctx.getKubernetesClient()); reconciler.reconcile(ctx); } catch (Exception e) { - eventRecorder.triggerEvent( - flinkSessionJob, - EventRecorder.Type.Warning, - "SessionJobException", - ExceptionUtils.getExceptionMessage(e), - EventRecorder.Component.Job, - josdkContext.getClient()); + triggerErrorEvent(ctx, e); throw new ReconciliationException(e); } statusRecorder.patchAndCacheStatus(flinkSessionJob, ctx.getKubernetesClient()); @@ -167,6 +161,16 @@ public DeleteControl cleanup(FlinkSessionJob sessionJob, Context josdkContext) { return deleteControl; } + private void triggerErrorEvent(FlinkResourceContext ctx, Exception e) { + eventRecorder.triggerEvent( + ctx.getResource(), + EventRecorder.Type.Warning, + EventRecorder.Reason.Error.name(), + ExceptionUtils.getExceptionMessage(e), + EventRecorder.Component.Job, + ctx.getKubernetesClient()); + } + @Override public ErrorStatusUpdateControl updateErrorStatus( FlinkSessionJob sessionJob, Context context, Exception e) { diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserver.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserver.java index c94c1231b3..444d0a0ad5 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserver.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserver.java @@ -27,6 +27,7 @@ import org.apache.flink.kubernetes.operator.controller.FlinkResourceContext; import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; import org.apache.flink.kubernetes.operator.utils.EventRecorder; +import org.apache.flink.kubernetes.operator.utils.ExceptionUtils; import org.apache.flink.runtime.client.JobStatusMessage; import org.slf4j.Logger; @@ -182,7 +183,7 @@ private void updateJobStatus(FlinkResourceContext ctx, JobStatusMessage clust markSuspended(resource); } - setErrorIfPresent(ctx, clusterJobStatus); + recordJobErrorIfPresent(ctx, clusterJobStatus); eventRecorder.triggerEvent( resource, EventRecorder.Type.Normal, @@ -203,7 +204,8 @@ private static void markSuspended(AbstractFlinkResource resource) { }); } - private void setErrorIfPresent(FlinkResourceContext ctx, JobStatusMessage clusterJobStatus) { + private void recordJobErrorIfPresent( + FlinkResourceContext ctx, JobStatusMessage clusterJobStatus) { if (clusterJobStatus.getJobState() == JobStatus.FAILED) { try { var result = @@ -215,10 +217,14 @@ private void setErrorIfPresent(FlinkResourceContext ctx, JobStatusMessage clu t -> { updateFlinkResourceException( t, ctx.getResource(), ctx.getOperatorConfig()); - LOG.error( - "Job {} failed with error: {}", - clusterJobStatus.getJobId(), - t.getFullStringifiedStackTrace()); + + eventRecorder.triggerEvent( + ctx.getResource(), + EventRecorder.Type.Warning, + EventRecorder.Reason.Error, + EventRecorder.Component.Job, + ExceptionUtils.getExceptionMessage(t), + ctx.getKubernetesClient()); }); } catch (Exception e) { LOG.warn("Failed to request the job result", e); diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/EventRecorder.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/EventRecorder.java index 4af3f1906e..1989de0e14 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/EventRecorder.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/EventRecorder.java @@ -314,6 +314,7 @@ public enum Reason { Scaling, UnsupportedFlinkVersion, SnapshotError, - SnapshotAbandoned + SnapshotAbandoned, + Error } } diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/ExceptionUtils.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/ExceptionUtils.java index ad7bd6be92..002c43b79d 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/ExceptionUtils.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/ExceptionUtils.java @@ -75,7 +75,7 @@ public static Optional findThrowableSerializedAware( * → cause3" */ public static String getExceptionMessage(Throwable throwable) { - return getExceptionMessage(throwable, 0); + return getExceptionMessage(throwable, 1); } /** @@ -93,11 +93,12 @@ private static String getExceptionMessage(Throwable throwable, int level) { } if (throwable instanceof SerializedThrowable) { + var serialized = ((SerializedThrowable) throwable); var deserialized = - ((SerializedThrowable) throwable) - .deserializeError(Thread.currentThread().getContextClassLoader()); + serialized.deserializeError(Thread.currentThread().getContextClassLoader()); if (deserialized == throwable) { - return "Unknown Error (SerializedThrowable)"; + var msg = serialized.getMessage(); + return msg != null ? msg : serialized.getOriginalErrorClassName(); } else { return getExceptionMessage(deserialized, level); } 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 3dfc34a909..fa893e7c49 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 @@ -52,6 +52,7 @@ import org.apache.flink.kubernetes.operator.service.SuspendMode; import org.apache.flink.kubernetes.operator.standalone.StandaloneKubernetesConfigOptionsInternal; import org.apache.flink.runtime.client.JobStatusMessage; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; @@ -141,6 +142,7 @@ public class TestingFlinkService extends AbstractFlinkService { @Getter private final Map checkpointTriggers = new HashMap<>(); private final Map checkpointStats = new HashMap<>(); @Setter private boolean throwCheckpointingDisabledError = false; + @Setter private Throwable jobFailedErr; @Getter private int desiredReplicas = 0; @Getter private int cancelJobCallCount = 0; @@ -301,9 +303,29 @@ public Optional getJobStatus(Configuration conf, JobID jobID) if (!isPortReady) { throw new TimeoutException("JM port is unavailable"); } + + if (jobFailedErr != null) { + return Optional.of(new JobStatusMessage(jobID, "n", JobStatus.FAILED, 0)); + } + return super.getJobStatus(conf, jobID); } + @Override + public JobResult requestJobResult(Configuration conf, JobID jobID) throws Exception { + if (jobFailedErr != null) { + return new JobResult.Builder() + .jobId(jobID) + .serializedThrowable(new SerializedThrowable(jobFailedErr)) + .netRuntime(1) + .accumulatorResults(new HashMap<>()) + .applicationStatus(ApplicationStatus.FAILED) + .build(); + } + + return super.requestJobResult(conf, jobID); + } + public List> listJobs() { return jobs; } diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentControllerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentControllerTest.java index 18273c6372..8c3bd33fb3 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentControllerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentControllerTest.java @@ -981,7 +981,7 @@ public void testEventOfNonDeploymentFailedException() throws Exception { var event = testController.flinkResourceEvents().remove(); assertEquals("Submit", event.getReason()); event = testController.flinkResourceEvents().remove(); - assertEquals("ClusterDeploymentException", event.getReason()); + assertEquals("Error", event.getReason()); assertEquals("Deployment failure", event.getMessage()); } @@ -1006,7 +1006,7 @@ public void testEventOfNonDeploymentFailedChainedException() { var event = testController.flinkResourceEvents().remove(); assertEquals("Submit", event.getReason()); event = testController.flinkResourceEvents().remove(); - assertEquals("ClusterDeploymentException", event.getReason()); + assertEquals("Error", event.getReason()); assertEquals( "Deployment Failure -> IllegalStateException -> actual failure reason", event.getMessage()); @@ -1112,7 +1112,7 @@ public void testErrorOnReconcileWithChainedExceptions() throws Exception { var event = testController.flinkResourceEvents().remove(); assertEquals("Submit", event.getReason()); event = testController.flinkResourceEvents().remove(); - assertEquals("ClusterDeploymentException", event.getReason()); + assertEquals("Error", event.getReason()); assertEquals( "Deployment Failure -> IllegalStateException -> actual failure reason", event.getMessage()); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobControllerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobControllerTest.java index b7ad6f1352..f0489cd21a 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobControllerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobControllerTest.java @@ -103,7 +103,7 @@ public void testSubmitJobButException() { var event = testController.events().remove(); Assertions.assertEquals(EventRecorder.Type.Warning.toString(), event.getType()); - Assertions.assertEquals("SessionJobException", event.getReason()); + Assertions.assertEquals("Error", event.getReason()); testController.cleanup(sessionJob, context); } @@ -635,7 +635,7 @@ public void testErrorOnReconcileWithChainedExceptions() throws Exception { var event = testController.events().remove(); assertEquals("Submit", event.getReason()); event = testController.events().remove(); - assertEquals("SessionJobException", event.getReason()); + assertEquals("Error", event.getReason()); assertEquals( "Deployment Failure -> IllegalStateException -> actual failure reason", event.getMessage()); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserverTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserverTest.java index 512e9f7c49..aee49e3c3c 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserverTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserverTest.java @@ -29,10 +29,12 @@ import org.apache.flink.kubernetes.operator.api.spec.UpgradeMode; import org.apache.flink.kubernetes.operator.controller.FlinkResourceContext; import org.apache.flink.kubernetes.operator.utils.EventRecorder; +import org.apache.flink.util.SerializedThrowable; import io.fabric8.kubernetes.client.KubernetesClient; import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient; import lombok.Getter; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.EnumSource; @@ -42,6 +44,7 @@ import java.util.stream.Stream; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; /** Tests for the {@link JobStatusObserver}. */ @EnableKubernetesMockClient(crud = true) @@ -114,6 +117,36 @@ void testCancellingToTerminal(JobStatus fromStatus) throws Exception { .getState()); } + @Test + void testFailed() throws Exception { + var observer = new JobStatusObserver<>(eventRecorder); + var deployment = initDeployment(); + var status = deployment.getStatus(); + var jobStatus = status.getJobStatus(); + jobStatus.setState(JobStatus.RUNNING); + FlinkResourceContext> ctx = getResourceContext(deployment); + flinkService.submitApplicationCluster( + deployment.getSpec().getJob(), ctx.getDeployConfig(deployment.getSpec()), false); + + // Mark failed + flinkService.setJobFailedErr( + new Exception("job err", new SerializedThrowable(new Exception("root")))); + observer.observe(ctx); + + // First event should be job error reported + var jobErrorEvent = flinkResourceEventCollector.events.poll(); + assertEquals(EventRecorder.Reason.Error.name(), jobErrorEvent.getReason()); + assertEquals("job err -> root", jobErrorEvent.getMessage()); + + // Make sure job status still reported + assertEquals( + EventRecorder.Reason.JobStatusChanged.name(), + flinkResourceEventCollector.events.poll().getReason()); + + observer.observe(ctx); + assertTrue(flinkResourceEventCollector.events.isEmpty()); + } + private static Stream cancellingArgs() { var args = new ArrayList(); for (var status : JobStatus.values()) { diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/ExceptionUtilsTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/ExceptionUtilsTest.java index 1bd8873d36..144ec3189c 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/ExceptionUtilsTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/ExceptionUtilsTest.java @@ -47,17 +47,30 @@ void testGetExceptionMessage_differentKindsOfExceptions() { var ex2 = new RuntimeException("Cause 2", new SerializedThrowable(ex3)); var ex = new RuntimeException("Cause 1", ex2); assertThat(ExceptionUtils.getExceptionMessage(ex)) - .isEqualTo("Cause 1 -> Cause 2 -> Cause 3 -> Cause 4"); + .isEqualTo("Cause 1 -> Cause 2 -> Cause 3"); } @Test void testSerializedThrowableError() { - var serializedException = new SerializedThrowable(new NonSerializableException()); - assertThat(ExceptionUtils.getExceptionMessage(serializedException)) - .isEqualTo("Unknown Error (SerializedThrowable)"); + assertThat( + ExceptionUtils.getExceptionMessage( + new SerializedThrowable(new NonSerializableException("Message")))) + .isEqualTo(String.format("%s: Message", NonSerializableException.class.getName())); + + assertThat( + ExceptionUtils.getExceptionMessage( + new SerializedThrowable(new NonSerializableException()))) + .isEqualTo(NonSerializableException.class.getName()); } private static class NonSerializableException extends Exception { + + public NonSerializableException(String message) { + super(message); + } + + public NonSerializableException() {} + private void writeObject(java.io.ObjectOutputStream stream) throws IOException { throw new IOException(); } From 1578a2960c987435b79113db5152ea4c57f4e9de Mon Sep 17 00:00:00 2001 From: Gyula Fora Date: Fri, 28 Mar 2025 11:01:47 +0100 Subject: [PATCH 16/19] [FLINK-37470] Improve JobManager Deployment / Pod error handling --- flink-kubernetes-operator/pom.xml | 7 + .../exception/DeploymentFailedException.java | 43 +++-- .../AbstractFlinkDeploymentObserver.java | 37 +++-- .../kubernetes/operator/utils/EventUtils.java | 93 ++++++++++- .../flink/kubernetes/operator/TestUtils.java | 21 ++- .../FlinkDeploymentControllerTest.java | 94 +---------- .../DeploymentFailedExceptionTest.java | 69 ++++++++ .../deployment/ApplicationObserverTest.java | 27 +++- .../operator/utils/PodErrorTest.java | 148 ++++++++++++++++++ 9 files changed, 407 insertions(+), 132 deletions(-) create mode 100644 flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/exception/DeploymentFailedExceptionTest.java create mode 100644 flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/PodErrorTest.java diff --git a/flink-kubernetes-operator/pom.xml b/flink-kubernetes-operator/pom.xml index b455fffa5a..cff4a43f9e 100644 --- a/flink-kubernetes-operator/pom.xml +++ b/flink-kubernetes-operator/pom.xml @@ -184,6 +184,13 @@ under the License. + + io.fabric8 + kube-api-test-client-inject + ${fabric8.version} + test + + com.squareup.okhttp3 mockwebserver diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/exception/DeploymentFailedException.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/exception/DeploymentFailedException.java index a8f9abf0b3..9fc6143f06 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/exception/DeploymentFailedException.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/exception/DeploymentFailedException.java @@ -17,15 +17,25 @@ package org.apache.flink.kubernetes.operator.exception; -import io.fabric8.kubernetes.api.model.ContainerStateWaiting; +import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableSet; + +import io.fabric8.kubernetes.api.model.ContainerStatus; import io.fabric8.kubernetes.api.model.apps.DeploymentCondition; +import java.util.Optional; +import java.util.Set; + /** Exception to signal terminal deployment failure. */ public class DeploymentFailedException extends RuntimeException { - public static final String REASON_CRASH_LOOP_BACKOFF = "CrashLoopBackOff"; - public static final String REASON_IMAGE_PULL_BACKOFF = "ImagePullBackOff"; - public static final String REASON_ERR_IMAGE_PULL = "ErrImagePull"; + public static final Set CONTAINER_ERROR_REASONS = + ImmutableSet.of( + "CrashLoopBackOff", + "ImagePullBackOff", + "ErrImagePull", + "RunContainerError", + "CreateContainerConfigError", + "OOMKilled"); private static final long serialVersionUID = -1070179896083579221L; @@ -36,11 +46,6 @@ public DeploymentFailedException(DeploymentCondition deployCondition) { this.reason = deployCondition.getReason(); } - public DeploymentFailedException(ContainerStateWaiting stateWaiting) { - super(stateWaiting.getMessage()); - this.reason = stateWaiting.getReason(); - } - public DeploymentFailedException(String message, String reason) { super(message); this.reason = reason; @@ -49,4 +54,24 @@ public DeploymentFailedException(String message, String reason) { public String getReason() { return reason; } + + public static DeploymentFailedException forContainerStatus(ContainerStatus status) { + var waiting = status.getState().getWaiting(); + var lastState = status.getLastState(); + String message = null; + if ("CrashLoopBackOff".equals(waiting.getReason()) + && lastState != null + && lastState.getTerminated() != null) { + message = + Optional.ofNullable(lastState.getTerminated().getMessage()) + .map(err -> "CrashLoop - " + err) + .orElse(null); + } + + if (message == null) { + message = waiting.getMessage(); + } + return new DeploymentFailedException( + String.format("[%s] %s", status.getName(), message), waiting.getReason()); + } } diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/deployment/AbstractFlinkDeploymentObserver.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/deployment/AbstractFlinkDeploymentObserver.java index cd817ec89d..54e1573bfb 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/deployment/AbstractFlinkDeploymentObserver.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/deployment/AbstractFlinkDeploymentObserver.java @@ -30,9 +30,9 @@ import org.apache.flink.kubernetes.operator.observer.AbstractFlinkResourceObserver; import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; import org.apache.flink.kubernetes.operator.utils.EventRecorder; +import org.apache.flink.kubernetes.operator.utils.EventUtils; import org.apache.flink.kubernetes.operator.utils.FlinkUtils; -import io.fabric8.kubernetes.api.model.ContainerStateWaiting; import io.fabric8.kubernetes.api.model.ContainerStatus; import io.fabric8.kubernetes.api.model.Pod; import io.fabric8.kubernetes.api.model.PodList; @@ -45,7 +45,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.Set; +import java.util.stream.Stream; /** Base observer for session and application clusters. */ public abstract class AbstractFlinkDeploymentObserver @@ -134,7 +134,7 @@ protected void observeJmDeployment(FlinkResourceContext ctx) { try { checkFailedCreate(status); // checking the pod is expensive; only do it when the deployment isn't ready - checkContainerBackoff(ctx); + checkContainerErrors(ctx); } catch (DeploymentFailedException dfe) { // throw only when not already in error status to allow for spec update deploymentStatus.getJobStatus().setState(JobStatus.RECONCILING); @@ -171,21 +171,28 @@ private void checkFailedCreate(DeploymentStatus status) { } } - private void checkContainerBackoff(FlinkResourceContext ctx) { + private void checkContainerErrors(FlinkResourceContext ctx) { PodList jmPods = ctx.getFlinkService().getJmPodList(ctx.getResource(), ctx.getObserveConfig()); for (Pod pod : jmPods.getItems()) { - for (ContainerStatus cs : pod.getStatus().getContainerStatuses()) { - ContainerStateWaiting csw = cs.getState().getWaiting(); - if (csw != null - && Set.of( - DeploymentFailedException.REASON_CRASH_LOOP_BACKOFF, - DeploymentFailedException.REASON_IMAGE_PULL_BACKOFF, - DeploymentFailedException.REASON_ERR_IMAGE_PULL) - .contains(csw.getReason())) { - throw new DeploymentFailedException(csw); - } - } + var podStatus = pod.getStatus(); + Stream.concat( + podStatus.getContainerStatuses().stream(), + podStatus.getInitContainerStatuses().stream()) + .forEach(AbstractFlinkDeploymentObserver::checkContainerError); + + // No obvious errors were found, check for volume mount issues + EventUtils.checkForVolumeMountErrors(ctx.getKubernetesClient(), pod); + } + } + + private static void checkContainerError(ContainerStatus cs) { + if (cs.getState() == null || cs.getState().getWaiting() == null) { + return; + } + if (DeploymentFailedException.CONTAINER_ERROR_REASONS.contains( + cs.getState().getWaiting().getReason())) { + throw DeploymentFailedException.forContainerStatus(cs); } } diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/EventUtils.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/EventUtils.java index b515cdda38..fb0dbfc663 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/EventUtils.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/EventUtils.java @@ -17,11 +17,16 @@ package org.apache.flink.kubernetes.operator.utils; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.kubernetes.operator.exception.DeploymentFailedException; + import io.fabric8.kubernetes.api.model.Event; import io.fabric8.kubernetes.api.model.EventBuilder; import io.fabric8.kubernetes.api.model.HasMetadata; import io.fabric8.kubernetes.api.model.ObjectMeta; -import io.fabric8.kubernetes.api.model.ObjectReferenceBuilder; +import io.fabric8.kubernetes.api.model.ObjectReference; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.PodCondition; import io.fabric8.kubernetes.client.KubernetesClient; import io.fabric8.kubernetes.client.KubernetesClientException; import org.slf4j.Logger; @@ -32,10 +37,14 @@ import java.net.HttpURLConnection; import java.time.Duration; import java.time.Instant; +import java.util.ArrayList; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.function.Consumer; +import java.util.function.Function; import java.util.function.Predicate; +import java.util.stream.Collectors; /** * The util to generate an event for the target resource. It is copied from @@ -182,13 +191,7 @@ private static Event buildEvent( String eventName) { return new EventBuilder() .withApiVersion("v1") - .withInvolvedObject( - new ObjectReferenceBuilder() - .withKind(target.getKind()) - .withUid(target.getMetadata().getUid()) - .withName(target.getMetadata().getName()) - .withNamespace(target.getMetadata().getNamespace()) - .build()) + .withInvolvedObject(getObjectReference(target)) .withType(type.name()) .withReason(reason) .withFirstTimestamp(Instant.now().toString()) @@ -235,4 +238,78 @@ private static Optional createOrReplaceEvent(KubernetesClient client, Eve } return Optional.empty(); } + + private static List getPodEvents(KubernetesClient client, Pod pod) { + var ref = getObjectReference(pod); + + var eventList = + client.v1() + .events() + .inNamespace(pod.getMetadata().getNamespace()) + .withInvolvedObject(ref) + .list(); + + if (eventList == null) { + return new ArrayList<>(); + } + + var items = eventList.getItems(); + if (items == null) { + return new ArrayList<>(); + } + return items; + } + + @VisibleForTesting + protected static ObjectReference getObjectReference(HasMetadata resource) { + var ref = new ObjectReference(); + ref.setApiVersion(resource.getApiVersion()); + ref.setKind(resource.getKind()); + ref.setName(resource.getMetadata().getName()); + ref.setNamespace(resource.getMetadata().getNamespace()); + ref.setUid(resource.getMetadata().getUid()); + return ref; + } + + /** + * Check that pod is stuck during volume mount stage and throw {@link DeploymentFailedException} + * with the right reason message if that's the case. + * + * @param client Kubernetes client + * @param pod Pod to be checked + */ + public static void checkForVolumeMountErrors(KubernetesClient client, Pod pod) { + var conditions = pod.getStatus().getConditions(); + if (conditions == null) { + return; + } + var conditionMap = + conditions.stream() + .collect(Collectors.toMap(PodCondition::getType, Function.identity())); + + // We use PodReadyToStartContainers if available otherwise use Initialized, but it's only + // there k8s 1.29+ + boolean failedInitialization = + checkStatusWasAlways( + pod, + conditionMap.getOrDefault( + "PodReadyToStartContainers", conditionMap.get("Initialized")), + "False"); + + boolean notReady = checkStatusWasAlways(pod, conditionMap.get("Ready"), "False"); + + if (notReady && failedInitialization) { + getPodEvents(client, pod).stream() + .filter(e -> e.getReason().equals("FailedMount")) + .findAny() + .ifPresent( + e -> { + throw new DeploymentFailedException(e.getMessage(), e.getReason()); + }); + } + } + + private static boolean checkStatusWasAlways(Pod pod, PodCondition condition, String status) { + return condition != null && condition.getStatus().equals(status); + } } 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 d9492bfd60..e3a3ae25b3 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 @@ -97,6 +97,24 @@ public class TestUtils extends BaseTestUtils { public static PodList createFailedPodList(String crashLoopMessage, String reason) { ContainerStatus cs = new ContainerStatusBuilder() + .withName("c1") + .withNewState() + .withNewWaiting() + .withReason(reason) + .withMessage(crashLoopMessage) + .endWaiting() + .endState() + .build(); + + Pod pod = getTestPod("host", "apiVersion", Collections.emptyList()); + pod.setStatus(new PodStatusBuilder().withContainerStatuses(cs).build()); + return new PodListBuilder().withItems(pod).build(); + } + + public static PodList createFailedInitContainerPodList(String crashLoopMessage, String reason) { + ContainerStatus cs = + new ContainerStatusBuilder() + .withName("c1") .withNewState() .withNewWaiting() .withReason(reason) @@ -108,7 +126,8 @@ public static PodList createFailedPodList(String crashLoopMessage, String reason Pod pod = getTestPod("host", "apiVersion", Collections.emptyList()); pod.setStatus( new PodStatusBuilder() - .withContainerStatuses(Collections.singletonList(cs)) + .withContainerStatuses(new ContainerStatusBuilder().withReady().build()) + .withInitContainerStatuses(cs) .build()); return new PodListBuilder().withItems(pod).build(); } diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentControllerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentControllerTest.java index 8c3bd33fb3..2384fb4da1 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentControllerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentControllerTest.java @@ -35,7 +35,6 @@ import org.apache.flink.kubernetes.operator.api.status.ReconciliationStatus; import org.apache.flink.kubernetes.operator.api.status.TaskManagerInfo; import org.apache.flink.kubernetes.operator.config.FlinkConfigManager; -import org.apache.flink.kubernetes.operator.exception.DeploymentFailedException; import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; import org.apache.flink.kubernetes.operator.reconciler.deployment.AbstractFlinkResourceReconciler; import org.apache.flink.kubernetes.operator.utils.EventRecorder; @@ -59,7 +58,6 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; import org.junit.jupiter.params.provider.MethodSource; -import org.junit.jupiter.params.provider.ValueSource; import java.util.ArrayList; import java.util.Map; @@ -299,94 +297,6 @@ public void verifyFailedDeployment() throws Exception { updateControl.getScheduleDelay().get()); } - @ParameterizedTest() - @ValueSource( - strings = { - DeploymentFailedException.REASON_CRASH_LOOP_BACKOFF, - DeploymentFailedException.REASON_IMAGE_PULL_BACKOFF, - DeploymentFailedException.REASON_ERR_IMAGE_PULL - }) - public void verifyInProgressDeploymentWithError(String reason) throws Exception { - String crashLoopMessage = "container fails"; - - var submittedEventValidatingResponseProvider = - new TestUtils.ValidatingResponseProvider<>( - mockedEvent, - r -> - assertTrue( - r.getBody() - .readUtf8() - .contains( - AbstractFlinkResourceReconciler - .MSG_SUBMIT))); - mockServer - .expect() - .post() - .withPath("/api/v1/namespaces/flink-operator-test/events") - .andReply(submittedEventValidatingResponseProvider) - .once(); - - var validatingResponseProvider = - new TestUtils.ValidatingResponseProvider<>( - mockedEvent, - r -> { - String recordedRequestBody = r.getBody().readUtf8(); - assertTrue(recordedRequestBody.contains(reason)); - assertTrue(recordedRequestBody.contains(crashLoopMessage)); - }); - mockServer - .expect() - .post() - .withPath("/api/v1/namespaces/flink-operator-test/events") - .andReply(validatingResponseProvider) - .once(); - - flinkService.setPodList(TestUtils.createFailedPodList(crashLoopMessage, reason)); - - FlinkDeployment appCluster = TestUtils.buildApplicationCluster(); - UpdateControl updateControl; - - testController.reconcile(appCluster, context); - updateControl = - testController.reconcile( - appCluster, - TestUtils.createContextWithInProgressDeployment(kubernetesClient)); - submittedEventValidatingResponseProvider.assertValidated(); - assertFalse(updateControl.isUpdateStatus()); - assertEquals( - Optional.of( - configManager.getOperatorConfiguration().getReconcileInterval().toMillis()), - updateControl.getScheduleDelay()); - - assertEquals( - JobManagerDeploymentStatus.ERROR, - appCluster.getStatus().getJobManagerDeploymentStatus()); - assertEquals( - org.apache.flink.api.common.JobStatus.RECONCILING, - appCluster.getStatus().getJobStatus().getState()); - - // Validate status status - assertNotNull(appCluster.getStatus().getError()); - - // next cycle should not create another event - updateControl = - testController.reconcile( - appCluster, - TestUtils.createContextWithFailedJobManagerDeployment(kubernetesClient)); - assertEquals( - JobManagerDeploymentStatus.ERROR, - appCluster.getStatus().getJobManagerDeploymentStatus()); - assertFalse(updateControl.isUpdateStatus()); - assertEquals( - ReconciliationUtils.rescheduleAfter( - JobManagerDeploymentStatus.READY, - appCluster, - configManager.getOperatorConfiguration()) - .toMillis(), - updateControl.getScheduleDelay().get()); - validatingResponseProvider.assertValidated(); - } - @ParameterizedTest @MethodSource("org.apache.flink.kubernetes.operator.TestUtils#flinkVersions") public void verifyUpgradeFromSavepointLegacyMode(FlinkVersion flinkVersion) throws Exception { @@ -915,9 +825,7 @@ private void testUpgradeNotReadyCluster(FlinkDeployment appCluster) throws Excep @Test public void testSuccessfulObservationShouldClearErrors() throws Exception { final String crashLoopMessage = "deploy errors"; - flinkService.setPodList( - TestUtils.createFailedPodList( - crashLoopMessage, DeploymentFailedException.REASON_CRASH_LOOP_BACKOFF)); + flinkService.setPodList(TestUtils.createFailedPodList(crashLoopMessage, "ErrImagePull")); FlinkDeployment appCluster = TestUtils.buildApplicationCluster(); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/exception/DeploymentFailedExceptionTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/exception/DeploymentFailedExceptionTest.java new file mode 100644 index 0000000000..2437d3b990 --- /dev/null +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/exception/DeploymentFailedExceptionTest.java @@ -0,0 +1,69 @@ +/* + * 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.exception; + +import io.fabric8.kubernetes.api.model.ContainerState; +import io.fabric8.kubernetes.api.model.ContainerStateTerminated; +import io.fabric8.kubernetes.api.model.ContainerStateWaiting; +import io.fabric8.kubernetes.api.model.ContainerStatus; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** Tests for {@link DeploymentFailedException}. */ +public class DeploymentFailedExceptionTest { + + @Test + public void testErrorFromContainerStatus() { + var containerStatus = new ContainerStatus(); + containerStatus.setName("c1"); + var state = new ContainerState(); + var waiting = new ContainerStateWaiting(); + waiting.setMessage("msg"); + waiting.setReason("r"); + state.setWaiting(waiting); + + containerStatus.setState(state); + + var ex = DeploymentFailedException.forContainerStatus(containerStatus); + assertEquals("[c1] msg", ex.getMessage()); + assertEquals("r", ex.getReason()); + + waiting.setReason("CrashLoopBackOff"); + waiting.setMessage("backing off"); + ex = DeploymentFailedException.forContainerStatus(containerStatus); + assertEquals("[c1] backing off", ex.getMessage()); + assertEquals("CrashLoopBackOff", ex.getReason()); + + // Last state set but not terminated + var lastState = new ContainerState(); + containerStatus.setLastState(lastState); + + ex = DeploymentFailedException.forContainerStatus(containerStatus); + assertEquals("[c1] backing off", ex.getMessage()); + assertEquals("CrashLoopBackOff", ex.getReason()); + + var terminated = new ContainerStateTerminated(); + terminated.setMessage("crash"); + lastState.setTerminated(terminated); + + ex = DeploymentFailedException.forContainerStatus(containerStatus); + assertEquals("[c1] CrashLoop - crash", ex.getMessage()); + assertEquals("CrashLoopBackOff", ex.getReason()); + } +} diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/ApplicationObserverTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/ApplicationObserverTest.java index aa4898d827..68233d397e 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/ApplicationObserverTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/ApplicationObserverTest.java @@ -48,12 +48,16 @@ import lombok.Getter; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import java.time.Duration; import java.time.Instant; import java.util.HashMap; import java.util.Map; import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.apache.flink.kubernetes.operator.api.utils.FlinkResourceUtils.getCheckpointInfo; import static org.apache.flink.kubernetes.operator.api.utils.FlinkResourceUtils.getJobStatus; @@ -753,8 +757,9 @@ private void bringToReadyStatus(FlinkDeployment deployment) { deployment.getStatus().setJobManagerDeploymentStatus(JobManagerDeploymentStatus.READY); } - @Test - public void observeListJobsError() { + @ParameterizedTest + @MethodSource("containerFailureReasons") + public void observeListJobsError(String reason, boolean initError) { bringToReadyStatus(deployment); observer.observe(deployment, readyContext); assertEquals( @@ -762,9 +767,12 @@ public void observeListJobsError() { deployment.getStatus().getJobManagerDeploymentStatus()); // simulate deployment failure String podFailedMessage = "list jobs error"; - flinkService.setPodList( - TestUtils.createFailedPodList( - podFailedMessage, DeploymentFailedException.REASON_CRASH_LOOP_BACKOFF)); + if (initError) { + flinkService.setPodList( + TestUtils.createFailedInitContainerPodList(podFailedMessage, reason)); + } else { + flinkService.setPodList(TestUtils.createFailedPodList(podFailedMessage, reason)); + } flinkService.setPortReady(false); Exception exception = assertThrows( @@ -774,7 +782,14 @@ public void observeListJobsError() { deployment, TestUtils.createContextWithInProgressDeployment( kubernetesClient))); - assertEquals(podFailedMessage, exception.getMessage()); + assertEquals("[c1] " + podFailedMessage, exception.getMessage()); + } + + private static Stream containerFailureReasons() { + return DeploymentFailedException.CONTAINER_ERROR_REASONS.stream() + .flatMap( + reason -> + Stream.of(Arguments.of(reason, true), Arguments.of(reason, false))); } @Test diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/PodErrorTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/PodErrorTest.java new file mode 100644 index 0000000000..d4e578097f --- /dev/null +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/PodErrorTest.java @@ -0,0 +1,148 @@ +/* + * 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; + +import org.apache.flink.kubernetes.operator.exception.DeploymentFailedException; + +import io.fabric8.kubeapitest.junit.EnableKubeAPIServer; +import io.fabric8.kubernetes.api.model.EventBuilder; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.PodBuilder; +import io.fabric8.kubernetes.api.model.PodCondition; +import io.fabric8.kubernetes.api.model.PodConditionBuilder; +import io.fabric8.kubernetes.client.KubernetesClient; +import org.junit.jupiter.api.Test; + +import java.time.Instant; +import java.util.ArrayList; +import java.util.HashMap; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.fail; + +/** Test for {@link EventUtils}. */ +@EnableKubeAPIServer +public class PodErrorTest { + + static KubernetesClient client; + + @Test + public void testVolumeMountErrors() { + var pod = + new PodBuilder() + .withNewMetadata() + .withName("test") + .withNamespace("default") + .endMetadata() + .withNewStatus() + .endStatus() + .build(); + + // No conditions, no error expected + EventUtils.checkForVolumeMountErrors(client, pod); + + var conditions = new ArrayList(); + pod.getStatus().setConditions(conditions); + + // No conditions, no error expected + EventUtils.checkForVolumeMountErrors(client, pod); + + // Create error events + createPodEvent("e1", "reason1", "msg1", pod); + createPodEvent("e2", "FailedMount", "mountErr", pod); + + var conditionMap = new HashMap(); + + // Pod initialized completely, shouldn't check events + conditionMap.put("Initialized", "True"); + conditionMap.put("Ready", "False"); + + conditions.clear(); + conditionMap.forEach( + (t, s) -> + conditions.add( + new PodConditionBuilder().withType(t).withStatus(s).build())); + EventUtils.checkForVolumeMountErrors(client, pod); + + // Pod initialized completely, shouldn't check events + conditionMap.put("PodReadyToStartContainers", "True"); + conditionMap.put("Initialized", "False"); + + conditions.clear(); + conditionMap.forEach( + (t, s) -> + conditions.add( + new PodConditionBuilder().withType(t).withStatus(s).build())); + EventUtils.checkForVolumeMountErrors(client, pod); + + // Check event only when not ready to start + conditionMap.put("PodReadyToStartContainers", "False"); + conditions.clear(); + conditionMap.forEach( + (t, s) -> + conditions.add( + new PodConditionBuilder().withType(t).withStatus(s).build())); + + try { + EventUtils.checkForVolumeMountErrors(client, pod); + fail("Exception not thrown"); + } catch (DeploymentFailedException dfe) { + assertEquals("FailedMount", dfe.getReason()); + assertEquals("mountErr", dfe.getMessage()); + } + + // Old kubernetes without PodReadyToStartContainers + conditionMap.remove("PodReadyToStartContainers"); + conditionMap.put("Initialized", "False"); + conditions.clear(); + conditionMap.forEach( + (t, s) -> + conditions.add( + new PodConditionBuilder().withType(t).withStatus(s).build())); + + try { + EventUtils.checkForVolumeMountErrors(client, pod); + fail("Exception not thrown"); + } catch (DeploymentFailedException dfe) { + assertEquals("FailedMount", dfe.getReason()); + assertEquals("mountErr", dfe.getMessage()); + } + } + + private void createPodEvent(String name, String reason, String msg, Pod pod) { + var event = + new EventBuilder() + .withApiVersion("v1") + .withInvolvedObject(EventUtils.getObjectReference(pod)) + .withType("type") + .withReason(reason) + .withFirstTimestamp(Instant.now().toString()) + .withLastTimestamp(Instant.now().toString()) + .withNewSource() + .withComponent("pod") + .endSource() + .withCount(1) + .withMessage(msg) + .withNewMetadata() + .withName(name) + .withNamespace(pod.getMetadata().getNamespace()) + .endMetadata() + .build(); + client.resource(event).create(); + } +} From 0e65a5a1aeef4ebd8b827ad3bdec76036751328f Mon Sep 17 00:00:00 2001 From: Gyula Fora Date: Fri, 21 Mar 2025 08:51:56 +0100 Subject: [PATCH 17/19] [FLINK-37530] Record upgrade savepoint correctly in savepointInfo as well --- .../deployment/AbstractJobReconciler.java | 25 +++++++++++++++---- .../deployment/ApplicationReconciler.java | 4 ++- .../sessionjob/SessionJobReconciler.java | 5 +++- .../deployment/ApplicationReconcilerTest.java | 8 ++++++ 4 files changed, 35 insertions(+), 7 deletions(-) diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/deployment/AbstractJobReconciler.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/deployment/AbstractJobReconciler.java index 71ec3dee5e..023396b52a 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/deployment/AbstractJobReconciler.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/deployment/AbstractJobReconciler.java @@ -31,6 +31,7 @@ import org.apache.flink.kubernetes.operator.api.status.CommonStatus; import org.apache.flink.kubernetes.operator.api.status.JobStatus; import org.apache.flink.kubernetes.operator.api.status.ReconciliationState; +import org.apache.flink.kubernetes.operator.api.status.Savepoint; import org.apache.flink.kubernetes.operator.api.status.SavepointFormatType; import org.apache.flink.kubernetes.operator.api.status.SnapshotTriggerType; import org.apache.flink.kubernetes.operator.autoscaler.KubernetesJobAutoScalerContext; @@ -398,21 +399,35 @@ protected void restoreJob( * * @param ctx context * @param savepointLocation location of savepoint taken + * @param cancelTs Timestamp when upgrade/cancel was triggered */ - protected void setUpgradeSavepointPath(FlinkResourceContext ctx, String savepointLocation) { + protected void setUpgradeSavepointPath( + FlinkResourceContext ctx, String savepointLocation, Instant cancelTs) { var conf = ctx.getObserveConfig(); var savepointFormatType = - ctx.getObserveConfig() - .get(KubernetesOperatorConfigOptions.OPERATOR_SAVEPOINT_FORMAT_TYPE); + SavepointFormatType.valueOf( + conf.get(KubernetesOperatorConfigOptions.OPERATOR_SAVEPOINT_FORMAT_TYPE) + .name()); FlinkStateSnapshotUtils.createUpgradeSnapshotResource( conf, ctx.getOperatorConfig(), ctx.getKubernetesClient(), ctx.getResource(), - SavepointFormatType.valueOf(savepointFormatType.name()), + savepointFormatType, savepointLocation); - ctx.getResource().getStatus().getJobStatus().setUpgradeSavepointPath(savepointLocation); + var jobStatus = ctx.getResource().getStatus().getJobStatus(); + jobStatus.setUpgradeSavepointPath(savepointLocation); + + // Register created savepoint in the now deprecated savepoint info and history + var savepoint = + new Savepoint( + cancelTs.toEpochMilli(), + savepointLocation, + SnapshotTriggerType.UPGRADE, + savepointFormatType, + null); + jobStatus.getSavepointInfo().updateLastSavepoint(savepoint); } /** diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconciler.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconciler.java index 71cf417958..b5486e0ec7 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconciler.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconciler.java @@ -223,10 +223,12 @@ private void setJobIdIfNecessary( @Override protected boolean cancelJob(FlinkResourceContext ctx, SuspendMode suspendMode) throws Exception { + var cancelTs = Instant.now(); var result = ctx.getFlinkService() .cancelJob(ctx.getResource(), suspendMode, ctx.getObserveConfig()); - result.getSavepointPath().ifPresent(location -> setUpgradeSavepointPath(ctx, location)); + result.getSavepointPath() + .ifPresent(location -> setUpgradeSavepointPath(ctx, location, cancelTs)); return result.isPending(); } diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/sessionjob/SessionJobReconciler.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/sessionjob/SessionJobReconciler.java index f1fa88859a..4932940a4f 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/sessionjob/SessionJobReconciler.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/sessionjob/SessionJobReconciler.java @@ -41,6 +41,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.time.Instant; import java.util.Optional; /** The reconciler for the {@link FlinkSessionJob}. */ @@ -100,10 +101,12 @@ public void deploy( @Override protected boolean cancelJob(FlinkResourceContext ctx, SuspendMode suspendMode) throws Exception { + var cancelTs = Instant.now(); var result = ctx.getFlinkService() .cancelSessionJob(ctx.getResource(), suspendMode, ctx.getObserveConfig()); - result.getSavepointPath().ifPresent(location -> setUpgradeSavepointPath(ctx, location)); + result.getSavepointPath() + .ifPresent(location -> setUpgradeSavepointPath(ctx, location, cancelTs)); return result.isPending(); } diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerTest.java index 47ce5e4eb2..19d5446113 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerTest.java @@ -98,6 +98,7 @@ import java.time.Duration; import java.time.Instant; import java.time.ZoneId; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Optional; @@ -303,6 +304,13 @@ public void testUpgrade(FlinkVersion flinkVersion) throws Exception { assertEquals(0, flinkService.getRunningCount()); + var spInfo = statefulUpgrade.getStatus().getJobStatus().getSavepointInfo(); + assertEquals("savepoint_0", spInfo.getLastSavepoint().getLocation()); + assertEquals(SnapshotTriggerType.UPGRADE, spInfo.getLastSavepoint().getTriggerType()); + assertEquals( + spInfo.getLastSavepoint(), + new LinkedList<>(spInfo.getSavepointHistory()).getLast()); + reconciler.reconcile(statefulUpgrade, context); runningJobs = flinkService.listJobs(); From c81ed916ba46a2046e4487d97e91e321fe1e4f6a Mon Sep 17 00:00:00 2001 From: schongloo Date: Tue, 1 Apr 2025 18:48:07 -0700 Subject: [PATCH 18/19] Added logic to SUSPEND the FlinkDeployment accordingly if it fails to start in a timely manner. --- .../FlinkBlueGreenDeploymentController.java | 13 ++++++++----- .../FlinkBlueGreenDeploymentControllerTest.java | 7 +++++-- 2 files changed, 13 insertions(+), 7 deletions(-) diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentController.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentController.java index 486532aa75..7eeb9f59f9 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentController.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentController.java @@ -65,7 +65,7 @@ @ControllerConfiguration public class FlinkBlueGreenDeploymentController implements Reconciler, - EventSourceInitializer { + EventSourceInitializer { private static final Logger LOG = LoggerFactory.getLogger(FlinkDeploymentController.class); private static final int DEFAULT_MAX_NUM_RETRIES = 5; @@ -206,7 +206,13 @@ private UpdateControl monitorTransition( if (deploymentStatus.getNumRetries() >= maxNumRetries) { // ABORT - return patchStatusUpdateControl(bgDeployment, deploymentStatus, null, null, false); + // Suspend the nextDeployment (FlinkDeployment) + nextDeployment.getStatus().getJobStatus().setState(JobStatus.SUSPENDED); + josdkContext.getClient().resource(nextDeployment).update(); + + // If the current running FlinkDeployment is not in RUNNING/STABLE, + // we flag this Blue/Green as FAILING + return patchStatusUpdateControl(bgDeployment, deploymentStatus, null, JobStatus.FAILING, false); } else { // RETRY deploymentStatus.setNumRetries(deploymentStatus.getNumRetries() + 1); @@ -242,8 +248,6 @@ private UpdateControl deleteAndFinalize( deploymentStatus.setLastReconciledSpec( SpecUtils.serializeObject(bgDeployment.getSpec(), "spec")); - // TODO: set the last reconciled spec here - return patchStatusUpdateControl( bgDeployment, deploymentStatus, nextState, JobStatus.RUNNING, false); } @@ -265,7 +269,6 @@ private UpdateControl checkAndInitiateDeployment( ? deployments.getFlinkDeploymentBlue() : deployments.getFlinkDeploymentGreen(); - // TODO: if the current deployment is not STABLE/RUNNING we'll revert to the last stable // spec, report the error and abort if (isDeploymentReady(currentFlinkDeployment, josdkContext, deploymentStatus)) { diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentControllerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentControllerTest.java index 1832df166b..2180bd66fd 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentControllerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentControllerTest.java @@ -302,7 +302,8 @@ public void verifyFailureDuringTransition(FlinkVersion flinkVersion) throws Exce assertSpec(rs, minReconciliationTs); - // The first job should be running OK, the second should be left in its broken state + // The first job should be RUNNING, the second should be SUSPENDED + assertEquals(JobStatus.FAILING, rs.reconciledStatus.getJobStatus().getState()); var flinkDeployments = getFlinkDeployments(); assertEquals(2, flinkDeployments.size()); assertEquals( @@ -310,7 +311,9 @@ public void verifyFailureDuringTransition(FlinkVersion flinkVersion) throws Exce assertEquals( ReconciliationState.DEPLOYED, flinkDeployments.get(0).getStatus().getReconciliationStatus().getState()); - assertNull(flinkDeployments.get(1).getStatus().getJobStatus().getState()); + assertEquals( + JobStatus.SUSPENDED, + flinkDeployments.get(1).getStatus().getJobStatus().getState()); assertEquals( ReconciliationState.UPGRADING, flinkDeployments.get(1).getStatus().getReconciliationStatus().getState()); From 1f3da3e5ddff5f6e57a666c93db1af70a02d3db9 Mon Sep 17 00:00:00 2001 From: schongloo Date: Mon, 7 Apr 2025 16:48:49 -0700 Subject: [PATCH 19/19] Correctly "resuming" a previously SUSPENDED deployment. --- .../FlinkBlueGreenDeploymentStatus.java | 4 +- .../FlinkBlueGreenDeploymentController.java | 49 +++++++---- ...linkBlueGreenDeploymentControllerTest.java | 83 +++++++++++++++---- 3 files changed, 101 insertions(+), 35 deletions(-) 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 index 1cf75f02ee..0820803056 100644 --- 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 @@ -41,10 +41,10 @@ public class FlinkBlueGreenDeploymentStatus { /** The state of the blue/green transition. */ private FlinkBlueGreenDeploymentState blueGreenState; - /** Last reconciled (serialized) deployment spec for A. */ + /** Last reconciled (serialized) deployment spec. */ private String lastReconciledSpec; - /** Timestamp of last reconciliation (either A or B). */ + /** Timestamp of last reconciliation. */ private Long lastReconciledTimestamp; /** Current number of retries. */ diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentController.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentController.java index 7eeb9f59f9..5717a75cca 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentController.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentController.java @@ -65,7 +65,7 @@ @ControllerConfiguration public class FlinkBlueGreenDeploymentController implements Reconciler, - EventSourceInitializer { + EventSourceInitializer { private static final Logger LOG = LoggerFactory.getLogger(FlinkDeploymentController.class); private static final int DEFAULT_MAX_NUM_RETRIES = 5; @@ -196,8 +196,6 @@ private UpdateControl monitorTransition( // could get stuck // (e.g. waiting for resources) // TODO: figure out the course of action for error/failure cases - // TODO: should we reschedule indefinitely? Should this case check the "should abort" - // condition? int maxNumRetries = bgDeployment.getSpec().getTemplate().getMaxNumRetries(); if (maxNumRetries <= 0) { @@ -210,29 +208,37 @@ private UpdateControl monitorTransition( nextDeployment.getStatus().getJobStatus().setState(JobStatus.SUSPENDED); josdkContext.getClient().resource(nextDeployment).update(); + // We indicate this Blue/Green deployment is no longer Transitioning + // and rollback the state value + deploymentStatus.setBlueGreenState( + nextState == FlinkBlueGreenDeploymentState.ACTIVE_BLUE + ? FlinkBlueGreenDeploymentState.ACTIVE_GREEN + : FlinkBlueGreenDeploymentState.ACTIVE_BLUE); + // If the current running FlinkDeployment is not in RUNNING/STABLE, // we flag this Blue/Green as FAILING - return patchStatusUpdateControl(bgDeployment, deploymentStatus, null, JobStatus.FAILING, false); + return patchStatusUpdateControl( + bgDeployment, deploymentStatus, null, JobStatus.FAILING, false); } else { // RETRY deploymentStatus.setNumRetries(deploymentStatus.getNumRetries() + 1); LOG.info("Deployment " + nextDeployment.getMetadata().getName() + " not ready yet"); - int reconciliationReschedInterval = - bgDeployment - .getSpec() - .getTemplate() - .getReconciliationReschedulingIntervalMs(); - if (reconciliationReschedInterval <= 0) { - reconciliationReschedInterval = DEFAULT_RECONCILIATION_RESCHEDULING_INTERVAL_MS; - } - return patchStatusUpdateControl(bgDeployment, deploymentStatus, null, null, false) - .rescheduleAfter(reconciliationReschedInterval); + .rescheduleAfter(getReconciliationReschedInterval(bgDeployment)); } } } + private static int getReconciliationReschedInterval(FlinkBlueGreenDeployment bgDeployment) { + int reconciliationReschedInterval = + bgDeployment.getSpec().getTemplate().getReconciliationReschedulingIntervalMs(); + if (reconciliationReschedInterval <= 0) { + reconciliationReschedInterval = DEFAULT_RECONCILIATION_RESCHEDULING_INTERVAL_MS; + } + return reconciliationReschedInterval; + } + private UpdateControl deleteAndFinalize( FlinkBlueGreenDeployment bgDeployment, FlinkBlueGreenDeploymentStatus deploymentStatus, @@ -248,6 +254,8 @@ private UpdateControl deleteAndFinalize( deploymentStatus.setLastReconciledSpec( SpecUtils.serializeObject(bgDeployment.getSpec(), "spec")); + // TODO: Set the new child job STATUS to RUNNING too + return patchStatusUpdateControl( bgDeployment, deploymentStatus, nextState, JobStatus.RUNNING, false); } @@ -387,7 +395,7 @@ private static Savepoint configureSavepoint( // TODO 1: check the last CP age with the logic from // AbstractJobReconciler.changeLastStateIfCheckpointTooOld - // TODO 2: if no checkpoint is available, take a savepoint + // TODO 2: if no checkpoint is available, take a savepoint? throw error? if (lastCheckpoint.isEmpty()) { throw new IllegalStateException( "Last Checkpoint for Job " @@ -406,14 +414,23 @@ private UpdateControl initiateDeployment( Context josdkContext, boolean isFirstDeployment) throws JsonProcessingException { + deploy( flinkBlueGreenDeployment, nextDeploymentType, lastCheckpoint, josdkContext, isFirstDeployment); + + // TODO: set child job status to JobStatus.INITIALIZING + return patchStatusUpdateControl( - flinkBlueGreenDeployment, deploymentStatus, nextState, null, isFirstDeployment); + flinkBlueGreenDeployment, + deploymentStatus, + nextState, + null, + isFirstDeployment) + .rescheduleAfter(getReconciliationReschedInterval(flinkBlueGreenDeployment)); } private boolean isDeploymentReady( diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentControllerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentControllerTest.java index 2180bd66fd..fb36dafa8f 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentControllerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkBlueGreenDeploymentControllerTest.java @@ -76,6 +76,7 @@ public class FlinkBlueGreenDeploymentControllerTest { 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 final FlinkConfigManager configManager = new FlinkConfigManager(new Configuration()); private TestingFlinkService flinkService; private Context context; @@ -148,10 +149,10 @@ public void verifyBasicTransition(FlinkVersion flinkVersion) throws Exception { var blueGreenDeployment = buildSessionCluster(TEST_DEPLOYMENT_NAME, TEST_NAMESPACE, flinkVersion); - // 1. Initiate the Green deployment + // 1. Initiate the Blue deployment var rs = reconcile(blueGreenDeployment); - // 2. Finalize the Green deployment + // 2. Finalize the Blue deployment simulateSubmitAndSuccessfulJobStart(getFlinkDeployments().get(0)); rs = reconcile(rs.deployment); @@ -159,15 +160,27 @@ public void verifyBasicTransition(FlinkVersion flinkVersion) throws Exception { var rs2 = reconcile(rs.deployment); assertTrue(rs2.updateControl.isNoUpdate()); - // 3. Simulate a change in the spec to trigger a Blue deployment - simulateChangeInSpec(rs.deployment); + // 3. Simulate a change in the spec to trigger a Green deployment + String customValue = UUID.randomUUID().toString(); + simulateChangeInSpec(rs.deployment, customValue); - // 4. Initiate the Blue deployment + // 4. Initiate the Green deployment var bgUpdatedSpec = rs.deployment.getSpec(); Long minReconciliationTs = System.currentTimeMillis() - 1; rs = reconcile(rs.deployment); + + testTransitionToGreen(rs, minReconciliationTs, customValue, bgUpdatedSpec); + } + + private void testTransitionToGreen( + TestingFlinkBlueGreenDeploymentController.BlueGreenReconciliationResult rs, + Long minReconciliationTs, + String customValue, + FlinkBlueGreenDeploymentSpec bgUpdatedSpec) + throws Exception { + TestingFlinkBlueGreenDeploymentController.BlueGreenReconciliationResult rs2; var flinkDeployments = getFlinkDeployments(); - var blueDeploymentName = flinkDeployments.get(1).getMetadata().getName(); + var greenDeploymentName = flinkDeployments.get(1).getMetadata().getName(); assertSpec(rs, minReconciliationTs); @@ -178,18 +191,26 @@ public void verifyBasicTransition(FlinkVersion flinkVersion) throws Exception { assertEquals( FlinkBlueGreenDeploymentState.TRANSITIONING_TO_GREEN, rs.reconciledStatus.getBlueGreenState()); + assertEquals( + customValue, + rs.deployment + .getSpec() + .getTemplate() + .getSpec() + .getFlinkConfiguration() + .get(CUSTOM_CONFIG_FIELD)); assertNotNull(rs.reconciledStatus.getLastReconciledSpec()); - // 5. New Blue deployment successfully started + // New Blue deployment successfully started simulateSuccessfulJobStart(getFlinkDeployments().get(1)); rs2 = reconcile(rs.deployment); assertTrue(rs2.updateControl.isNoUpdate()); + // Old Blue deployment deleted, Green is the active one flinkDeployments = getFlinkDeployments(); assertEquals(1, flinkDeployments.size()); - assertEquals(blueDeploymentName, flinkDeployments.get(0).getMetadata().getName()); + assertEquals(greenDeploymentName, flinkDeployments.get(0).getMetadata().getName()); - // 6. Old Green deployment deleted, Blue promoted to Green minReconciliationTs = System.currentTimeMillis() - 1; rs = reconcile(rs.deployment); @@ -222,7 +243,7 @@ public void verifyFailureBeforeTransition(FlinkVersion flinkVersion) throws Exce rs = reconcile(rs.deployment); // 3. Simulate a change in the spec to trigger a Blue deployment - simulateChangeInSpec(rs.deployment); + simulateChangeInSpec(rs.deployment, UUID.randomUUID().toString()); // TODO: simulate a failure in the running deployment simulateJobFailure(getFlinkDeployments().get(0)); @@ -273,11 +294,25 @@ public void verifyFailureDuringTransition(FlinkVersion flinkVersion) throws Exce rs = reconcile(rs.deployment); // 3. Simulate a change in the spec to trigger a Blue deployment - simulateChangeInSpec(rs.deployment); + String customValue = UUID.randomUUID().toString(); + simulateChangeInSpec(rs.deployment, customValue); // 4. Initiate the Blue 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)); + // 4a. Simulating the Blue deployment doesn't start correctly (status will remain the same) // Asserting the status retry count is incremented by 1 long lastTs = System.currentTimeMillis(); @@ -304,6 +339,9 @@ public void verifyFailureDuringTransition(FlinkVersion flinkVersion) throws Exce // 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( @@ -312,11 +350,21 @@ public void verifyFailureDuringTransition(FlinkVersion flinkVersion) throws Exce ReconciliationState.DEPLOYED, flinkDeployments.get(0).getStatus().getReconciliationStatus().getState()); assertEquals( - JobStatus.SUSPENDED, - flinkDeployments.get(1).getStatus().getJobStatus().getState()); + JobStatus.SUSPENDED, flinkDeployments.get(1).getStatus().getJobStatus().getState()); assertEquals( ReconciliationState.UPGRADING, flinkDeployments.get(1).getStatus().getReconciliationStatus().getState()); + + // 5. Simulate another change in the spec to trigger a redeployment + customValue = UUID.randomUUID().toString(); + simulateChangeInSpec(rs.deployment, customValue); + + // 6. Initiate the redeployment + var bgUpdatedSpec = rs.deployment.getSpec(); + minReconciliationTs = System.currentTimeMillis() - 1; + rs = reconcile(rs.deployment); + + testTransitionToGreen(rs, minReconciliationTs, customValue, bgUpdatedSpec); } private static void assertSpec( @@ -326,9 +374,10 @@ private static void assertSpec( assertTrue(minReconciliationTs < rs.reconciledStatus.getLastReconciledTimestamp()); } - private void simulateChangeInSpec(FlinkBlueGreenDeployment blueGreenDeployment) { + private void simulateChangeInSpec( + FlinkBlueGreenDeployment blueGreenDeployment, String customValue) { FlinkDeploymentSpec spec = blueGreenDeployment.getSpec().getTemplate().getSpec(); - spec.getFlinkConfiguration().put(TaskManagerOptions.NUM_TASK_SLOTS.key(), "10"); + spec.getFlinkConfiguration().put(CUSTOM_CONFIG_FIELD, customValue); blueGreenDeployment.getSpec().getTemplate().setSpec(spec); kubernetesClient.resource(blueGreenDeployment).createOrReplace(); } @@ -366,13 +415,13 @@ private void simulateSuccessfulJobStart(FlinkDeployment deployment) { .getReconciliationStatus() .serializeAndSetLastReconciledSpec(deployment.getSpec(), deployment); deployment.getStatus().getReconciliationStatus().markReconciledSpecAsStable(); - kubernetesClient.resource(deployment).createOrReplace(); + kubernetesClient.resource(deployment).update(); } private void simulateJobFailure(FlinkDeployment deployment) { deployment.getStatus().getJobStatus().setState(JobStatus.RECONCILING); deployment.getStatus().getReconciliationStatus().setState(ReconciliationState.UPGRADING); - kubernetesClient.resource(deployment).createOrReplace(); + kubernetesClient.resource(deployment).update(); } private static void verifyOwnerReferences(