Skip to content

Commit

Permalink
[FLINK-17230] Fix incorrect returned address of Endpoint for external…
Browse files Browse the repository at this point in the history
… Service of ClusterIP type

This closes apache#12277 .
  • Loading branch information
zhengcanbin committed May 25, 2020
1 parent e92b2bf commit b19058a
Show file tree
Hide file tree
Showing 10 changed files with 65 additions and 44 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@

import org.apache.flink.configuration.Configuration;
import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
import org.apache.flink.kubernetes.kubeclient.decorators.ExternalServiceDecorator;
import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPod;
import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPodsWatcher;
import org.apache.flink.kubernetes.kubeclient.resources.KubernetesService;
Expand Down Expand Up @@ -62,7 +63,7 @@ public class Fabric8FlinkKubeClient implements FlinkKubeClient {

private final KubernetesClient internalClient;
private final String clusterId;
private final String nameSpace;
private final String namespace;

private final ExecutorService kubeClientExecutorService;

Expand All @@ -73,7 +74,7 @@ public Fabric8FlinkKubeClient(
this.internalClient = checkNotNull(client);
this.clusterId = checkNotNull(flinkConfig.getString(KubernetesConfigOptions.CLUSTER_ID));

this.nameSpace = flinkConfig.getString(KubernetesConfigOptions.NAMESPACE);
this.namespace = flinkConfig.getString(KubernetesConfigOptions.NAMESPACE);

this.kubeClientExecutorService = asyncExecutorFactory.get();
}
Expand All @@ -88,15 +89,15 @@ public void createJobManagerComponent(KubernetesJobManagerSpecification kubernet
final Deployment createdDeployment = this.internalClient
.apps()
.deployments()
.inNamespace(this.nameSpace)
.inNamespace(this.namespace)
.create(deployment);

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

this.internalClient
.resourceList(accompanyingResources)
.inNamespace(this.nameSpace)
.inNamespace(this.namespace)
.createOrReplace();
}

Expand All @@ -107,13 +108,13 @@ public CompletableFuture<Void> createTaskManagerPod(KubernetesPod kubernetesPod)
final Deployment masterDeployment = this.internalClient
.apps()
.deployments()
.inNamespace(this.nameSpace)
.inNamespace(this.namespace)
.withName(KubernetesUtils.getDeploymentName(clusterId))
.get();

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

// Note that we should use the uid of the master Deployment for the OwnerReference.
Expand All @@ -125,7 +126,7 @@ public CompletableFuture<Void> createTaskManagerPod(KubernetesPod kubernetesPod)

this.internalClient
.pods()
.inNamespace(this.nameSpace)
.inNamespace(this.namespace)
.create(kubernetesPod.getInternalResource());
},
kubeClientExecutorService);
Expand All @@ -150,10 +151,10 @@ public Optional<Endpoint> getRestEndpoint(String clusterId) {
final KubernetesConfigOptions.ServiceExposedType serviceExposedType =
KubernetesConfigOptions.ServiceExposedType.valueOf(service.getSpec().getType());

// Return the service.namespace directly when use ClusterIP.
// Return the external service.namespace directly when using ClusterIP.
if (serviceExposedType == KubernetesConfigOptions.ServiceExposedType.ClusterIP) {
return Optional.of(
new Endpoint(KubernetesUtils.getInternalServiceName(clusterId) + "." + nameSpace, restPort));
new Endpoint(ExternalServiceDecorator.getNamespacedExternalServiceName(clusterId, namespace), restPort));
}

