Skip to content

Commit

Permalink
[FLINK-15116] Return a ClusterClientProvider from ClusterDescriptor m…
Browse files Browse the repository at this point in the history
…ethods

This allows the consumer of the methods to create a new ClusterClient
with a separate lifecycle whenever necessary.
  • Loading branch information
aljoscha committed Dec 8, 2019
1 parent 27488a4 commit 6825f80
Show file tree
Hide file tree
Showing 21 changed files with 179 additions and 98 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -862,7 +862,7 @@ private <ClusterID> void runClusterAction(CustomCommandLine activeCommandLine, C
"you would like to connect.");
} else {
try {
final ClusterClient<ClusterID> clusterClient = clusterDescriptor.retrieve(clusterId);
final ClusterClient<ClusterID> clusterClient = clusterDescriptor.retrieve(clusterId).getClusterClient();

try {
clusterAction.runAction(clusterClient);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,9 @@ public CompletableFuture<JobClient> execute(@Nonnull final Pipeline pipeline, @N

final ClusterSpecification clusterSpecification = clusterClientFactory.getClusterSpecification(configuration);

final ClusterClient<ClusterID> clusterClient = clusterDescriptor.deployJobCluster(clusterSpecification, jobGraph, configAccessor.getDetachedMode());
final ClusterClient<ClusterID> clusterClient = clusterDescriptor
.deployJobCluster(clusterSpecification, jobGraph, configAccessor.getDetachedMode())
.getClusterClient();
LOG.info("Job has been submitted with JobID " + jobGraph.getJobID());

final boolean withShutdownHook = !configAccessor.getDetachedMode() && configAccessor.isShutdownOnAttachedExit();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,10 @@ public CompletableFuture<JobClient> execute(@Nonnull final Pipeline pipeline, @N
final ClusterID clusterID = clusterClientFactory.getClusterId(configuration);
checkState(clusterID != null);

final ClusterClient<ClusterID> clusterClient = clusterDescriptor.retrieve(clusterID);
final ClusterClient<ClusterID> clusterClient = clusterDescriptor
.retrieve(clusterID)
.getClusterClient();

return clusterClient
.submitJob(jobGraph)
.thenApply(jobID -> new ClusterClientJobClientAdapter<ClusterID>(clusterClient, jobID) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@

package org.apache.flink.client.deployment;

import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.client.program.ClusterClientProvider;
import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.util.FlinkException;

Expand All @@ -41,15 +41,15 @@ public interface ClusterDescriptor<T> extends AutoCloseable {
* @return Client for the cluster
* @throws ClusterRetrieveException if the cluster client could not be retrieved
*/
ClusterClient<T> retrieve(T clusterId) throws ClusterRetrieveException;
ClusterClientProvider<T> retrieve(T clusterId) throws ClusterRetrieveException;

/**
* Triggers deployment of a cluster.
* @param clusterSpecification Cluster specification defining the cluster to deploy
* @return Client for the cluster
* @throws ClusterDeploymentException if the cluster could not be deployed
*/
ClusterClient<T> deploySessionCluster(ClusterSpecification clusterSpecification) throws ClusterDeploymentException;
ClusterClientProvider<T> deploySessionCluster(ClusterSpecification clusterSpecification) throws ClusterDeploymentException;

/**
* Deploys a per-job cluster with the given job on the cluster.
Expand All @@ -61,7 +61,7 @@ public interface ClusterDescriptor<T> extends AutoCloseable {
* @return Cluster client to talk to the Flink cluster
* @throws ClusterDeploymentException if the cluster could not be deployed
*/
ClusterClient<T> deployJobCluster(
ClusterClientProvider<T> deployJobCluster(
final ClusterSpecification clusterSpecification,
final JobGraph jobGraph,
final boolean detached) throws ClusterDeploymentException;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.flink.client.deployment;

import org.apache.flink.client.program.ClusterClientProvider;
import org.apache.flink.client.program.rest.RestClusterClient;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.JobManagerOptions;
Expand All @@ -44,21 +45,23 @@ public String getClusterDescription() {
}

@Override
public RestClusterClient<StandaloneClusterId> retrieve(StandaloneClusterId standaloneClusterId) throws ClusterRetrieveException {
try {
return new RestClusterClient<>(config, standaloneClusterId);
} catch (Exception e) {
throw new ClusterRetrieveException("Couldn't retrieve standalone cluster", e);
}
public ClusterClientProvider<StandaloneClusterId> retrieve(StandaloneClusterId standaloneClusterId) throws ClusterRetrieveException {
return () -> {
try {
return new RestClusterClient<>(config, standaloneClusterId);
} catch (Exception e) {
throw new RuntimeException("Couldn't retrieve standalone cluster", e);
}
};
}

@Override
public RestClusterClient<StandaloneClusterId> deploySessionCluster(ClusterSpecification clusterSpecification) {
public ClusterClientProvider<StandaloneClusterId> deploySessionCluster(ClusterSpecification clusterSpecification) {
throw new UnsupportedOperationException("Can't deploy a standalone cluster.");
}

@Override
public RestClusterClient<StandaloneClusterId> deployJobCluster(
public ClusterClientProvider<StandaloneClusterId> deployJobCluster(
ClusterSpecification clusterSpecification,
JobGraph jobGraph,
boolean detached) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
/*
* 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:https://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.client.program;

import org.apache.flink.annotation.Internal;

/**
* Factory for {@link ClusterClient ClusterClients}.
*/
@Internal
public interface ClusterClientProvider<T> {

/**
* Creates and returns a new {@link ClusterClient}. The returned client needs to be closed via
* {@link ClusterClient#close()} after use.
*/
ClusterClient<T> getClusterClient();
}
Original file line number Diff line number Diff line change
Expand Up @@ -109,6 +109,6 @@ private ClusterClient<?> getClusterClient(AbstractCustomCommandLine defaultCLI,
checkState(clusterFactory != null);

final ClusterDescriptor<StandaloneClusterId> clusterDescriptor = clusterFactory.createClusterDescriptor(executorConfig);
return clusterDescriptor.retrieve(clusterFactory.getClusterId(executorConfig));
return clusterDescriptor.retrieve(clusterFactory.getClusterId(executorConfig)).getClusterClient();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.flink.client.deployment.ClusterDescriptor;
import org.apache.flink.client.deployment.ClusterSpecification;
import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.client.program.ClusterClientProvider;
import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.util.Preconditions;

Expand All @@ -41,21 +42,21 @@ public String getClusterDescription() {
}

@Override
public ClusterClient<T> retrieve(T clusterId) {
return clusterClient;
public ClusterClientProvider<T> retrieve(T clusterId) {
return () -> clusterClient;
}

@Override
public ClusterClient<T> deploySessionCluster(ClusterSpecification clusterSpecification) {
return clusterClient;
public ClusterClientProvider<T> deploySessionCluster(ClusterSpecification clusterSpecification) {
return () -> clusterClient;
}

@Override
public ClusterClient<T> deployJobCluster(
public ClusterClientProvider<T> deployJobCluster(
ClusterSpecification clusterSpecification,
JobGraph jobGraph,
boolean detached) {
return clusterClient;
return () -> clusterClient;
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -555,7 +555,9 @@ public void testRESTManualConfigurationOverride() throws Exception {
checkState(clusterFactory != null);

final ClusterDescriptor<StandaloneClusterId> clusterDescriptor = clusterFactory.createClusterDescriptor(executorConfig);
final RestClusterClient<?> clusterClient = (RestClusterClient<?>) clusterDescriptor.retrieve(clusterFactory.getClusterId(executorConfig));
final RestClusterClient<?> clusterClient = (RestClusterClient<?>) clusterDescriptor
.retrieve(clusterFactory.getClusterId(executorConfig))
.getClusterClient();

URL webMonitorBaseUrl = clusterClient.getWebMonitorBaseUrl().get();
assertThat(webMonitorBaseUrl.getHost(), equalTo(manualHostname));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import org.apache.flink.client.deployment.ClusterRetrieveException;
import org.apache.flink.client.deployment.ClusterSpecification;
import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.client.program.ClusterClientProvider;
import org.apache.flink.client.program.rest.RestClusterClient;
import org.apache.flink.configuration.BlobServerOptions;
import org.apache.flink.configuration.Configuration;
Expand Down Expand Up @@ -74,60 +75,61 @@ public String getClusterDescription() {
return CLUSTER_DESCRIPTION;
}

private ClusterClient<String> createClusterClient(String clusterId) throws Exception {
final Configuration configuration = new Configuration(flinkConfig);
private ClusterClientProvider<String> createClusterClientProvider(String clusterId) {
return () -> {
final Configuration configuration = new Configuration(flinkConfig);

final Endpoint restEndpoint = client.getRestEndpoint(clusterId);
final Endpoint restEndpoint = client.getRestEndpoint(clusterId);

if (restEndpoint != null) {
configuration.setString(RestOptions.ADDRESS, restEndpoint.getAddress());
configuration.setInteger(RestOptions.PORT, restEndpoint.getPort());
} else {
throw new ClusterRetrieveException("Could not get the rest endpoint of " + clusterId);
}
if (restEndpoint != null) {
configuration.setString(RestOptions.ADDRESS, restEndpoint.getAddress());
configuration.setInteger(RestOptions.PORT, restEndpoint.getPort());
} else {
throw new RuntimeException(
new ClusterRetrieveException(
"Could not get the rest endpoint of " + clusterId));
}

return new RestClusterClient<>(configuration, clusterId);
try {

RestClusterClient<String> resultClient = new RestClusterClient<>(
configuration, clusterId);
LOG.info(
"Succesfully retrieved cluster client for cluster {}, JobManager Web Interface : {}",
clusterId,
resultClient.getWebInterfaceURL());
return resultClient;
} catch (Exception e) {
client.handleException(e);
throw new RuntimeException(new ClusterRetrieveException("Could not create the RestClusterClient.", e));
}
};
}

@Override
public ClusterClient<String> retrieve(String clusterId) throws ClusterRetrieveException {
try {
final ClusterClient<String> retrievedClient = createClusterClient(clusterId);
LOG.info(
"Retrieve flink cluster {} successfully, JobManager Web Interface : {}",
clusterId,
retrievedClient.getWebInterfaceURL());
return retrievedClient;
} catch (Exception e) {
client.handleException(e);
throw new ClusterRetrieveException("Could not create the RestClusterClient.", e);
}
public ClusterClientProvider<String> retrieve(String clusterId) {
return createClusterClientProvider(clusterId);
}

@Override
public ClusterClient<String> deploySessionCluster(ClusterSpecification clusterSpecification) throws ClusterDeploymentException {
final ClusterClient<String> clusterClient = deployClusterInternal(
public ClusterClientProvider<String> deploySessionCluster(ClusterSpecification clusterSpecification) throws ClusterDeploymentException {
final ClusterClientProvider<String> clusterClient = deployClusterInternal(
KubernetesSessionClusterEntrypoint.class.getName(),
clusterSpecification,
false);

LOG.info(
"Create flink session cluster {} successfully, JobManager Web Interface: {}",
clusterId,
clusterClient.getWebInterfaceURL());

return clusterClient;
}

@Override
public ClusterClient<String> deployJobCluster(
public ClusterClientProvider<String> deployJobCluster(
ClusterSpecification clusterSpecification,
JobGraph jobGraph,
boolean detached) throws ClusterDeploymentException {
throw new ClusterDeploymentException("Per job could not be supported now.");
}

private ClusterClient<String> deployClusterInternal(
private ClusterClientProvider<String> deployClusterInternal(
String entryPoint,
ClusterSpecification clusterSpecification,
boolean detached) throws ClusterDeploymentException {
Expand Down Expand Up @@ -178,7 +180,16 @@ private ClusterClient<String> deployClusterInternal(
client.createConfigMap();
client.createFlinkMasterDeployment(clusterSpecification);

return createClusterClient(clusterId);
ClusterClientProvider<String> clusterClientProvider = createClusterClientProvider(clusterId);

try (ClusterClient<String> clusterClient = clusterClientProvider.getClusterClient()) {
LOG.info(
"Create flink session cluster {} successfully, JobManager Web Interface: {}",
clusterId,
clusterClient.getWebInterfaceURL());
}

return clusterClientProvider;
} catch (Exception e) {
client.handleException(e);
throw new ClusterDeploymentException("Could not create Kubernetes cluster " + clusterId, e);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -242,10 +242,12 @@ private int run(String[] args) throws CliArgsException, FlinkException {

// Retrieve or create a session cluster.
if (clusterId != null && kubeClient.getInternalService(clusterId) != null) {
clusterClient = kubernetesClusterDescriptor.retrieve(clusterId);
clusterClient = kubernetesClusterDescriptor.retrieve(clusterId).getClusterClient();
} else {
clusterClient = kubernetesClusterDescriptor.deploySessionCluster(
kubernetesClusterClientFactory.getClusterSpecification(configuration));
clusterClient = kubernetesClusterDescriptor
.deploySessionCluster(
kubernetesClusterClientFactory.getClusterSpecification(configuration))
.getClusterClient();
clusterId = clusterClient.getClusterId();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,8 @@ public void testDeploySessionCluster() throws Exception {
.setSlotsPerTaskManager(1)
.createClusterSpecification();

final ClusterClient<String> clusterClient = descriptor.deploySessionCluster(clusterSpecification);
final ClusterClient<String> clusterClient =
descriptor.deploySessionCluster(clusterSpecification).getClusterClient();

assertEquals(CLUSTER_ID, clusterClient.getClusterId());
assertEquals(String.format("http:https://%s:8081", MOCK_SERVICE_IP), clusterClient.getWebInterfaceURL());
Expand Down Expand Up @@ -91,6 +92,8 @@ public void testDeploySessionCluster() throws Exception {
assertEquals(
clusterSpecification.getMasterMemoryMB() + Constants.RESOURCE_UNIT_MB,
jmContainer.getResources().getLimits().get(Constants.RESOURCE_NAME_MEMORY).getAmount());

clusterClient.close();
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -251,7 +251,9 @@ object FlinkShell {
val clusterSpecification = clientFactory.getClusterSpecification(executorConfig)

val clusterClient = try {
clusterDescriptor.deploySessionCluster(clusterSpecification)
clusterDescriptor
.deploySessionCluster(clusterSpecification)
.getClusterClient
} finally {
clusterDescriptor.close()
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1671,10 +1671,6 @@ public CompletableFuture<JobClient> executeAsync(String jobName) throws Exceptio
* the program that have resulted in a "sink" operation. Sink operations are
* for example printing results or forwarding them to a message queue.
*
* <p><b>ATTENTION:</b> The caller of this method is responsible for managing the lifecycle of
* the returned {@link JobClient}. This means calling {@link JobClient#close()} at the end of
* its usage. In other case, there may be resource leaks depending on the JobClient implementation.
*
* @param streamGraph the stream graph representing the transformations
* @return A future of {@link JobClient} that can be used to communicate with the submitted job, completed on submission succeeded.
* @throws Exception which occurs during job execution.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -547,7 +547,7 @@ private <T> void cancelQueryInternal(ExecutionContext<T> context, String resultI
ClusterClient<T> clusterClient = null;
try {
// retrieve existing cluster
clusterClient = clusterDescriptor.retrieve(context.getClusterId());
clusterClient = clusterDescriptor.retrieve(context.getClusterId()).getClusterClient();
try {
clusterClient.cancel(new JobID(StringUtils.hexStringToByte(resultId))).get();
} catch (Throwable t) {
Expand Down
Loading

0 comments on commit 6825f80

Please sign in to comment.