Fabric8FlinkKubeClient.java 14.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.configuration.KubernetesLeaderElectionConfiguration;
24
import org.apache.flink.kubernetes.kubeclient.decorators.ExternalServiceDecorator;
25 26 27
import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap;
import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMapWatcher;
import org.apache.flink.kubernetes.kubeclient.resources.KubernetesException;
28
import org.apache.flink.kubernetes.kubeclient.resources.KubernetesLeaderElector;
29
import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPod;
30
import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPodsWatcher;
31
import org.apache.flink.kubernetes.kubeclient.resources.KubernetesService;
32
import org.apache.flink.kubernetes.kubeclient.resources.KubernetesWatch;
33
import org.apache.flink.kubernetes.utils.Constants;
34
import org.apache.flink.kubernetes.utils.KubernetesUtils;
35 36
import org.apache.flink.runtime.concurrent.FutureUtils;
import org.apache.flink.util.ExceptionUtils;
37

38
import io.fabric8.kubernetes.api.model.ConfigMap;
39
import io.fabric8.kubernetes.api.model.HasMetadata;
40
import io.fabric8.kubernetes.api.model.LoadBalancerStatus;
41 42
import io.fabric8.kubernetes.api.model.OwnerReference;
import io.fabric8.kubernetes.api.model.OwnerReferenceBuilder;
43 44 45
import io.fabric8.kubernetes.api.model.Pod;
import io.fabric8.kubernetes.api.model.Service;
import io.fabric8.kubernetes.api.model.ServicePort;
46
import io.fabric8.kubernetes.api.model.apps.Deployment;
47
import io.fabric8.kubernetes.client.KubernetesClient;
48
import io.fabric8.kubernetes.client.KubernetesClientException;
49
import io.fabric8.kubernetes.client.NamespacedKubernetesClient;
50 51 52 53 54 55 56
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
57 58
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
59
import java.util.concurrent.CompletionException;
60
import java.util.concurrent.Executor;
61
import java.util.function.Function;
62
import java.util.function.Supplier;
63 64 65 66 67 68 69 70 71 72 73 74 75
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;
76
	private final String namespace;
77
	private final int maxRetryAttempts;
78

79
	private final Executor kubeClientExecutorService;
80 81 82 83

	public Fabric8FlinkKubeClient(
			Configuration flinkConfig,
			KubernetesClient client,
84
			Supplier<Executor> asyncExecutorFactory) {
85 86 87
		this.internalClient = checkNotNull(client);
		this.clusterId = checkNotNull(flinkConfig.getString(KubernetesConfigOptions.CLUSTER_ID));

88
		this.namespace = flinkConfig.getString(KubernetesConfigOptions.NAMESPACE);
89

90 91 92
		this.maxRetryAttempts = flinkConfig.getInteger(
			KubernetesConfigOptions.KUBERNETES_TRANSACTIONAL_OPERATION_MAX_RETRIES);

93
		this.kubeClientExecutorService = asyncExecutorFactory.get();
94 95 96
	}

	@Override
97 98 99 100 101 102 103 104 105
	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()
106
			.inNamespace(this.namespace)
107 108 109 110 111 112 113
			.create(deployment);

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

		this.internalClient
			.resourceList(accompanyingResources)
114
			.inNamespace(this.namespace)
115 116 117 118
			.createOrReplace();
	}

	@Override
119 120 121 122 123 124
	public CompletableFuture<Void> createTaskManagerPod(KubernetesPod kubernetesPod) {
		return CompletableFuture.runAsync(
			() -> {
				final Deployment masterDeployment = this.internalClient
					.apps()
					.deployments()
125
					.inNamespace(this.namespace)
126 127 128 129 130
					.withName(KubernetesUtils.getDeploymentName(clusterId))
					.get();

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

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

137 138 139
				LOG.debug("Start to create pod with metadata {}, spec {}",
					kubernetesPod.getInternalResource().getMetadata(),
					kubernetesPod.getInternalResource().getSpec());
140

141 142
				this.internalClient
					.pods()
143
					.inNamespace(this.namespace)
144 145 146
					.create(kubernetesPod.getInternalResource());
				},
			kubeClientExecutorService);
147 148 149
	}

	@Override
150 151 152 153
	public CompletableFuture<Void> stopPod(String podName) {
		return CompletableFuture.runAsync(
			() -> this.internalClient.pods().withName(podName).delete(),
			kubeClientExecutorService);
154 155 156
	}

	@Override
157 158 159 160
	public Optional<Endpoint> getRestEndpoint(String clusterId) {
		Optional<KubernetesService> restService = getRestService(clusterId);
		if (!restService.isPresent()) {
			return Optional.empty();
161
		}
162
		final Service service = restService.get().getInternalResource();
163 164 165 166
		final int restPort = getRestPortFromExternalService(service);

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

168
		// Return the external service.namespace directly when using ClusterIP.
169
		if (serviceExposedType == KubernetesConfigOptions.ServiceExposedType.ClusterIP) {
170
			return Optional.of(
171
				new Endpoint(ExternalServiceDecorator.getNamespacedExternalServiceName(clusterId, namespace), restPort));
172 173
		}

174
		return getRestEndPointFromService(service, restPort);
175 176 177 178 179 180
	}

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

181
		if (podList == null || podList.isEmpty()) {
182 183 184 185 186
			return new ArrayList<>();
		}

		return podList
			.stream()
187
			.map(KubernetesPod::new)
