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/02/27 08:05:21 UTC

[GitHub] [flink] zhengcanbin opened a new pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

zhengcanbin opened a new pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233
 
 
   ## What is the purpose of the change
   
   So far, Flink has made efforts for the native integration of Kubernetes. However, it is always essential to evaluate the existing design and consider alternatives that have better design and are easier to maintain in the long run. We have suffered from some problems while developing new features base on the current code. Here is some of them:
   
   1. We don’t have a unified monadic-step based orchestrator architecture to construct all the Kubernetes resources.
   2. We don’t have dedicated objects or tools for centrally parsing, verifying, and managing the Kubernetes parameters, which has raised some maintenance and inconsistency issues. 
   
   The ultimate goal of this PR is to evolve some of the designs. Here is a summary of the main evolution.
   
   1. Introduce a unified monadic-step based orchestrator architecture that has a better, cleaner and consistent abstraction for the Kubernetes resources construction process.
   2. Introduce some dedicated tools for centrally parsing, verifying, and managing the Kubernetes parameters.
   
   ## Open Questions
   1. For the new design, we change the owner from the internal Service to the Deployment for GC. There are several concerns:
   - We do not need the internal Service to forward request from TaskManger to JobManager in the HA mode, that Service would be removed in such a scenario in another issue.
   - Things like Deployment are the first citizen in Kubernetes, it is reasonable that one deletes the controller that runs the master leads to clean-up of all the other resources together representing that Application.
   
   2. For the new design, we don't listen to **ADD** event when creating the rest Service. The previous design assumes that the Service is ready once the client receives the **ADD** event. However, this is incorrect, no matter for the LB or the NodePort type. We plan to open another issue to fix this problem.
   
   ## Brief change log
   
   Main changes are:
   
   - [e629bbc](https://github.com/apache/flink/commit/e629bbc4091e9288f74e2d6a9cfd689daabeb4a3) Trivial code clean-up and test code normalization.
   - [675151e](https://github.com/apache/flink/commit/675151e02d7b91e0736963ed2b24f2b8c3ff7046): Remove the existing decorator design patterns.
   - [0355f0a](https://github.com/apache/flink/commit/0355f0a6d95bca530b4018fecf11db7560626956): Refactor and simplify KubernetesTestBase.
   - [edc3d23](https://github.com/apache/flink/commit/edc3d23742d64dcbd07b24b72b674c17ce06b6e7): Remove the Flink Configuration out of KubernetesResource.
   - [8d6e520](https://github.com/apache/flink/commit/8d6e5201b336a6238923292f96b9f0563a4f9029): Introduce some dedicated Kubernetes parameters parsing tools.
   - [c41a9a2](https://github.com/apache/flink/commit/c41a9a2b5a5f23b820cae038a0611d2d071c4ce9) to [23ed312](https://github.com/apache/flink/commit/23ed31201c0cd08d6bdb715721bba857ead2b520): Introduce the new Kubernetes decorator design pattern.
   - [710984c](https://github.com/apache/flink/commit/710984c24f05169a2c1e644676e05dbc573e6c3a): Rework the FlinkKubeClient to employ the new decorator pattern.
   - [a47f12d](https://github.com/apache/flink/commit/a47f12d54f832485ad54f273bc5a2f4901d4dce7) to [fb57917](https://github.com/apache/flink/commit/fb57917227853d0477aa1383d399a619146d7170): Minor improvements
   
   
   ## Verifying this change
   
   This PR adds several test classes and many unit tests to catch most of the test branch for the newly decorator design pattern.
   
   ## 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)
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r385615804
 
 

 ##########
 File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesClusterDescriptorTest.java
 ##########
 @@ -47,19 +55,29 @@
  * Tests for the {@link KubernetesClusterDescriptor}.
  */
 public class KubernetesClusterDescriptorTest extends KubernetesTestBase {
-
+	private static final String MOCK_SERVICE_HOST_NAME = "mock-host-name-of-service";
 	private static final String MOCK_SERVICE_IP = "192.168.0.1";
 
 	private final ClusterSpecification clusterSpecification = new ClusterSpecification.ClusterSpecificationBuilder()
 		.createClusterSpecification();
 
+	@Before
+	public void setup() throws Exception {
+		super.setup();
+
+		mockRestServiceWithLoadBalancer(MOCK_SERVICE_HOST_NAME, MOCK_SERVICE_IP);
+	}
+
 	@Test
 	public void testDeploySessionCluster() throws Exception {
+		mockRestServiceWithLoadBalancer(MOCK_SERVICE_HOST_NAME, MOCK_SERVICE_IP);
 
 Review comment:
   `mockRestServiceWithLoadBalancer` could be removed. It has been called in the `setup`. Other tests is in the same situation.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r386258439
 
 

 ##########
 File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactoryTest.java
 ##########
 @@ -0,0 +1,203 @@
+/*
+ * 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.factory;
+
+import org.apache.flink.kubernetes.KubernetesTestUtils;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptionsInternal;
+import org.apache.flink.kubernetes.entrypoint.KubernetesSessionClusterEntrypoint;
+import org.apache.flink.kubernetes.kubeclient.KubernetesJobManagerSpecification;
+import org.apache.flink.kubernetes.kubeclient.KubernetesJobManagerTestBase;
+import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters;
+import org.apache.flink.kubernetes.utils.Constants;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+
+import io.fabric8.kubernetes.api.model.ConfigMap;
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.HasMetadata;
+import io.fabric8.kubernetes.api.model.PodSpec;
+import io.fabric8.kubernetes.api.model.Quantity;
+import io.fabric8.kubernetes.api.model.Service;
+import io.fabric8.kubernetes.api.model.apps.Deployment;
+import io.fabric8.kubernetes.api.model.apps.DeploymentSpec;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.configuration.GlobalConfiguration.FLINK_CONF_FILENAME;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * General tests for the {@link KubernetesJobManagerFactory}.
+ */
+public class KubernetesJobManagerFactoryTest extends KubernetesJobManagerTestBase {
+
+	private static final String SERVICE_ACCOUNT_NAME = "service-test";
+	private static final String ENTRY_POINT_CLASS = KubernetesSessionClusterEntrypoint.class.getCanonicalName();
+
+	private KubernetesJobManagerSpecification kubernetesJobManagerSpecification;
+
+	@Before
+	public void setup() throws Exception {
+		super.setup();
+
+		KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "logback.xml");
+		KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "log4j.properties");
+
+		flinkConfig.set(KubernetesConfigOptionsInternal.ENTRY_POINT_CLASS, ENTRY_POINT_CLASS);
+		flinkConfig.set(KubernetesConfigOptions.JOB_MANAGER_SERVICE_ACCOUNT, SERVICE_ACCOUNT_NAME);
+
+		this.kubernetesJobManagerSpecification =
+			KubernetesJobManagerFactory.createJobManagerComponent(kubernetesJobManagerParameters);
+	}
+
+	@Test
+	public void testDeploymentMetadata() {
+		final Deployment resultDeployment = this.kubernetesJobManagerSpecification.getDeployment();
+		assertEquals(Constants.APPS_API_VERSION, resultDeployment.getApiVersion());
+		assertEquals(KubernetesUtils.getDeploymentName(CLUSTER_ID), resultDeployment.getMetadata().getName());
+		final Map<String, String> expectedLabels = getCommonLabels();
+		expectedLabels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_JOB_MANAGER);
+		assertEquals(expectedLabels, resultDeployment.getMetadata().getLabels());
+	}
+
+	@Test
+	public void testDeploymentSpec() {
+		final DeploymentSpec resultDeploymentSpec = this.kubernetesJobManagerSpecification.getDeployment().getSpec();
+		assertEquals(1, resultDeploymentSpec.getReplicas().intValue());
+
+		final Map<String, String> expectedLabels =  new HashMap<>(getCommonLabels());
+		expectedLabels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_JOB_MANAGER);
+
+		assertEquals(expectedLabels, resultDeploymentSpec.getTemplate().getMetadata().getLabels());
+		assertEquals(expectedLabels, resultDeploymentSpec.getSelector().getMatchLabels());
+
+		assertNotNull(resultDeploymentSpec.getTemplate().getSpec());
+	}
+
+	@Test
+	public void testPodSpec() {
+		final PodSpec resultPodSpec =
+			this.kubernetesJobManagerSpecification.getDeployment().getSpec().getTemplate().getSpec();
+
+		assertEquals(1, resultPodSpec.getContainers().size());
+		assertEquals(SERVICE_ACCOUNT_NAME, resultPodSpec.getServiceAccountName());
+		assertEquals(1, resultPodSpec.getVolumes().size());
+
+		final Container resultedMainContainer = resultPodSpec.getContainers().get(0);
+		assertEquals(KubernetesJobManagerParameters.JOB_MANAGER_MAIN_CONTAINER_NAME, resultedMainContainer.getName());
+		assertEquals(CONTAINER_IMAGE, resultedMainContainer.getImage());
+		assertEquals(CONTAINER_IMAGE_PULL_POLICY, resultedMainContainer.getImagePullPolicy());
+
+		assertEquals(3, resultedMainContainer.getEnv().size());
+		assertTrue(resultedMainContainer.getEnv()
+				.stream()
+				.anyMatch(envVar -> envVar.getName().equals("key1")));
+
+		assertEquals(3, resultedMainContainer.getPorts().size());
+
+		final Map<String, Quantity> requests = resultedMainContainer.getResources().getRequests();
+		assertEquals(Double.toString(JOB_MANAGER_CPU), requests.get("cpu").getAmount());
+		assertEquals(JOB_MANAGER_MEMORY + "Mi", requests.get("memory").getAmount());
+
+		assertEquals(1, resultedMainContainer.getCommand().size());
+		assertEquals(3, resultedMainContainer.getArgs().size());
+
+		assertEquals(1, resultedMainContainer.getVolumeMounts().size());
+	}
+
+	@Test
+	public void testAdditionalResourcesSize() {
+		final List<HasMetadata> resultAdditionalResources = this.kubernetesJobManagerSpecification.getAccompanyingResources();
+		assertEquals(3, resultAdditionalResources.size());
+
+		final List<HasMetadata> resultServices = resultAdditionalResources
+			.stream()
+			.filter(x -> x instanceof Service)
+			.collect(Collectors.toList());
+		assertEquals(2, resultServices.size());
+
+		final List<HasMetadata> resultConfigMaps = resultAdditionalResources
+			.stream()
+			.filter(x -> x instanceof ConfigMap)
+			.collect(Collectors.toList());
+		assertEquals(1, resultConfigMaps.size());
+	}
+
+	@Test
+	public void testServices() {
+		final List<Service> resultServices = this.kubernetesJobManagerSpecification.getAccompanyingResources()
+			.stream()
+			.filter(x -> x instanceof Service)
+			.map(x -> (Service) x)
+			.collect(Collectors.toList());
+
+		assertEquals(2, resultServices.size());
+
+		final List<Service> internalServiceCandidates = resultServices
+				.stream()
+				.filter(x -> x.getMetadata().getName().equals(KubernetesUtils.getInternalServiceName(CLUSTER_ID)))
+				.collect(Collectors.toList());
+		assertEquals(1, internalServiceCandidates.size());
+
+		final List<Service> restServiceCandidates = resultServices
+				.stream()
+				.filter(x -> x.getMetadata().getName().equals(KubernetesUtils.getRestServiceName(CLUSTER_ID)))
+				.collect(Collectors.toList());
+		assertEquals(1, restServiceCandidates.size());
+
+		final Service resultInternalService = internalServiceCandidates.get(0);
+		assertEquals(2, resultInternalService.getMetadata().getLabels().size());
 
 Review comment:
   Since we have checked the real labels in `InternalServiceDecoratorTest` and `ExternalServiceDecoratorTest`, I don't think it's necessary to check them again in the final test of `KubernetesJobManagerFactoryTest`, rough assert such as size verifying could be enough, otherwise, there is repeated test work. 

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 19e32e6bca1a90f608010fdff1236fdfdbb53e3b Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/151155366) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765) 
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be UNKNOWN
   * abfef134eb5940d7831dab4c794d654c2fa70263 UNKNOWN
   * d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4 UNKNOWN
   * 78750a20196de5f01b01b59b1cd15c752bdee5c0 UNKNOWN
   * 08f07deb2243554d24e2e4171e6cb23a5f934cc8 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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fb57917227853d0477aa1383d399a619146d7170 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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r385996299
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactory.java
 ##########
 @@ -0,0 +1,109 @@
+/*
+ * 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.factory;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.KubernetesJobManagerSpecification;
+import org.apache.flink.kubernetes.kubeclient.decorators.ExternalServiceDecorator;
+import org.apache.flink.kubernetes.kubeclient.decorators.FlinkConfMountDecorator;
+import org.apache.flink.kubernetes.kubeclient.decorators.InitJobManagerDecorator;
+import org.apache.flink.kubernetes.kubeclient.decorators.InternalServiceDecorator;
+import org.apache.flink.kubernetes.kubeclient.decorators.JavaCmdJobManagerDecorator;
+import org.apache.flink.kubernetes.kubeclient.decorators.KubernetesStepDecorator;
+import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters;
+import org.apache.flink.kubernetes.utils.Constants;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.HasMetadata;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.PodBuilder;
+import io.fabric8.kubernetes.api.model.apps.Deployment;
+import io.fabric8.kubernetes.api.model.apps.DeploymentBuilder;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Utility class for constructing all the Kubernetes components on the client-side. This can
+ * include the Deployment, the ConfigMap(s), and the Service(s).
+ */
+public class KubernetesJobManagerFactory {
+
+	public static KubernetesJobManagerSpecification createJobManagerComponent(
+			KubernetesJobManagerParameters kubernetesJobManagerParameters) throws IOException {
+		FlinkPod flinkPod = new FlinkPod.Builder().build();
+		List<HasMetadata> accompanyingResources = new ArrayList<>();
+
+		final KubernetesStepDecorator[] stepDecorators = new KubernetesStepDecorator[] {
+			new InitJobManagerDecorator(kubernetesJobManagerParameters),
+			new JavaCmdJobManagerDecorator(kubernetesJobManagerParameters),
+			new InternalServiceDecorator(kubernetesJobManagerParameters),
+			new ExternalServiceDecorator(kubernetesJobManagerParameters),
+			new FlinkConfMountDecorator(kubernetesJobManagerParameters)};
+
+		for (KubernetesStepDecorator stepDecorator: stepDecorators) {
+			flinkPod = stepDecorator.decorateFlinkPod(flinkPod);
+			accompanyingResources.addAll(stepDecorator.buildAccompanyingKubernetesResources());
+		}
+
+		final Deployment deployment = createJobManagerDeployment(flinkPod, kubernetesJobManagerParameters);
+
+		KubernetesUtils.setOwnerReference(deployment, accompanyingResources);
+
+		return new KubernetesJobManagerSpecification(deployment, accompanyingResources);
+	}
+
+	private static Deployment createJobManagerDeployment(
+			FlinkPod flinkPod,
+			KubernetesJobManagerParameters kubernetesJobManagerParameters) {
+		final Container resolvedMainContainer = flinkPod.getMainContainer();
+
+		final Pod resolvedPod = new PodBuilder(flinkPod.getPod())
+			.editOrNewSpec()
+				.addToContainers(resolvedMainContainer)
+				.endSpec()
+			.build();
+
+		final Map<String, String> labels = resolvedPod.getMetadata().getLabels();
 
 Review comment:
   The basic logic is:
   1. We use the Pod object built by the decorators to fill the pod template of the deployment.
   2. Then set the deployment.metadata.labels separately to help others realize that the labels of the deployment could be different from the labels of the underlying pod.
   
   For the usual scenarios, the labels could be the same. For this PR,  resolvedPod.getMetadata().getLabels() and kubernetesJobManagerParameters.getLabels() shares the same values. 
   
   So I think it is reasonable to use the resolvedPod.getMetadata().getLabels to fill the spec of the deployment.
   
   For the deployment metadata, maybe we could introduce a wrapper private method in KubernetesJobManagerFactory to get the deployment labels.
   
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151292878",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151373127",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151373127",
       "triggerID" : "593715190",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828",
       "triggerID" : "593715190",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151475534",
       "triggerID" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5846",
       "triggerID" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64137c6034da191208680bbe89948d911eec6e52",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151480044",
       "triggerID" : "64137c6034da191208680bbe89948d911eec6e52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64137c6034da191208680bbe89948d911eec6e52",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5848",
       "triggerID" : "64137c6034da191208680bbe89948d911eec6e52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f539fe185165e1071e3a54a8f781977a4872995",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5916",
       "triggerID" : "3f539fe185165e1071e3a54a8f781977a4872995",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f539fe185165e1071e3a54a8f781977a4872995",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151723903",
       "triggerID" : "3f539fe185165e1071e3a54a8f781977a4872995",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0072b0b681bfb099d135c59152dd1f61bca8b34",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f0072b0b681bfb099d135c59152dd1f61bca8b34",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be UNKNOWN
   * abfef134eb5940d7831dab4c794d654c2fa70263 UNKNOWN
   * d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4 UNKNOWN
   * 78750a20196de5f01b01b59b1cd15c752bdee5c0 UNKNOWN
   * 3f539fe185165e1071e3a54a8f781977a4872995 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/151723903) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5916) 
   * f0072b0b681bfb099d135c59152dd1f61bca8b34 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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591837567
 
 
   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 fb57917227853d0477aa1383d399a619146d7170 (Thu Feb 27 08:08:16 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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 786a5056c957863c05ad24b00ca1dca032905eb0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/150962647) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717) 
   
   <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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151292878",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be UNKNOWN
   * abfef134eb5940d7831dab4c794d654c2fa70263 UNKNOWN
   * d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4 UNKNOWN
   * 78750a20196de5f01b01b59b1cd15c752bdee5c0 UNKNOWN
   * 08f07deb2243554d24e2e4171e6cb23a5f934cc8 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/151292878) Azure: [CANCELED](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791) 
   
   <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


With regards,
Apache Git Services

[GitHub] [flink] wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r386202186
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/JavaCmdTaskManagerDecorator.java
 ##########
 @@ -0,0 +1,109 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesTaskManagerParameters;
+import org.apache.flink.kubernetes.taskmanager.KubernetesTaskExecutorRunner;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils;
+import org.apache.flink.runtime.entrypoint.parser.CommandLineOptions;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.ContainerBuilder;
+
+import javax.annotation.Nullable;
+
+import java.util.Arrays;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Attach the command and args to the main container for running the TaskManager code.
+ */
+public class JavaCmdTaskManagerDecorator extends AbstractKubernetesStepDecorator {
+
+	private final KubernetesTaskManagerParameters kubernetesTaskManagerParameters;
+
+	public JavaCmdTaskManagerDecorator(KubernetesTaskManagerParameters kubernetesTaskManagerParameters) {
+		this.kubernetesTaskManagerParameters = checkNotNull(kubernetesTaskManagerParameters);
+	}
+
+	@Override
+	public FlinkPod decorateFlinkPod(FlinkPod flinkPod) {
+		final Container mainContainerWithStartCmd = new ContainerBuilder(flinkPod.getMainContainer())
+			.withCommand(kubernetesTaskManagerParameters.getContainerEntrypoint())
+			.withArgs(Arrays.asList("/bin/bash", "-c", getTaskManagerStartCommand()))
+			.build();
+
+		return new FlinkPod.Builder(flinkPod)
+			.withMainContainer(mainContainerWithStartCmd)
+			.build();
+	}
+
+	private String getTaskManagerStartCommand() {
+		final String confDirInPod = kubernetesTaskManagerParameters.getFlinkConfDirInPod();
+
+		final String logDirInPod = kubernetesTaskManagerParameters.getFlinkLogDirInPod();
+
+		final String mainClassArgs = "--" + CommandLineOptions.CONFIG_DIR_OPTION.getLongOpt() + " " +
+			confDirInPod + " " + kubernetesTaskManagerParameters.getDynamicProperties();
+
+		return getTaskManagerStartCommand(
+			kubernetesTaskManagerParameters.getFlinkConfiguration(),
+			kubernetesTaskManagerParameters.getContaineredTaskManagerParameters(),
+			confDirInPod,
+			logDirInPod,
+			kubernetesTaskManagerParameters.hasLogback(),
+			kubernetesTaskManagerParameters.hasLog4j(),
+			KubernetesTaskExecutorRunner.class.getCanonicalName(),
+			mainClassArgs);
+	}
+
+	private static String getTaskManagerStartCommand(
+			Configuration flinkConfig,
+			ContaineredTaskManagerParameters tmParams,
+			String configDirectory,
+			String logDirectory,
+			boolean hasLogback,
+			boolean hasLog4j,
+			String mainClass,
+			@Nullable String mainArgs) {
 
 Review comment:
   `@Nullable` could be removed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151292878",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151373127",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151373127",
       "triggerID" : "593715190",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828",
       "triggerID" : "593715190",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151475534",
       "triggerID" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5846",
       "triggerID" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64137c6034da191208680bbe89948d911eec6e52",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151480044",
       "triggerID" : "64137c6034da191208680bbe89948d911eec6e52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64137c6034da191208680bbe89948d911eec6e52",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5848",
       "triggerID" : "64137c6034da191208680bbe89948d911eec6e52",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be UNKNOWN
   * abfef134eb5940d7831dab4c794d654c2fa70263 UNKNOWN
   * d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4 UNKNOWN
   * 78750a20196de5f01b01b59b1cd15c752bdee5c0 UNKNOWN
   * 64137c6034da191208680bbe89948d911eec6e52 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/151480044) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5848) 
   
   <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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r385996299
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactory.java
 ##########
 @@ -0,0 +1,109 @@
+/*
+ * 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.factory;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.KubernetesJobManagerSpecification;
+import org.apache.flink.kubernetes.kubeclient.decorators.ExternalServiceDecorator;
+import org.apache.flink.kubernetes.kubeclient.decorators.FlinkConfMountDecorator;
+import org.apache.flink.kubernetes.kubeclient.decorators.InitJobManagerDecorator;
+import org.apache.flink.kubernetes.kubeclient.decorators.InternalServiceDecorator;
+import org.apache.flink.kubernetes.kubeclient.decorators.JavaCmdJobManagerDecorator;
+import org.apache.flink.kubernetes.kubeclient.decorators.KubernetesStepDecorator;
+import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters;
+import org.apache.flink.kubernetes.utils.Constants;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.HasMetadata;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.PodBuilder;
+import io.fabric8.kubernetes.api.model.apps.Deployment;
+import io.fabric8.kubernetes.api.model.apps.DeploymentBuilder;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Utility class for constructing all the Kubernetes components on the client-side. This can
+ * include the Deployment, the ConfigMap(s), and the Service(s).
+ */
+public class KubernetesJobManagerFactory {
+
+	public static KubernetesJobManagerSpecification createJobManagerComponent(
+			KubernetesJobManagerParameters kubernetesJobManagerParameters) throws IOException {
+		FlinkPod flinkPod = new FlinkPod.Builder().build();
+		List<HasMetadata> accompanyingResources = new ArrayList<>();
+
+		final KubernetesStepDecorator[] stepDecorators = new KubernetesStepDecorator[] {
+			new InitJobManagerDecorator(kubernetesJobManagerParameters),
+			new JavaCmdJobManagerDecorator(kubernetesJobManagerParameters),
+			new InternalServiceDecorator(kubernetesJobManagerParameters),
+			new ExternalServiceDecorator(kubernetesJobManagerParameters),
+			new FlinkConfMountDecorator(kubernetesJobManagerParameters)};
+
+		for (KubernetesStepDecorator stepDecorator: stepDecorators) {
+			flinkPod = stepDecorator.decorateFlinkPod(flinkPod);
+			accompanyingResources.addAll(stepDecorator.buildAccompanyingKubernetesResources());
+		}
+
+		final Deployment deployment = createJobManagerDeployment(flinkPod, kubernetesJobManagerParameters);
+
+		KubernetesUtils.setOwnerReference(deployment, accompanyingResources);
+
+		return new KubernetesJobManagerSpecification(deployment, accompanyingResources);
+	}
+
+	private static Deployment createJobManagerDeployment(
+			FlinkPod flinkPod,
+			KubernetesJobManagerParameters kubernetesJobManagerParameters) {
+		final Container resolvedMainContainer = flinkPod.getMainContainer();
+
+		final Pod resolvedPod = new PodBuilder(flinkPod.getPod())
+			.editOrNewSpec()
+				.addToContainers(resolvedMainContainer)
+				.endSpec()
+			.build();
+
+		final Map<String, String> labels = resolvedPod.getMetadata().getLabels();
 
 Review comment:
   The basic logic is:
   1. We use the Pod object built by the decorators to fill the pod template of the deployment.
   2. Then set the {{deployment.metadata.labels}} separately to help other developers realize that the labels of the deployment could be different from the labels of the pod.
   
   For the usual scenarios, the labels could be the same. For this PR,  resolvedPod.getMetadata().getLabels() and kubernetesJobManagerParameters.getLabels() shares the same values. 
   
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r385616888
 
 

 ##########
 File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesClusterDescriptorTest.java
 ##########
 @@ -147,4 +173,30 @@ public void testKillCluster() throws Exception {
 
 		return clusterClient;
 	}
+
+	private void mockRestServiceWithLoadBalancer(@Nullable String hostname, @Nullable String ip) {
+		final String restServiceName = KubernetesUtils.getRestServiceName(CLUSTER_ID);
+
+		final String path = String.format("/api/v1/namespaces/%s/services/%s", NAMESPACE, restServiceName);
+		server.expect()
+			.withPath(path)
+			.andReturn(200, buildMockRestService(hostname, ip))
+			.always();
+	}
+
+	private Service buildMockRestService(@Nullable String hostname, @Nullable String ip) {
 
 Review comment:
   `@Nullable` could be removed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-593314187
 
 
   > ```
   > echo 'stop' | ./bin/kubernetes-session.sh -Dkubernetes.cluster-id=flink-native-k8s-session-1 -Dexecution.attached=true
   > ```
   > 
   > Probably because we do not include `io.fabric8:zjsonpatch` in the pom.xml of `flink-kubernetes` module.
   > 
   > ```
   > 2020-03-02 14:30:32,386 INFO  org.apache.flink.kubernetes.KubernetesClusterDescriptor      [] - Retrieve flink cluster flink-native-k8s-session-1 successfully, JobManager Web Interface: http://11.164.91.5:31318
   > Exception in thread "main" java.lang.NoClassDefFoundError: io/fabric8/zjsonpatch/JsonDiff
   > 	at io.fabric8.kubernetes.client.dsl.base.OperationSupport.handlePatch(OperationSupport.java:297)
   > 	at io.fabric8.kubernetes.client.dsl.base.BaseOperation.handlePatch(BaseOperation.java:808)
   > 	at io.fabric8.kubernetes.client.dsl.base.HasMetadataOperation.lambda$patch$2(HasMetadataOperation.java:145)
   > 	at io.fabric8.kubernetes.api.model.apps.DoneableDeployment.done(DoneableDeployment.java:27)
   > 	at io.fabric8.kubernetes.api.model.apps.DoneableDeployment.done(DoneableDeployment.java:6)
   > 	at io.fabric8.kubernetes.client.dsl.base.HasMetadataOperation.patch(HasMetadataOperation.java:151)
   > 	at io.fabric8.kubernetes.client.dsl.internal.RollableScalableResourceOperation.patch(RollableScalableResourceOperation.java:167)
   > 	at io.fabric8.kubernetes.client.dsl.internal.DeploymentOperationsImpl.patch(DeploymentOperationsImpl.java:113)
   > 	at io.fabric8.kubernetes.client.dsl.internal.DeploymentOperationsImpl.patch(DeploymentOperationsImpl.java:45)
   > 	at io.fabric8.kubernetes.client.dsl.base.HasMetadataOperation.lambda$edit$0(HasMetadataOperation.java:53)
   > 	at io.fabric8.kubernetes.api.model.apps.DoneableDeployment.done(DoneableDeployment.java:27)
   > 	at io.fabric8.kubernetes.client.dsl.internal.DeploymentOperationsImpl$DeploymentReaper.reap(DeploymentOperationsImpl.java:245)
   > 	at io.fabric8.kubernetes.client.dsl.base.BaseOperation.delete(BaseOperation.java:642)
   > 	at io.fabric8.kubernetes.client.dsl.base.BaseOperation.delete(BaseOperation.java:63)
   > 	at org.apache.flink.kubernetes.kubeclient.Fabric8FlinkKubeClient.stopAndCleanupCluster(Fabric8FlinkKubeClient.java:182)
   > 	at org.apache.flink.kubernetes.KubernetesClusterDescriptor.killCluster(KubernetesClusterDescriptor.java:193)
   > 	at org.apache.flink.kubernetes.KubernetesClusterDescriptor.killCluster(KubernetesClusterDescriptor.java:59)
   > 	at org.apache.flink.kubernetes.cli.KubernetesSessionCli.run(KubernetesSessionCli.java:125)
   > 	at org.apache.flink.kubernetes.cli.KubernetesSessionCli.lambda$main$0(KubernetesSessionCli.java:185)
   > 	at org.apache.flink.runtime.security.contexts.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:30)
   > 	at org.apache.flink.kubernetes.cli.KubernetesSessionCli.main(KubernetesSessionCli.java:185)
   > Caused by: java.lang.ClassNotFoundException: io.fabric8.zjsonpatch.JsonDiff
   > 	at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
   > 	at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
   > 	at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:335)
   > 	at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
   > 	... 21 more
   > ```
   
   Good catch. Fixed by [b088e0e](https://github.com/apache/flink/pull/11233/commits/b088e0e0530fa3178a547b26b7e950165181caaf)

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r386008661
 
 

 ##########
 File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesClusterDescriptorTest.java
 ##########
 @@ -47,19 +55,29 @@
  * Tests for the {@link KubernetesClusterDescriptor}.
  */
 public class KubernetesClusterDescriptorTest extends KubernetesTestBase {
-
+	private static final String MOCK_SERVICE_HOST_NAME = "mock-host-name-of-service";
 	private static final String MOCK_SERVICE_IP = "192.168.0.1";
 
 	private final ClusterSpecification clusterSpecification = new ClusterSpecification.ClusterSpecificationBuilder()
 		.createClusterSpecification();
 
+	@Before
+	public void setup() throws Exception {
+		super.setup();
+
+		mockRestServiceWithLoadBalancer(MOCK_SERVICE_HOST_NAME, MOCK_SERVICE_IP);
+	}
+
 	@Test
 	public void testDeploySessionCluster() throws Exception {
+		mockRestServiceWithLoadBalancer(MOCK_SERVICE_HOST_NAME, MOCK_SERVICE_IP);
 
 Review comment:
   Fixed by [e55c0d2](https://github.com/apache/flink/pull/11233/commits/e55c0d2867d9c79b977bd57997e28ecb2310a616)

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] TisonKun edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
TisonKun edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-593774185
 
 
   ```
   2020-03-03T04:18:38.2171847Z [ERROR] Failed to execute goal org.apache.maven.plugins:maven-surefire-plugin:2.22.1:test (integration-tests) on project flink-state-processor-api_2.11: There are test failures.
   2020-03-03T04:18:38.2172997Z [ERROR] 
   2020-03-03T04:18:38.2173799Z [ERROR] Please refer to /__w/1/s/flink-libraries/flink-state-processing-api/target/surefire-reports for the individual test results.
   2020-03-03T04:18:38.2174597Z [ERROR] Please refer to dump files (if any exist) [date].dump, [date]-jvmRun[N].dump and [date].dumpstream.
   2020-03-03T04:18:38.2175072Z [ERROR] ExecutionException Error occurred in starting fork, check output in log
   2020-03-03T04:18:38.2175671Z [ERROR] org.apache.maven.surefire.booter.SurefireBooterForkException: ExecutionException Error occurred in starting fork, check output in log
   2020-03-03T04:18:38.2176787Z [ERROR] at org.apache.maven.plugin.surefire.booterclient.ForkStarter.awaitResultsDone(ForkStarter.java:510)
   2020-03-03T04:18:38.2177445Z [ERROR] at org.apache.maven.plugin.surefire.booterclient.ForkStarter.runSuitesForkPerTestSet(ForkStarter.java:457)
   2020-03-03T04:18:38.2178053Z [ERROR] at org.apache.maven.plugin.surefire.booterclient.ForkStarter.run(ForkStarter.java:298)
   2020-03-03T04:18:38.2178813Z [ERROR] at org.apache.maven.plugin.surefire.booterclient.ForkStarter.run(ForkStarter.java:246)
   2020-03-03T04:18:38.2179652Z [ERROR] at org.apache.maven.plugin.surefire.AbstractSurefireMojo.executeProvider(AbstractSurefireMojo.java:1183)
   2020-03-03T04:18:38.2180320Z [ERROR] at org.apache.maven.plugin.surefire.AbstractSurefireMojo.executeAfterPreconditionsChecked(AbstractSurefireMojo.java:1011)
   2020-03-03T04:18:38.2180974Z [ERROR] at org.apache.maven.plugin.surefire.AbstractSurefireMojo.execute(AbstractSurefireMojo.java:857)
   2020-03-03T04:18:38.2181725Z [ERROR] at org.apache.maven.plugin.DefaultBuildPluginManager.executeMojo(DefaultBuildPluginManager.java:132)
   2020-03-03T04:18:38.2182381Z [ERROR] at org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:208)
   2020-03-03T04:18:38.2182907Z [ERROR] at org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:153)
   2020-03-03T04:18:38.2183440Z [ERROR] at org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:145)
   2020-03-03T04:18:38.2184009Z [ERROR] at org.apache.maven.lifecycle.internal.LifecycleModuleBuilder.buildProject(LifecycleModuleBuilder.java:116)
   2020-03-03T04:18:38.2184645Z [ERROR] at org.apache.maven.lifecycle.internal.LifecycleModuleBuilder.buildProject(LifecycleModuleBuilder.java:80)
   2020-03-03T04:18:38.2185293Z [ERROR] at org.apache.maven.lifecycle.internal.builder.singlethreaded.SingleThreadedBuilder.build(SingleThreadedBuilder.java:51)
   2020-03-03T04:18:38.2185929Z [ERROR] at org.apache.maven.lifecycle.internal.LifecycleStarter.execute(LifecycleStarter.java:120)
   2020-03-03T04:18:38.2186545Z [ERROR] at org.apache.maven.DefaultMaven.doExecute(DefaultMaven.java:355)
   2020-03-03T04:18:38.2187050Z [ERROR] at org.apache.maven.DefaultMaven.execute(DefaultMaven.java:155)
   2020-03-03T04:18:38.2187800Z [ERROR] at org.apache.maven.cli.MavenCli.execute(MavenCli.java:584)
   2020-03-03T04:18:38.2188225Z [ERROR] at org.apache.maven.cli.MavenCli.doMain(MavenCli.java:216)
   2020-03-03T04:18:38.2188733Z [ERROR] at org.apache.maven.cli.MavenCli.main(MavenCli.java:160)
   2020-03-03T04:18:38.2189123Z [ERROR] at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   2020-03-03T04:18:38.2189580Z [ERROR] at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   2020-03-03T04:18:38.2190104Z [ERROR] at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   2020-03-03T04:18:38.2190716Z [ERROR] at java.lang.reflect.Method.invoke(Method.java:498)
   2020-03-03T04:18:38.2191179Z [ERROR] at org.codehaus.plexus.classworlds.launcher.Launcher.launchEnhanced(Launcher.java:289)
   2020-03-03T04:18:38.2191713Z [ERROR] at org.codehaus.plexus.classworlds.launcher.Launcher.launch(Launcher.java:229)
   2020-03-03T04:18:38.2192306Z [ERROR] at org.codehaus.plexus.classworlds.launcher.Launcher.mainWithExitCode(Launcher.java:415)
   2020-03-03T04:18:38.2192961Z [ERROR] at org.codehaus.plexus.classworlds.launcher.Launcher.main(Launcher.java:356)
   2020-03-03T04:18:38.2193642Z [ERROR] Caused by: org.apache.maven.surefire.booter.SurefireBooterForkException: Error occurred in starting fork, check output in log
   2020-03-03T04:18:38.2194259Z [ERROR] at org.apache.maven.plugin.surefire.booterclient.ForkStarter.fork(ForkStarter.java:622)
   2020-03-03T04:18:38.2194809Z [ERROR] at org.apache.maven.plugin.surefire.booterclient.ForkStarter.access$600(ForkStarter.java:115)
   2020-03-03T04:18:38.2195379Z [ERROR] at org.apache.maven.plugin.surefire.booterclient.ForkStarter$2.call(ForkStarter.java:444)
   2020-03-03T04:18:38.2195921Z [ERROR] at org.apache.maven.plugin.surefire.booterclient.ForkStarter$2.call(ForkStarter.java:420)
   2020-03-03T04:18:38.2196512Z [ERROR] at java.util.concurrent.FutureTask.run(FutureTask.java:266)
   2020-03-03T04:18:38.2196985Z [ERROR] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
   2020-03-03T04:18:38.2197519Z [ERROR] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
   2020-03-03T04:18:38.2197971Z [ERROR] at java.lang.Thread.run(Thread.java:748)
   2020-03-03T04:18:38.2198739Z [ERROR] -> [Help 1]
   ```
   
   azure failed on 'library' profile which shouldn't be affected by this PR. It seems we failed in irrelevant PRs and master in this case( https://dev.azure.com/rmetzger/Flink/_pipeline/analytics/stageawareoutcome?definitionId=4 ). Is it a known issue? cc @zentol @tzulitai 

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151292878",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151373127",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151373127",
       "triggerID" : "593715190",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828",
       "triggerID" : "593715190",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151475534",
       "triggerID" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5846",
       "triggerID" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64137c6034da191208680bbe89948d911eec6e52",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151480044",
       "triggerID" : "64137c6034da191208680bbe89948d911eec6e52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64137c6034da191208680bbe89948d911eec6e52",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5848",
       "triggerID" : "64137c6034da191208680bbe89948d911eec6e52",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be UNKNOWN
   * abfef134eb5940d7831dab4c794d654c2fa70263 UNKNOWN
   * d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4 UNKNOWN
   * 78750a20196de5f01b01b59b1cd15c752bdee5c0 UNKNOWN
   * 64137c6034da191208680bbe89948d911eec6e52 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/151480044) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5848) 
   
   <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


With regards,
Apache Git Services

[GitHub] [flink] wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r386766284
 
 

 ##########
 File path: flink-kubernetes/src/main/resources/META-INF/NOTICE
 ##########
 @@ -15,6 +15,7 @@ This project bundles the following dependencies under the Apache Software Licens
 - com.squareup.okhttp3:okhttp:3.12.1
 - com.squareup.okio:okio:1.15.0
 - io.fabric8:kubernetes-client:4.5.2
+- io.fabric8:zjsonpatch:0.3.0
 
 Review comment:
   Nit: Maybe we could put the dependencies in alphabetical order.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-592917200
 
 
   > @zhengcanbin Thanks a lot for your excellent work. The design and implementation are really good. After this refactor, i believe that adding new features will become easier.
   > 
   > I have gone with a quick review and leave some comments. I have not tested in a minkube or real k8s cluster. So please hold on and i will give more feedback in this weekend.
   
   Thanks for the review, @wangyang0918.  Kindly expect to see the new feedback by testing this PR in a Minikube or a real k8s cluster.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-593715235
 
 
   @flinkbot run azure

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-592323880
 
 
   > [0001-hotfix-Refactor-subclass-isXXX-to-inherit-methods.txt](https://github.com/apache/flink/files/4263005/0001-hotfix-Refactor-subclass-isXXX-to-inherit-methods.txt)
   > 
   > @zhengcanbin here is a patch for code quality advice that you can take a look at. As we discuss previously it is often better to use override methods instead of control super classes' logic by subclasses' isXXX method.
   > 
   > Generally changes looks good. I will double check the client side code and wait for response from @wangyang0918 and possibly @tillrohrmann .
   
   Hi, @TisonKun , thanks for the comment, a new commit [786a505](https://github.com/apache/flink/pull/11233/commits/786a5056c957863c05ad24b00ca1dca032905eb0) resolves the problem.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 786a5056c957863c05ad24b00ca1dca032905eb0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/150962647) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717) 
   * 19e32e6bca1a90f608010fdff1236fdfdbb53e3b 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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-593317565
 
 
   @flinkbot run azure

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r386008359
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
 ##########
 @@ -66,8 +70,52 @@ public Fabric8FlinkKubeClient(Configuration flinkConfig, KubernetesClient client
 	}
 
 	@Override
-	public void createTaskManagerPod(TaskManagerPodParameter parameter) {
-		// todo
+	public void createJobManagerComponent(KubernetesJobManagerSpecification kubernetesJMSpec) {
+		final Deployment deployment = kubernetesJMSpec.getDeployment();
+		final List<HasMetadata> accompanyingResources = kubernetesJMSpec.getAccompanyingResources();
+
+		// create Deployment
+		LOG.debug("Start to create deployment with spec {}", deployment.getSpec().toString());
+		final Deployment createdDeployment = this.internalClient
+			.apps()
+			.deployments()
+			.inNamespace(this.nameSpace)
+			.create(deployment);
+
+		// Note, we should use the server-side uid of the created Deployment for the OwnerReference.
+		setOwnerReference(createdDeployment, accompanyingResources);
 
 Review comment:
   Fixed by [abfef13](https://github.com/apache/flink/pull/11233/commits/abfef134eb5940d7831dab4c794d654c2fa70263)

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fb57917227853d0477aa1383d399a619146d7170 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/150793256) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676) 
   
   <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


With regards,
Apache Git Services

[GitHub] [flink] wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r385575128
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/AbstractKubernetesStepDecorator.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 io.fabric8.kubernetes.api.model.HasMetadata;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * An abstract {@link KubernetesStepDecorator} contains common implementations for different plug-in features
+ * while providing two additional methods.
 
 Review comment:
   The document should be updated since we do not provide any additional methods now.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r386779843
 
 

 ##########
 File path: flink-kubernetes/src/main/resources/META-INF/NOTICE
 ##########
 @@ -15,6 +15,7 @@ This project bundles the following dependencies under the Apache Software Licens
 - com.squareup.okhttp3:okhttp:3.12.1
 - com.squareup.okio:okio:1.15.0
 - io.fabric8:kubernetes-client:4.5.2
+- io.fabric8:zjsonpatch:0.3.0
 
 Review comment:
   Fixed by [64137c6](https://github.com/apache/flink/pull/11233/commits/64137c6034da191208680bbe89948d911eec6e52)

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r385598341
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkConfMountDecorator.java
 ##########
 @@ -0,0 +1,174 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.client.cli.CliFrontend;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.parameters.AbstractKubernetesParameters;
+
+import org.apache.flink.shaded.guava18.com.google.common.io.Files;
+
+import io.fabric8.kubernetes.api.model.ConfigMap;
+import io.fabric8.kubernetes.api.model.ConfigMapBuilder;
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.ContainerBuilder;
+import io.fabric8.kubernetes.api.model.HasMetadata;
+import io.fabric8.kubernetes.api.model.KeyToPath;
+import io.fabric8.kubernetes.api.model.KeyToPathBuilder;
+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 java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.configuration.GlobalConfiguration.FLINK_CONF_FILENAME;
+import static org.apache.flink.kubernetes.utils.Constants.CONFIG_FILE_LOG4J_NAME;
+import static org.apache.flink.kubernetes.utils.Constants.CONFIG_FILE_LOGBACK_NAME;
+import static org.apache.flink.kubernetes.utils.Constants.CONFIG_MAP_PREFIX;
+import static org.apache.flink.kubernetes.utils.Constants.FLINK_CONF_VOLUME;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Mounts the log4j.properties, logback.xml, and flink-conf.yaml configuration on the JobManager or TaskManager pod.
+ */
+public class FlinkConfMountDecorator extends AbstractKubernetesStepDecorator {
+
+	private final AbstractKubernetesParameters kubernetesComponentConf;
+
+	public FlinkConfMountDecorator(AbstractKubernetesParameters kubernetesComponentConf) {
+		this.kubernetesComponentConf = checkNotNull(kubernetesComponentConf);
+	}
+
+	@Override
+	public FlinkPod decorateFlinkPod(FlinkPod flinkPod) {
+		final Pod mountedPod = decoratePod(flinkPod.getPod());
+
+		final Container mountedMainContainer = new ContainerBuilder(flinkPod.getMainContainer())
+			.addNewVolumeMount()
+				.withName(FLINK_CONF_VOLUME)
+				.withMountPath(kubernetesComponentConf.getFlinkConfDirInPod())
+				.endVolumeMount()
+			.build();
+
+		return new FlinkPod.Builder(flinkPod)
+			.withPod(mountedPod)
+			.withMainContainer(mountedMainContainer)
+			.build();
+	}
+
+	private Pod decoratePod(Pod pod) {
+		final List<KeyToPath> keyToPaths = getLocalLogConfFiles().stream()
+			.map(file -> new KeyToPathBuilder()
+				.withKey(file.getName())
+				.withPath(file.getName())
+				.build())
+			.collect(Collectors.toList());
+		keyToPaths.add(new KeyToPathBuilder()
+			.withKey(FLINK_CONF_FILENAME)
+			.withPath(FLINK_CONF_FILENAME)
+			.build());
+
+		final Volume flinkConfVolume = new VolumeBuilder()
+			.withName(FLINK_CONF_VOLUME)
+			.withNewConfigMap()
+				.withName(getFlinkConfConfigMapName(kubernetesComponentConf.getClusterId()))
+				.withItems(keyToPaths)
+				.endConfigMap()
+			.build();
+
+		return new PodBuilder(pod)
+			.editSpec()
+				.addNewVolumeLike(flinkConfVolume)
+				.endVolume()
+				.endSpec()
+			.build();
+	}
+
+	@Override
+	public List<HasMetadata> buildAccompanyingKubernetesResources() throws IOException {
+		final String clusterId = kubernetesComponentConf.getClusterId();
+
+		final Map<String, String> data = new HashMap<>();
+
+		final List<File> localLogFiles = getLocalLogConfFiles();
+		for (File file : localLogFiles) {
+			data.put(file.getName(), Files.toString(file, StandardCharsets.UTF_8));
+		}
+		data.put(FLINK_CONF_FILENAME, getFlinkConfData(kubernetesComponentConf.getFlinkConfiguration()));
+
+		final ConfigMap flinkConfConfigMap = new ConfigMapBuilder()
+			.withNewMetadata()
+				.withName(getFlinkConfConfigMapName(clusterId))
+				.withLabels(kubernetesComponentConf.getCommonLabels())
+				.endMetadata()
+			.addToData(data)
+			.build();
+
+		return Collections.singletonList(flinkConfConfigMap);
+	}
+
+	@VisibleForTesting
+	String getFlinkConfData(Configuration configuration) throws IOException {
+		try (StringWriter sw = new StringWriter();
+			PrintWriter out = new PrintWriter(sw)) {
+			for (String key : configuration.keySet()) {
 
 Review comment:
   I suggest to use `for (Map.Entry<String, String> entry : configuration.toMap().entrySet())` here. Since the `getString` is deprecated here.
   
   Also we need to remove some keys should not be taken to jobmanager and taskmanager. For example, `KubernetesConfigOptions.KUBE_CONFIG_FILE`.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151292878",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151373127",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151373127",
       "triggerID" : "593715190",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828",
       "triggerID" : "593715190",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151475534",
       "triggerID" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5846",
       "triggerID" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64137c6034da191208680bbe89948d911eec6e52",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "64137c6034da191208680bbe89948d911eec6e52",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be UNKNOWN
   * abfef134eb5940d7831dab4c794d654c2fa70263 UNKNOWN
   * d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4 UNKNOWN
   * 78750a20196de5f01b01b59b1cd15c752bdee5c0 UNKNOWN
   * afdf9578347a7782b8ee40925c04dbe00ddc9415 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/151475534) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5846) 
   * 64137c6034da191208680bbe89948d911eec6e52 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


With regards,
Apache Git Services

[GitHub] [flink] TisonKun commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
TisonKun commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-593771974
 
 
   > @TisonKun Do you think we need to reorganize the commit message or squash them when merging? Currently, i think there are too many `hotfix` to address the comments or previous commits.
   
   Yes it would be nice that if @zhengcanbin can reorganize the commits and squash some intermediate hotfixes. I tend to avoid the PR as a "squash-and-merge" monolithic one or such a series of handy fixups.
   
   Or I will try to reorganize the commits on 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


With regards,
Apache Git Services

[GitHub] [flink] TisonKun edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
TisonKun edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-593771974
 
 
   > @TisonKun Do you think we need to reorganize the commit message or squash them when merging? Currently, i think there are too many `hotfix` to address the comments or previous commits.
   
   Yes it would be nice that if @zhengcanbin can reorganize the commits and squash some intermediate hotfixes. I tend to avoid merging this PR as a "squash-and-merge" monolithic one or such a series of handy fixups.
   
   Or I will try to reorganize the commits on 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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r385996299
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactory.java
 ##########
 @@ -0,0 +1,109 @@
+/*
+ * 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.factory;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.KubernetesJobManagerSpecification;
+import org.apache.flink.kubernetes.kubeclient.decorators.ExternalServiceDecorator;
+import org.apache.flink.kubernetes.kubeclient.decorators.FlinkConfMountDecorator;
+import org.apache.flink.kubernetes.kubeclient.decorators.InitJobManagerDecorator;
+import org.apache.flink.kubernetes.kubeclient.decorators.InternalServiceDecorator;
+import org.apache.flink.kubernetes.kubeclient.decorators.JavaCmdJobManagerDecorator;
+import org.apache.flink.kubernetes.kubeclient.decorators.KubernetesStepDecorator;
+import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters;
+import org.apache.flink.kubernetes.utils.Constants;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.HasMetadata;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.PodBuilder;
+import io.fabric8.kubernetes.api.model.apps.Deployment;
+import io.fabric8.kubernetes.api.model.apps.DeploymentBuilder;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Utility class for constructing all the Kubernetes components on the client-side. This can
+ * include the Deployment, the ConfigMap(s), and the Service(s).
+ */
+public class KubernetesJobManagerFactory {
+
+	public static KubernetesJobManagerSpecification createJobManagerComponent(
+			KubernetesJobManagerParameters kubernetesJobManagerParameters) throws IOException {
+		FlinkPod flinkPod = new FlinkPod.Builder().build();
+		List<HasMetadata> accompanyingResources = new ArrayList<>();
+
+		final KubernetesStepDecorator[] stepDecorators = new KubernetesStepDecorator[] {
+			new InitJobManagerDecorator(kubernetesJobManagerParameters),
+			new JavaCmdJobManagerDecorator(kubernetesJobManagerParameters),
+			new InternalServiceDecorator(kubernetesJobManagerParameters),
+			new ExternalServiceDecorator(kubernetesJobManagerParameters),
+			new FlinkConfMountDecorator(kubernetesJobManagerParameters)};
+
+		for (KubernetesStepDecorator stepDecorator: stepDecorators) {
+			flinkPod = stepDecorator.decorateFlinkPod(flinkPod);
+			accompanyingResources.addAll(stepDecorator.buildAccompanyingKubernetesResources());
+		}
+
+		final Deployment deployment = createJobManagerDeployment(flinkPod, kubernetesJobManagerParameters);
+
+		KubernetesUtils.setOwnerReference(deployment, accompanyingResources);
+
+		return new KubernetesJobManagerSpecification(deployment, accompanyingResources);
+	}
+
+	private static Deployment createJobManagerDeployment(
+			FlinkPod flinkPod,
+			KubernetesJobManagerParameters kubernetesJobManagerParameters) {
+		final Container resolvedMainContainer = flinkPod.getMainContainer();
+
+		final Pod resolvedPod = new PodBuilder(flinkPod.getPod())
+			.editOrNewSpec()
+				.addToContainers(resolvedMainContainer)
+				.endSpec()
+			.build();
+
+		final Map<String, String> labels = resolvedPod.getMetadata().getLabels();
 
 Review comment:
   The basic logic is:
   1. We use the Pod object built by the decorators to fill the pod template of the deployment.
   2. Then set the deployment.metadata.labels separately to help others realize that the labels of the deployment could be different from the labels of the underlying pod.
   
   For the usual scenarios, the labels could be the same. For this PR,  resolvedPod.getMetadata().getLabels() and kubernetesJobManagerParameters.getLabels() shares the same values. 
   
   So I think it is reasonable to use the resolvedPod.getMetadata().getLabels to fill the spec of the deployment.
   
   For the deployment metadata, maybe we could introduce a private wrapper method in KubernetesJobManagerFactory to get the deployment labels.
   
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 786a5056c957863c05ad24b00ca1dca032905eb0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/150962647) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717) 
   * 19e32e6bca1a90f608010fdff1236fdfdbb53e3b Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/151155366) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765) 
   * 0017b8a8930024d0e9ce3355d050565a5007831d 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


With regards,
Apache Git Services

[GitHub] [flink] TisonKun commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
TisonKun commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-593774185
 
 
   ```
   2020-03-03T04:18:38.2171847Z [ERROR] Failed to execute goal org.apache.maven.plugins:maven-surefire-plugin:2.22.1:test (integration-tests) on project flink-state-processor-api_2.11: There are test failures.
   2020-03-03T04:18:38.2172997Z [ERROR] 
   2020-03-03T04:18:38.2173799Z [ERROR] Please refer to /__w/1/s/flink-libraries/flink-state-processing-api/target/surefire-reports for the individual test results.
   2020-03-03T04:18:38.2174597Z [ERROR] Please refer to dump files (if any exist) [date].dump, [date]-jvmRun[N].dump and [date].dumpstream.
   2020-03-03T04:18:38.2175072Z [ERROR] ExecutionException Error occurred in starting fork, check output in log
   2020-03-03T04:18:38.2175671Z [ERROR] org.apache.maven.surefire.booter.SurefireBooterForkException: ExecutionException Error occurred in starting fork, check output in log
   2020-03-03T04:18:38.2176787Z [ERROR] at org.apache.maven.plugin.surefire.booterclient.ForkStarter.awaitResultsDone(ForkStarter.java:510)
   2020-03-03T04:18:38.2177445Z [ERROR] at org.apache.maven.plugin.surefire.booterclient.ForkStarter.runSuitesForkPerTestSet(ForkStarter.java:457)
   2020-03-03T04:18:38.2178053Z [ERROR] at org.apache.maven.plugin.surefire.booterclient.ForkStarter.run(ForkStarter.java:298)
   2020-03-03T04:18:38.2178813Z [ERROR] at org.apache.maven.plugin.surefire.booterclient.ForkStarter.run(ForkStarter.java:246)
   2020-03-03T04:18:38.2179652Z [ERROR] at org.apache.maven.plugin.surefire.AbstractSurefireMojo.executeProvider(AbstractSurefireMojo.java:1183)
   2020-03-03T04:18:38.2180320Z [ERROR] at org.apache.maven.plugin.surefire.AbstractSurefireMojo.executeAfterPreconditionsChecked(AbstractSurefireMojo.java:1011)
   2020-03-03T04:18:38.2180974Z [ERROR] at org.apache.maven.plugin.surefire.AbstractSurefireMojo.execute(AbstractSurefireMojo.java:857)
   2020-03-03T04:18:38.2181725Z [ERROR] at org.apache.maven.plugin.DefaultBuildPluginManager.executeMojo(DefaultBuildPluginManager.java:132)
   2020-03-03T04:18:38.2182381Z [ERROR] at org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:208)
   2020-03-03T04:18:38.2182907Z [ERROR] at org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:153)
   2020-03-03T04:18:38.2183440Z [ERROR] at org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:145)
   2020-03-03T04:18:38.2184009Z [ERROR] at org.apache.maven.lifecycle.internal.LifecycleModuleBuilder.buildProject(LifecycleModuleBuilder.java:116)
   2020-03-03T04:18:38.2184645Z [ERROR] at org.apache.maven.lifecycle.internal.LifecycleModuleBuilder.buildProject(LifecycleModuleBuilder.java:80)
   2020-03-03T04:18:38.2185293Z [ERROR] at org.apache.maven.lifecycle.internal.builder.singlethreaded.SingleThreadedBuilder.build(SingleThreadedBuilder.java:51)
   2020-03-03T04:18:38.2185929Z [ERROR] at org.apache.maven.lifecycle.internal.LifecycleStarter.execute(LifecycleStarter.java:120)
   2020-03-03T04:18:38.2186545Z [ERROR] at org.apache.maven.DefaultMaven.doExecute(DefaultMaven.java:355)
   2020-03-03T04:18:38.2187050Z [ERROR] at org.apache.maven.DefaultMaven.execute(DefaultMaven.java:155)
   2020-03-03T04:18:38.2187800Z [ERROR] at org.apache.maven.cli.MavenCli.execute(MavenCli.java:584)
   2020-03-03T04:18:38.2188225Z [ERROR] at org.apache.maven.cli.MavenCli.doMain(MavenCli.java:216)
   2020-03-03T04:18:38.2188733Z [ERROR] at org.apache.maven.cli.MavenCli.main(MavenCli.java:160)
   2020-03-03T04:18:38.2189123Z [ERROR] at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   2020-03-03T04:18:38.2189580Z [ERROR] at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   2020-03-03T04:18:38.2190104Z [ERROR] at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   2020-03-03T04:18:38.2190716Z [ERROR] at java.lang.reflect.Method.invoke(Method.java:498)
   2020-03-03T04:18:38.2191179Z [ERROR] at org.codehaus.plexus.classworlds.launcher.Launcher.launchEnhanced(Launcher.java:289)
   2020-03-03T04:18:38.2191713Z [ERROR] at org.codehaus.plexus.classworlds.launcher.Launcher.launch(Launcher.java:229)
   2020-03-03T04:18:38.2192306Z [ERROR] at org.codehaus.plexus.classworlds.launcher.Launcher.mainWithExitCode(Launcher.java:415)
   2020-03-03T04:18:38.2192961Z [ERROR] at org.codehaus.plexus.classworlds.launcher.Launcher.main(Launcher.java:356)
   2020-03-03T04:18:38.2193642Z [ERROR] Caused by: org.apache.maven.surefire.booter.SurefireBooterForkException: Error occurred in starting fork, check output in log
   2020-03-03T04:18:38.2194259Z [ERROR] at org.apache.maven.plugin.surefire.booterclient.ForkStarter.fork(ForkStarter.java:622)
   2020-03-03T04:18:38.2194809Z [ERROR] at org.apache.maven.plugin.surefire.booterclient.ForkStarter.access$600(ForkStarter.java:115)
   2020-03-03T04:18:38.2195379Z [ERROR] at org.apache.maven.plugin.surefire.booterclient.ForkStarter$2.call(ForkStarter.java:444)
   2020-03-03T04:18:38.2195921Z [ERROR] at org.apache.maven.plugin.surefire.booterclient.ForkStarter$2.call(ForkStarter.java:420)
   2020-03-03T04:18:38.2196512Z [ERROR] at java.util.concurrent.FutureTask.run(FutureTask.java:266)
   2020-03-03T04:18:38.2196985Z [ERROR] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
   2020-03-03T04:18:38.2197519Z [ERROR] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
   2020-03-03T04:18:38.2197971Z [ERROR] at java.lang.Thread.run(Thread.java:748)
   2020-03-03T04:18:38.2198739Z [ERROR] -> [Help 1]
   ```
   
   azure failed on 'library' profile which doesn't affected by this PR. It seems we failed in irrelevant PRs and master in this case( https://dev.azure.com/rmetzger/Flink/_pipeline/analytics/stageawareoutcome?definitionId=4 ). Is it a known issue? cc @zentol @tzulitai 

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-592323880
 
 
   > [0001-hotfix-Refactor-subclass-isXXX-to-inherit-methods.txt](https://github.com/apache/flink/files/4263005/0001-hotfix-Refactor-subclass-isXXX-to-inherit-methods.txt)
   > 
   > @zhengcanbin here is a patch for code quality advice that you can take a look at. As we discuss previously it is often better to use override methods instead of control super classes' logic by subclasses' isXXX method.
   > 
   > Generally changes looks good. I will double check the client side code and wait for response from @wangyang0918 and possibly @tillrohrmann .
   
   Hi, tison, thanks for the comment, a new commit [786a505](https://github.com/apache/flink/pull/11233/commits/786a5056c957863c05ad24b00ca1dca032905eb0) resolves the problem.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r386008661
 
 

 ##########
 File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesClusterDescriptorTest.java
 ##########
 @@ -47,19 +55,29 @@
  * Tests for the {@link KubernetesClusterDescriptor}.
  */
 public class KubernetesClusterDescriptorTest extends KubernetesTestBase {
-
+	private static final String MOCK_SERVICE_HOST_NAME = "mock-host-name-of-service";
 	private static final String MOCK_SERVICE_IP = "192.168.0.1";
 
 	private final ClusterSpecification clusterSpecification = new ClusterSpecification.ClusterSpecificationBuilder()
 		.createClusterSpecification();
 
+	@Before
+	public void setup() throws Exception {
+		super.setup();
+
+		mockRestServiceWithLoadBalancer(MOCK_SERVICE_HOST_NAME, MOCK_SERVICE_IP);
+	}
+
 	@Test
 	public void testDeploySessionCluster() throws Exception {
+		mockRestServiceWithLoadBalancer(MOCK_SERVICE_HOST_NAME, MOCK_SERVICE_IP);
 
 Review comment:
   Fixed by [d78c1b1](https://github.com/apache/flink/pull/11233/commits/d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4)

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151292878",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151373127",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151373127",
       "triggerID" : "593715190",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be UNKNOWN
   * abfef134eb5940d7831dab4c794d654c2fa70263 UNKNOWN
   * d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4 UNKNOWN
   * 78750a20196de5f01b01b59b1cd15c752bdee5c0 UNKNOWN
   * d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/151373127) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828) 
   
   <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


With regards,
Apache Git Services

[GitHub] [flink] wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r385613358
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
 ##########
 @@ -66,8 +70,52 @@ public Fabric8FlinkKubeClient(Configuration flinkConfig, KubernetesClient client
 	}
 
 	@Override
-	public void createTaskManagerPod(TaskManagerPodParameter parameter) {
-		// todo
+	public void createJobManagerComponent(KubernetesJobManagerSpecification kubernetesJMSpec) {
+		final Deployment deployment = kubernetesJMSpec.getDeployment();
+		final List<HasMetadata> accompanyingResources = kubernetesJMSpec.getAccompanyingResources();
+
+		// create Deployment
+		LOG.debug("Start to create deployment with spec {}", deployment.getSpec().toString());
+		final Deployment createdDeployment = this.internalClient
+			.apps()
+			.deployments()
+			.inNamespace(this.nameSpace)
+			.create(deployment);
+
+		// Note, we should use the server-side uid of the created Deployment for the OwnerReference.
+		setOwnerReference(createdDeployment, accompanyingResources);
 
 Review comment:
   Since we change the owner reference here, the document needs to be updated.
   https://ci.apache.org/projects/flink/flink-docs-master/ops/deployment/native_kubernetes.html#manual-resource-cleanup

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151292878",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151373127",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151373127",
       "triggerID" : "593715190",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828",
       "triggerID" : "593715190",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151475534",
       "triggerID" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5846",
       "triggerID" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be UNKNOWN
   * abfef134eb5940d7831dab4c794d654c2fa70263 UNKNOWN
   * d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4 UNKNOWN
   * 78750a20196de5f01b01b59b1cd15c752bdee5c0 UNKNOWN
   * d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/151373127) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828) 
   * afdf9578347a7782b8ee40925c04dbe00ddc9415 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/151475534) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5846) 
   
   <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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 19e32e6bca1a90f608010fdff1236fdfdbb53e3b Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/151155366) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765) 
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be 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


With regards,
Apache Git Services

[GitHub] [flink] wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r385604376
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactory.java
 ##########
 @@ -0,0 +1,109 @@
+/*
+ * 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.factory;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.KubernetesJobManagerSpecification;
+import org.apache.flink.kubernetes.kubeclient.decorators.ExternalServiceDecorator;
+import org.apache.flink.kubernetes.kubeclient.decorators.FlinkConfMountDecorator;
+import org.apache.flink.kubernetes.kubeclient.decorators.InitJobManagerDecorator;
+import org.apache.flink.kubernetes.kubeclient.decorators.InternalServiceDecorator;
+import org.apache.flink.kubernetes.kubeclient.decorators.JavaCmdJobManagerDecorator;
+import org.apache.flink.kubernetes.kubeclient.decorators.KubernetesStepDecorator;
+import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters;
+import org.apache.flink.kubernetes.utils.Constants;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.HasMetadata;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.PodBuilder;
+import io.fabric8.kubernetes.api.model.apps.Deployment;
+import io.fabric8.kubernetes.api.model.apps.DeploymentBuilder;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Utility class for constructing all the Kubernetes components on the client-side. This can
+ * include the Deployment, the ConfigMap(s), and the Service(s).
+ */
+public class KubernetesJobManagerFactory {
+
+	public static KubernetesJobManagerSpecification createJobManagerComponent(
+			KubernetesJobManagerParameters kubernetesJobManagerParameters) throws IOException {
+		FlinkPod flinkPod = new FlinkPod.Builder().build();
+		List<HasMetadata> accompanyingResources = new ArrayList<>();
+
+		final KubernetesStepDecorator[] stepDecorators = new KubernetesStepDecorator[] {
+			new InitJobManagerDecorator(kubernetesJobManagerParameters),
+			new JavaCmdJobManagerDecorator(kubernetesJobManagerParameters),
+			new InternalServiceDecorator(kubernetesJobManagerParameters),
+			new ExternalServiceDecorator(kubernetesJobManagerParameters),
+			new FlinkConfMountDecorator(kubernetesJobManagerParameters)};
+
+		for (KubernetesStepDecorator stepDecorator: stepDecorators) {
+			flinkPod = stepDecorator.decorateFlinkPod(flinkPod);
+			accompanyingResources.addAll(stepDecorator.buildAccompanyingKubernetesResources());
+		}
+
+		final Deployment deployment = createJobManagerDeployment(flinkPod, kubernetesJobManagerParameters);
+
+		KubernetesUtils.setOwnerReference(deployment, accompanyingResources);
+
+		return new KubernetesJobManagerSpecification(deployment, accompanyingResources);
+	}
+
+	private static Deployment createJobManagerDeployment(
+			FlinkPod flinkPod,
+			KubernetesJobManagerParameters kubernetesJobManagerParameters) {
+		final Container resolvedMainContainer = flinkPod.getMainContainer();
+
+		final Pod resolvedPod = new PodBuilder(flinkPod.getPod())
+			.editOrNewSpec()
+				.addToContainers(resolvedMainContainer)
+				.endSpec()
+			.build();
+
+		final Map<String, String> labels = resolvedPod.getMetadata().getLabels();
 
 Review comment:
   Sometimes we use `resolvedPod.getMetadata().getLabels()`, or `kubernetesJobManagerParameters.getLabels()`. What's the differences? Is it possible that the labels of deployment and pod are different?

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] TisonKun commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
TisonKun commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-593724440
 
 
   It seems azure failed on mesos e2e cases. @KarmaGYZ do you have an idea or know an existing issue? (log: https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_apis/build/builds/5828/logs/118 )

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 19e32e6bca1a90f608010fdff1236fdfdbb53e3b Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/151155366) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765) 
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be UNKNOWN
   * abfef134eb5940d7831dab4c794d654c2fa70263 UNKNOWN
   * d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4 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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591942827
 
 
   @flinkbot run azure

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 19e32e6bca1a90f608010fdff1236fdfdbb53e3b Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/151155366) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765) 
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be 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


With regards,
Apache Git Services

[GitHub] [flink] TisonKun commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
TisonKun commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r387862667
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/KubernetesJobManagerSpecification.java
 ##########
 @@ -16,19 +16,32 @@
  * limitations under the License.
  */
 
-package org.apache.flink.kubernetes.kubeclient.resources;
-
-import org.apache.flink.configuration.Configuration;
+package org.apache.flink.kubernetes.kubeclient;
 
+import io.fabric8.kubernetes.api.model.HasMetadata;
 import io.fabric8.kubernetes.api.model.apps.Deployment;
-import io.fabric8.kubernetes.api.model.apps.DeploymentBuilder;
+
+import java.util.List;
 
 /**
- * Represent KubernetesDeployment resource in kubernetes.
+ *	Composition of the created Kubernetes components that represents a Flink application.
 
 Review comment:
   You can edit the commit introduce this instead of adding fixup. I think we agree on most of codes now.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r386007422
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
 ##########
 @@ -66,8 +70,52 @@ public Fabric8FlinkKubeClient(Configuration flinkConfig, KubernetesClient client
 	}
 
 	@Override
-	public void createTaskManagerPod(TaskManagerPodParameter parameter) {
-		// todo
+	public void createJobManagerComponent(KubernetesJobManagerSpecification kubernetesJMSpec) {
+		final Deployment deployment = kubernetesJMSpec.getDeployment();
+		final List<HasMetadata> accompanyingResources = kubernetesJMSpec.getAccompanyingResources();
+
+		// create Deployment
+		LOG.debug("Start to create deployment with spec {}", deployment.getSpec().toString());
+		final Deployment createdDeployment = this.internalClient
 
 Review comment:
   The deployment would not fail since the Kubernetes has strategies for such a case:
   1. Generally, the kubelet component helps retry pulling the missing ConfigMap.
   2. The controller helps re-deploy the Pod if strategy 1 fails.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151292878",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be UNKNOWN
   * abfef134eb5940d7831dab4c794d654c2fa70263 UNKNOWN
   * d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4 UNKNOWN
   * 78750a20196de5f01b01b59b1cd15c752bdee5c0 UNKNOWN
   * b088e0e0530fa3178a547b26b7e950165181caaf Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/151326230) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_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


With regards,
Apache Git Services

[GitHub] [flink] TisonKun commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
TisonKun commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-592078248
 
 
   
   [0001-hotfix-Refactor-subclass-isXXX-to-inherit-methods.txt](https://github.com/apache/flink/files/4263005/0001-hotfix-Refactor-subclass-isXXX-to-inherit-methods.txt)
   
   @zhengcanbin here is a patch for code quality advice that you can take a look at. As we discuss previously it is often better to use override methods instead of control super classes' logic by subclasses' isXXX method.
   
   Generally changes looks good. I will double check the client side code and wait for response from @wangyang0918 and possibly @tillrohrmann .

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r386286390
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
 ##########
 @@ -66,8 +70,52 @@ public Fabric8FlinkKubeClient(Configuration flinkConfig, KubernetesClient client
 	}
 
 	@Override
-	public void createTaskManagerPod(TaskManagerPodParameter parameter) {
-		// todo
+	public void createJobManagerComponent(KubernetesJobManagerSpecification kubernetesJMSpec) {
+		final Deployment deployment = kubernetesJMSpec.getDeployment();
+		final List<HasMetadata> accompanyingResources = kubernetesJMSpec.getAccompanyingResources();
+
+		// create Deployment
+		LOG.debug("Start to create deployment with spec {}", deployment.getSpec().toString());
+		final Deployment createdDeployment = this.internalClient
+			.apps()
+			.deployments()
+			.inNamespace(this.nameSpace)
+			.create(deployment);
+
+		// Note, we should use the server-side uid of the created Deployment for the OwnerReference.
+		setOwnerReference(createdDeployment, accompanyingResources);
 
 Review comment:
   Good catch. Fixed by [b088e0e](https://github.com/apache/flink/pull/11233/commits/b088e0e0530fa3178a547b26b7e950165181caaf)

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r386258439
 
 

 ##########
 File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactoryTest.java
 ##########
 @@ -0,0 +1,203 @@
+/*
+ * 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.factory;
+
+import org.apache.flink.kubernetes.KubernetesTestUtils;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptionsInternal;
+import org.apache.flink.kubernetes.entrypoint.KubernetesSessionClusterEntrypoint;
+import org.apache.flink.kubernetes.kubeclient.KubernetesJobManagerSpecification;
+import org.apache.flink.kubernetes.kubeclient.KubernetesJobManagerTestBase;
+import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters;
+import org.apache.flink.kubernetes.utils.Constants;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+
+import io.fabric8.kubernetes.api.model.ConfigMap;
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.HasMetadata;
+import io.fabric8.kubernetes.api.model.PodSpec;
+import io.fabric8.kubernetes.api.model.Quantity;
+import io.fabric8.kubernetes.api.model.Service;
+import io.fabric8.kubernetes.api.model.apps.Deployment;
+import io.fabric8.kubernetes.api.model.apps.DeploymentSpec;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.configuration.GlobalConfiguration.FLINK_CONF_FILENAME;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * General tests for the {@link KubernetesJobManagerFactory}.
+ */
+public class KubernetesJobManagerFactoryTest extends KubernetesJobManagerTestBase {
+
+	private static final String SERVICE_ACCOUNT_NAME = "service-test";
+	private static final String ENTRY_POINT_CLASS = KubernetesSessionClusterEntrypoint.class.getCanonicalName();
+
+	private KubernetesJobManagerSpecification kubernetesJobManagerSpecification;
+
+	@Before
+	public void setup() throws Exception {
+		super.setup();
+
+		KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "logback.xml");
+		KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "log4j.properties");
+
+		flinkConfig.set(KubernetesConfigOptionsInternal.ENTRY_POINT_CLASS, ENTRY_POINT_CLASS);
+		flinkConfig.set(KubernetesConfigOptions.JOB_MANAGER_SERVICE_ACCOUNT, SERVICE_ACCOUNT_NAME);
+
+		this.kubernetesJobManagerSpecification =
+			KubernetesJobManagerFactory.createJobManagerComponent(kubernetesJobManagerParameters);
+	}
+
+	@Test
+	public void testDeploymentMetadata() {
+		final Deployment resultDeployment = this.kubernetesJobManagerSpecification.getDeployment();
+		assertEquals(Constants.APPS_API_VERSION, resultDeployment.getApiVersion());
+		assertEquals(KubernetesUtils.getDeploymentName(CLUSTER_ID), resultDeployment.getMetadata().getName());
+		final Map<String, String> expectedLabels = getCommonLabels();
+		expectedLabels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_JOB_MANAGER);
+		assertEquals(expectedLabels, resultDeployment.getMetadata().getLabels());
+	}
+
+	@Test
+	public void testDeploymentSpec() {
+		final DeploymentSpec resultDeploymentSpec = this.kubernetesJobManagerSpecification.getDeployment().getSpec();
+		assertEquals(1, resultDeploymentSpec.getReplicas().intValue());
+
+		final Map<String, String> expectedLabels =  new HashMap<>(getCommonLabels());
+		expectedLabels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_JOB_MANAGER);
+
+		assertEquals(expectedLabels, resultDeploymentSpec.getTemplate().getMetadata().getLabels());
+		assertEquals(expectedLabels, resultDeploymentSpec.getSelector().getMatchLabels());
+
+		assertNotNull(resultDeploymentSpec.getTemplate().getSpec());
+	}
+
+	@Test
+	public void testPodSpec() {
+		final PodSpec resultPodSpec =
+			this.kubernetesJobManagerSpecification.getDeployment().getSpec().getTemplate().getSpec();
+
+		assertEquals(1, resultPodSpec.getContainers().size());
+		assertEquals(SERVICE_ACCOUNT_NAME, resultPodSpec.getServiceAccountName());
+		assertEquals(1, resultPodSpec.getVolumes().size());
+
+		final Container resultedMainContainer = resultPodSpec.getContainers().get(0);
+		assertEquals(KubernetesJobManagerParameters.JOB_MANAGER_MAIN_CONTAINER_NAME, resultedMainContainer.getName());
+		assertEquals(CONTAINER_IMAGE, resultedMainContainer.getImage());
+		assertEquals(CONTAINER_IMAGE_PULL_POLICY, resultedMainContainer.getImagePullPolicy());
+
+		assertEquals(3, resultedMainContainer.getEnv().size());
+		assertTrue(resultedMainContainer.getEnv()
+				.stream()
+				.anyMatch(envVar -> envVar.getName().equals("key1")));
+
+		assertEquals(3, resultedMainContainer.getPorts().size());
+
+		final Map<String, Quantity> requests = resultedMainContainer.getResources().getRequests();
+		assertEquals(Double.toString(JOB_MANAGER_CPU), requests.get("cpu").getAmount());
+		assertEquals(JOB_MANAGER_MEMORY + "Mi", requests.get("memory").getAmount());
+
+		assertEquals(1, resultedMainContainer.getCommand().size());
+		assertEquals(3, resultedMainContainer.getArgs().size());
+
+		assertEquals(1, resultedMainContainer.getVolumeMounts().size());
+	}
+
+	@Test
+	public void testAdditionalResourcesSize() {
+		final List<HasMetadata> resultAdditionalResources = this.kubernetesJobManagerSpecification.getAccompanyingResources();
+		assertEquals(3, resultAdditionalResources.size());
+
+		final List<HasMetadata> resultServices = resultAdditionalResources
+			.stream()
+			.filter(x -> x instanceof Service)
+			.collect(Collectors.toList());
+		assertEquals(2, resultServices.size());
+
+		final List<HasMetadata> resultConfigMaps = resultAdditionalResources
+			.stream()
+			.filter(x -> x instanceof ConfigMap)
+			.collect(Collectors.toList());
+		assertEquals(1, resultConfigMaps.size());
+	}
+
+	@Test
+	public void testServices() {
+		final List<Service> resultServices = this.kubernetesJobManagerSpecification.getAccompanyingResources()
+			.stream()
+			.filter(x -> x instanceof Service)
+			.map(x -> (Service) x)
+			.collect(Collectors.toList());
+
+		assertEquals(2, resultServices.size());
+
+		final List<Service> internalServiceCandidates = resultServices
+				.stream()
+				.filter(x -> x.getMetadata().getName().equals(KubernetesUtils.getInternalServiceName(CLUSTER_ID)))
+				.collect(Collectors.toList());
+		assertEquals(1, internalServiceCandidates.size());
+
+		final List<Service> restServiceCandidates = resultServices
+				.stream()
+				.filter(x -> x.getMetadata().getName().equals(KubernetesUtils.getRestServiceName(CLUSTER_ID)))
+				.collect(Collectors.toList());
+		assertEquals(1, restServiceCandidates.size());
+
+		final Service resultInternalService = internalServiceCandidates.get(0);
+		assertEquals(2, resultInternalService.getMetadata().getLabels().size());
 
 Review comment:
   Since we have checked the real labels in `InternalServiceDecoratorTest` and `ExternalServiceDecoratorTest`, I don't think it's necessary to check them again in the final test of `KubernetesJobManagerFactoryTest`, rough assert such as size verifying could be enough, otherwise, there are repeated test work. 

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r386010368
 
 

 ##########
 File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesClusterDescriptorTest.java
 ##########
 @@ -124,13 +144,19 @@ public void testKillCluster() throws Exception {
 		descriptor.deploySessionCluster(clusterSpecification);
 
 		final KubernetesClient kubeClient = server.getClient();
-		assertEquals(2, kubeClient.services().list().getItems().size());
+
 
 Review comment:
   Actually we mock the request of **ANY** kind, not only the **GET**.  I attach another commit [78750a2](https://github.com/apache/flink/pull/11233/commits/78750a20196de5f01b01b59b1cd15c752bdee5c0) to fix this problem.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r385622389
 
 

 ##########
 File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesClusterDescriptorTest.java
 ##########
 @@ -124,13 +144,19 @@ public void testKillCluster() throws Exception {
 		descriptor.deploySessionCluster(clusterSpecification);
 
 		final KubernetesClient kubeClient = server.getClient();
-		assertEquals(2, kubeClient.services().list().getItems().size());
+
 
 Review comment:
   I am not sure why could not get the rest svc here. Since we just mock the get request, not the post.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] TisonKun commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
TisonKun commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r387862136
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/KubernetesJobManagerSpecification.java
 ##########
 @@ -16,19 +16,32 @@
  * limitations under the License.
  */
 
-package org.apache.flink.kubernetes.kubeclient.resources;
-
-import org.apache.flink.configuration.Configuration;
+package org.apache.flink.kubernetes.kubeclient;
 
+import io.fabric8.kubernetes.api.model.HasMetadata;
 import io.fabric8.kubernetes.api.model.apps.Deployment;
-import io.fabric8.kubernetes.api.model.apps.DeploymentBuilder;
+
+import java.util.List;
 
 /**
- * Represent KubernetesDeployment resource in kubernetes.
+ *	Composition of the created Kubernetes components that represents a Flink application.
 
 Review comment:
   ```suggestion
    *  Composition of the created Kubernetes components that represents a Flink application.
   ```

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 786a5056c957863c05ad24b00ca1dca032905eb0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/150962647) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717) 
   
   <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


With regards,
Apache Git Services

[GitHub] [flink] wangyang0918 commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-593385215
 
 
   @zhengcanbin Thanks for addressing all the comments. The changes look almost good to me now. Last one comment about the `NOTICE` file. Since we have included a new dependency `zjsonpatch`, it should also be added into `flink-kubernetes/src/main/resources/META-INF/NOTICE`. 

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151292878",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 19e32e6bca1a90f608010fdff1236fdfdbb53e3b Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/151155366) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765) 
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be UNKNOWN
   * abfef134eb5940d7831dab4c794d654c2fa70263 UNKNOWN
   * d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4 UNKNOWN
   * 78750a20196de5f01b01b59b1cd15c752bdee5c0 UNKNOWN
   * 08f07deb2243554d24e2e4171e6cb23a5f934cc8 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/151292878) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791) 
   
   <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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fb57917227853d0477aa1383d399a619146d7170 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/150793256) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676) 
   * 786a5056c957863c05ad24b00ca1dca032905eb0 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/150962647) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717) 
   
   <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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151292878",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be UNKNOWN
   * abfef134eb5940d7831dab4c794d654c2fa70263 UNKNOWN
   * d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4 UNKNOWN
   * 78750a20196de5f01b01b59b1cd15c752bdee5c0 UNKNOWN
   * 08f07deb2243554d24e2e4171e6cb23a5f934cc8 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/151292878) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791) 
   
   <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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fb57917227853d0477aa1383d399a619146d7170 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/150793256) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676) 
   * 786a5056c957863c05ad24b00ca1dca032905eb0 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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151292878",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be UNKNOWN
   * abfef134eb5940d7831dab4c794d654c2fa70263 UNKNOWN
   * d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4 UNKNOWN
   * 78750a20196de5f01b01b59b1cd15c752bdee5c0 UNKNOWN
   * 08f07deb2243554d24e2e4171e6cb23a5f934cc8 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/151292878) Azure: [CANCELED](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791) 
   * b088e0e0530fa3178a547b26b7e950165181caaf 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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-593317405
 
 
   @flinkbot run travis

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] TisonKun commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
TisonKun commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r387861569
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkPod.java
 ##########
 @@ -0,0 +1,98 @@
+/*
+ * 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;
+
+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 static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A collection of variables that composes a JobManager/TaskManager Pod. This can include
+ * the Pod, the main Container, and the InitContainer, etc.
+ */
+public class FlinkPod {
+
+	private Pod pod;
+
+	private Container mainContainer;
+
+	public FlinkPod(Pod pod, Container mainContainer) {
+		this.pod = pod;
+		this.mainContainer = mainContainer;
+	}
+
+	public Pod getPod() {
+		return pod;
+	}
+
+	public void setPod(Pod pod) {
+		this.pod = pod;
+	}
+
+	public Container getMainContainer() {
+		return mainContainer;
+	}
+
+	public void setMainContainer(Container mainContainer) {
+		this.mainContainer = mainContainer;
+	}
+
+	/**
+	 * Builder for creating a {@link FlinkPod}.
+	 */
+	public static class Builder {
+
+		private Pod pod;
+		private Container mainContainer;
+
+		public Builder() {
+			this.pod = new PodBuilder()
+				.withNewMetadata()
+				.endMetadata()
+				.withNewSpec()
+				.endSpec()
+				.build();
+
+			this.mainContainer = new ContainerBuilder().build();
+		}
+
+		public Builder(FlinkPod flinkPod) {
 
 Review comment:
   As the document of `AbstractKubernetesStepDecorator`, I'm ok that this Builder follows  fabric8's Builder pattern. However, Flink has its own Builder and even a [style guide](http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/CODE-STYLE-Builder-pattern-tt32225.html) for it. We can defer the change decision and leave as is now.
   
   cc @gyfora seems we forget adding the code style of Builder pattern in flink document.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151292878",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151373127",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151373127",
       "triggerID" : "593715190",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828",
       "triggerID" : "593715190",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151475534",
       "triggerID" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5846",
       "triggerID" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64137c6034da191208680bbe89948d911eec6e52",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151480044",
       "triggerID" : "64137c6034da191208680bbe89948d911eec6e52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64137c6034da191208680bbe89948d911eec6e52",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5848",
       "triggerID" : "64137c6034da191208680bbe89948d911eec6e52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f539fe185165e1071e3a54a8f781977a4872995",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3f539fe185165e1071e3a54a8f781977a4872995",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be UNKNOWN
   * abfef134eb5940d7831dab4c794d654c2fa70263 UNKNOWN
   * d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4 UNKNOWN
   * 78750a20196de5f01b01b59b1cd15c752bdee5c0 UNKNOWN
   * 64137c6034da191208680bbe89948d911eec6e52 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/151480044) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5848) 
   * 3f539fe185165e1071e3a54a8f781977a4872995 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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r386009061
 
 

 ##########
 File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestBase.java
 ##########
 @@ -18,102 +18,70 @@
 
 package org.apache.flink.kubernetes;
 
-import org.apache.flink.configuration.BlobServerOptions;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.ResourceManagerOptions;
-import org.apache.flink.configuration.TaskManagerOptions;
 import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
-import org.apache.flink.kubernetes.configuration.KubernetesConfigOptionsInternal;
 import org.apache.flink.kubernetes.kubeclient.Fabric8FlinkKubeClient;
 import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
-import org.apache.flink.kubernetes.kubeclient.decorators.Decorator;
-import org.apache.flink.kubernetes.kubeclient.decorators.InitializerDecorator;
-import org.apache.flink.kubernetes.kubeclient.decorators.OwnerReferenceDecorator;
-import org.apache.flink.kubernetes.kubeclient.decorators.ServiceDecorator;
-import org.apache.flink.kubernetes.kubeclient.resources.KubernetesService;
 import org.apache.flink.kubernetes.utils.Constants;
 import org.apache.flink.runtime.clusterframework.BootstrapTools;
 import org.apache.flink.test.util.TestBaseUtils;
 import org.apache.flink.util.TestLogger;
 
-import io.fabric8.kubernetes.api.model.LoadBalancerIngress;
-import io.fabric8.kubernetes.api.model.LoadBalancerStatus;
-import io.fabric8.kubernetes.api.model.Service;
-import io.fabric8.kubernetes.api.model.ServiceStatusBuilder;
-import io.fabric8.kubernetes.api.model.WatchEvent;
 import io.fabric8.kubernetes.client.KubernetesClient;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.rules.TemporaryFolder;
 
-import javax.annotation.Nullable;
-
 import java.io.File;
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 
 /**
  * Base test class for Kubernetes.
  */
 public class KubernetesTestBase extends TestLogger {
-	@Rule
-	public MixedKubernetesServer server = new MixedKubernetesServer(true, true);
-
-	@Rule
-	public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
-	private File flinkConfDir;
 
 	protected static final String NAMESPACE = "test";
-
-	protected static final Configuration FLINK_CONFIG = new Configuration();
-
 	protected static final String CLUSTER_ID = "my-flink-cluster1";
-
 	protected static final String CONTAINER_IMAGE = "flink-k8s-test:latest";
+	protected static final String CONTAINER_IMAGE_PULL_POLICY = "IfNotPresent";
 
-	protected static final String MOCK_SERVICE_ID = "mock-uuid-of-service";
+	@Rule
+	public MixedKubernetesServer server = new MixedKubernetesServer(true, true);
 
-	protected static final String MOCK_SERVICE_HOST_NAME = "mock-host-name-of-service";
+	@Rule
+	public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	protected File flinkConfDir;
 
-	protected static final String MOCK_SERVICE_IP = "192.168.0.1";
+	protected final Configuration flinkConfig = new Configuration();
 
-	protected static final String FLINK_MASTER_ENV_KEY = "LD_LIBRARY_PATH";
+	protected KubernetesClient kubeClient;
 
 Review comment:
   Fixed by [d78c1b1](https://github.com/apache/flink/pull/11233/commits/d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4)

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151292878",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151373127",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be UNKNOWN
   * abfef134eb5940d7831dab4c794d654c2fa70263 UNKNOWN
   * d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4 UNKNOWN
   * 78750a20196de5f01b01b59b1cd15c752bdee5c0 UNKNOWN
   * d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/151373127) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828) 
   
   <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


With regards,
Apache Git Services

[GitHub] [flink] wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r385630595
 
 

 ##########
 File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestBase.java
 ##########
 @@ -18,102 +18,70 @@
 
 package org.apache.flink.kubernetes;
 
-import org.apache.flink.configuration.BlobServerOptions;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.ResourceManagerOptions;
-import org.apache.flink.configuration.TaskManagerOptions;
 import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
-import org.apache.flink.kubernetes.configuration.KubernetesConfigOptionsInternal;
 import org.apache.flink.kubernetes.kubeclient.Fabric8FlinkKubeClient;
 import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
-import org.apache.flink.kubernetes.kubeclient.decorators.Decorator;
-import org.apache.flink.kubernetes.kubeclient.decorators.InitializerDecorator;
-import org.apache.flink.kubernetes.kubeclient.decorators.OwnerReferenceDecorator;
-import org.apache.flink.kubernetes.kubeclient.decorators.ServiceDecorator;
-import org.apache.flink.kubernetes.kubeclient.resources.KubernetesService;
 import org.apache.flink.kubernetes.utils.Constants;
 import org.apache.flink.runtime.clusterframework.BootstrapTools;
 import org.apache.flink.test.util.TestBaseUtils;
 import org.apache.flink.util.TestLogger;
 
-import io.fabric8.kubernetes.api.model.LoadBalancerIngress;
-import io.fabric8.kubernetes.api.model.LoadBalancerStatus;
-import io.fabric8.kubernetes.api.model.Service;
-import io.fabric8.kubernetes.api.model.ServiceStatusBuilder;
-import io.fabric8.kubernetes.api.model.WatchEvent;
 import io.fabric8.kubernetes.client.KubernetesClient;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.rules.TemporaryFolder;
 
-import javax.annotation.Nullable;
-
 import java.io.File;
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 
 /**
  * Base test class for Kubernetes.
  */
 public class KubernetesTestBase extends TestLogger {
-	@Rule
-	public MixedKubernetesServer server = new MixedKubernetesServer(true, true);
-
-	@Rule
-	public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
-	private File flinkConfDir;
 
 	protected static final String NAMESPACE = "test";
-
-	protected static final Configuration FLINK_CONFIG = new Configuration();
-
 	protected static final String CLUSTER_ID = "my-flink-cluster1";
-
 	protected static final String CONTAINER_IMAGE = "flink-k8s-test:latest";
+	protected static final String CONTAINER_IMAGE_PULL_POLICY = "IfNotPresent";
 
-	protected static final String MOCK_SERVICE_ID = "mock-uuid-of-service";
+	@Rule
+	public MixedKubernetesServer server = new MixedKubernetesServer(true, true);
 
-	protected static final String MOCK_SERVICE_HOST_NAME = "mock-host-name-of-service";
+	@Rule
+	public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	protected File flinkConfDir;
 
-	protected static final String MOCK_SERVICE_IP = "192.168.0.1";
+	protected final Configuration flinkConfig = new Configuration();
 
-	protected static final String FLINK_MASTER_ENV_KEY = "LD_LIBRARY_PATH";
+	protected KubernetesClient kubeClient;
 
 Review comment:
   Since we make `kubeClient` and `flinkKubeClient` as `protected`, these methods `getFabric8FlinkKubeClient(), getKubeClient()` could be removed now.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151292878",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be UNKNOWN
   * abfef134eb5940d7831dab4c794d654c2fa70263 UNKNOWN
   * d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4 UNKNOWN
   * 78750a20196de5f01b01b59b1cd15c752bdee5c0 UNKNOWN
   * b088e0e0530fa3178a547b26b7e950165181caaf Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/151326230) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r386255489
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/AbstractServiceDecorator.java
 ##########
 @@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.kubeclient.decorators;
+
+import org.apache.flink.configuration.RestOptions;
+import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters;
+import org.apache.flink.kubernetes.utils.Constants;
+
+import io.fabric8.kubernetes.api.model.HasMetadata;
+import io.fabric8.kubernetes.api.model.Service;
+import io.fabric8.kubernetes.api.model.ServiceBuilder;
+import io.fabric8.kubernetes.api.model.ServicePort;
+import io.fabric8.kubernetes.api.model.ServicePortBuilder;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * An abstract class containing some common implementations for the internal/external Services.
+ */
+public abstract class AbstractServiceDecorator extends AbstractKubernetesStepDecorator {
+
+	protected final KubernetesJobManagerParameters kubernetesJobManagerParameters;
+
+	public AbstractServiceDecorator(KubernetesJobManagerParameters kubernetesJobManagerParameters) {
+		this.kubernetesJobManagerParameters = checkNotNull(kubernetesJobManagerParameters);
+	}
+
+	@Override
+	public List<HasMetadata> buildAccompanyingKubernetesResources() throws IOException {
+		final Service service = new ServiceBuilder()
+			.withApiVersion(Constants.API_VERSION)
+			.withNewMetadata()
+				.withName(getServiceName())
+				.withLabels(kubernetesJobManagerParameters.getCommonLabels())
+				.endMetadata()
+			.withNewSpec()
+				.withType(getServiceType())
+				.withPorts(getServicePorts())
+				.withSelector(kubernetesJobManagerParameters.getLabels())
+				.endSpec()
+			.build();
+
+		return Collections.singletonList(service);
+	}
+
+	protected abstract String getServiceType();
+
+	protected abstract String getServiceName();
+
+	protected List<ServicePort> getServicePorts() {
 
 Review comment:
   I think it's good to keep the current implementation. BTW, these pieces of code would be deleted with another PR for clarifying the design of the so-called internal/external Service.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] KarmaGYZ commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
KarmaGYZ commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-593729602
 
 
   > It seems azure failed on mesos e2e cases. @KarmaGYZ do you have an idea or know an existing issue? (log: https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_apis/build/builds/5828/logs/118 )
   
   Root cause: FLINK-16288. Please rebase on the latest master branch.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r385585549
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecorator.java
 ##########
 @@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.kubeclient.decorators;
+
+import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+
+/**
+ * Creates an external Service to expose the ports of the Flink JobManager(s).
+ * This can include the rest port, and the blob server port.
 
 Review comment:
   Only the rest port needs to be exposed on external service.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151292878",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151373127",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be UNKNOWN
   * abfef134eb5940d7831dab4c794d654c2fa70263 UNKNOWN
   * d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4 UNKNOWN
   * 78750a20196de5f01b01b59b1cd15c752bdee5c0 UNKNOWN
   * d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/151373127) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828) 
   
   <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


With regards,
Apache Git Services

[GitHub] [flink] TisonKun commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
TisonKun commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r387857955
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkPod.java
 ##########
 @@ -0,0 +1,98 @@
+/*
+ * 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;
+
+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 static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A collection of variables that composes a JobManager/TaskManager Pod. This can include
+ * the Pod, the main Container, and the InitContainer, etc.
+ */
+public class FlinkPod {
+
+	private Pod pod;
+
+	private Container mainContainer;
+
+	public FlinkPod(Pod pod, Container mainContainer) {
+		this.pod = pod;
+		this.mainContainer = mainContainer;
+	}
+
+	public Pod getPod() {
+		return pod;
+	}
+
+	public void setPod(Pod pod) {
 
 Review comment:
   nit: you can edit the FlinkPod commit to make it a value class, i.e., `pod` and `mainContainer` are `final` and no getters.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r385994708
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkConfMountDecorator.java
 ##########
 @@ -0,0 +1,174 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.client.cli.CliFrontend;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.parameters.AbstractKubernetesParameters;
+
+import org.apache.flink.shaded.guava18.com.google.common.io.Files;
+
+import io.fabric8.kubernetes.api.model.ConfigMap;
+import io.fabric8.kubernetes.api.model.ConfigMapBuilder;
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.ContainerBuilder;
+import io.fabric8.kubernetes.api.model.HasMetadata;
+import io.fabric8.kubernetes.api.model.KeyToPath;
+import io.fabric8.kubernetes.api.model.KeyToPathBuilder;
+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 java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.configuration.GlobalConfiguration.FLINK_CONF_FILENAME;
+import static org.apache.flink.kubernetes.utils.Constants.CONFIG_FILE_LOG4J_NAME;
+import static org.apache.flink.kubernetes.utils.Constants.CONFIG_FILE_LOGBACK_NAME;
+import static org.apache.flink.kubernetes.utils.Constants.CONFIG_MAP_PREFIX;
+import static org.apache.flink.kubernetes.utils.Constants.FLINK_CONF_VOLUME;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Mounts the log4j.properties, logback.xml, and flink-conf.yaml configuration on the JobManager or TaskManager pod.
+ */
+public class FlinkConfMountDecorator extends AbstractKubernetesStepDecorator {
+
+	private final AbstractKubernetesParameters kubernetesComponentConf;
+
+	public FlinkConfMountDecorator(AbstractKubernetesParameters kubernetesComponentConf) {
+		this.kubernetesComponentConf = checkNotNull(kubernetesComponentConf);
+	}
+
+	@Override
+	public FlinkPod decorateFlinkPod(FlinkPod flinkPod) {
+		final Pod mountedPod = decoratePod(flinkPod.getPod());
+
+		final Container mountedMainContainer = new ContainerBuilder(flinkPod.getMainContainer())
+			.addNewVolumeMount()
+				.withName(FLINK_CONF_VOLUME)
+				.withMountPath(kubernetesComponentConf.getFlinkConfDirInPod())
+				.endVolumeMount()
+			.build();
+
+		return new FlinkPod.Builder(flinkPod)
+			.withPod(mountedPod)
+			.withMainContainer(mountedMainContainer)
+			.build();
+	}
+
+	private Pod decoratePod(Pod pod) {
+		final List<KeyToPath> keyToPaths = getLocalLogConfFiles().stream()
+			.map(file -> new KeyToPathBuilder()
+				.withKey(file.getName())
+				.withPath(file.getName())
+				.build())
+			.collect(Collectors.toList());
+		keyToPaths.add(new KeyToPathBuilder()
+			.withKey(FLINK_CONF_FILENAME)
+			.withPath(FLINK_CONF_FILENAME)
+			.build());
+
+		final Volume flinkConfVolume = new VolumeBuilder()
+			.withName(FLINK_CONF_VOLUME)
+			.withNewConfigMap()
+				.withName(getFlinkConfConfigMapName(kubernetesComponentConf.getClusterId()))
+				.withItems(keyToPaths)
+				.endConfigMap()
+			.build();
+
+		return new PodBuilder(pod)
+			.editSpec()
+				.addNewVolumeLike(flinkConfVolume)
+				.endVolume()
+				.endSpec()
+			.build();
+	}
+
+	@Override
+	public List<HasMetadata> buildAccompanyingKubernetesResources() throws IOException {
+		final String clusterId = kubernetesComponentConf.getClusterId();
+
+		final Map<String, String> data = new HashMap<>();
+
+		final List<File> localLogFiles = getLocalLogConfFiles();
+		for (File file : localLogFiles) {
+			data.put(file.getName(), Files.toString(file, StandardCharsets.UTF_8));
+		}
+		data.put(FLINK_CONF_FILENAME, getFlinkConfData(kubernetesComponentConf.getFlinkConfiguration()));
+
+		final ConfigMap flinkConfConfigMap = new ConfigMapBuilder()
+			.withNewMetadata()
+				.withName(getFlinkConfConfigMapName(clusterId))
+				.withLabels(kubernetesComponentConf.getCommonLabels())
+				.endMetadata()
+			.addToData(data)
+			.build();
+
+		return Collections.singletonList(flinkConfConfigMap);
+	}
+
+	@VisibleForTesting
+	String getFlinkConfData(Configuration configuration) throws IOException {
+		try (StringWriter sw = new StringWriter();
+			PrintWriter out = new PrintWriter(sw)) {
+			for (String key : configuration.keySet()) {
 
 Review comment:
   Fixed by [978c6f0](https://github.com/apache/flink/pull/11233/commits/978c6f00810a66a87325af6454e49e2f084e45be).

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r386216470
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
 ##########
 @@ -66,8 +70,52 @@ public Fabric8FlinkKubeClient(Configuration flinkConfig, KubernetesClient client
 	}
 
 	@Override
-	public void createTaskManagerPod(TaskManagerPodParameter parameter) {
-		// todo
+	public void createJobManagerComponent(KubernetesJobManagerSpecification kubernetesJMSpec) {
+		final Deployment deployment = kubernetesJMSpec.getDeployment();
+		final List<HasMetadata> accompanyingResources = kubernetesJMSpec.getAccompanyingResources();
+
+		// create Deployment
+		LOG.debug("Start to create deployment with spec {}", deployment.getSpec().toString());
+		final Deployment createdDeployment = this.internalClient
+			.apps()
+			.deployments()
+			.inNamespace(this.nameSpace)
+			.create(deployment);
+
+		// Note, we should use the server-side uid of the created Deployment for the OwnerReference.
+		setOwnerReference(createdDeployment, accompanyingResources);
 
 Review comment:
   The e2e test  `test_kubernetes_session.sh` also needs to be updated.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r385996299
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactory.java
 ##########
 @@ -0,0 +1,109 @@
+/*
+ * 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.factory;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.KubernetesJobManagerSpecification;
+import org.apache.flink.kubernetes.kubeclient.decorators.ExternalServiceDecorator;
+import org.apache.flink.kubernetes.kubeclient.decorators.FlinkConfMountDecorator;
+import org.apache.flink.kubernetes.kubeclient.decorators.InitJobManagerDecorator;
+import org.apache.flink.kubernetes.kubeclient.decorators.InternalServiceDecorator;
+import org.apache.flink.kubernetes.kubeclient.decorators.JavaCmdJobManagerDecorator;
+import org.apache.flink.kubernetes.kubeclient.decorators.KubernetesStepDecorator;
+import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters;
+import org.apache.flink.kubernetes.utils.Constants;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.HasMetadata;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.PodBuilder;
+import io.fabric8.kubernetes.api.model.apps.Deployment;
+import io.fabric8.kubernetes.api.model.apps.DeploymentBuilder;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Utility class for constructing all the Kubernetes components on the client-side. This can
+ * include the Deployment, the ConfigMap(s), and the Service(s).
+ */
+public class KubernetesJobManagerFactory {
+
+	public static KubernetesJobManagerSpecification createJobManagerComponent(
+			KubernetesJobManagerParameters kubernetesJobManagerParameters) throws IOException {
+		FlinkPod flinkPod = new FlinkPod.Builder().build();
+		List<HasMetadata> accompanyingResources = new ArrayList<>();
+
+		final KubernetesStepDecorator[] stepDecorators = new KubernetesStepDecorator[] {
+			new InitJobManagerDecorator(kubernetesJobManagerParameters),
+			new JavaCmdJobManagerDecorator(kubernetesJobManagerParameters),
+			new InternalServiceDecorator(kubernetesJobManagerParameters),
+			new ExternalServiceDecorator(kubernetesJobManagerParameters),
+			new FlinkConfMountDecorator(kubernetesJobManagerParameters)};
+
+		for (KubernetesStepDecorator stepDecorator: stepDecorators) {
+			flinkPod = stepDecorator.decorateFlinkPod(flinkPod);
+			accompanyingResources.addAll(stepDecorator.buildAccompanyingKubernetesResources());
+		}
+
+		final Deployment deployment = createJobManagerDeployment(flinkPod, kubernetesJobManagerParameters);
+
+		KubernetesUtils.setOwnerReference(deployment, accompanyingResources);
+
+		return new KubernetesJobManagerSpecification(deployment, accompanyingResources);
+	}
+
+	private static Deployment createJobManagerDeployment(
+			FlinkPod flinkPod,
+			KubernetesJobManagerParameters kubernetesJobManagerParameters) {
+		final Container resolvedMainContainer = flinkPod.getMainContainer();
+
+		final Pod resolvedPod = new PodBuilder(flinkPod.getPod())
+			.editOrNewSpec()
+				.addToContainers(resolvedMainContainer)
+				.endSpec()
+			.build();
+
+		final Map<String, String> labels = resolvedPod.getMetadata().getLabels();
 
 Review comment:
   The basic logic is:
   1. We use the Pod object built by the decorators to fill the pod template of the deployment.
   2. Then set the deployment.metadata.labels separately to help others realize that the labels of the deployment could be different from the labels of the underlying pod.
   
   For the usual case, the labels could be the same. For this PR,  resolvedPod.getMetadata().getLabels() and kubernetesJobManagerParameters.getLabels() shares the same values. 
   
   So I think it is reasonable to use the resolvedPod.getMetadata().getLabels to fill the spec of the deployment.
   
   For the deployment metadata, maybe we could introduce a private wrapper method in KubernetesJobManagerFactory to get the deployment labels.
   
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] TisonKun closed pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
TisonKun closed pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233
 
 
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151292878",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151373127",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151373127",
       "triggerID" : "593715190",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828",
       "triggerID" : "593715190",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be UNKNOWN
   * abfef134eb5940d7831dab4c794d654c2fa70263 UNKNOWN
   * d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4 UNKNOWN
   * 78750a20196de5f01b01b59b1cd15c752bdee5c0 UNKNOWN
   * d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/151373127) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828) 
   
   <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


With regards,
Apache Git Services

[GitHub] [flink] wangyang0918 commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-593246987
 
 
   ```
   echo 'stop' | ./bin/kubernetes-session.sh -Dkubernetes.cluster-id=flink-native-k8s-session-1 -Dexecution.attached=true
   ```
   Probably because we do not include `io.fabric8:zjsonpatch` in the pom.xml of `flink-kubernetes` module.
   
   ```
   2020-03-02 14:30:32,386 INFO  org.apache.flink.kubernetes.KubernetesClusterDescriptor      [] - Retrieve flink cluster flink-native-k8s-session-1 successfully, JobManager Web Interface: http://11.164.91.5:31318
   Exception in thread "main" java.lang.NoClassDefFoundError: io/fabric8/zjsonpatch/JsonDiff
   	at io.fabric8.kubernetes.client.dsl.base.OperationSupport.handlePatch(OperationSupport.java:297)
   	at io.fabric8.kubernetes.client.dsl.base.BaseOperation.handlePatch(BaseOperation.java:808)
   	at io.fabric8.kubernetes.client.dsl.base.HasMetadataOperation.lambda$patch$2(HasMetadataOperation.java:145)
   	at io.fabric8.kubernetes.api.model.apps.DoneableDeployment.done(DoneableDeployment.java:27)
   	at io.fabric8.kubernetes.api.model.apps.DoneableDeployment.done(DoneableDeployment.java:6)
   	at io.fabric8.kubernetes.client.dsl.base.HasMetadataOperation.patch(HasMetadataOperation.java:151)
   	at io.fabric8.kubernetes.client.dsl.internal.RollableScalableResourceOperation.patch(RollableScalableResourceOperation.java:167)
   	at io.fabric8.kubernetes.client.dsl.internal.DeploymentOperationsImpl.patch(DeploymentOperationsImpl.java:113)
   	at io.fabric8.kubernetes.client.dsl.internal.DeploymentOperationsImpl.patch(DeploymentOperationsImpl.java:45)
   	at io.fabric8.kubernetes.client.dsl.base.HasMetadataOperation.lambda$edit$0(HasMetadataOperation.java:53)
   	at io.fabric8.kubernetes.api.model.apps.DoneableDeployment.done(DoneableDeployment.java:27)
   	at io.fabric8.kubernetes.client.dsl.internal.DeploymentOperationsImpl$DeploymentReaper.reap(DeploymentOperationsImpl.java:245)
   	at io.fabric8.kubernetes.client.dsl.base.BaseOperation.delete(BaseOperation.java:642)
   	at io.fabric8.kubernetes.client.dsl.base.BaseOperation.delete(BaseOperation.java:63)
   	at org.apache.flink.kubernetes.kubeclient.Fabric8FlinkKubeClient.stopAndCleanupCluster(Fabric8FlinkKubeClient.java:182)
   	at org.apache.flink.kubernetes.KubernetesClusterDescriptor.killCluster(KubernetesClusterDescriptor.java:193)
   	at org.apache.flink.kubernetes.KubernetesClusterDescriptor.killCluster(KubernetesClusterDescriptor.java:59)
   	at org.apache.flink.kubernetes.cli.KubernetesSessionCli.run(KubernetesSessionCli.java:125)
   	at org.apache.flink.kubernetes.cli.KubernetesSessionCli.lambda$main$0(KubernetesSessionCli.java:185)
   	at org.apache.flink.runtime.security.contexts.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:30)
   	at org.apache.flink.kubernetes.cli.KubernetesSessionCli.main(KubernetesSessionCli.java:185)
   Caused by: java.lang.ClassNotFoundException: io.fabric8.zjsonpatch.JsonDiff
   	at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
   	at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
   	at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:335)
   	at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
   	... 21 more
   ```

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r386210683
 
 

 ##########
 File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactoryTest.java
 ##########
 @@ -0,0 +1,203 @@
+/*
+ * 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.factory;
+
+import org.apache.flink.kubernetes.KubernetesTestUtils;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptionsInternal;
+import org.apache.flink.kubernetes.entrypoint.KubernetesSessionClusterEntrypoint;
+import org.apache.flink.kubernetes.kubeclient.KubernetesJobManagerSpecification;
+import org.apache.flink.kubernetes.kubeclient.KubernetesJobManagerTestBase;
+import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters;
+import org.apache.flink.kubernetes.utils.Constants;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+
+import io.fabric8.kubernetes.api.model.ConfigMap;
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.HasMetadata;
+import io.fabric8.kubernetes.api.model.PodSpec;
+import io.fabric8.kubernetes.api.model.Quantity;
+import io.fabric8.kubernetes.api.model.Service;
+import io.fabric8.kubernetes.api.model.apps.Deployment;
+import io.fabric8.kubernetes.api.model.apps.DeploymentSpec;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.configuration.GlobalConfiguration.FLINK_CONF_FILENAME;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * General tests for the {@link KubernetesJobManagerFactory}.
+ */
+public class KubernetesJobManagerFactoryTest extends KubernetesJobManagerTestBase {
+
+	private static final String SERVICE_ACCOUNT_NAME = "service-test";
+	private static final String ENTRY_POINT_CLASS = KubernetesSessionClusterEntrypoint.class.getCanonicalName();
+
+	private KubernetesJobManagerSpecification kubernetesJobManagerSpecification;
+
+	@Before
+	public void setup() throws Exception {
+		super.setup();
+
+		KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "logback.xml");
+		KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "log4j.properties");
+
+		flinkConfig.set(KubernetesConfigOptionsInternal.ENTRY_POINT_CLASS, ENTRY_POINT_CLASS);
+		flinkConfig.set(KubernetesConfigOptions.JOB_MANAGER_SERVICE_ACCOUNT, SERVICE_ACCOUNT_NAME);
+
+		this.kubernetesJobManagerSpecification =
+			KubernetesJobManagerFactory.createJobManagerComponent(kubernetesJobManagerParameters);
+	}
+
+	@Test
+	public void testDeploymentMetadata() {
+		final Deployment resultDeployment = this.kubernetesJobManagerSpecification.getDeployment();
+		assertEquals(Constants.APPS_API_VERSION, resultDeployment.getApiVersion());
+		assertEquals(KubernetesUtils.getDeploymentName(CLUSTER_ID), resultDeployment.getMetadata().getName());
+		final Map<String, String> expectedLabels = getCommonLabels();
+		expectedLabels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_JOB_MANAGER);
+		assertEquals(expectedLabels, resultDeployment.getMetadata().getLabels());
+	}
+
+	@Test
+	public void testDeploymentSpec() {
+		final DeploymentSpec resultDeploymentSpec = this.kubernetesJobManagerSpecification.getDeployment().getSpec();
+		assertEquals(1, resultDeploymentSpec.getReplicas().intValue());
+
+		final Map<String, String> expectedLabels =  new HashMap<>(getCommonLabels());
+		expectedLabels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_JOB_MANAGER);
+
+		assertEquals(expectedLabels, resultDeploymentSpec.getTemplate().getMetadata().getLabels());
+		assertEquals(expectedLabels, resultDeploymentSpec.getSelector().getMatchLabels());
+
+		assertNotNull(resultDeploymentSpec.getTemplate().getSpec());
+	}
+
+	@Test
+	public void testPodSpec() {
+		final PodSpec resultPodSpec =
+			this.kubernetesJobManagerSpecification.getDeployment().getSpec().getTemplate().getSpec();
+
+		assertEquals(1, resultPodSpec.getContainers().size());
+		assertEquals(SERVICE_ACCOUNT_NAME, resultPodSpec.getServiceAccountName());
+		assertEquals(1, resultPodSpec.getVolumes().size());
+
+		final Container resultedMainContainer = resultPodSpec.getContainers().get(0);
+		assertEquals(KubernetesJobManagerParameters.JOB_MANAGER_MAIN_CONTAINER_NAME, resultedMainContainer.getName());
+		assertEquals(CONTAINER_IMAGE, resultedMainContainer.getImage());
+		assertEquals(CONTAINER_IMAGE_PULL_POLICY, resultedMainContainer.getImagePullPolicy());
+
+		assertEquals(3, resultedMainContainer.getEnv().size());
+		assertTrue(resultedMainContainer.getEnv()
+				.stream()
+				.anyMatch(envVar -> envVar.getName().equals("key1")));
+
+		assertEquals(3, resultedMainContainer.getPorts().size());
+
+		final Map<String, Quantity> requests = resultedMainContainer.getResources().getRequests();
+		assertEquals(Double.toString(JOB_MANAGER_CPU), requests.get("cpu").getAmount());
+		assertEquals(JOB_MANAGER_MEMORY + "Mi", requests.get("memory").getAmount());
+
+		assertEquals(1, resultedMainContainer.getCommand().size());
+		assertEquals(3, resultedMainContainer.getArgs().size());
+
+		assertEquals(1, resultedMainContainer.getVolumeMounts().size());
+	}
+
+	@Test
+	public void testAdditionalResourcesSize() {
+		final List<HasMetadata> resultAdditionalResources = this.kubernetesJobManagerSpecification.getAccompanyingResources();
+		assertEquals(3, resultAdditionalResources.size());
+
+		final List<HasMetadata> resultServices = resultAdditionalResources
+			.stream()
+			.filter(x -> x instanceof Service)
+			.collect(Collectors.toList());
+		assertEquals(2, resultServices.size());
+
+		final List<HasMetadata> resultConfigMaps = resultAdditionalResources
+			.stream()
+			.filter(x -> x instanceof ConfigMap)
+			.collect(Collectors.toList());
+		assertEquals(1, resultConfigMaps.size());
+	}
+
+	@Test
+	public void testServices() {
+		final List<Service> resultServices = this.kubernetesJobManagerSpecification.getAccompanyingResources()
+			.stream()
+			.filter(x -> x instanceof Service)
+			.map(x -> (Service) x)
+			.collect(Collectors.toList());
+
+		assertEquals(2, resultServices.size());
+
+		final List<Service> internalServiceCandidates = resultServices
+				.stream()
+				.filter(x -> x.getMetadata().getName().equals(KubernetesUtils.getInternalServiceName(CLUSTER_ID)))
+				.collect(Collectors.toList());
+		assertEquals(1, internalServiceCandidates.size());
+
+		final List<Service> restServiceCandidates = resultServices
+				.stream()
+				.filter(x -> x.getMetadata().getName().equals(KubernetesUtils.getRestServiceName(CLUSTER_ID)))
+				.collect(Collectors.toList());
+		assertEquals(1, restServiceCandidates.size());
+
+		final Service resultInternalService = internalServiceCandidates.get(0);
+		assertEquals(2, resultInternalService.getMetadata().getLabels().size());
 
 Review comment:
   Maybe we could use `getCommonLabels()` to assert the real labels, not only check the size. I think other `getLabels().size()` is in the same situation.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151292878",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151373127",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151373127",
       "triggerID" : "593715190",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828",
       "triggerID" : "593715190",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151475534",
       "triggerID" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5846",
       "triggerID" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64137c6034da191208680bbe89948d911eec6e52",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151480044",
       "triggerID" : "64137c6034da191208680bbe89948d911eec6e52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64137c6034da191208680bbe89948d911eec6e52",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5848",
       "triggerID" : "64137c6034da191208680bbe89948d911eec6e52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f539fe185165e1071e3a54a8f781977a4872995",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5916",
       "triggerID" : "3f539fe185165e1071e3a54a8f781977a4872995",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f539fe185165e1071e3a54a8f781977a4872995",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151723903",
       "triggerID" : "3f539fe185165e1071e3a54a8f781977a4872995",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0072b0b681bfb099d135c59152dd1f61bca8b34",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f0072b0b681bfb099d135c59152dd1f61bca8b34",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be UNKNOWN
   * abfef134eb5940d7831dab4c794d654c2fa70263 UNKNOWN
   * d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4 UNKNOWN
   * 78750a20196de5f01b01b59b1cd15c752bdee5c0 UNKNOWN
   * 64137c6034da191208680bbe89948d911eec6e52 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/151480044) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5848) 
   * 3f539fe185165e1071e3a54a8f781977a4872995 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/151723903) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5916) 
   * f0072b0b681bfb099d135c59152dd1f61bca8b34 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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 19e32e6bca1a90f608010fdff1236fdfdbb53e3b Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/151155366) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765) 
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be UNKNOWN
   * abfef134eb5940d7831dab4c794d654c2fa70263 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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151292878",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151373127",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151373127",
       "triggerID" : "593715190",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828",
       "triggerID" : "593715190",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151475534",
       "triggerID" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5846",
       "triggerID" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64137c6034da191208680bbe89948d911eec6e52",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151480044",
       "triggerID" : "64137c6034da191208680bbe89948d911eec6e52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64137c6034da191208680bbe89948d911eec6e52",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5848",
       "triggerID" : "64137c6034da191208680bbe89948d911eec6e52",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be UNKNOWN
   * abfef134eb5940d7831dab4c794d654c2fa70263 UNKNOWN
   * d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4 UNKNOWN
   * 78750a20196de5f01b01b59b1cd15c752bdee5c0 UNKNOWN
   * afdf9578347a7782b8ee40925c04dbe00ddc9415 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/151475534) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5846) 
   * 64137c6034da191208680bbe89948d911eec6e52 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/151480044) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5848) 
   
   <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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r385992323
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/KubernetesStepDecorator.java
 ##########
 @@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.kubeclient.decorators;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+
+import io.fabric8.kubernetes.api.model.HasMetadata;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * A set of functions that together represent a feature in pods that are deployed for
+ * the JobManager(s) or the TaskManager(s), which provides an extension to the way the
+ * given Flink application works.
+ */
+public interface KubernetesStepDecorator {
+
+	/**
+	 * Apply transformations to the given FlinkPod in accordance with this feature. This can include adding
+	 * labels/annotations, mounting volumes, and setting startup command or parameters, etc.
+	 */
+	FlinkPod decorateFlinkPod(FlinkPod flinkPod);
+
+	/**
+	 * Build the accompanying Kubernetes resources that should be introduced to support this feature. This could
+	 * only applicable on the client-side submission process.
 
 Review comment:
   That's right.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-593732437
 
 
   > > It seems azure failed on mesos e2e cases. @KarmaGYZ do you have an idea or know an existing issue? (log: https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_apis/build/builds/5828/logs/118 )
   > 
   > Root cause: FLINK-16288. Please rebase on the latest master branch.
   
   @KarmaGYZ Thanks for the information.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-593792257
 
 
   > > @TisonKun Do you think we need to reorganize the commit message or squash them when merging? Currently, i think there are too many `hotfix` to address the comments or previous commits.
   > 
   > Yes it would be nice that if @zhengcanbin can reorganize the commits and squash some intermediate hotfixes. I tend to avoid merging this PR as a "squash-and-merge" monolithic one or such a series of handy fixups.
   > 
   > Or I will try to reorganize the commits on merging.
   
   Thanks! I will reorganize the commits later to remove the handy fixups.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151292878",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151373127",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be UNKNOWN
   * abfef134eb5940d7831dab4c794d654c2fa70263 UNKNOWN
   * d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4 UNKNOWN
   * 78750a20196de5f01b01b59b1cd15c752bdee5c0 UNKNOWN
   * b088e0e0530fa3178a547b26b7e950165181caaf Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/151326230) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809) 
   * d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/151373127) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828) 
   
   <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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r385993430
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecorator.java
 ##########
 @@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.kubeclient.decorators;
+
+import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+
+/**
+ * Creates an external Service to expose the ports of the Flink JobManager(s).
+ * This can include the rest port, and the blob server port.
 
 Review comment:
   Fixed by [0017b8a](https://github.com/apache/flink/pull/11233/commits/0017b8a8930024d0e9ce3355d050565a5007831d)

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r385612691
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
 ##########
 @@ -66,8 +70,52 @@ public Fabric8FlinkKubeClient(Configuration flinkConfig, KubernetesClient client
 	}
 
 	@Override
-	public void createTaskManagerPod(TaskManagerPodParameter parameter) {
-		// todo
+	public void createJobManagerComponent(KubernetesJobManagerSpecification kubernetesJMSpec) {
+		final Deployment deployment = kubernetesJMSpec.getDeployment();
+		final List<HasMetadata> accompanyingResources = kubernetesJMSpec.getAccompanyingResources();
+
+		// create Deployment
+		LOG.debug("Start to create deployment with spec {}", deployment.getSpec().toString());
+		final Deployment createdDeployment = this.internalClient
 
 Review comment:
   We create the deployment first, then the accompanyingResources(e.g. configmap, service) are created. Does it have chance that the deployment is created with failure? Because the it needs to mount an nonexistent `ConfigMap`.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fb57917227853d0477aa1383d399a619146d7170 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/150793256) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676) 
   
   <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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fb57917227853d0477aa1383d399a619146d7170 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/150793256) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676) 
   
   <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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r385992882
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/AbstractKubernetesStepDecorator.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 io.fabric8.kubernetes.api.model.HasMetadata;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * An abstract {@link KubernetesStepDecorator} contains common implementations for different plug-in features
+ * while providing two additional methods.
 
 Review comment:
   Fixed by [19e32e6](https://github.com/apache/flink/pull/11233/commits/19e32e6bca1a90f608010fdff1236fdfdbb53e3b).

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r386009089
 
 

 ##########
 File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesClusterDescriptorTest.java
 ##########
 @@ -147,4 +173,30 @@ public void testKillCluster() throws Exception {
 
 		return clusterClient;
 	}
+
+	private void mockRestServiceWithLoadBalancer(@Nullable String hostname, @Nullable String ip) {
+		final String restServiceName = KubernetesUtils.getRestServiceName(CLUSTER_ID);
+
+		final String path = String.format("/api/v1/namespaces/%s/services/%s", NAMESPACE, restServiceName);
+		server.expect()
+			.withPath(path)
+			.andReturn(200, buildMockRestService(hostname, ip))
+			.always();
+	}
+
+	private Service buildMockRestService(@Nullable String hostname, @Nullable String ip) {
 
 Review comment:
   Fixed by [d78c1b1](https://github.com/apache/flink/pull/11233/commits/d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4)

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151292878",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be UNKNOWN
   * abfef134eb5940d7831dab4c794d654c2fa70263 UNKNOWN
   * d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4 UNKNOWN
   * 78750a20196de5f01b01b59b1cd15c752bdee5c0 UNKNOWN
   * 08f07deb2243554d24e2e4171e6cb23a5f934cc8 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/151292878) Azure: [CANCELED](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791) 
   * b088e0e0530fa3178a547b26b7e950165181caaf Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/151326230) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151292878",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151373127",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151373127",
       "triggerID" : "593715190",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828",
       "triggerID" : "593715190",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be UNKNOWN
   * abfef134eb5940d7831dab4c794d654c2fa70263 UNKNOWN
   * d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4 UNKNOWN
   * 78750a20196de5f01b01b59b1cd15c752bdee5c0 UNKNOWN
   * d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/151373127) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828) 
   * afdf9578347a7782b8ee40925c04dbe00ddc9415 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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151292878",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151373127",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151373127",
       "triggerID" : "593715190",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5828",
       "triggerID" : "593715190",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151475534",
       "triggerID" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5846",
       "triggerID" : "afdf9578347a7782b8ee40925c04dbe00ddc9415",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64137c6034da191208680bbe89948d911eec6e52",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151480044",
       "triggerID" : "64137c6034da191208680bbe89948d911eec6e52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64137c6034da191208680bbe89948d911eec6e52",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5848",
       "triggerID" : "64137c6034da191208680bbe89948d911eec6e52",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f539fe185165e1071e3a54a8f781977a4872995",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5916",
       "triggerID" : "3f539fe185165e1071e3a54a8f781977a4872995",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f539fe185165e1071e3a54a8f781977a4872995",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151723903",
       "triggerID" : "3f539fe185165e1071e3a54a8f781977a4872995",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0072b0b681bfb099d135c59152dd1f61bca8b34",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f0072b0b681bfb099d135c59152dd1f61bca8b34",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be UNKNOWN
   * abfef134eb5940d7831dab4c794d654c2fa70263 UNKNOWN
   * d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4 UNKNOWN
   * 78750a20196de5f01b01b59b1cd15c752bdee5c0 UNKNOWN
   * 3f539fe185165e1071e3a54a8f781977a4872995 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/151723903) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5916) 
   * f0072b0b681bfb099d135c59152dd1f61bca8b34 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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5791",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151292878",
       "triggerID" : "08f07deb2243554d24e2e4171e6cb23a5f934cc8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151326230",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b088e0e0530fa3178a547b26b7e950165181caaf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809",
       "triggerID" : "593317405",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be UNKNOWN
   * abfef134eb5940d7831dab4c794d654c2fa70263 UNKNOWN
   * d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4 UNKNOWN
   * 78750a20196de5f01b01b59b1cd15c752bdee5c0 UNKNOWN
   * b088e0e0530fa3178a547b26b7e950165181caaf Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/151326230) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5809) 
   * d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9 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


With regards,
Apache Git Services

[GitHub] [flink] wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r385570563
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/KubernetesStepDecorator.java
 ##########
 @@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.kubeclient.decorators;
+
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+
+import io.fabric8.kubernetes.api.model.HasMetadata;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * A set of functions that together represent a feature in pods that are deployed for
+ * the JobManager(s) or the TaskManager(s), which provides an extension to the way the
+ * given Flink application works.
+ */
+public interface KubernetesStepDecorator {
+
+	/**
+	 * Apply transformations to the given FlinkPod in accordance with this feature. This can include adding
+	 * labels/annotations, mounting volumes, and setting startup command or parameters, etc.
+	 */
+	FlinkPod decorateFlinkPod(FlinkPod flinkPod);
+
+	/**
+	 * Build the accompanying Kubernetes resources that should be introduced to support this feature. This could
+	 * only applicable on the client-side submission process.
 
 Review comment:
   I assume the "client-side submission process" means Flink creates the K8s resource on client side. Right?

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-593715190
 
 
   @flinkbot run travis

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r386010368
 
 

 ##########
 File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesClusterDescriptorTest.java
 ##########
 @@ -124,13 +144,19 @@ public void testKillCluster() throws Exception {
 		descriptor.deploySessionCluster(clusterSpecification);
 
 		final KubernetesClient kubeClient = server.getClient();
-		assertEquals(2, kubeClient.services().list().getItems().size());
+
 
 Review comment:
   Actually we mock the request of **ANY** kind, not only the **GET**.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r386286836
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/JavaCmdTaskManagerDecorator.java
 ##########
 @@ -0,0 +1,109 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.kubernetes.kubeclient.FlinkPod;
+import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesTaskManagerParameters;
+import org.apache.flink.kubernetes.taskmanager.KubernetesTaskExecutorRunner;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils;
+import org.apache.flink.runtime.entrypoint.parser.CommandLineOptions;
+
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.ContainerBuilder;
+
+import javax.annotation.Nullable;
+
+import java.util.Arrays;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Attach the command and args to the main container for running the TaskManager code.
+ */
+public class JavaCmdTaskManagerDecorator extends AbstractKubernetesStepDecorator {
+
+	private final KubernetesTaskManagerParameters kubernetesTaskManagerParameters;
+
+	public JavaCmdTaskManagerDecorator(KubernetesTaskManagerParameters kubernetesTaskManagerParameters) {
+		this.kubernetesTaskManagerParameters = checkNotNull(kubernetesTaskManagerParameters);
+	}
+
+	@Override
+	public FlinkPod decorateFlinkPod(FlinkPod flinkPod) {
+		final Container mainContainerWithStartCmd = new ContainerBuilder(flinkPod.getMainContainer())
+			.withCommand(kubernetesTaskManagerParameters.getContainerEntrypoint())
+			.withArgs(Arrays.asList("/bin/bash", "-c", getTaskManagerStartCommand()))
+			.build();
+
+		return new FlinkPod.Builder(flinkPod)
+			.withMainContainer(mainContainerWithStartCmd)
+			.build();
+	}
+
+	private String getTaskManagerStartCommand() {
+		final String confDirInPod = kubernetesTaskManagerParameters.getFlinkConfDirInPod();
+
+		final String logDirInPod = kubernetesTaskManagerParameters.getFlinkLogDirInPod();
+
+		final String mainClassArgs = "--" + CommandLineOptions.CONFIG_DIR_OPTION.getLongOpt() + " " +
+			confDirInPod + " " + kubernetesTaskManagerParameters.getDynamicProperties();
+
+		return getTaskManagerStartCommand(
+			kubernetesTaskManagerParameters.getFlinkConfiguration(),
+			kubernetesTaskManagerParameters.getContaineredTaskManagerParameters(),
+			confDirInPod,
+			logDirInPod,
+			kubernetesTaskManagerParameters.hasLogback(),
+			kubernetesTaskManagerParameters.hasLog4j(),
+			KubernetesTaskExecutorRunner.class.getCanonicalName(),
+			mainClassArgs);
+	}
+
+	private static String getTaskManagerStartCommand(
+			Configuration flinkConfig,
+			ContaineredTaskManagerParameters tmParams,
+			String configDirectory,
+			String logDirectory,
+			boolean hasLogback,
+			boolean hasLog4j,
+			String mainClass,
+			@Nullable String mainArgs) {
 
 Review comment:
   Fixed by [b088e0e](https://github.com/apache/flink/pull/11233/commits/b088e0e0530fa3178a547b26b7e950165181caaf)

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r386198450
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/AbstractServiceDecorator.java
 ##########
 @@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.kubeclient.decorators;
+
+import org.apache.flink.configuration.RestOptions;
+import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters;
+import org.apache.flink.kubernetes.utils.Constants;
+
+import io.fabric8.kubernetes.api.model.HasMetadata;
+import io.fabric8.kubernetes.api.model.Service;
+import io.fabric8.kubernetes.api.model.ServiceBuilder;
+import io.fabric8.kubernetes.api.model.ServicePort;
+import io.fabric8.kubernetes.api.model.ServicePortBuilder;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * An abstract class containing some common implementations for the internal/external Services.
+ */
+public abstract class AbstractServiceDecorator extends AbstractKubernetesStepDecorator {
+
+	protected final KubernetesJobManagerParameters kubernetesJobManagerParameters;
+
+	public AbstractServiceDecorator(KubernetesJobManagerParameters kubernetesJobManagerParameters) {
+		this.kubernetesJobManagerParameters = checkNotNull(kubernetesJobManagerParameters);
+	}
+
+	@Override
+	public List<HasMetadata> buildAccompanyingKubernetesResources() throws IOException {
+		final Service service = new ServiceBuilder()
+			.withApiVersion(Constants.API_VERSION)
+			.withNewMetadata()
+				.withName(getServiceName())
+				.withLabels(kubernetesJobManagerParameters.getCommonLabels())
+				.endMetadata()
+			.withNewSpec()
+				.withType(getServiceType())
+				.withPorts(getServicePorts())
+				.withSelector(kubernetesJobManagerParameters.getLabels())
+				.endSpec()
+			.build();
+
+		return Collections.singletonList(service);
+	}
+
+	protected abstract String getServiceType();
+
+	protected abstract String getServiceName();
+
+	protected List<ServicePort> getServicePorts() {
 
 Review comment:
   Just like @TisonKun said, maybe we could also make `getServicePorts()` as `abstract` and leave different implementation to derived classes.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-593316748
 
 
   > @zhengcanbin Thanks a lot for addressing the comments. I have gone though again and the changes look really good to me. I just leave few minor comments.
   > 
   > For the real K8s cluster tests.
   > 
   >     * Non-HA session cluster
   >       
   >       * deploy [SUCCEED]
   >       * submit Flink job via `CLI` [SUCCEED]
   >       * submit Flink job via webui [SUCCEED]
   >       * cancel all jobs and taskmanager released by idle [SUCCEED]
   >       * stop via `kubernetes-session.sh` [FAILED]
   >       * stop via `kubectl delete deployment/<Cluster_ID>` [SUCCEED]
   > 
   >     * HA session cluster
   >       
   >       * Same operations as above [SUCCEED]
   >       * Kill taskmanager, job failover and resume successfully [SUCCEED]
   >       * Kill jobmanager, job failover and resume successfully [SUCCEED]
   
   @wangyang0918 Thanks for the testing work and all the comments. I have resolved the new comments via commit [b088e0e](https://github.com/apache/flink/pull/11233/commits/b088e0e0530fa3178a547b26b7e950165181caaf).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] TisonKun commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
TisonKun commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r387861569
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkPod.java
 ##########
 @@ -0,0 +1,98 @@
+/*
+ * 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;
+
+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 static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A collection of variables that composes a JobManager/TaskManager Pod. This can include
+ * the Pod, the main Container, and the InitContainer, etc.
+ */
+public class FlinkPod {
+
+	private Pod pod;
+
+	private Container mainContainer;
+
+	public FlinkPod(Pod pod, Container mainContainer) {
+		this.pod = pod;
+		this.mainContainer = mainContainer;
+	}
+
+	public Pod getPod() {
+		return pod;
+	}
+
+	public void setPod(Pod pod) {
+		this.pod = pod;
+	}
+
+	public Container getMainContainer() {
+		return mainContainer;
+	}
+
+	public void setMainContainer(Container mainContainer) {
+		this.mainContainer = mainContainer;
+	}
+
+	/**
+	 * Builder for creating a {@link FlinkPod}.
+	 */
+	public static class Builder {
+
+		private Pod pod;
+		private Container mainContainer;
+
+		public Builder() {
+			this.pod = new PodBuilder()
+				.withNewMetadata()
+				.endMetadata()
+				.withNewSpec()
+				.endSpec()
+				.build();
+
+			this.mainContainer = new ContainerBuilder().build();
+		}
+
+		public Builder(FlinkPod flinkPod) {
 
 Review comment:
   As the document of `AbstractKubernetesStepDecorator`, I'm ok that this Builder follows  fabric8's Builder pattern. However, Flink has its own Builder pattern and even a [style guide](http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/CODE-STYLE-Builder-pattern-tt32225.html) for it. We can defer the change decision and leave as is now.
   
   cc @gyfora seems we forget adding the code style of Builder pattern in flink document.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-591843120
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150793256",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fb57917227853d0477aa1383d399a619146d7170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5676",
       "triggerID" : "fb57917227853d0477aa1383d399a619146d7170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150962647",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5717",
       "triggerID" : "786a5056c957863c05ad24b00ca1dca032905eb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151155366",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765",
       "triggerID" : "19e32e6bca1a90f608010fdff1236fdfdbb53e3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0017b8a8930024d0e9ce3355d050565a5007831d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "978c6f00810a66a87325af6454e49e2f084e45be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "abfef134eb5940d7831dab4c794d654c2fa70263",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "78750a20196de5f01b01b59b1cd15c752bdee5c0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 19e32e6bca1a90f608010fdff1236fdfdbb53e3b Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/151155366) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5765) 
   * 0017b8a8930024d0e9ce3355d050565a5007831d UNKNOWN
   * 978c6f00810a66a87325af6454e49e2f084e45be UNKNOWN
   * abfef134eb5940d7831dab4c794d654c2fa70263 UNKNOWN
   * d78c1b1b282a3d7fcfc797bf909dca2b5f2264e4 UNKNOWN
   * 78750a20196de5f01b01b59b1cd15c752bdee5c0 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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on a change in pull request #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#discussion_r386007422
 
 

 ##########
 File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
 ##########
 @@ -66,8 +70,52 @@ public Fabric8FlinkKubeClient(Configuration flinkConfig, KubernetesClient client
 	}
 
 	@Override
-	public void createTaskManagerPod(TaskManagerPodParameter parameter) {
-		// todo
+	public void createJobManagerComponent(KubernetesJobManagerSpecification kubernetesJMSpec) {
+		final Deployment deployment = kubernetesJMSpec.getDeployment();
+		final List<HasMetadata> accompanyingResources = kubernetesJMSpec.getAccompanyingResources();
+
+		// create Deployment
+		LOG.debug("Start to create deployment with spec {}", deployment.getSpec().toString());
+		final Deployment createdDeployment = this.internalClient
 
 Review comment:
   The deployment would never fail since the Kubernetes has strategies for such a case:
   1. Generally, the kubelet component helps retry pulling the missing ConfigMap.
   2. The controller helps re-deploy the Pod if strategy 1 fails.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [flink] zhengcanbin commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design

Posted by GitBox <gi...@apache.org>.
zhengcanbin commented on issue #11233: [FLINK-16194][k8s] Refactor the Kubernetes decorator design
URL: https://github.com/apache/flink/pull/11233#issuecomment-593416441
 
 
   > @zhengcanbin Thanks for addressing all the comments. The changes look almost good to me now. Last one comment about the `NOTICE` file. Since we have included a new dependency `zjsonpatch`, it should also be added into `flink-kubernetes/src/main/resources/META-INF/NOTICE`.
   
   Fixed by [d77ca1c](https://github.com/apache/flink/commit/d77ca1c8ccec2ec273d3a261c79e8ad1a31633f9). 

----------------------------------------------------------------
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


With regards,
Apache Git Services