You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/07/14 12:43:18 UTC

[GitHub] [flink] zhoulii opened a new pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

zhoulii opened a new pull request #12899:
URL: https://github.com/apache/flink/pull/12899


   ## What is the purpose of the change
   
   Kubernetes Secrets can be used to provide credentials for a Flink application to access secured services.  This ticket proposes to
   
   - Support to mount user-specified K8s Secrets into the JobManager/TaskManager Container
   - Support to use a user-specified K8s Secret through an environment variable.
   
   
   ## Brief change log
   
   - Introduce two new KubernetesStepDecorator implementations named `MountSecretsDecorator`、`EnvSecretsDecorator `.
   - Add `MountSecretsDecorator`、`EnvSecretsDecorator` to the decorator chains in KubernetesJobManagerFactory and KubernetesTaskManagerFactory.
   - Introduce two new configs  `kubernetes.secrets.` and `kubernetes.env.secretKeyRef.`.
   
   
   ## Verifying this change
   
   - Covered by unit test.
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (yes / **no**)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (yes / **no**)
     - The serializers: (yes / **no** / don't know)
     - The runtime per-record code paths (performance sensitive): (yes / **no** / don't know)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: (yes / **no** / don't know)
     - The S3 file system connector: (yes / **no** / don't know)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (**yes** / no)
     - If yes, how is the feature documented? (not applicable / **docs** / JavaDocs / not documented)
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zhoulii commented on a change in pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
zhoulii commented on a change in pull request #12899:
URL: https://github.com/apache/flink/pull/12899#discussion_r469673226



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesEnvVar.java
##########
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.kubeclient.resources;
+
+import io.fabric8.kubernetes.api.model.EnvVar;
+import io.fabric8.kubernetes.api.model.EnvVarBuilder;
+
+import java.util.Map;
+
+/**
+ * Represent EnvVar resource in kubernetes.
+ */
+public class KubernetesEnvVar extends KubernetesResource<EnvVar> {
+
+	private static final String ENV = "env";
+	private static final String SECRET = "secret";
+	private static final String KEY = "key";
+
+	private KubernetesEnvVar(EnvVar envVar) {
+		super(envVar);
+	}
+
+	public static KubernetesEnvVar fromMap(Map<String, String> stringMap) {

Review comment:
       you are right, I will change the class name to `KubernetesSecretEnvVar`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #12899:
URL: https://github.com/apache/flink/pull/12899#discussion_r471427288



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/MountSecretsDecorator.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.kubeclient.decorators;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.parameters.AbstractKubernetesParameters;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.ContainerBuilder;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.PodBuilder;
+import io.fabric8.kubernetes.api.model.Volume;
+import io.fabric8.kubernetes.api.model.VolumeBuilder;
+import io.fabric8.kubernetes.api.model.VolumeMount;
+import io.fabric8.kubernetes.api.model.VolumeMountBuilder;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * support mounting Secrets on the JobManager or TaskManager pod..
+ */
+public class MountSecretsDecorator extends AbstractKubernetesStepDecorator {
+
+	private final AbstractKubernetesParameters kubernetesComponentConf;
+
+	public MountSecretsDecorator(AbstractKubernetesParameters kubernetesComponentConf) {
+		this.kubernetesComponentConf = checkNotNull(kubernetesComponentConf);
+	}
+
+	@Override
+	public FlinkPod decorateFlinkPod(FlinkPod flinkPod) {
+		final Pod podWithMount = decoratePod(flinkPod.getPod());
+		final Container containerWithMount = decorateMainContainer(flinkPod.getMainContainer());
+
+		return new FlinkPod.Builder(flinkPod)
+			.withPod(podWithMount)
+			.withMainContainer(containerWithMount)
+			.build();
+	}
+
+	private Container decorateMainContainer(Container container) {
+		final VolumeMount[] volumeMounts = kubernetesComponentConf

Review comment:
       ```
   		final VolumeMount[] volumeMounts = kubernetesComponentConf.getSecretNamesToMountPaths().entrySet().stream()
   			.map(secretNameToPath -> new VolumeMountBuilder()
   				.withName(secretVolumeName(secretNameToPath.getKey()))
   				.withMountPath(secretNameToPath.getValue())
   				.build()
   			)
   			.toArray(VolumeMount[]::new);
   ```
   
   Maybe this format is better. Please check the code style for more information.
   https://flink.apache.org/contributing/code-style-and-quality-formatting.html#breaking-the-lines-of-too-long-statements




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zhoulii commented on a change in pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
zhoulii commented on a change in pull request #12899:
URL: https://github.com/apache/flink/pull/12899#discussion_r471872714



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/MountSecretsDecorator.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.kubeclient.decorators;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.parameters.AbstractKubernetesParameters;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.ContainerBuilder;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.PodBuilder;
+import io.fabric8.kubernetes.api.model.Volume;
+import io.fabric8.kubernetes.api.model.VolumeBuilder;
+import io.fabric8.kubernetes.api.model.VolumeMount;
+import io.fabric8.kubernetes.api.model.VolumeMountBuilder;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * support mounting Secrets on the JobManager or TaskManager pod..
+ */
+public class MountSecretsDecorator extends AbstractKubernetesStepDecorator {
+
+	private final AbstractKubernetesParameters kubernetesComponentConf;
+
+	public MountSecretsDecorator(AbstractKubernetesParameters kubernetesComponentConf) {
+		this.kubernetesComponentConf = checkNotNull(kubernetesComponentConf);
+	}
+
+	@Override
+	public FlinkPod decorateFlinkPod(FlinkPod flinkPod) {
+		final Pod podWithMount = decoratePod(flinkPod.getPod());
+		final Container containerWithMount = decorateMainContainer(flinkPod.getMainContainer());
+
+		return new FlinkPod.Builder(flinkPod)
+			.withPod(podWithMount)
+			.withMainContainer(containerWithMount)
+			.build();
+	}
+
+	private Container decorateMainContainer(Container container) {
+		final VolumeMount[] volumeMounts = kubernetesComponentConf

Review comment:
       done in https://github.com/apache/flink/pull/12899/files#diff-0e5823ba44ea4819a4c33767020ee54bR58




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zhoulii commented on a change in pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
zhoulii commented on a change in pull request #12899:
URL: https://github.com/apache/flink/pull/12899#discussion_r476051896



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/MountSecretsDecorator.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.kubeclient.decorators;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.parameters.AbstractKubernetesParameters;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.ContainerBuilder;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.PodBuilder;
+import io.fabric8.kubernetes.api.model.Volume;
+import io.fabric8.kubernetes.api.model.VolumeBuilder;
+import io.fabric8.kubernetes.api.model.VolumeMount;
+import io.fabric8.kubernetes.api.model.VolumeMountBuilder;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * support mounting Secrets on the JobManager or TaskManager pod..

Review comment:
       done in https://github.com/apache/flink/pull/12899/files#diff-0e5823ba44ea4819a4c33767020ee54bR36




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2f0285b88d653612277714e12e96d48e2f82fc51 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522) 
   * 46841d653da7282e6340ff9a25d6c2bc39abd00f UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 77f95464c6036283d77d6e81e2bd21ab27f29d54 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506) 
   * 2f0285b88d653612277714e12e96d48e2f82fc51 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474",
       "triggerID" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479",
       "triggerID" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1393e3d777e722703e850187861d33841b2f888c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5486",
       "triggerID" : "1393e3d777e722703e850187861d33841b2f888c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5637",
       "triggerID" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "triggerType" : "PUSH"
     }, {
       "hash" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5646",
       "triggerID" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0282e24a53a29e90a68216b79fbd749cde3dd808",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5800",
       "triggerID" : "0282e24a53a29e90a68216b79fbd749cde3dd808",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ca2f1b495438e216320525f6b74db203f08279d9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5803",
       "triggerID" : "ca2f1b495438e216320525f6b74db203f08279d9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 99f04f52689e520f48ebc1cfb263926f347a22c4 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5646) 
   * 0282e24a53a29e90a68216b79fbd749cde3dd808 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5800) 
   * ca2f1b495438e216320525f6b74db203f08279d9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5803) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-669843223


   cc @wangyang0918 could you help with reviewing this PR?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #12899:
URL: https://github.com/apache/flink/pull/12899#discussion_r471422158



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
##########
@@ -221,6 +221,21 @@
 	/** Defines the configuration key of that external resource in Kubernetes. This is used as a suffix in an actual config. */
 	public static final String EXTERNAL_RESOURCE_KUBERNETES_CONFIG_KEY_SUFFIX = "kubernetes.config-key";
 
+	public static final ConfigOption<Map<String, String>> KUBERNETES_SECRETS =
+		key("kubernetes.secrets")
+			.mapType()
+			.noDefaultValue()
+			.withDescription("The user-specified secrets that will be mounted into Flink container. The value should be in " +
+				"the form of foo:/opt/secrets-foo,bar:/opt/secrets-bar. Users could also specify this config option via -D cli option.");

Review comment:
       ```suggestion
   				"the form of foo:/opt/secrets-foo,bar:/opt/secrets-bar.");
   ```

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/EnvSecretsDecorator.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.kubeclient.decorators;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.parameters.AbstractKubernetesParameters;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesSecretEnvVar;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.ContainerBuilder;
+import io.fabric8.kubernetes.api.model.EnvVar;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * support setting environment variables via Secrets.
+ */
+public class EnvSecretsDecorator extends AbstractKubernetesStepDecorator {
+
+	private final AbstractKubernetesParameters kubernetesComponentConf;
+
+	public EnvSecretsDecorator(AbstractKubernetesParameters kubernetesComponentConf) {
+		this.kubernetesComponentConf = checkNotNull(kubernetesComponentConf);
+	}
+
+	@Override
+	public FlinkPod decorateFlinkPod(FlinkPod flinkPod) {
+		final Container basicMainContainer = new ContainerBuilder(flinkPod.getMainContainer())
+			.addAllToEnv(getSecretEnvs())
+			.build();
+
+		return new FlinkPod.Builder(flinkPod)
+			.withPod(flinkPod.getPod())

Review comment:
       Maybe we remove this line since we do not decorate the existing pod.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/VolumeTestUtils.java
##########
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.Pod;
+
+/**
+ * Utilities for the Kubernetes tests.
+ */
+public class VolumeTestUtils {
+
+	public static boolean podHasVolume(Pod pod, String volumeName){
+		return pod.getSpec()

Review comment:
       Same as above about how to break the long line.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/MountSecretsDecorator.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.kubeclient.decorators;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.parameters.AbstractKubernetesParameters;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.ContainerBuilder;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.PodBuilder;
+import io.fabric8.kubernetes.api.model.Volume;
+import io.fabric8.kubernetes.api.model.VolumeBuilder;
+import io.fabric8.kubernetes.api.model.VolumeMount;
+import io.fabric8.kubernetes.api.model.VolumeMountBuilder;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * support mounting Secrets on the JobManager or TaskManager pod..
+ */
+public class MountSecretsDecorator extends AbstractKubernetesStepDecorator {
+
+	private final AbstractKubernetesParameters kubernetesComponentConf;
+
+	public MountSecretsDecorator(AbstractKubernetesParameters kubernetesComponentConf) {
+		this.kubernetesComponentConf = checkNotNull(kubernetesComponentConf);
+	}
+
+	@Override
+	public FlinkPod decorateFlinkPod(FlinkPod flinkPod) {
+		final Pod podWithMount = decoratePod(flinkPod.getPod());
+		final Container containerWithMount = decorateMainContainer(flinkPod.getMainContainer());
+
+		return new FlinkPod.Builder(flinkPod)
+			.withPod(podWithMount)
+			.withMainContainer(containerWithMount)
+			.build();
+	}
+
+	private Container decorateMainContainer(Container container) {
+		final VolumeMount[] volumeMounts = kubernetesComponentConf

Review comment:
       ```
   final VolumeMount[] volumeMounts = kubernetesComponentConf.getSecretNamesToMountPaths().entrySet().stream()
   			.map(secretNameToPath -> new VolumeMountBuilder()
   				.withName(secretVolumeName(secretNameToPath.getKey()))
   				.withMountPath(secretNameToPath.getValue())
   				.build()
   			)
   			.toArray(VolumeMount[]::new);
   ```
   
   Maybe this format is better. Please check the code style for more information.
   https://flink.apache.org/contributing/code-style-and-quality-formatting.html#breaking-the-lines-of-too-long-statements

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/MountSecretsDecorator.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.kubeclient.decorators;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.parameters.AbstractKubernetesParameters;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.ContainerBuilder;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.PodBuilder;
+import io.fabric8.kubernetes.api.model.Volume;
+import io.fabric8.kubernetes.api.model.VolumeBuilder;
+import io.fabric8.kubernetes.api.model.VolumeMount;
+import io.fabric8.kubernetes.api.model.VolumeMountBuilder;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * support mounting Secrets on the JobManager or TaskManager pod..
+ */
+public class MountSecretsDecorator extends AbstractKubernetesStepDecorator {
+
+	private final AbstractKubernetesParameters kubernetesComponentConf;
+
+	public MountSecretsDecorator(AbstractKubernetesParameters kubernetesComponentConf) {
+		this.kubernetesComponentConf = checkNotNull(kubernetesComponentConf);
+	}
+
+	@Override
+	public FlinkPod decorateFlinkPod(FlinkPod flinkPod) {
+		final Pod podWithMount = decoratePod(flinkPod.getPod());
+		final Container containerWithMount = decorateMainContainer(flinkPod.getMainContainer());
+
+		return new FlinkPod.Builder(flinkPod)
+			.withPod(podWithMount)
+			.withMainContainer(containerWithMount)
+			.build();
+	}
+
+	private Container decorateMainContainer(Container container) {
+		final VolumeMount[] volumeMounts = kubernetesComponentConf
+			.getSecretNamesToMountPaths()
+			.entrySet()
+			.stream()
+			.map(secretNameToPath ->
+				new VolumeMountBuilder()
+					.withName(secretVolumeName(secretNameToPath.getKey()))
+					.withMountPath(secretNameToPath.getValue())
+					.build()
+			).toArray(VolumeMount[]::new);
+
+		return new ContainerBuilder(container)
+			.addToVolumeMounts(volumeMounts)
+			.build();
+	}
+
+	private Pod decoratePod(Pod pod) {
+		final Volume[] volumes = kubernetesComponentConf.getSecretNamesToMountPaths()

Review comment:
       Same as above.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474",
       "triggerID" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479",
       "triggerID" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1393e3d777e722703e850187861d33841b2f888c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5486",
       "triggerID" : "1393e3d777e722703e850187861d33841b2f888c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5637",
       "triggerID" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "triggerType" : "PUSH"
     }, {
       "hash" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5646",
       "triggerID" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0282e24a53a29e90a68216b79fbd749cde3dd808",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5800",
       "triggerID" : "0282e24a53a29e90a68216b79fbd749cde3dd808",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ca2f1b495438e216320525f6b74db203f08279d9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5803",
       "triggerID" : "ca2f1b495438e216320525f6b74db203f08279d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31ac34fb5f8aa55f31464f703e29815e09ade9de",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "31ac34fb5f8aa55f31464f703e29815e09ade9de",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7c7e89e76154ab9ae40c4f79a5e83ad644cab60",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7c7e89e76154ab9ae40c4f79a5e83ad644cab60",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4769b098d0e6667767a4dcafedbae81f66b182ca",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5813",
       "triggerID" : "4769b098d0e6667767a4dcafedbae81f66b182ca",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0282e24a53a29e90a68216b79fbd749cde3dd808 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5800) 
   * ca2f1b495438e216320525f6b74db203f08279d9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5803) 
   * 31ac34fb5f8aa55f31464f703e29815e09ade9de UNKNOWN
   * e7c7e89e76154ab9ae40c4f79a5e83ad644cab60 UNKNOWN
   * 4769b098d0e6667767a4dcafedbae81f66b182ca Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5813) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474",
       "triggerID" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479",
       "triggerID" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 46841d653da7282e6340ff9a25d6c2bc39abd00f Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474) 
   * c6edb8841a675096cab4c0e866c3b95c1f3801bf Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474",
       "triggerID" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479",
       "triggerID" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1393e3d777e722703e850187861d33841b2f888c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5486",
       "triggerID" : "1393e3d777e722703e850187861d33841b2f888c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5637",
       "triggerID" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "triggerType" : "PUSH"
     }, {
       "hash" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5646",
       "triggerID" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0282e24a53a29e90a68216b79fbd749cde3dd808",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5800",
       "triggerID" : "0282e24a53a29e90a68216b79fbd749cde3dd808",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ca2f1b495438e216320525f6b74db203f08279d9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ca2f1b495438e216320525f6b74db203f08279d9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 99f04f52689e520f48ebc1cfb263926f347a22c4 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5646) 
   * 0282e24a53a29e90a68216b79fbd749cde3dd808 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5800) 
   * ca2f1b495438e216320525f6b74db203f08279d9 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zhoulii commented on a change in pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
zhoulii commented on a change in pull request #12899:
URL: https://github.com/apache/flink/pull/12899#discussion_r471934466



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/EnvSecretsDecorator.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.kubeclient.decorators;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.parameters.AbstractKubernetesParameters;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesSecretEnvVar;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.ContainerBuilder;
+import io.fabric8.kubernetes.api.model.EnvVar;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * support setting environment variables via Secrets.
+ */
+public class EnvSecretsDecorator extends AbstractKubernetesStepDecorator {
+
+	private final AbstractKubernetesParameters kubernetesComponentConf;
+
+	public EnvSecretsDecorator(AbstractKubernetesParameters kubernetesComponentConf) {
+		this.kubernetesComponentConf = checkNotNull(kubernetesComponentConf);
+	}
+
+	@Override
+	public FlinkPod decorateFlinkPod(FlinkPod flinkPod) {
+		final Container basicMainContainer = new ContainerBuilder(flinkPod.getMainContainer())
+			.addAllToEnv(getSecretEnvs())
+			.build();
+
+		return new FlinkPod.Builder(flinkPod)
+			.withMainContainer(basicMainContainer)
+			.build();
+	}
+
+	private List<EnvVar> getSecretEnvs() {
+		return kubernetesComponentConf.getEnvironmentsFromSecrets()

Review comment:
       done in https://github.com/apache/flink/pull/12899/files#diff-5937442e6c2456441334306f869a6121R57




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zhoulii commented on a change in pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
zhoulii commented on a change in pull request #12899:
URL: https://github.com/apache/flink/pull/12899#discussion_r476050973



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
##########
@@ -221,6 +221,21 @@
 	/** Defines the configuration key of that external resource in Kubernetes. This is used as a suffix in an actual config. */
 	public static final String EXTERNAL_RESOURCE_KUBERNETES_CONFIG_KEY_SUFFIX = "kubernetes.config-key";
 
+	public static final ConfigOption<Map<String, String>> KUBERNETES_SECRETS =
+		key("kubernetes.secrets")
+			.mapType()
+			.noDefaultValue()
+			.withDescription("The user-specified secrets that will be mounted into Flink container. The value should be in " +
+				"the form of foo:/opt/secrets-foo,bar:/opt/secrets-bar.");

Review comment:
       add some description in [native_kubernetes.md](https://github.com/apache/flink/pull/12899/files#diff-8ac5ff0acb9a268297bd157249c52acdR214)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474",
       "triggerID" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479",
       "triggerID" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c6edb8841a675096cab4c0e866c3b95c1f3801bf Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zhoulii commented on a change in pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
zhoulii commented on a change in pull request #12899:
URL: https://github.com/apache/flink/pull/12899#discussion_r469670097



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/EnvSecretsDecorator.java
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.kubeclient.decorators;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.parameters.AbstractKubernetesParameters;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesEnvVar;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.ContainerBuilder;
+import io.fabric8.kubernetes.api.model.EnvVar;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * support setting environment variables via Secrets.
+ */
+public class EnvSecretsDecorator extends AbstractKubernetesStepDecorator {
+
+	private final AbstractKubernetesParameters kubernetesComponentConf;
+
+	public EnvSecretsDecorator(AbstractKubernetesParameters kubernetesComponentConf) {
+		this.kubernetesComponentConf = checkNotNull(kubernetesComponentConf);
+	}
+
+	@Override
+	public FlinkPod decorateFlinkPod(FlinkPod flinkPod) {
+		final Container basicMainContainer = decorateMainContainer(flinkPod.getMainContainer());

Review comment:
       is the logic of method `decorateMainContainer` too simple, does not need to be wrapped as a method ?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 77f95464c6036283d77d6e81e2bd21ab27f29d54 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zhoulii commented on a change in pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
zhoulii commented on a change in pull request #12899:
URL: https://github.com/apache/flink/pull/12899#discussion_r471872180



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
##########
@@ -221,6 +221,21 @@
 	/** Defines the configuration key of that external resource in Kubernetes. This is used as a suffix in an actual config. */
 	public static final String EXTERNAL_RESOURCE_KUBERNETES_CONFIG_KEY_SUFFIX = "kubernetes.config-key";
 
+	public static final ConfigOption<Map<String, String>> KUBERNETES_SECRETS =
+		key("kubernetes.secrets")
+			.mapType()
+			.noDefaultValue()
+			.withDescription("The user-specified secrets that will be mounted into Flink container. The value should be in " +
+				"the form of foo:/opt/secrets-foo,bar:/opt/secrets-bar. Users could also specify this config option via -D cli option.");

Review comment:
       done in https://github.com/apache/flink/pull/12899/files#diff-1cd32b8ad754e869567bc0d78d39139fR237




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474",
       "triggerID" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479",
       "triggerID" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1393e3d777e722703e850187861d33841b2f888c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5486",
       "triggerID" : "1393e3d777e722703e850187861d33841b2f888c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5637",
       "triggerID" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "triggerType" : "PUSH"
     }, {
       "hash" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5646",
       "triggerID" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0282e24a53a29e90a68216b79fbd749cde3dd808",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5800",
       "triggerID" : "0282e24a53a29e90a68216b79fbd749cde3dd808",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ca2f1b495438e216320525f6b74db203f08279d9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5803",
       "triggerID" : "ca2f1b495438e216320525f6b74db203f08279d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31ac34fb5f8aa55f31464f703e29815e09ade9de",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "31ac34fb5f8aa55f31464f703e29815e09ade9de",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7c7e89e76154ab9ae40c4f79a5e83ad644cab60",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7c7e89e76154ab9ae40c4f79a5e83ad644cab60",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0282e24a53a29e90a68216b79fbd749cde3dd808 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5800) 
   * ca2f1b495438e216320525f6b74db203f08279d9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5803) 
   * 31ac34fb5f8aa55f31464f703e29815e09ade9de UNKNOWN
   * e7c7e89e76154ab9ae40c4f79a5e83ad644cab60 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474",
       "triggerID" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479",
       "triggerID" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1393e3d777e722703e850187861d33841b2f888c",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5486",
       "triggerID" : "1393e3d777e722703e850187861d33841b2f888c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1393e3d777e722703e850187861d33841b2f888c Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5486) 
   * ffa55544892c9d4eadf42ca2fcead4ba53701510 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474",
       "triggerID" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479",
       "triggerID" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1393e3d777e722703e850187861d33841b2f888c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5486",
       "triggerID" : "1393e3d777e722703e850187861d33841b2f888c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5637",
       "triggerID" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "triggerType" : "PUSH"
     }, {
       "hash" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5646",
       "triggerID" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ffa55544892c9d4eadf42ca2fcead4ba53701510 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5637) 
   * 99f04f52689e520f48ebc1cfb263926f347a22c4 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5646) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] wangyang0918 edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
wangyang0918 edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-675294429


   cc @tillrohrmann @rmetzger, this PR looks good to me. And i have verified it in a real K8s cluster. Could you have a final check and help with merging?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474",
       "triggerID" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479",
       "triggerID" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1393e3d777e722703e850187861d33841b2f888c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5486",
       "triggerID" : "1393e3d777e722703e850187861d33841b2f888c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5637",
       "triggerID" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "triggerType" : "PUSH"
     }, {
       "hash" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5646",
       "triggerID" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0282e24a53a29e90a68216b79fbd749cde3dd808",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5800",
       "triggerID" : "0282e24a53a29e90a68216b79fbd749cde3dd808",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ca2f1b495438e216320525f6b74db203f08279d9",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5803",
       "triggerID" : "ca2f1b495438e216320525f6b74db203f08279d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31ac34fb5f8aa55f31464f703e29815e09ade9de",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "31ac34fb5f8aa55f31464f703e29815e09ade9de",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7c7e89e76154ab9ae40c4f79a5e83ad644cab60",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7c7e89e76154ab9ae40c4f79a5e83ad644cab60",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4769b098d0e6667767a4dcafedbae81f66b182ca",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5813",
       "triggerID" : "4769b098d0e6667767a4dcafedbae81f66b182ca",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ca2f1b495438e216320525f6b74db203f08279d9 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5803) 
   * 31ac34fb5f8aa55f31464f703e29815e09ade9de UNKNOWN
   * e7c7e89e76154ab9ae40c4f79a5e83ad644cab60 UNKNOWN
   * 4769b098d0e6667767a4dcafedbae81f66b182ca Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5813) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zhoulii commented on a change in pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
zhoulii commented on a change in pull request #12899:
URL: https://github.com/apache/flink/pull/12899#discussion_r470372722



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
##########
@@ -221,6 +221,22 @@
 	/** Defines the configuration key of that external resource in Kubernetes. This is used as a suffix in an actual config. */
 	public static final String EXTERNAL_RESOURCE_KUBERNETES_CONFIG_KEY_SUFFIX = "kubernetes.config-key";
 
+	public static final ConfigOption<Map<String, String>> KUBERNETES_SECRETS =
+		key("kubernetes.secrets")
+			.mapType()
+			.noDefaultValue()
+			.withDescription("The user-specified secrets that are mounted into Flink container. The value should be in " +
+				"the form of a1:v1,a2:v2. For example for mounting secrets foo to path /opt/secrets , just need to set " +
+				"kubernetes.secrets: foo:/opt/secrets in the flink-conf.yaml");
+
+	public static final ConfigOption<List<Map<String, String>>> KUBERNETES_ENV_SECRET_KEY_REF =
+		key("kubernetes.env.secretKeyRef")
+			.mapType()
+			.asList()
+			.noDefaultValue()
+			.withDescription("the user-specified secrets to set env variable in Flink container. The value should be in " +

Review comment:
       fixed




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #12899:
URL: https://github.com/apache/flink/pull/12899#discussion_r471985089



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
##########
@@ -221,6 +221,21 @@
 	/** Defines the configuration key of that external resource in Kubernetes. This is used as a suffix in an actual config. */
 	public static final String EXTERNAL_RESOURCE_KUBERNETES_CONFIG_KEY_SUFFIX = "kubernetes.config-key";
 
+	public static final ConfigOption<Map<String, String>> KUBERNETES_SECRETS =
+		key("kubernetes.secrets")
+			.mapType()
+			.noDefaultValue()
+			.withDescription("The user-specified secrets that will be mounted into Flink container. The value should be in " +
+				"the form of foo:/opt/secrets-foo,bar:/opt/secrets-bar.");
+
+	public static final ConfigOption<List<Map<String, String>>> KUBERNETES_ENV_SECRET_KEY_REF =
+		key("kubernetes.env.secretKeyRef")
+			.mapType()
+			.asList()
+			.noDefaultValue()
+			.withDescription("The user-specified secrets to set env variables in Flink container. The value should be in " +
+				"the form of env:FOO_ENV,secret:foo_secret,key:foo_key;env:BAR_ENV,secret:bar_secret,key:bar_key.");

Review comment:
       Same here with `TextElement.code` and `env:FOO_ENV,secret:foo_secret,key:foo_key;env:BAR_ENV,secret:bar_secret,key:bar_key`

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
##########
@@ -221,6 +221,21 @@
 	/** Defines the configuration key of that external resource in Kubernetes. This is used as a suffix in an actual config. */
 	public static final String EXTERNAL_RESOURCE_KUBERNETES_CONFIG_KEY_SUFFIX = "kubernetes.config-key";
 
+	public static final ConfigOption<Map<String, String>> KUBERNETES_SECRETS =
+		key("kubernetes.secrets")
+			.mapType()
+			.noDefaultValue()
+			.withDescription("The user-specified secrets that will be mounted into Flink container. The value should be in " +
+				"the form of foo:/opt/secrets-foo,bar:/opt/secrets-bar.");

Review comment:
       ```suggestion
   			.withDescription(
   			    Description.builder()
   			        .text("The user-specified secrets that will be mounted into Flink container. The value should be in " +
   				        "the form of %s.", TextElement.code("foo:/opt/secrets-foo,bar:/opt/secrets-bar"))
   				    .build());
   ```

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesSecretEnvVar.java
##########
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.kubeclient.resources;
+
+import io.fabric8.kubernetes.api.model.EnvVar;
+import io.fabric8.kubernetes.api.model.EnvVarBuilder;
+
+import java.util.Map;
+
+/**
+ * Represent EnvVar resource in kubernetes.

Review comment:
       ```suggestion
    * Represents EnvVar resource in Kubernetes.
   ```

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/EnvSecretsDecorator.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.kubeclient.decorators;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.parameters.AbstractKubernetesParameters;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesSecretEnvVar;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.ContainerBuilder;
+import io.fabric8.kubernetes.api.model.EnvVar;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * support setting environment variables via Secrets.

Review comment:
       ```suggestion
    * Support setting environment variables via Secrets.
   ```

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/MountSecretsDecorator.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.kubeclient.decorators;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.parameters.AbstractKubernetesParameters;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.ContainerBuilder;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.PodBuilder;
+import io.fabric8.kubernetes.api.model.Volume;
+import io.fabric8.kubernetes.api.model.VolumeBuilder;
+import io.fabric8.kubernetes.api.model.VolumeMount;
+import io.fabric8.kubernetes.api.model.VolumeMountBuilder;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * support mounting Secrets on the JobManager or TaskManager pod..

Review comment:
       ```suggestion
    * Support mounting Secrets on the JobManager or TaskManager pod..
   ```

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java
##########
@@ -110,4 +110,14 @@
 	 * The local directory to locate the custom Hadoop configuration.
 	 */
 	Optional<String> getLocalHadoopConfigurationDirectory();
+
+	/**
+	 * A collection of Secrets and path that are mounted to the JobManager and TaskManager Container(s).

Review comment:
       ```suggestion
   	 * A collection of secret and path pairs that are mounted to the JobManager and TaskManager container(s).
   ```

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java
##########
@@ -110,4 +110,14 @@
 	 * The local directory to locate the custom Hadoop configuration.
 	 */
 	Optional<String> getLocalHadoopConfigurationDirectory();
+
+	/**
+	 * A collection of Secrets and path that are mounted to the JobManager and TaskManager Container(s).
+	 */
+	Map<String, String> getSecretNamesToMountPaths();
+
+	/**
+	 * A collection of customized environments that are attached to the JobManager and TaskManager Container(s).

Review comment:
       ```suggestion
   	 * A collection of customized environments that are attached to the JobManager and TaskManager container(s).
   ```

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
##########
@@ -221,6 +221,21 @@
 	/** Defines the configuration key of that external resource in Kubernetes. This is used as a suffix in an actual config. */
 	public static final String EXTERNAL_RESOURCE_KUBERNETES_CONFIG_KEY_SUFFIX = "kubernetes.config-key";
 
+	public static final ConfigOption<Map<String, String>> KUBERNETES_SECRETS =
+		key("kubernetes.secrets")
+			.mapType()
+			.noDefaultValue()
+			.withDescription("The user-specified secrets that will be mounted into Flink container. The value should be in " +
+				"the form of foo:/opt/secrets-foo,bar:/opt/secrets-bar.");
+
+	public static final ConfigOption<List<Map<String, String>>> KUBERNETES_ENV_SECRET_KEY_REF =
+		key("kubernetes.env.secretKeyRef")
+			.mapType()
+			.asList()
+			.noDefaultValue()
+			.withDescription("The user-specified secrets to set env variables in Flink container. The value should be in " +
+				"the form of env:FOO_ENV,secret:foo_secret,key:foo_key;env:BAR_ENV,secret:bar_secret,key:bar_key.");

Review comment:
       To me it is not very clear how to use this configuration option without more context or even a small example which we could add to `native_kubernetes.md`. Moreover, we should add a link to https://kubernetes.io/docs/concepts/configuration/secret/ for reference.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
##########
@@ -221,6 +221,21 @@
 	/** Defines the configuration key of that external resource in Kubernetes. This is used as a suffix in an actual config. */
 	public static final String EXTERNAL_RESOURCE_KUBERNETES_CONFIG_KEY_SUFFIX = "kubernetes.config-key";
 
+	public static final ConfigOption<Map<String, String>> KUBERNETES_SECRETS =
+		key("kubernetes.secrets")
+			.mapType()
+			.noDefaultValue()
+			.withDescription("The user-specified secrets that will be mounted into Flink container. The value should be in " +
+				"the form of foo:/opt/secrets-foo,bar:/opt/secrets-bar.");

Review comment:
       I think it would be good to add a bit more description (might even want to add this description to `native_kubernetes.md`) for how to configure the secrets. At least we should add a link to the K8s documentation https://kubernetes.io/docs/concepts/configuration/secret/.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474",
       "triggerID" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479",
       "triggerID" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1393e3d777e722703e850187861d33841b2f888c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5486",
       "triggerID" : "1393e3d777e722703e850187861d33841b2f888c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5637",
       "triggerID" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "triggerType" : "PUSH"
     }, {
       "hash" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5646",
       "triggerID" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0282e24a53a29e90a68216b79fbd749cde3dd808",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5800",
       "triggerID" : "0282e24a53a29e90a68216b79fbd749cde3dd808",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ca2f1b495438e216320525f6b74db203f08279d9",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5803",
       "triggerID" : "ca2f1b495438e216320525f6b74db203f08279d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31ac34fb5f8aa55f31464f703e29815e09ade9de",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "31ac34fb5f8aa55f31464f703e29815e09ade9de",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7c7e89e76154ab9ae40c4f79a5e83ad644cab60",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7c7e89e76154ab9ae40c4f79a5e83ad644cab60",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4769b098d0e6667767a4dcafedbae81f66b182ca",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5813",
       "triggerID" : "4769b098d0e6667767a4dcafedbae81f66b182ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ad037b9f4ff761d21e063c8dd7f92982437fb6c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2ad037b9f4ff761d21e063c8dd7f92982437fb6c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ca2f1b495438e216320525f6b74db203f08279d9 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5803) 
   * 31ac34fb5f8aa55f31464f703e29815e09ade9de UNKNOWN
   * e7c7e89e76154ab9ae40c4f79a5e83ad644cab60 UNKNOWN
   * 4769b098d0e6667767a4dcafedbae81f66b182ca Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5813) 
   * 2ad037b9f4ff761d21e063c8dd7f92982437fb6c UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 77f95464c6036283d77d6e81e2bd21ab27f29d54 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506) 
   * 2f0285b88d653612277714e12e96d48e2f82fc51 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 77f95464c6036283d77d6e81e2bd21ab27f29d54 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474",
       "triggerID" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479",
       "triggerID" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1393e3d777e722703e850187861d33841b2f888c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1393e3d777e722703e850187861d33841b2f888c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c6edb8841a675096cab4c0e866c3b95c1f3801bf Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479) 
   * 1393e3d777e722703e850187861d33841b2f888c UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zhoulii commented on a change in pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
zhoulii commented on a change in pull request #12899:
URL: https://github.com/apache/flink/pull/12899#discussion_r471934259



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
##########
@@ -221,6 +221,21 @@
 	/** Defines the configuration key of that external resource in Kubernetes. This is used as a suffix in an actual config. */
 	public static final String EXTERNAL_RESOURCE_KUBERNETES_CONFIG_KEY_SUFFIX = "kubernetes.config-key";
 
+	public static final ConfigOption<Map<String, String>> KUBERNETES_SECRETS =
+		key("kubernetes.secrets")
+			.mapType()
+			.noDefaultValue()
+			.withDescription("The user-specified secrets that will be mounted into Flink container. The value should be in " +
+				"the form of foo:/opt/secrets-foo,bar:/opt/secrets-bar.");
+
+	public static final ConfigOption<List<Map<String, String>>> KUBERNETES_ENV_SECRET_KEY_REF =
+		key("kubernetes.env.secretKeyRef")
+			.mapType()
+			.asList()
+			.noDefaultValue()
+			.withDescription("The user-specified secrets to set env variables in Flink container. The value should be in " +
+				"the form of env:FOO_ENV,secret:foo_secret,key:foo_key;env:BAR_ENV,secret:bar_secret,key:bar_key");

Review comment:
       done in https://github.com/apache/flink/pull/12899/files#diff-1cd32b8ad754e869567bc0d78d39139fR237




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474",
       "triggerID" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479",
       "triggerID" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1393e3d777e722703e850187861d33841b2f888c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5486",
       "triggerID" : "1393e3d777e722703e850187861d33841b2f888c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c6edb8841a675096cab4c0e866c3b95c1f3801bf Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479) 
   * 1393e3d777e722703e850187861d33841b2f888c Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5486) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474",
       "triggerID" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479",
       "triggerID" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1393e3d777e722703e850187861d33841b2f888c",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5486",
       "triggerID" : "1393e3d777e722703e850187861d33841b2f888c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5637",
       "triggerID" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "triggerType" : "PUSH"
     }, {
       "hash" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1393e3d777e722703e850187861d33841b2f888c Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5486) 
   * ffa55544892c9d4eadf42ca2fcead4ba53701510 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5637) 
   * 99f04f52689e520f48ebc1cfb263926f347a22c4 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 77f95464c6036283d77d6e81e2bd21ab27f29d54 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-669861807


   @tillrohrmann Thanks for you kindly reminding. I will take a look the PR shortly.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474",
       "triggerID" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479",
       "triggerID" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1393e3d777e722703e850187861d33841b2f888c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5486",
       "triggerID" : "1393e3d777e722703e850187861d33841b2f888c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5637",
       "triggerID" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "triggerType" : "PUSH"
     }, {
       "hash" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5646",
       "triggerID" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0282e24a53a29e90a68216b79fbd749cde3dd808",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5800",
       "triggerID" : "0282e24a53a29e90a68216b79fbd749cde3dd808",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ca2f1b495438e216320525f6b74db203f08279d9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5803",
       "triggerID" : "ca2f1b495438e216320525f6b74db203f08279d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31ac34fb5f8aa55f31464f703e29815e09ade9de",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "31ac34fb5f8aa55f31464f703e29815e09ade9de",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7c7e89e76154ab9ae40c4f79a5e83ad644cab60",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7c7e89e76154ab9ae40c4f79a5e83ad644cab60",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4769b098d0e6667767a4dcafedbae81f66b182ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5813",
       "triggerID" : "4769b098d0e6667767a4dcafedbae81f66b182ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ad037b9f4ff761d21e063c8dd7f92982437fb6c",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5809",
       "triggerID" : "2ad037b9f4ff761d21e063c8dd7f92982437fb6c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 31ac34fb5f8aa55f31464f703e29815e09ade9de UNKNOWN
   * e7c7e89e76154ab9ae40c4f79a5e83ad644cab60 UNKNOWN
   * 2ad037b9f4ff761d21e063c8dd7f92982437fb6c Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5809) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zhoulii commented on a change in pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