188 189 190 191 192
			.collect(Collectors.toList());
	}

	@Override
	public void stopAndCleanupCluster(String clusterId) {
193 194 195
		this.internalClient
			.apps()
			.deployments()
196
			.inNamespace(this.namespace)
197 198 199
			.withName(KubernetesUtils.getDeploymentName(clusterId))
			.cascading(true)
			.delete();
200 201 202 203
	}

	@Override
	public void handleException(Exception e) {
204
		LOG.error("A Kubernetes exception occurred.", e);
205 206 207
	}

	@Override
208
	public Optional<KubernetesService> getRestService(String clusterId) {
209
		final String serviceName = ExternalServiceDecorator.getExternalServiceName(clusterId);
210 211 212

		final Service service = this.internalClient
			.services()
213
			.inNamespace(namespace)
214 215 216 217 218 219
			.withName(serviceName)
			.fromServer()
			.get();

		if (service == null) {
			LOG.debug("Service {} does not exist", serviceName);
220
			return Optional.empty();
221 222
		}

223
		return Optional.of(new KubernetesService(service));
224 225 226
	}

	@Override
227 228 229
	public KubernetesWatch watchPodsAndDoCallback(
			Map<String, String> labels,
			WatchCallbackHandler<KubernetesPod> podCallbackHandler) {
230 231 232 233
		return new KubernetesWatch(
			this.internalClient.pods()
				.withLabels(labels)
				.watch(new KubernetesPodsWatcher(podCallbackHandler)));
234 235
	}

236 237 238 239 240 241 242 243 244 245 246
	@Override
	public KubernetesLeaderElector createLeaderElector(
			KubernetesLeaderElectionConfiguration leaderElectionConfiguration,
			KubernetesLeaderElector.LeaderCallbackHandler leaderCallbackHandler) {
		return new KubernetesLeaderElector(
			(NamespacedKubernetesClient) this.internalClient,
			namespace,
			leaderElectionConfiguration,
			leaderCallbackHandler);
	}

247 248 249 250 251 252 253 254 255 256 257 258 259 260 261 262 263 264 265 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 305 306 307 308 309 310 311 312 313 314 315 316 317 318 319 320
	@Override
	public CompletableFuture<Void> createConfigMap(KubernetesConfigMap configMap) {
		final String configMapName = configMap.getName();
		return CompletableFuture.runAsync(
			() -> this.internalClient.configMaps().inNamespace(namespace).create(configMap.getInternalResource()),
			kubeClientExecutorService)
			.exceptionally(
				throwable -> {
					throw new CompletionException(
						new KubernetesException("Failed to create ConfigMap " + configMapName, throwable));
				});
	}

	@Override
	public Optional<KubernetesConfigMap> getConfigMap(String name) {
		final ConfigMap configMap = this.internalClient.configMaps().inNamespace(namespace).withName(name).get();
		return configMap == null ? Optional.empty() : Optional.of(new KubernetesConfigMap(configMap));
	}

	@Override
	public CompletableFuture<Boolean> checkAndUpdateConfigMap(
			String configMapName,
			Function<KubernetesConfigMap, Optional<KubernetesConfigMap>> function) {
		return FutureUtils.retry(
			() -> CompletableFuture.supplyAsync(
				() -> getConfigMap(configMapName)
					.map(
						configMap -> function.apply(configMap).map(
							updatedConfigMap -> {
								try {
									this.internalClient.configMaps()
										.inNamespace(namespace)
										.withName(configMapName)
										.lockResourceVersion(updatedConfigMap.getResourceVersion())
										.replace(updatedConfigMap.getInternalResource());
								} catch (Throwable throwable) {
									LOG.debug("Failed to update ConfigMap {} with data {} because of concurrent " +
										"modifications. Trying again.", configMap.getName(), configMap.getData());
									throw throwable;
								}
								return true;
							}).orElse(false))
					.orElseThrow(() -> new CompletionException(
						new KubernetesException("Cannot retry checkAndUpdateConfigMap with configMap "
							+ configMapName + " because it does not exist."))),
				kubeClientExecutorService),
			maxRetryAttempts,
			// Only KubernetesClientException is retryable
			throwable -> ExceptionUtils.findThrowable(throwable, KubernetesClientException.class).isPresent(),
			kubeClientExecutorService);
	}

	@Override
	public KubernetesWatch watchConfigMaps(
			String name,
			WatchCallbackHandler<KubernetesConfigMap> callbackHandler) {
		return new KubernetesWatch(
			this.internalClient.configMaps().withName(name).watch(new KubernetesConfigMapWatcher(callbackHandler)));
	}

	@Override
	public CompletableFuture<Void> deleteConfigMapsByLabels(Map<String, String> labels) {
		return CompletableFuture.runAsync(
			() -> this.internalClient.configMaps().inNamespace(namespace).withLabels(labels).delete(),
			kubeClientExecutorService);
	}

	@Override
	public CompletableFuture<Void> deleteConfigMap(String configMapName) {
		return CompletableFuture.runAsync(
			() -> this.internalClient.configMaps().inNamespace(namespace).withName(configMapName).delete(),
			kubeClientExecutorService);
	}

321 322 323 324 325
	@Override
	public void close() {
		this.internalClient.close();
	}

326 327 328 329 330 331 332 333 334 335 336 337 338
	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)));
	}

339
	/**
340
	 * Get rest port from the external Service.
341
	 */
342 343 344 345 346 347 348 349
	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 \"" +
350
				ExternalServiceDecorator.getExternalServiceName(this.clusterId) + "\"");
351 352 353 354 355 356 357 358 359 360 361 362 363 364 365
		}

		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);
366 367
		}
	}
368 369 370 371 372 373 374 375 376 377 378 379 380 381 382 383 384 385 386 387 388 389 390 391 392 393 394 395 396 397 398 399 400 401 402 403 404

	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));
	}
405
}