return getRestEndPointFromService(service, restPort);
Expand All @@ -178,7 +179,7 @@ public void stopAndCleanupCluster(String clusterId) {
this.internalClient
.apps()
.deployments()
.inNamespace(this.nameSpace)
.inNamespace(this.namespace)
.withName(KubernetesUtils.getDeploymentName(clusterId))
.cascading(true)
.delete();
Expand All @@ -191,11 +192,11 @@ public void handleException(Exception e) {

@Override
public Optional<KubernetesService> getRestService(String clusterId) {
final String serviceName = KubernetesUtils.getRestServiceName(clusterId);
final String serviceName = ExternalServiceDecorator.getExternalServiceName(clusterId);

final Service service = this.internalClient
.services()
.inNamespace(nameSpace)
.inNamespace(namespace)
.withName(serviceName)
.fromServer()
.get();
Expand Down Expand Up @@ -246,7 +247,7 @@ private int getRestPortFromExternalService(Service externalService) {

if (servicePortCandidates.isEmpty()) {
throw new RuntimeException("Failed to find port \"" + Constants.REST_PORT_NAME + "\" in Service \"" +
KubernetesUtils.getRestServiceName(this.clusterId) + "\"");
ExternalServiceDecorator.getExternalServiceName(this.clusterId) + "\"");
}

final ServicePort externalServicePort = servicePortCandidates.get(0);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@

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.HasMetadata;
import io.fabric8.kubernetes.api.model.Service;
Expand All @@ -46,7 +45,7 @@ public ExternalServiceDecorator(KubernetesJobManagerParameters kubernetesJobMana
@Override
public List<HasMetadata> buildAccompanyingKubernetesResources() throws IOException {
final String serviceName =
KubernetesUtils.getRestServiceName(kubernetesJobManagerParameters.getClusterId());
getExternalServiceName(kubernetesJobManagerParameters.getClusterId());

final Service externalService = new ServiceBuilder()
.withApiVersion(Constants.API_VERSION)
Expand All @@ -68,4 +67,18 @@ public List<HasMetadata> buildAccompanyingKubernetesResources() throws IOExcepti

return Collections.singletonList(externalService);
}

/**
* Generate name of the external Service.
*/
public static String getExternalServiceName(String clusterId) {
return clusterId + Constants.FLINK_REST_SERVICE_SUFFIX;
}

/**
* Generate namespaced name of the external Service.
*/
public static String getNamespacedExternalServiceName(String clusterId, String namespace) {
return getExternalServiceName(clusterId) + "." + namespace;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
import org.apache.flink.configuration.JobManagerOptions;
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.HasMetadata;
import io.fabric8.kubernetes.api.model.Service;
Expand Down Expand Up @@ -53,7 +52,7 @@ public List<HasMetadata> buildAccompanyingKubernetesResources() throws IOExcepti
return Collections.emptyList();
}

final String serviceName = KubernetesUtils.getInternalServiceName(kubernetesJobManagerParameters.getClusterId());
final String serviceName = getInternalServiceName(kubernetesJobManagerParameters.getClusterId());

final Service headlessService = new ServiceBuilder()
.withApiVersion(Constants.API_VERSION)
Expand All @@ -77,10 +76,27 @@ public List<HasMetadata> buildAccompanyingKubernetesResources() throws IOExcepti

// Set job manager address to namespaced service name
final String namespace = kubernetesJobManagerParameters.getNamespace();
kubernetesJobManagerParameters.getFlinkConfiguration().setString(JobManagerOptions.ADDRESS, serviceName + "." + namespace);
kubernetesJobManagerParameters.getFlinkConfiguration().setString(
JobManagerOptions.ADDRESS,
getNamespacedInternalServiceName(serviceName, namespace));

return Collections.singletonList(headlessService);
}


/**
* Generate name of the internal Service.
*/
public static String getInternalServiceName(String clusterId) {
return clusterId;
}

/**
* Generate namespaced name of the internal Service.
*/
public static String getNamespacedInternalServiceName(String clusterId, String namespace) {
return getInternalServiceName(clusterId) + "." + namespace;
}
}


Original file line number Diff line number Diff line change
Expand Up @@ -91,20 +91,6 @@ public static Integer parsePort(Configuration flinkConfig, ConfigOption<String>
}
}

/**
* Generate name of the internal Service.
*/
public static String getInternalServiceName(String clusterId) {
return clusterId;
}

/**
* Generate name of the external Service.
*/
public static String getRestServiceName(String clusterId) {
return clusterId + Constants.FLINK_REST_SERVICE_SUFFIX;
}

/**
* Generate name of the Deployment.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,8 @@
package org.apache.flink.kubernetes;

import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
import org.apache.flink.kubernetes.kubeclient.decorators.ExternalServiceDecorator;
import org.apache.flink.kubernetes.utils.Constants;
import org.apache.flink.kubernetes.utils.KubernetesUtils;

import io.fabric8.kubernetes.api.model.LoadBalancerIngress;
import io.fabric8.kubernetes.api.model.LoadBalancerStatus;
Expand Down Expand Up @@ -107,7 +107,7 @@ private Service buildExternalService(
@Nullable ServiceStatus serviceStatus) {
final ServiceBuilder serviceBuilder = new ServiceBuilder()
.editOrNewMetadata()
.withName(KubernetesUtils.getRestServiceName(CLUSTER_ID))
.withName(ExternalServiceDecorator.getExternalServiceName(CLUSTER_ID))
.endMetadata()
.editOrNewSpec()
.withType(serviceExposedType.name())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,8 +30,8 @@
import org.apache.flink.configuration.PipelineOptions;
import org.apache.flink.configuration.TaskManagerOptions;
import org.apache.flink.kubernetes.configuration.KubernetesDeploymentTarget;
import org.apache.flink.kubernetes.kubeclient.decorators.InternalServiceDecorator;
import org.apache.flink.kubernetes.utils.Constants;
import org.apache.flink.kubernetes.utils.KubernetesUtils;
import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;

import io.fabric8.kubernetes.api.model.Container;
Expand Down Expand Up @@ -191,8 +191,9 @@ private void checkUpdatedConfigAndResourceSetting() {
// Check updated flink config options
assertEquals(String.valueOf(Constants.BLOB_SERVER_PORT), flinkConfig.getString(BlobServerOptions.PORT));
assertEquals(String.valueOf(Constants.TASK_MANAGER_RPC_PORT), flinkConfig.getString(TaskManagerOptions.RPC_PORT));
assertEquals(KubernetesUtils.getInternalServiceName(CLUSTER_ID) + "." +
NAMESPACE, flinkConfig.getString(JobManagerOptions.ADDRESS));
assertEquals(
InternalServiceDecorator.getNamespacedInternalServiceName(CLUSTER_ID, NAMESPACE),
flinkConfig.getString(JobManagerOptions.ADDRESS));

final Deployment jmDeployment = kubeClient
.apps()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
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.decorators.ExternalServiceDecorator;
import org.apache.flink.kubernetes.kubeclient.factory.KubernetesJobManagerFactory;
import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters;
import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPod;
Expand Down Expand Up @@ -219,6 +220,9 @@ public void testClusterIPService() {

final Optional<Endpoint> resultEndpoint = flinkKubeClient.getRestEndpoint(CLUSTER_ID);
assertThat(resultEndpoint.isPresent(), is(true));
assertThat(
resultEndpoint.get().getAddress(),
is(ExternalServiceDecorator.getNamespacedExternalServiceName(CLUSTER_ID, NAMESPACE)));
assertThat(resultEndpoint.get().getPort(), is(REST_PORT));
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
import org.apache.flink.kubernetes.kubeclient.KubernetesJobManagerTestBase;
import org.apache.flink.kubernetes.utils.Constants;
import org.apache.flink.kubernetes.utils.KubernetesUtils;

import io.fabric8.kubernetes.api.model.HasMetadata;
import io.fabric8.kubernetes.api.model.Service;
Expand Down Expand Up @@ -71,7 +70,7 @@ public void testBuildAccompanyingKubernetesResources() throws IOException {

assertEquals(Constants.API_VERSION, restService.getApiVersion());

assertEquals(KubernetesUtils.getRestServiceName(CLUSTER_ID), restService.getMetadata().getName());
assertEquals(ExternalServiceDecorator.getExternalServiceName(CLUSTER_ID), restService.getMetadata().getName());

final Map<String, String> expectedLabels = getCommonLabels();
assertEquals(expectedLabels, restService.getMetadata().getLabels());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import org.apache.flink.configuration.JobManagerOptions;
import org.apache.flink.kubernetes.kubeclient.KubernetesJobManagerTestBase;
import org.apache.flink.kubernetes.utils.Constants;
import org.apache.flink.kubernetes.utils.KubernetesUtils;
import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;

import io.fabric8.kubernetes.api.model.HasMetadata;
Expand Down Expand Up @@ -59,14 +58,14 @@ public void testBuildAccompanyingKubernetesResources() throws IOException {
assertEquals(1, resources.size());

assertEquals(
KubernetesUtils.getInternalServiceName(CLUSTER_ID) + "." + NAMESPACE,
InternalServiceDecorator.getNamespacedInternalServiceName(CLUSTER_ID, NAMESPACE),
this.flinkConfig.getString(JobManagerOptions.ADDRESS));

final Service internalService = (Service) resources.get(0);

assertEquals(Constants.API_VERSION, internalService.getApiVersion());

assertEquals(KubernetesUtils.getInternalServiceName(CLUSTER_ID), internalService.getMetadata().getName());
assertEquals(InternalServiceDecorator.getInternalServiceName(CLUSTER_ID), internalService.getMetadata().getName());

final Map<String, String> expectedLabels = getCommonLabels();
assertEquals(expectedLabels, internalService.getMetadata().getLabels());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,8 +24,10 @@
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.decorators.ExternalServiceDecorator;
import org.apache.flink.kubernetes.kubeclient.decorators.FlinkConfMountDecorator;
import org.apache.flink.kubernetes.kubeclient.decorators.HadoopConfMountDecorator;
import org.apache.flink.kubernetes.kubeclient.decorators.InternalServiceDecorator;
import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters;
import org.apache.flink.kubernetes.utils.Constants;
import org.apache.flink.kubernetes.utils.KubernetesUtils;
Expand Down Expand Up @@ -171,13 +173,13 @@ public void testServices() {

final List<Service> internalServiceCandidates = resultServices
.stream()
.filter(x -> x.getMetadata().getName().equals(KubernetesUtils.getInternalServiceName(CLUSTER_ID)))
.filter(x -> x.getMetadata().getName().equals(InternalServiceDecorator.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)))
.filter(x -> x.getMetadata().getName().equals(ExternalServiceDecorator.getExternalServiceName(CLUSTER_ID)))
.collect(Collectors.toList());
assertEquals(1, restServiceCandidates.size());

Expand Down

0 comments on commit b19058a

Please sign in to comment.