zhoulii commented on a change in pull request #12899:
URL: https://github.com/apache/flink/pull/12899#discussion_r476052224



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/EnvSecretsDecorator.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.kubeclient.decorators;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.parameters.AbstractKubernetesParameters;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesSecretEnvVar;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.ContainerBuilder;
+import io.fabric8.kubernetes.api.model.EnvVar;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * support setting environment variables via Secrets.

Review comment:
       done in https://github.com/apache/flink/pull/12899/files#diff-5937442e6c2456441334306f869a6121R35




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658158568


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit 77f95464c6036283d77d6e81e2bd21ab27f29d54 (Tue Jul 14 12:46:05 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474",
       "triggerID" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479",
       "triggerID" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1393e3d777e722703e850187861d33841b2f888c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5486",
       "triggerID" : "1393e3d777e722703e850187861d33841b2f888c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5637",
       "triggerID" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "triggerType" : "PUSH"
     }, {
       "hash" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5646",
       "triggerID" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0282e24a53a29e90a68216b79fbd749cde3dd808",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0282e24a53a29e90a68216b79fbd749cde3dd808",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 99f04f52689e520f48ebc1cfb263926f347a22c4 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5646) 
   * 0282e24a53a29e90a68216b79fbd749cde3dd808 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474",
       "triggerID" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479",
       "triggerID" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1393e3d777e722703e850187861d33841b2f888c",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5486",
       "triggerID" : "1393e3d777e722703e850187861d33841b2f888c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5637",
       "triggerID" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1393e3d777e722703e850187861d33841b2f888c Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5486) 
   * ffa55544892c9d4eadf42ca2fcead4ba53701510 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5637) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zhoulii commented on a change in pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
