Fabric8FlinkKubeClientTest.java 8.9 KB
Newer Older
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20
/*
 * 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;

21 22 23 24
import org.apache.flink.client.deployment.ClusterSpecification;
import org.apache.flink.configuration.BlobServerOptions;
import org.apache.flink.configuration.JobManagerOptions;
import org.apache.flink.configuration.RestOptions;
25
import org.apache.flink.kubernetes.KubernetesClientTestBase;
26 27 28 29 30 31 32
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.factory.KubernetesJobManagerFactory;
import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters;
import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPod;
33

34 35 36 37 38 39 40
import io.fabric8.kubernetes.api.model.ConfigMap;
import io.fabric8.kubernetes.api.model.HasMetadata;
import io.fabric8.kubernetes.api.model.OwnerReference;
import io.fabric8.kubernetes.api.model.Pod;
import io.fabric8.kubernetes.api.model.PodBuilder;
import io.fabric8.kubernetes.api.model.Service;
import io.fabric8.kubernetes.api.model.apps.Deployment;
41 42 43
import org.junit.Before;
import org.junit.Test;

44 45
import java.util.List;

46
import static org.hamcrest.Matchers.is;
47 48 49
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
50
import static org.junit.Assert.assertThat;
51 52
import static org.junit.Assert.assertTrue;

53 54 55
/**
 * Tests for Fabric implementation of {@link FlinkKubeClient}.
 */
56
public class Fabric8FlinkKubeClientTest extends KubernetesClientTestBase {
57 58 59 60 61 62 63 64 65 66 67
	private static final int RPC_PORT = 7123;
	private static final int BLOB_SERVER_PORT = 8346;

	private static final double JOB_MANAGER_CPU = 2.0;
	private static final int JOB_MANAGER_MEMORY = 768;

	private static final String SERVICE_ACCOUNT_NAME = "service-test";

	private static final String ENTRY_POINT_CLASS = KubernetesSessionClusterEntrypoint.class.getCanonicalName();

	private KubernetesJobManagerSpecification kubernetesJobManagerSpecification;
68 69

	@Before
70 71
	public void setup() throws Exception {
		super.setup();
72 73 74 75 76 77 78 79 80 81 82 83 84 85 86 87 88 89 90 91 92 93

		KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "logback.xml");
		KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "log4j.properties");

		flinkConfig.set(KubernetesConfigOptions.CONTAINER_IMAGE_PULL_POLICY, CONTAINER_IMAGE_PULL_POLICY);
		flinkConfig.set(KubernetesConfigOptionsInternal.ENTRY_POINT_CLASS, ENTRY_POINT_CLASS);
		flinkConfig.set(RestOptions.PORT, REST_PORT);
		flinkConfig.set(JobManagerOptions.PORT, RPC_PORT);
		flinkConfig.set(BlobServerOptions.PORT, Integer.toString(BLOB_SERVER_PORT));
		flinkConfig.set(KubernetesConfigOptions.JOB_MANAGER_CPU, JOB_MANAGER_CPU);
		flinkConfig.set(KubernetesConfigOptions.JOB_MANAGER_SERVICE_ACCOUNT, SERVICE_ACCOUNT_NAME);

		final ClusterSpecification clusterSpecification = new ClusterSpecification.ClusterSpecificationBuilder()
			.setMasterMemoryMB(JOB_MANAGER_MEMORY)
			.setTaskManagerMemoryMB(1000)
			.setSlotsPerTaskManager(3)
			.createClusterSpecification();

		final KubernetesJobManagerParameters kubernetesJobManagerParameters =
			new KubernetesJobManagerParameters(flinkConfig, clusterSpecification);
		this.kubernetesJobManagerSpecification =
			KubernetesJobManagerFactory.createJobManagerComponent(kubernetesJobManagerParameters);
94 95 96
	}

	@Test
97 98 99 100 101 102 103 104 105 106 107 108 109 110 111 112 113 114 115 116 117 118 119 120 121 122 123 124 125 126 127 128 129 130 131 132 133 134 135 136 137 138 139 140 141 142 143 144 145 146 147 148 149 150
	public void testCreateFlinkMasterComponent() throws Exception {
		flinkKubeClient.createJobManagerComponent(this.kubernetesJobManagerSpecification);

		final List<Deployment> resultedDeployments = kubeClient.apps().deployments()
			.inNamespace(NAMESPACE)
			.list()
			.getItems();
		assertEquals(1, resultedDeployments.size());

		final List<ConfigMap> resultedConfigMaps = kubeClient.configMaps()
			.inNamespace(NAMESPACE)
			.list()
			.getItems();
		assertEquals(1, resultedConfigMaps.size());

		final List<Service> resultedServices = kubeClient.services()
			.inNamespace(NAMESPACE)
			.list()
			.getItems();
		assertEquals(2, resultedServices.size());

		testOwnerReferenceSetting(resultedDeployments.get(0), resultedConfigMaps);
		testOwnerReferenceSetting(resultedDeployments.get(0), resultedServices);
	}

	private <T extends HasMetadata> void testOwnerReferenceSetting(
		HasMetadata ownerReference,
		List<T> resources) {
		resources.forEach(resource -> {
			List<OwnerReference> ownerReferences = resource.getMetadata().getOwnerReferences();
			assertEquals(1, ownerReferences.size());
			assertEquals(ownerReference.getMetadata().getUid(), ownerReferences.get(0).getUid());
		});
	}

	@Test
	public void testCreateFlinkTaskManagerPod() throws Exception {
		this.flinkKubeClient.createJobManagerComponent(this.kubernetesJobManagerSpecification);

		final KubernetesPod kubernetesPod = new KubernetesPod(new PodBuilder()
			.editOrNewMetadata()
			.withName("mock-task-manager-pod")
			.endMetadata()
			.editOrNewSpec()
			.endSpec()
			.build());
		this.flinkKubeClient.createTaskManagerPod(kubernetesPod);

		final Pod resultTaskManagerPod =
			this.kubeClient.pods().inNamespace(NAMESPACE).withName("mock-task-manager-pod").get();

		assertEquals(
			this.kubeClient.apps().deployments().inNamespace(NAMESPACE).list().getItems().get(0).getMetadata().getUid(),
			resultTaskManagerPod.getMetadata().getOwnerReferences().get(0).getUid());
151
	}
