Fabric8FlinkKubeClient.java 10.6 KB
Newer Older
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22
/*
 * 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 org.apache.flink.configuration.Configuration;
import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
23
import org.apache.flink.kubernetes.kubeclient.decorators.ExternalServiceDecorator;
24
import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPod;
25
import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPodsWatcher;
26
import org.apache.flink.kubernetes.kubeclient.resources.KubernetesService;
27
import org.apache.flink.kubernetes.kubeclient.resources.KubernetesWatch;
28
import org.apache.flink.kubernetes.utils.Constants;
29
import org.apache.flink.kubernetes.utils.KubernetesUtils;
30
import org.apache.flink.util.ExecutorUtils;
31

32
import io.fabric8.kubernetes.api.model.HasMetadata;
33
import io.fabric8.kubernetes.api.model.LoadBalancerStatus;
34 35
import io.fabric8.kubernetes.api.model.OwnerReference;
import io.fabric8.kubernetes.api.model.OwnerReferenceBuilder;
36 37 38
import io.fabric8.kubernetes.api.model.Pod;
import io.fabric8.kubernetes.api.model.Service;
import io.fabric8.kubernetes.api.model.ServicePort;
39
import io.fabric8.kubernetes.api.model.apps.Deployment;
40 41 42 43 44 45 46 47
import io.fabric8.kubernetes.client.KubernetesClient;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
48 49 50 51 52
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.function.Supplier;
53 54 55 56 57 58 59 60 61 62 63 64 65
import java.util.stream.Collectors;

import static org.apache.flink.util.Preconditions.checkNotNull;

/**
 * The implementation of {@link FlinkKubeClient}.
 */
public class Fabric8FlinkKubeClient implements FlinkKubeClient {

	private static final Logger LOG = LoggerFactory.getLogger(Fabric8FlinkKubeClient.class);

	private final KubernetesClient internalClient;
	private final String clusterId;
66
	private final String namespace;
67

68 69 70 71 72 73
	private final ExecutorService kubeClientExecutorService;

	public Fabric8FlinkKubeClient(
			Configuration flinkConfig,
			KubernetesClient client,
			Supplier<ExecutorService> asyncExecutorFactory) {
74 75 76
		this.internalClient = checkNotNull(client);
		this.clusterId = checkNotNull(flinkConfig.getString(KubernetesConfigOptions.CLUSTER_ID));

77
		this.namespace = flinkConfig.getString(KubernetesConfigOptions.NAMESPACE);
78 79

		this.kubeClientExecutorService = asyncExecutorFactory.get();
80 81 82
	}

	@Override
83 84 85 86 87 88 89 90 91
	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()
92
			.inNamespace(this.namespace)
93 94 95 96 97 98 99
			.create(deployment);

		// Note that we should use the uid of the created Deployment for the OwnerReference.
		setOwnerReference(createdDeployment, accompanyingResources);

		this.internalClient
			.resourceList(accompanyingResources)
100
			.inNamespace(this.namespace)
101 102 103 104
			.createOrReplace();
	}

	@Override
105 106 107 108 109 110
	public CompletableFuture<Void> createTaskManagerPod(KubernetesPod kubernetesPod) {
		return CompletableFuture.runAsync(
			() -> {
				final Deployment masterDeployment = this.internalClient
					.apps()
					.deployments()
111
					.inNamespace(this.namespace)
112 113 114 115 116
					.withName(KubernetesUtils.getDeploymentName(clusterId))
					.get();

				if (masterDeployment == null) {
					throw new RuntimeException(
117
						"Failed to find Deployment named " + clusterId + " in namespace " + this.namespace);
118
				}
119

120 121
				// Note that we should use the uid of the master Deployment for the OwnerReference.
				setOwnerReference(masterDeployment, Collections.singletonList(kubernetesPod.getInternalResource()));
122

123 124 125
				LOG.debug("Start to create pod with metadata {}, spec {}",
					kubernetesPod.getInternalResource().getMetadata(),
					kubernetesPod.getInternalResource().getSpec());
126

127 128
				this.internalClient
					.pods()
129
					.inNamespace(this.namespace)
130 131 132
					.create(kubernetesPod.getInternalResource());
				},
			kubeClientExecutorService);
133 134 135
	}

	@Override
136 137 138 139
	public CompletableFuture<Void> stopPod(String podName) {
		return CompletableFuture.runAsync(
			() -> this.internalClient.pods().withName(podName).delete(),
			kubeClientExecutorService);
140 141 142
	}

	@Override
143 144 145 146
	public Optional<Endpoint> getRestEndpoint(String clusterId) {
		Optional<KubernetesService> restService = getRestService(clusterId);
		if (!restService.isPresent()) {
			return Optional.empty();
147
		}
148
		final Service service = restService.get().getInternalResource();
149 150 151 152
		final int restPort = getRestPortFromExternalService(service);

		final KubernetesConfigOptions.ServiceExposedType serviceExposedType =
			KubernetesConfigOptions.ServiceExposedType.valueOf(service.getSpec().getType());
153

154
		// Return the external service.namespace directly when using ClusterIP.
155
		if (serviceExposedType == KubernetesConfigOptions.ServiceExposedType.ClusterIP) {
156
			return Optional.of(
157
				new Endpoint(ExternalServiceDecorator.getNamespacedExternalServiceName(clusterId, namespace), restPort));
158 159
		}

160
		return getRestEndPointFromService(service, restPort);
161 162 163 164 165 166
	}

	@Override
	public List<KubernetesPod> getPodsWithLabels(Map<String, String> labels) {
		final List<Pod> podList = this.internalClient.pods().withLabels(labels).list().getItems();

167
		if (podList == null || podList.isEmpty()) {
168 169 170 171 172
			return new ArrayList<>();
		}

		return podList
			.stream()
173
			.map(KubernetesPod::new)
174 175 176 177 178
			.collect(Collectors.toList());
	}

	@Override
	public void stopAndCleanupCluster(String clusterId) {
179 180 181
		this.internalClient
			.apps()
			.deployments()
182
			.inNamespace(this.namespace)
183 184 185
			.withName(KubernetesUtils.getDeploymentName(clusterId))
			.cascading(true)
			.delete();
186 187 188 189
	}

	@Override
	public void handleException(Exception e) {
190
		LOG.error("A Kubernetes exception occurred.", e);
191 192 193
	}

	@Override