zhoulii commented on a change in pull request #12899:
URL: https://github.com/apache/flink/pull/12899#discussion_r476051560



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
##########
@@ -221,6 +221,21 @@
 	/** Defines the configuration key of that external resource in Kubernetes. This is used as a suffix in an actual config. */
 	public static final String EXTERNAL_RESOURCE_KUBERNETES_CONFIG_KEY_SUFFIX = "kubernetes.config-key";
 
+	public static final ConfigOption<Map<String, String>> KUBERNETES_SECRETS =
+		key("kubernetes.secrets")
+			.mapType()
+			.noDefaultValue()
+			.withDescription("The user-specified secrets that will be mounted into Flink container. The value should be in " +
+				"the form of foo:/opt/secrets-foo,bar:/opt/secrets-bar.");
+
+	public static final ConfigOption<List<Map<String, String>>> KUBERNETES_ENV_SECRET_KEY_REF =
+		key("kubernetes.env.secretKeyRef")
+			.mapType()
+			.asList()
+			.noDefaultValue()
+			.withDescription("The user-specified secrets to set env variables in Flink container. The value should be in " +
+				"the form of env:FOO_ENV,secret:foo_secret,key:foo_key;env:BAR_ENV,secret:bar_secret,key:bar_key.");

Review comment:
       done in https://github.com/apache/flink/pull/12899/files#diff-1cd32b8ad754e869567bc0d78d39139fR236




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zhoulii commented on a change in pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
zhoulii commented on a change in pull request #12899:
URL: https://github.com/apache/flink/pull/12899#discussion_r469670609



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/MountSecretsDecorator.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.kubeclient.decorators;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.parameters.AbstractKubernetesParameters;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.ContainerBuilder;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.PodBuilder;
+import io.fabric8.kubernetes.api.model.Volume;
+import io.fabric8.kubernetes.api.model.VolumeBuilder;
+import io.fabric8.kubernetes.api.model.VolumeMount;
+import io.fabric8.kubernetes.api.model.VolumeMountBuilder;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * support mounting Secrets on the JobManager or TaskManager pod..
+ */
+public class MountSecretsDecorator extends AbstractKubernetesStepDecorator {
+
+	private final AbstractKubernetesParameters kubernetesComponentConf;
+
+	public MountSecretsDecorator(AbstractKubernetesParameters kubernetesComponentConf) {
+		this.kubernetesComponentConf = checkNotNull(kubernetesComponentConf);
+	}
+
+	@Override
+	public FlinkPod decorateFlinkPod(FlinkPod flinkPod) {
+		final Pod podWithMount = decoratePod(flinkPod.getPod());
+		final Container containerWithMount = decorateMainContainer(flinkPod.getMainContainer());
+
+		return new FlinkPod.Builder(flinkPod)
+			.withPod(podWithMount)
+			.withMainContainer(containerWithMount)
+			.build();
+	}
+
+	private Container decorateMainContainer(Container container) {
+		VolumeMount[] volumeMounts = kubernetesComponentConf

Review comment:
       it is reasonable.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #12899:
URL: https://github.com/apache/flink/pull/12899#discussion_r471900279



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/EnvSecretsDecorator.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.kubeclient.decorators;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.parameters.AbstractKubernetesParameters;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesSecretEnvVar;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.ContainerBuilder;
+import io.fabric8.kubernetes.api.model.EnvVar;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * support setting environment variables via Secrets.
+ */
+public class EnvSecretsDecorator extends AbstractKubernetesStepDecorator {
+
+	private final AbstractKubernetesParameters kubernetesComponentConf;
+
+	public EnvSecretsDecorator(AbstractKubernetesParameters kubernetesComponentConf) {
+		this.kubernetesComponentConf = checkNotNull(kubernetesComponentConf);
+	}
+
+	@Override
+	public FlinkPod decorateFlinkPod(FlinkPod flinkPod) {
+		final Container basicMainContainer = new ContainerBuilder(flinkPod.getMainContainer())
+			.addAllToEnv(getSecretEnvs())
+			.build();
+
+		return new FlinkPod.Builder(flinkPod)
+			.withMainContainer(basicMainContainer)
+			.build();
+	}
+
+	private List<EnvVar> getSecretEnvs() {
+		return kubernetesComponentConf.getEnvironmentsFromSecrets()

Review comment:
       ```suggestion
   		return kubernetesComponentConf.getEnvironmentsFromSecrets().stream()
   ```

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
##########
@@ -221,6 +221,21 @@
 	/** Defines the configuration key of that external resource in Kubernetes. This is used as a suffix in an actual config. */
 	public static final String EXTERNAL_RESOURCE_KUBERNETES_CONFIG_KEY_SUFFIX = "kubernetes.config-key";
 
+	public static final ConfigOption<Map<String, String>> KUBERNETES_SECRETS =
+		key("kubernetes.secrets")
+			.mapType()
+			.noDefaultValue()
+			.withDescription("The user-specified secrets that will be mounted into Flink container. The value should be in " +
+				"the form of foo:/opt/secrets-foo,bar:/opt/secrets-bar.");
+
+	public static final ConfigOption<List<Map<String, String>>> KUBERNETES_ENV_SECRET_KEY_REF =
+		key("kubernetes.env.secretKeyRef")
+			.mapType()
+			.asList()
+			.noDefaultValue()
+			.withDescription("The user-specified secrets to set env variables in Flink container. The value should be in " +
+				"the form of env:FOO_ENV,secret:foo_secret,key:foo_key;env:BAR_ENV,secret:bar_secret,key:bar_key");

Review comment:
       ```suggestion
   				"the form of env:FOO_ENV,secret:foo_secret,key:foo_key;env:BAR_ENV,secret:bar_secret,key:bar_key.");
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zhoulii commented on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
zhoulii commented on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-679456963


   Hi @tillrohrmann , thanks for reviewing this pr. I add some description about how to use Secrets in native_kubernetes.md and update the code on your advice.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474",
       "triggerID" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2f0285b88d653612277714e12e96d48e2f82fc51 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522) 
   * 46841d653da7282e6340ff9a25d6c2bc39abd00f Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474) 
   * c6edb8841a675096cab4c0e866c3b95c1f3801bf UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474",
       "triggerID" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479",
       "triggerID" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1393e3d777e722703e850187861d33841b2f888c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5486",
       "triggerID" : "1393e3d777e722703e850187861d33841b2f888c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5637",
       "triggerID" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "triggerType" : "PUSH"
     }, {
       "hash" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5646",
       "triggerID" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0282e24a53a29e90a68216b79fbd749cde3dd808",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5800",
       "triggerID" : "0282e24a53a29e90a68216b79fbd749cde3dd808",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 99f04f52689e520f48ebc1cfb263926f347a22c4 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5646) 
   * 0282e24a53a29e90a68216b79fbd749cde3dd808 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5800) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #12899:
URL: https://github.com/apache/flink/pull/12899#discussion_r469246970



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
##########
@@ -221,6 +221,22 @@
 	/** Defines the configuration key of that external resource in Kubernetes. This is used as a suffix in an actual config. */
 	public static final String EXTERNAL_RESOURCE_KUBERNETES_CONFIG_KEY_SUFFIX = "kubernetes.config-key";
 
+	public static final ConfigOption<Map<String, String>> KUBERNETES_SECRETS =
+		key("kubernetes.secrets")
+			.mapType()
+			.noDefaultValue()
+			.withDescription("The user-specified secrets that are mounted into Flink container. The value should be in " +

Review comment:
       Could we update the description to the following?
   The user-specified secrets that will be mounted into Flink container. The value should be in the form of foo:/opt/secrets-foo,bar:/opt/secrets-bar.
   
   Users could also specify this config option via `-D` cli option.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/MountSecretsDecorator.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.kubeclient.decorators;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.parameters.AbstractKubernetesParameters;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.ContainerBuilder;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.PodBuilder;
+import io.fabric8.kubernetes.api.model.Volume;
+import io.fabric8.kubernetes.api.model.VolumeBuilder;
+import io.fabric8.kubernetes.api.model.VolumeMount;
+import io.fabric8.kubernetes.api.model.VolumeMountBuilder;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * support mounting Secrets on the JobManager or TaskManager pod..
+ */
+public class MountSecretsDecorator extends AbstractKubernetesStepDecorator {
+
+	private final AbstractKubernetesParameters kubernetesComponentConf;
+
+	public MountSecretsDecorator(AbstractKubernetesParameters kubernetesComponentConf) {
+		this.kubernetesComponentConf = checkNotNull(kubernetesComponentConf);
+	}
+
+	@Override
+	public FlinkPod decorateFlinkPod(FlinkPod flinkPod) {
+		final Pod podWithMount = decoratePod(flinkPod.getPod());
+		final Container containerWithMount = decorateMainContainer(flinkPod.getMainContainer());
+
+		return new FlinkPod.Builder(flinkPod)
+			.withPod(podWithMount)
+			.withMainContainer(containerWithMount)
+			.build();
+	}
+
+	private Container decorateMainContainer(Container container) {
+		VolumeMount[] volumeMounts = kubernetesComponentConf
+			.getSecretNamesToMountPaths()
+			.entrySet()
+			.stream()
+			.map(secretNameToPath ->
+				new VolumeMountBuilder()
+					.withName(secretVolumeName(secretNameToPath.getKey()))
+					.withMountPath(secretNameToPath.getValue())
+					.build()
+			).toArray(VolumeMount[]::new);
+
+		return new ContainerBuilder(container)
+			.addToVolumeMounts(volumeMounts)
+			.build();
+	}
+
+	private Pod decoratePod(Pod pod) {
+		Volume[] volumes = kubernetesComponentConf.getSecretNamesToMountPaths()

Review comment:
       ```suggestion
   		final Volume[] volumes = kubernetesComponentConf.getSecretNamesToMountPaths()
   ```

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
##########
@@ -221,6 +221,22 @@
 	/** Defines the configuration key of that external resource in Kubernetes. This is used as a suffix in an actual config. */
 	public static final String EXTERNAL_RESOURCE_KUBERNETES_CONFIG_KEY_SUFFIX = "kubernetes.config-key";
 
+	public static final ConfigOption<Map<String, String>> KUBERNETES_SECRETS =
+		key("kubernetes.secrets")
+			.mapType()
+			.noDefaultValue()
+			.withDescription("The user-specified secrets that are mounted into Flink container. The value should be in " +
+				"the form of a1:v1,a2:v2. For example for mounting secrets foo to path /opt/secrets , just need to set " +
+				"kubernetes.secrets: foo:/opt/secrets in the flink-conf.yaml");
+
+	public static final ConfigOption<List<Map<String, String>>> KUBERNETES_ENV_SECRET_KEY_REF =
+		key("kubernetes.env.secretKeyRef")
+			.mapType()
+			.asList()
+			.noDefaultValue()
+			.withDescription("the user-specified secrets to set env variable in Flink container. The value should be in " +
+				"the form of env:MY_ENV1,secret:my_secret1,key:my_key1;env:MY_ENV2,secret:my_secret2,key:my_key2");

Review comment:
       env:MY_ENV1,secret:my_secret1,key:my_key1;env:MY_ENV2,secret:my_secret2,key:my_key2
   
   I prefer not using **my** in the description. So it could be like following.
   env:FOO_ENV,secret:foo_secret,key:key;env:BAR_ENV,secret:bar_secret,key:key

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/EnvSecretsDecorator.java
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.kubeclient.decorators;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.parameters.AbstractKubernetesParameters;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesEnvVar;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.ContainerBuilder;
+import io.fabric8.kubernetes.api.model.EnvVar;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * support setting environment variables via Secrets.
+ */
+public class EnvSecretsDecorator extends AbstractKubernetesStepDecorator {
+
+	private final AbstractKubernetesParameters kubernetesComponentConf;
+
+	public EnvSecretsDecorator(AbstractKubernetesParameters kubernetesComponentConf) {
+		this.kubernetesComponentConf = checkNotNull(kubernetesComponentConf);
+	}
+
+	@Override
+	public FlinkPod decorateFlinkPod(FlinkPod flinkPod) {
+		final Container basicMainContainer = decorateMainContainer(flinkPod.getMainContainer());

Review comment:
       ```suggestion
   		final Container decoratedContainer = new ContainerBuilder(flinkPod.getMainContainer())
   			.addAllToEnv(getSecretEnvs())
   			.build();
   ```

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
##########
@@ -221,6 +221,22 @@
 	/** Defines the configuration key of that external resource in Kubernetes. This is used as a suffix in an actual config. */
 	public static final String EXTERNAL_RESOURCE_KUBERNETES_CONFIG_KEY_SUFFIX = "kubernetes.config-key";
 
+	public static final ConfigOption<Map<String, String>> KUBERNETES_SECRETS =
+		key("kubernetes.secrets")
+			.mapType()
+			.noDefaultValue()
+			.withDescription("The user-specified secrets that are mounted into Flink container. The value should be in " +
+				"the form of a1:v1,a2:v2. For example for mounting secrets foo to path /opt/secrets , just need to set " +
+				"kubernetes.secrets: foo:/opt/secrets in the flink-conf.yaml");
+
+	public static final ConfigOption<List<Map<String, String>>> KUBERNETES_ENV_SECRET_KEY_REF =
+		key("kubernetes.env.secretKeyRef")
+			.mapType()
+			.asList()
+			.noDefaultValue()
+			.withDescription("the user-specified secrets to set env variable in Flink container. The value should be in " +

Review comment:
       ```suggestion
   			.withDescription("The user-specified secrets to set env variables in Flink container. The value should be in " +
   ```

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/MountSecretsDecorator.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.kubeclient.decorators;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.parameters.AbstractKubernetesParameters;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.ContainerBuilder;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.PodBuilder;
+import io.fabric8.kubernetes.api.model.Volume;
+import io.fabric8.kubernetes.api.model.VolumeBuilder;
+import io.fabric8.kubernetes.api.model.VolumeMount;
+import io.fabric8.kubernetes.api.model.VolumeMountBuilder;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * support mounting Secrets on the JobManager or TaskManager pod..
+ */
+public class MountSecretsDecorator extends AbstractKubernetesStepDecorator {
+
+	private final AbstractKubernetesParameters kubernetesComponentConf;
+
+	public MountSecretsDecorator(AbstractKubernetesParameters kubernetesComponentConf) {
+		this.kubernetesComponentConf = checkNotNull(kubernetesComponentConf);
+	}
+
+	@Override
+	public FlinkPod decorateFlinkPod(FlinkPod flinkPod) {
+		final Pod podWithMount = decoratePod(flinkPod.getPod());
+		final Container containerWithMount = decorateMainContainer(flinkPod.getMainContainer());
+
+		return new FlinkPod.Builder(flinkPod)
+			.withPod(podWithMount)
+			.withMainContainer(containerWithMount)
+			.build();
+	}
+
+	private Container decorateMainContainer(Container container) {
+		VolumeMount[] volumeMounts = kubernetesComponentConf

Review comment:
       ```suggestion
   		final VolumeMount[] volumeMounts = kubernetesComponentConf
   ```

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesEnvVar.java
##########
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.kubeclient.resources;
+
+import io.fabric8.kubernetes.api.model.EnvVar;
+import io.fabric8.kubernetes.api.model.EnvVarBuilder;
+
+import java.util.Map;
+
+/**
+ * Represent EnvVar resource in kubernetes.
+ */
+public class KubernetesEnvVar extends KubernetesResource<EnvVar> {
+
+	private static final String ENV = "env";
+	private static final String SECRET = "secret";
+	private static final String KEY = "key";
+
+	private KubernetesEnvVar(EnvVar envVar) {
+		super(envVar);
+	}
+
+	public static KubernetesEnvVar fromMap(Map<String, String> stringMap) {

Review comment:
       Maybe this class should be `KubernetesSecretEnvVar` since we could only support secret key reference.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/EnvSecretsDecorator.java
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.kubeclient.decorators;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.parameters.AbstractKubernetesParameters;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesEnvVar;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.ContainerBuilder;
+import io.fabric8.kubernetes.api.model.EnvVar;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * support setting environment variables via Secrets.
+ */
+public class EnvSecretsDecorator extends AbstractKubernetesStepDecorator {
+
+	private final AbstractKubernetesParameters kubernetesComponentConf;
+
+	public EnvSecretsDecorator(AbstractKubernetesParameters kubernetesComponentConf) {
+		this.kubernetesComponentConf = checkNotNull(kubernetesComponentConf);
+	}
+
+	@Override
+	public FlinkPod decorateFlinkPod(FlinkPod flinkPod) {
+		final Container basicMainContainer = decorateMainContainer(flinkPod.getMainContainer());
+		return new FlinkPod.Builder(flinkPod)
+			.withPod(flinkPod.getPod())
+			.withMainContainer(basicMainContainer)
+			.build();
+	}
+
+	private Container decorateMainContainer(Container container) {

Review comment:
       ```suggestion
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zhoulii commented on a change in pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
zhoulii commented on a change in pull request #12899:
URL: https://github.com/apache/flink/pull/12899#discussion_r469665467



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
##########
@@ -221,6 +221,22 @@
 	/** Defines the configuration key of that external resource in Kubernetes. This is used as a suffix in an actual config. */
 	public static final String EXTERNAL_RESOURCE_KUBERNETES_CONFIG_KEY_SUFFIX = "kubernetes.config-key";
 
+	public static final ConfigOption<Map<String, String>> KUBERNETES_SECRETS =
+		key("kubernetes.secrets")
+			.mapType()
+			.noDefaultValue()
+			.withDescription("The user-specified secrets that are mounted into Flink container. The value should be in " +

Review comment:
       thanks, you are right, I will change the description.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] tillrohrmann closed pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
tillrohrmann closed pull request #12899:
URL: https://github.com/apache/flink/pull/12899


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474",
       "triggerID" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479",
       "triggerID" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1393e3d777e722703e850187861d33841b2f888c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5486",
       "triggerID" : "1393e3d777e722703e850187861d33841b2f888c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5637",
       "triggerID" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "triggerType" : "PUSH"
     }, {
       "hash" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5646",
       "triggerID" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0282e24a53a29e90a68216b79fbd749cde3dd808",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5800",
       "triggerID" : "0282e24a53a29e90a68216b79fbd749cde3dd808",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ca2f1b495438e216320525f6b74db203f08279d9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5803",
       "triggerID" : "ca2f1b495438e216320525f6b74db203f08279d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31ac34fb5f8aa55f31464f703e29815e09ade9de",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "31ac34fb5f8aa55f31464f703e29815e09ade9de",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7c7e89e76154ab9ae40c4f79a5e83ad644cab60",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7c7e89e76154ab9ae40c4f79a5e83ad644cab60",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4769b098d0e6667767a4dcafedbae81f66b182ca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4769b098d0e6667767a4dcafedbae81f66b182ca",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0282e24a53a29e90a68216b79fbd749cde3dd808 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5800) 
   * ca2f1b495438e216320525f6b74db203f08279d9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5803) 
   * 31ac34fb5f8aa55f31464f703e29815e09ade9de UNKNOWN
   * e7c7e89e76154ab9ae40c4f79a5e83ad644cab60 UNKNOWN
   * 4769b098d0e6667767a4dcafedbae81f66b182ca UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #12899:
URL: https://github.com/apache/flink/pull/12899#discussion_r476217310



##########
File path: docs/ops/deployment/native_kubernetes.md
##########
@@ -211,6 +211,77 @@ $ ./bin/flink run-application -p 8 -t kubernetes-application \
   local:///opt/flink/usrlib/my-flink-job.jar
 {% endhighlight %}
 
+## Using Secrets
+
+[Kubernetes Secrets](https://kubernetes.io/docs/concepts/configuration/secret/) is an object that contains a small amount of sensitive data such as a password, a token, or a key.
+Such information might otherwise be put in a Pod specification or in an image. Flink on Kubernetes can use Secrets in two ways:
+
+- Using Secrets as files from a pod;
+
+- Using Secrets as environment variables;
+
+### Using Secrets as files from a pod
+
+Here is an example of a Pod that mounts a Secret in a volume:
+
+{% highlight yaml %}
+apiVersion: v1
+kind: Pod
+metadata:
+  name: foo
+spec:
+  containers:
+  - name: foo
+    image: foo
+    volumeMounts:
+    - name: foo
+      mountPath: "/opt/foo"
+  volumes:
+  - name: foo
+    secret:
+      secretName: foo
+{% endhighlight %}
+
+By applying this yaml, each key in foo Secrets becomes the filename under `/opt/foo` path. Flink on Kubernetes can enable this feature by the following command:
+
+{% highlight bash %}
+$ ./bin/kubernetes-session.sh \
+  -Dkubernetes.cluster-id=<ClusterId> \
+  -Dkubernetes.container.image=<CustomImageName> \
+  -Dkubernetes.secrets=foo:/opt/foo
+{% endhighlight %}
+

Review comment:
       A link for more details to https://kubernetes.io/docs/concepts/configuration/secret/#using-secrets-as-files-from-a-pod could be helpful.

##########
File path: docs/ops/deployment/native_kubernetes.md
##########
@@ -211,6 +211,77 @@ $ ./bin/flink run-application -p 8 -t kubernetes-application \
   local:///opt/flink/usrlib/my-flink-job.jar
 {% endhighlight %}
 
+## Using Secrets
+
+[Kubernetes Secrets](https://kubernetes.io/docs/concepts/configuration/secret/) is an object that contains a small amount of sensitive data such as a password, a token, or a key.
+Such information might otherwise be put in a Pod specification or in an image. Flink on Kubernetes can use Secrets in two ways:
+
+- Using Secrets as files from a pod;
+
+- Using Secrets as environment variables;
+
+### Using Secrets as files from a pod
+
+Here is an example of a Pod that mounts a Secret in a volume:
+
+{% highlight yaml %}
+apiVersion: v1
+kind: Pod
+metadata:
+  name: foo
+spec:
+  containers:
+  - name: foo
+    image: foo
+    volumeMounts:
+    - name: foo
+      mountPath: "/opt/foo"
+  volumes:
+  - name: foo
+    secret:
+      secretName: foo
+{% endhighlight %}
+
+By applying this yaml, each key in foo Secrets becomes the filename under `/opt/foo` path. Flink on Kubernetes can enable this feature by the following command:
+
+{% highlight bash %}
+$ ./bin/kubernetes-session.sh \
+  -Dkubernetes.cluster-id=<ClusterId> \
+  -Dkubernetes.container.image=<CustomImageName> \
+  -Dkubernetes.secrets=foo:/opt/foo
+{% endhighlight %}
+
+### Using Secrets as environment variables
+
+Here is an example of a Pod that uses secrets from environment variables:
+
+{% highlight yaml %}
+apiVersion: v1
+kind: Pod
+metadata:
+  name: foo
+spec:
+  containers:
+  - name: foo
+    image: foo
+    env:
+      - name: FOO_ENV
+        valueFrom:
+          secretKeyRef:
+            name: foo_secret
+            key: foo_key
+{% endhighlight %}
+
+By applying this yaml, an environment variable named `FOO_ENV` is added into `foo` container, and `FOO_ENV` consumes the value of `foo_key` which is defined in Secrets `foo_secret`.
+Flink on Kubernetes can enable this feature by the following command:
+
+{% highlight bash %}
+$ ./bin/kubernetes-session.sh \
+  -Dkubernetes.cluster-id=<ClusterId> \
+  -Dkubernetes.container.image=<CustomImageName> \
+  -Dkubernetes.env.secretKeyRef=env:FOO_ENV,secret:foo_secret,key:foo_key
+{% endhighlight %}
+

Review comment:
       Same for the env configuration variant: https://kubernetes.io/docs/concepts/configuration/secret/#using-secrets-as-environment-variables




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zhoulii commented on a change in pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
zhoulii commented on a change in pull request #12899:
URL: https://github.com/apache/flink/pull/12899#discussion_r476053086



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java
##########
@@ -110,4 +110,14 @@
 	 * The local directory to locate the custom Hadoop configuration.
 	 */
 	Optional<String> getLocalHadoopConfigurationDirectory();
+
+	/**
+	 * A collection of Secrets and path that are mounted to the JobManager and TaskManager Container(s).
+	 */
+	Map<String, String> getSecretNamesToMountPaths();
+
+	/**
+	 * A collection of customized environments that are attached to the JobManager and TaskManager Container(s).

Review comment:
       done in https://github.com/apache/flink/pull/12899/files#diff-2f05594496213e9ac669e519d48c462dR120




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474",
       "triggerID" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479",
       "triggerID" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1393e3d777e722703e850187861d33841b2f888c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5486",
       "triggerID" : "1393e3d777e722703e850187861d33841b2f888c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5637",
       "triggerID" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "triggerType" : "PUSH"
     }, {
       "hash" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5646",
       "triggerID" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0282e24a53a29e90a68216b79fbd749cde3dd808",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5800",
       "triggerID" : "0282e24a53a29e90a68216b79fbd749cde3dd808",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ca2f1b495438e216320525f6b74db203f08279d9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5803",
       "triggerID" : "ca2f1b495438e216320525f6b74db203f08279d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31ac34fb5f8aa55f31464f703e29815e09ade9de",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "31ac34fb5f8aa55f31464f703e29815e09ade9de",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7c7e89e76154ab9ae40c4f79a5e83ad644cab60",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7c7e89e76154ab9ae40c4f79a5e83ad644cab60",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4769b098d0e6667767a4dcafedbae81f66b182ca",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5813",
       "triggerID" : "4769b098d0e6667767a4dcafedbae81f66b182ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2ad037b9f4ff761d21e063c8dd7f92982437fb6c",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5809",
       "triggerID" : "2ad037b9f4ff761d21e063c8dd7f92982437fb6c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 31ac34fb5f8aa55f31464f703e29815e09ade9de UNKNOWN
   * e7c7e89e76154ab9ae40c4f79a5e83ad644cab60 UNKNOWN
   * 4769b098d0e6667767a4dcafedbae81f66b182ca Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5813) 
   * 2ad037b9f4ff761d21e063c8dd7f92982437fb6c Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5809) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474",
       "triggerID" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479",
       "triggerID" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1393e3d777e722703e850187861d33841b2f888c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5486",
       "triggerID" : "1393e3d777e722703e850187861d33841b2f888c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5637",
       "triggerID" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "triggerType" : "PUSH"
     }, {
       "hash" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5646",
       "triggerID" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 99f04f52689e520f48ebc1cfb263926f347a22c4 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5646) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-675294429


   cc @tillrohrmann @rmetzger, this PR looks really good to me. And i have verified it in a real K8s cluster. Could you have a final check and help with merging?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474",
       "triggerID" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479",
       "triggerID" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2f0285b88d653612277714e12e96d48e2f82fc51 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522) 
   * 46841d653da7282e6340ff9a25d6c2bc39abd00f Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474) 
   * c6edb8841a675096cab4c0e866c3b95c1f3801bf Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zhoulii commented on a change in pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
zhoulii commented on a change in pull request #12899:
URL: https://github.com/apache/flink/pull/12899#discussion_r476053520



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesSecretEnvVar.java
##########
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.kubeclient.resources;
+
+import io.fabric8.kubernetes.api.model.EnvVar;
+import io.fabric8.kubernetes.api.model.EnvVarBuilder;
+
+import java.util.Map;
+
+/**
+ * Represent EnvVar resource in kubernetes.

Review comment:
       done in https://github.com/apache/flink/pull/12899/files#diff-cdc2fed9ec3c1184d4011356be0bdc65R27




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474",
       "triggerID" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2f0285b88d653612277714e12e96d48e2f82fc51 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522) 
   * 46841d653da7282e6340ff9a25d6c2bc39abd00f Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2f0285b88d653612277714e12e96d48e2f82fc51 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zhoulii commented on a change in pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
zhoulii commented on a change in pull request #12899:
URL: https://github.com/apache/flink/pull/12899#discussion_r471872580



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/EnvSecretsDecorator.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.kubeclient.decorators;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.parameters.AbstractKubernetesParameters;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesSecretEnvVar;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.ContainerBuilder;
+import io.fabric8.kubernetes.api.model.EnvVar;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * support setting environment variables via Secrets.
+ */
+public class EnvSecretsDecorator extends AbstractKubernetesStepDecorator {
+
+	private final AbstractKubernetesParameters kubernetesComponentConf;
+
+	public EnvSecretsDecorator(AbstractKubernetesParameters kubernetesComponentConf) {
+		this.kubernetesComponentConf = checkNotNull(kubernetesComponentConf);
+	}
+
+	@Override
+	public FlinkPod decorateFlinkPod(FlinkPod flinkPod) {
+		final Container basicMainContainer = new ContainerBuilder(flinkPod.getMainContainer())
+			.addAllToEnv(getSecretEnvs())
+			.build();
+
+		return new FlinkPod.Builder(flinkPod)
+			.withPod(flinkPod.getPod())

Review comment:
       done in https://github.com/apache/flink/pull/12899/files#diff-5937442e6c2456441334306f869a6121R52




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zhoulii commented on a change in pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
zhoulii commented on a change in pull request #12899:
URL: https://github.com/apache/flink/pull/12899#discussion_r470372369



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
##########
@@ -221,6 +221,22 @@
 	/** Defines the configuration key of that external resource in Kubernetes. This is used as a suffix in an actual config. */
 	public static final String EXTERNAL_RESOURCE_KUBERNETES_CONFIG_KEY_SUFFIX = "kubernetes.config-key";
 
+	public static final ConfigOption<Map<String, String>> KUBERNETES_SECRETS =
+		key("kubernetes.secrets")
+			.mapType()
+			.noDefaultValue()
+			.withDescription("The user-specified secrets that are mounted into Flink container. The value should be in " +
+				"the form of a1:v1,a2:v2. For example for mounting secrets foo to path /opt/secrets , just need to set " +
+				"kubernetes.secrets: foo:/opt/secrets in the flink-conf.yaml");
+
+	public static final ConfigOption<List<Map<String, String>>> KUBERNETES_ENV_SECRET_KEY_REF =
+		key("kubernetes.env.secretKeyRef")
+			.mapType()
+			.asList()
+			.noDefaultValue()
+			.withDescription("the user-specified secrets to set env variable in Flink container. The value should be in " +
+				"the form of env:MY_ENV1,secret:my_secret1,key:my_key1;env:MY_ENV2,secret:my_secret2,key:my_key2");

Review comment:
       fixed




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zhoulii commented on a change in pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
zhoulii commented on a change in pull request #12899:
URL: https://github.com/apache/flink/pull/12899#discussion_r476052791



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java
##########
@@ -110,4 +110,14 @@
 	 * The local directory to locate the custom Hadoop configuration.
 	 */
 	Optional<String> getLocalHadoopConfigurationDirectory();
+
+	/**
+	 * A collection of Secrets and path that are mounted to the JobManager and TaskManager Container(s).

Review comment:
       done in https://github.com/apache/flink/pull/12899/files#diff-2f05594496213e9ac669e519d48c462dR115




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zhoulii commented on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
zhoulii commented on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-673845701


   Hi @wangyang0918 , I have modified the code and commit message on your advice, and thanks for your time to review this pr. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474",
       "triggerID" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479",
       "triggerID" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1393e3d777e722703e850187861d33841b2f888c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5486",
       "triggerID" : "1393e3d777e722703e850187861d33841b2f888c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5637",
       "triggerID" : "ffa55544892c9d4eadf42ca2fcead4ba53701510",
       "triggerType" : "PUSH"
     }, {
       "hash" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5646",
       "triggerID" : "99f04f52689e520f48ebc1cfb263926f347a22c4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0282e24a53a29e90a68216b79fbd749cde3dd808",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5800",
       "triggerID" : "0282e24a53a29e90a68216b79fbd749cde3dd808",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ca2f1b495438e216320525f6b74db203f08279d9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5803",
       "triggerID" : "ca2f1b495438e216320525f6b74db203f08279d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31ac34fb5f8aa55f31464f703e29815e09ade9de",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "31ac34fb5f8aa55f31464f703e29815e09ade9de",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 99f04f52689e520f48ebc1cfb263926f347a22c4 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5646) 
   * 0282e24a53a29e90a68216b79fbd749cde3dd808 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5800) 
   * ca2f1b495438e216320525f6b74db203f08279d9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5803) 
   * 31ac34fb5f8aa55f31464f703e29815e09ade9de UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12899:
URL: https://github.com/apache/flink/pull/12899#issuecomment-658162074


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4506",
       "triggerID" : "77f95464c6036283d77d6e81e2bd21ab27f29d54",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4522",
       "triggerID" : "2f0285b88d653612277714e12e96d48e2f82fc51",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5474",
       "triggerID" : "46841d653da7282e6340ff9a25d6c2bc39abd00f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5479",
       "triggerID" : "c6edb8841a675096cab4c0e866c3b95c1f3801bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1393e3d777e722703e850187861d33841b2f888c",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5486",
       "triggerID" : "1393e3d777e722703e850187861d33841b2f888c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1393e3d777e722703e850187861d33841b2f888c Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5486) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zhoulii commented on a change in pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
zhoulii commented on a change in pull request #12899:
URL: https://github.com/apache/flink/pull/12899#discussion_r471872900



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/MountSecretsDecorator.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.kubeclient.decorators;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.parameters.AbstractKubernetesParameters;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.ContainerBuilder;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.PodBuilder;
+import io.fabric8.kubernetes.api.model.Volume;
+import io.fabric8.kubernetes.api.model.VolumeBuilder;
+import io.fabric8.kubernetes.api.model.VolumeMount;
+import io.fabric8.kubernetes.api.model.VolumeMountBuilder;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * support mounting Secrets on the JobManager or TaskManager pod..
+ */
+public class MountSecretsDecorator extends AbstractKubernetesStepDecorator {
+
+	private final AbstractKubernetesParameters kubernetesComponentConf;
+
+	public MountSecretsDecorator(AbstractKubernetesParameters kubernetesComponentConf) {
+		this.kubernetesComponentConf = checkNotNull(kubernetesComponentConf);
+	}
+
+	@Override
+	public FlinkPod decorateFlinkPod(FlinkPod flinkPod) {
+		final Pod podWithMount = decoratePod(flinkPod.getPod());
+		final Container containerWithMount = decorateMainContainer(flinkPod.getMainContainer());
+
+		return new FlinkPod.Builder(flinkPod)
+			.withPod(podWithMount)
+			.withMainContainer(containerWithMount)
+			.build();
+	}
+
+	private Container decorateMainContainer(Container container) {
+		final VolumeMount[] volumeMounts = kubernetesComponentConf
+			.getSecretNamesToMountPaths()
+			.entrySet()
+			.stream()
+			.map(secretNameToPath ->
+				new VolumeMountBuilder()
+					.withName(secretVolumeName(secretNameToPath.getKey()))
+					.withMountPath(secretNameToPath.getValue())
+					.build()
+			).toArray(VolumeMount[]::new);
+
+		return new ContainerBuilder(container)
+			.addToVolumeMounts(volumeMounts)
+			.build();
+	}
+
+	private Pod decoratePod(Pod pod) {
+		final Volume[] volumes = kubernetesComponentConf.getSecretNamesToMountPaths()

Review comment:
       done in https://github.com/apache/flink/pull/12899/files#diff-0e5823ba44ea4819a4c33767020ee54bR72




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zhoulii commented on a change in pull request #12899: [FLINK-16699][k8s] Support accessing secured services via K8s secrets

Posted by GitBox <gi...@apache.org>.
zhoulii commented on a change in pull request #12899:
URL: https://github.com/apache/flink/pull/12899#discussion_r471873117



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/VolumeTestUtils.java
##########
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.Pod;
+
+/**
+ * Utilities for the Kubernetes tests.
+ */
+public class VolumeTestUtils {
+
+	public static boolean podHasVolume(Pod pod, String volumeName){
+		return pod.getSpec()

Review comment:
       done in https://github.com/apache/flink/pull/12899/files#diff-9a0db1fd66568c4dee2b5298dbea2ebfR30




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org