152 153

	@Test
154 155 156 157 158 159 160 161 162 163 164 165 166 167 168
	public void testStopPod() {
		final String podName = "pod-for-delete";
		final Pod pod = new PodBuilder()
			.editOrNewMetadata()
			.withName(podName)
			.endMetadata()
			.editOrNewSpec()
			.endSpec()
			.build();

		this.kubeClient.pods().inNamespace(NAMESPACE).create(pod);
		assertNotNull(this.kubeClient.pods().inNamespace(NAMESPACE).withName(podName).get());

		this.flinkKubeClient.stopPod(podName);
		assertNull(this.kubeClient.pods().inNamespace(NAMESPACE).withName(podName).get());
169 170 171
	}

	@Test
172 173
	public void testServiceLoadBalancerWithNoIP() {
		final String hostName = "test-host-name";
174
		mockExpectedServiceFromServerSide(buildExternalServiceWithLoadBalancer(hostName, ""));
175 176 177 178 179 180 181 182 183

		final Endpoint resultEndpoint = flinkKubeClient.getRestEndpoint(CLUSTER_ID);

		assertEquals(hostName, resultEndpoint.getAddress());
		assertEquals(REST_PORT, resultEndpoint.getPort());
	}

	@Test
	public void testServiceLoadBalancerEmptyHostAndIP() {
184
		mockExpectedServiceFromServerSide(buildExternalServiceWithLoadBalancer("", ""));
185 186 187 188 189 190 191

		final Endpoint resultEndpoint1 = flinkKubeClient.getRestEndpoint(CLUSTER_ID);
		assertNull(resultEndpoint1);
	}

	@Test
	public void testServiceLoadBalancerNullHostAndIP() {
192
		mockExpectedServiceFromServerSide(buildExternalServiceWithLoadBalancer(null, null));
193 194 195 196 197

		final Endpoint resultEndpoint2 = flinkKubeClient.getRestEndpoint(CLUSTER_ID);
		assertNull(resultEndpoint2);
	}

198 199 200 201 202 203 204 205 206 207 208 209 210 211 212 213
	@Test
	public void testNodePortService() {
		mockExpectedServiceFromServerSide(buildExternalServiceWithNodePort());

		final Endpoint resultEndpoint = flinkKubeClient.getRestEndpoint(CLUSTER_ID);
		assertThat(resultEndpoint.getPort(), is(NODE_PORT));
	}

	@Test
	public void testClusterIPService() {
		mockExpectedServiceFromServerSide(buildExternalServiceWithClusterIP());
		final Endpoint resultEndpoint = flinkKubeClient.getRestEndpoint(CLUSTER_ID);

		assertThat(resultEndpoint.getPort(), is(REST_PORT));
	}

214 215 216 217 218 219 220 221 222 223 224 225 226 227 228 229 230 231 232 233 234
	@Test
	public void testStopAndCleanupCluster() throws Exception {
		this.flinkKubeClient.createJobManagerComponent(this.kubernetesJobManagerSpecification);

		final KubernetesPod kubernetesPod = new KubernetesPod(new PodBuilder()
			.editOrNewMetadata()
			.withName("mock-task-manager-pod")
			.endMetadata()
			.editOrNewSpec()
			.endSpec()
			.build());
		this.flinkKubeClient.createTaskManagerPod(kubernetesPod);

		assertEquals(1, this.kubeClient.apps().deployments().inNamespace(NAMESPACE).list().getItems().size());
		assertEquals(1, this.kubeClient.configMaps().inNamespace(NAMESPACE).list().getItems().size());
		assertEquals(2, this.kubeClient.services().inNamespace(NAMESPACE).list().getItems().size());
		assertEquals(1, this.kubeClient.pods().inNamespace(NAMESPACE).list().getItems().size());

		this.flinkKubeClient.stopAndCleanupCluster(CLUSTER_ID);
		assertTrue(this.kubeClient.apps().deployments().inNamespace(NAMESPACE).list().getItems().isEmpty());
	}
235
}