Skip to content

[FLINK-36876] Add proxy for restClient #956

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 3 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -140,6 +140,12 @@
<td>Duration</td>
<td>The timeout for the observer to wait the flink rest client to return.</td>
</tr>
<tr>
<td><h5>kubernetes.operator.flink.client.io.threads</h5></td>
<td style="word-wrap: break-word;">60</td>
<td>Integer</td>
<td>The maximum number of io threads used by the flink rest client.</td>
</tr>
<tr>
<td><h5>kubernetes.operator.health.canary.resource.timeout</h5></td>
<td style="word-wrap: break-word;">1 min</td>
Expand Down
6 changes: 6 additions & 0 deletions docs/layouts/shortcodes/generated/system_section.html
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,12 @@
<td>Duration</td>
<td>The timeout for the observer to wait the flink rest client to return.</td>
</tr>
<tr>
<td><h5>kubernetes.operator.flink.client.io.threads</h5></td>
<td style="word-wrap: break-word;">60</td>
<td>Integer</td>
<td>The maximum number of io threads used by the flink rest client.</td>
</tr>
<tr>
<td><h5>kubernetes.operator.leader-election.enabled</h5></td>
<td style="word-wrap: break-word;">false</td>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,7 @@ public class FlinkOperatorConfiguration {
Duration progressCheckInterval;
Duration restApiReadyDelay;
Duration flinkClientTimeout;
int flinkClientIOThreads;
String flinkServiceHostOverride;
Set<String> watchedNamespaces;
boolean dynamicNamespacesEnabled;
Expand Down Expand Up @@ -97,6 +98,10 @@ public static FlinkOperatorConfiguration fromConfiguration(Configuration operato
Duration flinkClientTimeout =
operatorConfig.get(KubernetesOperatorConfigOptions.OPERATOR_FLINK_CLIENT_TIMEOUT);

int flinkClientIOThreads =
operatorConfig.getInteger(
KubernetesOperatorConfigOptions.OPERATOR_FLINK_CLIENT_IO_THREADS);

Duration flinkCancelJobTimeout =
operatorConfig.get(
KubernetesOperatorConfigOptions.OPERATOR_FLINK_CLIENT_CANCEL_TIMEOUT);
Expand Down Expand Up @@ -201,6 +206,7 @@ public static FlinkOperatorConfiguration fromConfiguration(Configuration operato
progressCheckInterval,
restApiReadyDelay,
flinkClientTimeout,
flinkClientIOThreads,
flinkServiceHostOverride,
watchedNamespaces,
dynamicNamespacesEnabled,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -115,6 +115,14 @@ public static String operatorConfigKey(String key) {
.withDescription(
"The timeout for the observer to wait the flink rest client to return.");

@Documentation.Section(SECTION_SYSTEM)
public static final ConfigOption<Integer> OPERATOR_FLINK_CLIENT_IO_THREADS =
operatorConfig("flink.client.io.threads")
.intType()
.defaultValue(60)
.withDescription(
"The maximum number of io threads used by the flink rest client.");

@Documentation.Section(SECTION_SYSTEM)
public static final ConfigOption<Duration> OPERATOR_FLINK_CLIENT_CANCEL_TIMEOUT =
operatorConfig("flink.client.cancel.timeout")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,8 @@
import org.apache.flink.autoscaler.config.AutoScalerOptions;
import org.apache.flink.autoscaler.utils.JobStatusUtils;
import org.apache.flink.client.program.rest.RestClusterClient;
import org.apache.flink.client.program.rest.retry.ExponentialWaitStrategy;
import org.apache.flink.client.program.rest.retry.WaitStrategy;
import org.apache.flink.configuration.CheckpointingOptions;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.RestOptions;
Expand Down Expand Up @@ -55,6 +57,7 @@
import org.apache.flink.kubernetes.operator.utils.ExceptionUtils;
import org.apache.flink.kubernetes.operator.utils.FlinkUtils;
import org.apache.flink.runtime.client.JobStatusMessage;
import org.apache.flink.runtime.highavailability.ClientHighAvailabilityServicesFactory;
import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneClientHAServices;
import org.apache.flink.runtime.jobmaster.JobResult;
import org.apache.flink.runtime.messages.FlinkJobNotFoundException;
Expand Down Expand Up @@ -105,6 +108,7 @@
import org.apache.flink.util.Preconditions;

import org.apache.flink.shaded.guava31.com.google.common.collect.Iterables;
import org.apache.flink.shaded.netty4.io.netty.channel.EventLoopGroup;
import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;

import io.fabric8.kubernetes.api.model.DeletionPropagation;
Expand All @@ -125,6 +129,7 @@
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.lang.reflect.Constructor;
import java.net.HttpURLConnection;
import java.net.InetSocketAddress;
import java.net.MalformedURLException;
Expand Down Expand Up @@ -171,17 +176,20 @@ public abstract class AbstractFlinkService implements FlinkService {
protected final ExecutorService executorService;
protected final FlinkOperatorConfiguration operatorConfig;
protected final ArtifactManager artifactManager;
private final EventLoopGroup flinkClientSharedEventLoopGroup;
private static final String EMPTY_JAR = createEmptyJar();

public AbstractFlinkService(
KubernetesClient kubernetesClient,
ArtifactManager artifactManager,
ExecutorService executorService,
FlinkOperatorConfiguration operatorConfig) {
FlinkOperatorConfiguration operatorConfig,
EventLoopGroup flinkClientEventLoopGroup) {
this.kubernetesClient = kubernetesClient;
this.artifactManager = artifactManager;
this.executorService = executorService;
this.operatorConfig = operatorConfig;
this.flinkClientSharedEventLoopGroup = flinkClientEventLoopGroup;
}

protected abstract PodList getJmPodList(String namespace, String clusterId);
Expand Down Expand Up @@ -838,11 +846,34 @@ public RestClusterClient<String> getClusterClient(Configuration conf) throws Exc
operatorConfig.getFlinkServiceHostOverride(),
ExternalServiceDecorator.getNamespacedExternalServiceName(
clusterId, namespace));

final String restServerAddress = String.format("http://%s:%s", host, port);
return new RestClusterClient<>(
operatorRestConf,
clusterId,
(c, e) -> new StandaloneClientHAServices(restServerAddress));
RestClient restClient =
new RestClientProxy(
operatorRestConf,
executorService,
host,
port,
flinkClientSharedEventLoopGroup);

Constructor<?> clusterClientConstructor =
RestClusterClient.class.getDeclaredConstructor(
Configuration.class,
RestClient.class,
Object.class,
WaitStrategy.class,
ClientHighAvailabilityServicesFactory.class);

clusterClientConstructor.setAccessible(true);

return (RestClusterClient<String>)
clusterClientConstructor.newInstance(
operatorRestConf,
restClient,
clusterId,
new ExponentialWaitStrategy(10L, 2000L),
(ClientHighAvailabilityServicesFactory)
(c, e) -> new StandaloneClientHAServices(restServerAddress));
}

@VisibleForTesting
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,8 @@
import org.apache.flink.util.concurrent.ExecutorThreadFactory;

import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableMap;
import org.apache.flink.shaded.netty4.io.netty.channel.EventLoopGroup;
import org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup;

import io.javaoperatorsdk.operator.api.reconciler.Context;
import io.javaoperatorsdk.operator.processing.event.ResourceID;
Expand All @@ -60,6 +62,7 @@ public class FlinkResourceContextFactory {

protected final Map<Tuple2<Class<?>, ResourceID>, KubernetesResourceMetricGroup>
resourceMetricGroups = new ConcurrentHashMap<>();
private final EventLoopGroup flinkClientEventLoopGroup;

public FlinkResourceContextFactory(
FlinkConfigManager configManager,
Expand All @@ -73,6 +76,10 @@ public FlinkResourceContextFactory(
Executors.newFixedThreadPool(
configManager.getOperatorConfiguration().getReconcilerMaxParallelism(),
new ExecutorThreadFactory("Flink-RestClusterClient-IO"));
this.flinkClientEventLoopGroup =
new NioEventLoopGroup(
configManager.getOperatorConfiguration().getFlinkClientIOThreads(),
new ExecutorThreadFactory("flink-rest-client-netty-shared"));
}

public FlinkStateSnapshotContext getFlinkStateSnapshotContext(
Expand Down Expand Up @@ -123,13 +130,15 @@ protected FlinkService getFlinkService(FlinkResourceContext<?> ctx) {
artifactManager,
clientExecutorService,
ctx.getOperatorConfig(),
eventRecorder);
eventRecorder,
flinkClientEventLoopGroup);
case STANDALONE:
return new StandaloneFlinkService(
ctx.getKubernetesClient(),
artifactManager,
clientExecutorService,
ctx.getOperatorConfig());
ctx.getOperatorConfig(),
flinkClientEventLoopGroup);
default:
throw new UnsupportedOperationException(
String.format("Unsupported deployment mode: %s", deploymentMode));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,8 @@
import org.apache.flink.runtime.rest.messages.job.JobResourceRequirementsHeaders;
import org.apache.flink.runtime.rest.messages.job.JobResourcesRequirementsUpdateHeaders;

import org.apache.flink.shaded.netty4.io.netty.channel.EventLoopGroup;

import io.fabric8.kubernetes.api.model.DeletionPropagation;
import io.fabric8.kubernetes.api.model.PodList;
import io.fabric8.kubernetes.api.model.apps.Deployment;
Expand Down Expand Up @@ -89,7 +91,28 @@ public NativeFlinkService(
ExecutorService executorService,
FlinkOperatorConfiguration operatorConfig,
EventRecorder eventRecorder) {
super(kubernetesClient, artifactManager, executorService, operatorConfig);
this(
kubernetesClient,
artifactManager,
executorService,
operatorConfig,
eventRecorder,
null);
}

public NativeFlinkService(
KubernetesClient kubernetesClient,
ArtifactManager artifactManager,
ExecutorService executorService,
FlinkOperatorConfiguration operatorConfig,
EventRecorder eventRecorder,
EventLoopGroup flinkClientEventLoopGroup) {
super(
kubernetesClient,
artifactManager,
executorService,
operatorConfig,
flinkClientEventLoopGroup);
this.eventRecorder = eventRecorder;
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,110 @@
/*
* 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.operator.service;

import org.apache.flink.api.common.time.Time;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.rest.RestClient;
import org.apache.flink.util.ConfigurationException;
import org.apache.flink.util.Preconditions;

import org.apache.flink.shaded.netty4.io.netty.bootstrap.AbstractBootstrap;
import org.apache.flink.shaded.netty4.io.netty.bootstrap.Bootstrap;
import org.apache.flink.shaded.netty4.io.netty.channel.EventLoopGroup;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.lang.reflect.Field;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;

/** Proxy for the {@link RestClient}. */
public class RestClientProxy extends RestClient {
private static final Logger LOG = LoggerFactory.getLogger(RestClientProxy.class);

private final boolean useSharedEventLoopGroup;
private Field groupField;
private Bootstrap bootstrap;
private CompletableFuture<Void> terminationFuture;

public RestClientProxy(
Configuration configuration,
ExecutorService executor,
String host,
int port,
EventLoopGroup sharedGroup)
throws ConfigurationException, NoSuchFieldException, IllegalAccessException {
super(configuration, executor, host, port);

if (sharedGroup != null) {
Preconditions.checkArgument(
!sharedGroup.isShuttingDown() && !sharedGroup.isShutdown(),
"provided eventLoopGroup is shut/shutting down");

// get private field
Field bootstrapField = RestClient.class.getDeclaredField("bootstrap");
Field terminationFutureField = RestClient.class.getDeclaredField("terminationFuture");
Field groupField = AbstractBootstrap.class.getDeclaredField("group");

bootstrapField.setAccessible(true);
terminationFutureField.setAccessible(true);
groupField.setAccessible(true);

this.terminationFuture = (CompletableFuture<Void>) terminationFutureField.get(this);
this.bootstrap = (Bootstrap) bootstrapField.get(this);
this.groupField = groupField;

// close previous group
bootstrap.config().group().shutdown();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How will this actually solve the original problem? The rest client still creates an individual group (that we close here) won't that cause the same issue anyways?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Because PoolThreadCache is a request-thread-related cache, it is created and initialized when a request is made using the NioEventLoopGroup;

here it is created and closed immediately, with no requests in between, so there is no PoolThreadCache leakage issue.

// setup share group
groupField.set(bootstrap, sharedGroup);

useSharedEventLoopGroup = true;
} else {
useSharedEventLoopGroup = false;
}
}

@Override
public CompletableFuture<Void> closeAsync() {
if (useSharedEventLoopGroup) {
this.shutdownInternal();
}

return super.closeAsync();
}

@Override
public void shutdown(Time timeout) {
if (useSharedEventLoopGroup) {
this.shutdownInternal();
}
super.shutdown(timeout);
}

private void shutdownInternal() {
try {
// replace bootstrap's group to null to avoid shutdown shared group
groupField.set(bootstrap, null);
terminationFuture.complete(null);
} catch (IllegalAccessException e) {
LOG.error("Failed to setup rest client event group .", e);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,8 @@
import org.apache.flink.kubernetes.operator.utils.StandaloneKubernetesUtils;
import org.apache.flink.util.concurrent.ExecutorThreadFactory;

import org.apache.flink.shaded.netty4.io.netty.channel.EventLoopGroup;

import io.fabric8.kubernetes.api.model.DeletionPropagation;
import io.fabric8.kubernetes.api.model.PodList;
import io.fabric8.kubernetes.client.KubernetesClient;
Expand All @@ -61,7 +63,21 @@ public StandaloneFlinkService(
ArtifactManager artifactManager,
ExecutorService executorService,
FlinkOperatorConfiguration operatorConfig) {
super(kubernetesClient, artifactManager, executorService, operatorConfig);
this(kubernetesClient, artifactManager, executorService, operatorConfig, null);
}

public StandaloneFlinkService(
KubernetesClient kubernetesClient,
ArtifactManager artifactManager,
ExecutorService executorService,
FlinkOperatorConfiguration operatorConfig,
EventLoopGroup flinkClientEventLoopGroup) {
super(
kubernetesClient,
artifactManager,
executorService,
operatorConfig,
flinkClientEventLoopGroup);
}

@Override
Expand Down
Loading
Loading