194
	public Optional<KubernetesService> getRestService(String clusterId) {
195
		final String serviceName = ExternalServiceDecorator.getExternalServiceName(clusterId);
196 197 198

		final Service service = this.internalClient
			.services()
199
			.inNamespace(namespace)
200 201 202 203 204 205
			.withName(serviceName)
			.fromServer()
			.get();

		if (service == null) {
			LOG.debug("Service {} does not exist", serviceName);
206
			return Optional.empty();
207 208
		}

209
		return Optional.of(new KubernetesService(service));
210 211 212
	}

	@Override
213 214 215 216 217
	public KubernetesWatch watchPodsAndDoCallback(Map<String, String> labels, PodCallbackHandler podCallbackHandler) {
		return new KubernetesWatch(
			this.internalClient.pods()
				.withLabels(labels)
				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
218 219 220 221 222
	}

	@Override
	public void close() {
		this.internalClient.close();
223
		ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, this.kubeClientExecutorService);
224 225
	}

226 227 228 229 230 231 232 233 234 235 236 237 238
	private void setOwnerReference(Deployment deployment, List<HasMetadata> resources) {
		final OwnerReference deploymentOwnerReference = new OwnerReferenceBuilder()
			.withName(deployment.getMetadata().getName())
			.withApiVersion(deployment.getApiVersion())
			.withUid(deployment.getMetadata().getUid())
			.withKind(deployment.getKind())
			.withController(true)
			.withBlockOwnerDeletion(true)
			.build();
		resources.forEach(resource ->
			resource.getMetadata().setOwnerReferences(Collections.singletonList(deploymentOwnerReference)));
	}

239
	/**
240
	 * Get rest port from the external Service.
241
	 */
242 243 244 245 246 247 248 249
	private int getRestPortFromExternalService(Service externalService) {
		final List<ServicePort> servicePortCandidates = externalService.getSpec().getPorts()
			.stream()
			.filter(x -> x.getName().equals(Constants.REST_PORT_NAME))
			.collect(Collectors.toList());

		if (servicePortCandidates.isEmpty()) {
			throw new RuntimeException("Failed to find port \"" + Constants.REST_PORT_NAME + "\" in Service \"" +
250
				ExternalServiceDecorator.getExternalServiceName(this.clusterId) + "\"");
251 252 253 254 255 256 257 258 259 260 261 262 263 264 265
		}

		final ServicePort externalServicePort = servicePortCandidates.get(0);

		final KubernetesConfigOptions.ServiceExposedType externalServiceType =
			KubernetesConfigOptions.ServiceExposedType.valueOf(externalService.getSpec().getType());

		switch (externalServiceType) {
			case ClusterIP:
			case LoadBalancer:
				return externalServicePort.getPort();
			case NodePort:
				return externalServicePort.getNodePort();
			default:
				throw new RuntimeException("Unrecognized Service type: " + externalServiceType);
266 267
		}
	}
268 269 270 271 272 273 274 275 276 277 278 279 280 281 282 283 284 285 286 287 288 289 290 291 292 293 294 295 296 297 298 299 300 301 302 303 304

	private Optional<Endpoint> getRestEndPointFromService(Service service, int restPort) {
		if (service.getStatus() == null) {
			return Optional.empty();
		}

		LoadBalancerStatus loadBalancer = service.getStatus().getLoadBalancer();
		boolean hasExternalIP = service.getSpec() != null &&
			service.getSpec().getExternalIPs() != null && !service.getSpec().getExternalIPs().isEmpty();

		if (loadBalancer != null) {
			return getLoadBalancerRestEndpoint(loadBalancer, restPort);
		} else if (hasExternalIP) {
			final String address = service.getSpec().getExternalIPs().get(0);
			if (address != null && !address.isEmpty()) {
				return Optional.of(new Endpoint(address, restPort));
			}
		}
		return Optional.empty();
	}

	private Optional<Endpoint> getLoadBalancerRestEndpoint(LoadBalancerStatus loadBalancer, int restPort) {
		boolean hasIngress = loadBalancer.getIngress() != null && !loadBalancer.getIngress().isEmpty();
		String address;
		if (hasIngress) {
			address = loadBalancer.getIngress().get(0).getIp();
			// Use hostname when the ip address is null
			if (address == null || address.isEmpty()) {
				address = loadBalancer.getIngress().get(0).getHostname();
			}
		} else {
			// Use node port
			address = this.internalClient.getMasterUrl().getHost();
		}
		boolean noAddress = address == null || address.isEmpty();
		return noAddress ? Optional.empty() : Optional.of(new Endpoint(address, restPort));
	}
305
}