Skip to content

[FLINK-37579] Avoid duplicate generate ResourceProfile #26377

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 1 commit into
base: master
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 @@ -614,13 +614,22 @@ public static TaskExecutor startTaskManager(
final TaskExecutorResourceSpec taskExecutorResourceSpec =
TaskExecutorResourceUtils.resourceSpecFromConfig(configuration);

TaskManagerConfiguration taskManagerConfiguration =
TaskManagerConfiguration.fromConfiguration(
configuration,
taskExecutorResourceSpec,
externalAddress,
workingDirectory.getTmpDirectory());

TaskManagerServicesConfiguration taskManagerServicesConfiguration =
TaskManagerServicesConfiguration.fromConfiguration(
configuration,
resourceID,
externalAddress,
localCommunicationOnly,
taskExecutorResourceSpec,
taskManagerConfiguration.getDefaultSlotResourceProfile(),
taskManagerConfiguration.getTotalResourceProfile(),
workingDirectory);

Tuple2<TaskManagerMetricGroup, MetricGroup> taskManagerMetricGroup =
Expand Down Expand Up @@ -650,13 +659,6 @@ public static TaskExecutor startTaskManager(
taskManagerServices.getTaskSlotTable(),
taskManagerServices::getManagedMemorySize);

TaskManagerConfiguration taskManagerConfiguration =
TaskManagerConfiguration.fromConfiguration(
configuration,
taskExecutorResourceSpec,
externalAddress,
workingDirectory.getTmpDirectory());

String metricQueryServiceAddress = metricRegistry.getMetricQueryServiceGatewayRpcAddress();

return new TaskExecutor(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.apache.flink.runtime.blob.PermanentBlobService;
import org.apache.flink.runtime.broadcast.BroadcastVariableManager;
import org.apache.flink.runtime.clusterframework.types.AllocationID;
import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
import org.apache.flink.runtime.deployment.TaskDeploymentDescriptorFactory.ShuffleDescriptorGroup;
import org.apache.flink.runtime.entrypoint.WorkingDirectory;
import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager;
Expand Down Expand Up @@ -374,7 +375,8 @@ public static TaskManagerServices fromConfiguration(
final TaskSlotTable<Task> taskSlotTable =
createTaskSlotTable(
taskManagerServicesConfiguration.getNumberOfSlots(),
taskManagerServicesConfiguration.getTaskExecutorResourceSpec(),
taskManagerServicesConfiguration.getTotalResourceProfile(),
taskManagerServicesConfiguration.getDefaultSlotResourceProfile(),
taskManagerServicesConfiguration.getTimerServiceShutdownTimeout(),
taskManagerServicesConfiguration.getPageSize(),
ioExecutor);
Expand Down Expand Up @@ -467,7 +469,8 @@ public static TaskManagerServices fromConfiguration(

private static TaskSlotTable<Task> createTaskSlotTable(
final int numberOfSlots,
final TaskExecutorResourceSpec taskExecutorResourceSpec,
final ResourceProfile totalAvailableResourceProfile,
final ResourceProfile defaultSlotResourceProfile,
final long timerServiceShutdownTimeout,
final int pageSize,
final Executor memoryVerificationExecutor) {
Expand All @@ -476,10 +479,8 @@ private static TaskSlotTable<Task> createTaskSlotTable(
new ScheduledThreadPoolExecutor(1), timerServiceShutdownTimeout);
return new TaskSlotTableImpl<>(
numberOfSlots,
TaskExecutorResourceUtils.generateTotalAvailableResourceProfile(
taskExecutorResourceSpec),
TaskExecutorResourceUtils.generateDefaultSlotResourceProfile(
taskExecutorResourceSpec, numberOfSlots),
totalAvailableResourceProfile,
defaultSlotResourceProfile,
pageSize,
timerService,
memoryVerificationExecutor);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import org.apache.flink.configuration.TaskManagerOptions;
import org.apache.flink.configuration.TaskManagerOptionsInternal;
import org.apache.flink.runtime.clusterframework.types.ResourceID;
import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
import org.apache.flink.runtime.entrypoint.ClusterEntrypointUtils;
import org.apache.flink.runtime.entrypoint.WorkingDirectory;
import org.apache.flink.runtime.registration.RetryingRegistrationConfiguration;
Expand Down Expand Up @@ -91,6 +92,10 @@ public class TaskManagerServicesConfiguration {

private final TaskExecutorResourceSpec taskExecutorResourceSpec;

private final ResourceProfile totalResourceProfile;

private final ResourceProfile defaultSlotResourceProfile;

private final FlinkUserCodeClassLoaders.ResolveOrder classLoaderResolveOrder;

private final String[] alwaysParentFirstLoaderPatterns;
Expand All @@ -112,6 +117,8 @@ private TaskManagerServicesConfiguration(
int numberOfSlots,
int pageSize,
TaskExecutorResourceSpec taskExecutorResourceSpec,
ResourceProfile defaultSlotResourceProfile,
ResourceProfile totalAvailableResourceProfile,
long timerServiceShutdownTimeout,
RetryingRegistrationConfiguration retryingRegistrationConfiguration,
Optional<Duration> systemResourceMetricsProbingInterval,
Expand All @@ -136,6 +143,8 @@ private TaskManagerServicesConfiguration(
this.pageSize = pageSize;

this.taskExecutorResourceSpec = taskExecutorResourceSpec;
this.defaultSlotResourceProfile = defaultSlotResourceProfile;
this.totalResourceProfile = totalAvailableResourceProfile;
this.classLoaderResolveOrder = classLoaderResolveOrder;
this.alwaysParentFirstLoaderPatterns = alwaysParentFirstLoaderPatterns;
this.numIoThreads = numIoThreads;
Expand Down Expand Up @@ -213,6 +222,14 @@ public TaskExecutorResourceSpec getTaskExecutorResourceSpec() {
return taskExecutorResourceSpec;
}

public ResourceProfile getDefaultSlotResourceProfile() {
return defaultSlotResourceProfile;
}

public ResourceProfile getTotalResourceProfile() {
return totalResourceProfile;
}

public MemorySize getNetworkMemorySize() {
return taskExecutorResourceSpec.getNetworkMemSize();
}
Expand Down Expand Up @@ -273,6 +290,8 @@ public static TaskManagerServicesConfiguration fromConfiguration(
String externalAddress,
boolean localCommunicationOnly,
TaskExecutorResourceSpec taskExecutorResourceSpec,
ResourceProfile defaultSlotResourceProfile,
ResourceProfile totalAvailableResourceProfile,
WorkingDirectory workingDirectory)
throws Exception {
String[] localStateRootDirs = ConfigurationUtils.parseLocalStateDirectories(configuration);
Expand Down Expand Up @@ -342,6 +361,8 @@ public static TaskManagerServicesConfiguration fromConfiguration(
ConfigurationParserUtils.getSlot(configuration),
ConfigurationParserUtils.getPageSize(configuration),
taskExecutorResourceSpec,
defaultSlotResourceProfile,
totalAvailableResourceProfile,
timerServiceShutdownTimeout,
retryingRegistrationConfiguration,
ConfigurationUtils.getSystemResourceMetricsProbingInterval(configuration),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,10 +29,12 @@
import org.apache.flink.runtime.entrypoint.WorkingDirectory;
import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
import org.apache.flink.runtime.taskexecutor.TaskExecutorResourceSpec;
import org.apache.flink.runtime.taskexecutor.TaskExecutorResourceUtils;
import org.apache.flink.runtime.taskexecutor.TaskManagerServices;
import org.apache.flink.runtime.taskexecutor.TaskManagerServicesConfiguration;
import org.apache.flink.runtime.testutils.WorkingDirectoryExtension;
import org.apache.flink.runtime.util.ConfigurationParserUtils;
import org.apache.flink.testutils.junit.utils.TempDirUtils;
import org.apache.flink.util.FileUtils;
import org.apache.flink.util.Reference;
Expand Down Expand Up @@ -423,12 +425,18 @@ private void checkRootDirsClean(File[] rootDirs) {

private TaskManagerServicesConfiguration createTaskManagerServiceConfiguration(
Configuration config, WorkingDirectory workingDirectory) throws Exception {
final TaskExecutorResourceSpec taskExecutorResourceSpec =
TaskExecutorResourceUtils.resourceSpecFromConfigForLocalExecution(config);
return TaskManagerServicesConfiguration.fromConfiguration(
config,
ResourceID.generate(),
InetAddress.getLocalHost().getHostName(),
true,
TaskExecutorResourceUtils.resourceSpecFromConfigForLocalExecution(config),
taskExecutorResourceSpec,
TaskExecutorResourceUtils.generateDefaultSlotResourceProfile(
taskExecutorResourceSpec, ConfigurationParserUtils.getSlot(config)),
TaskExecutorResourceUtils.generateTotalAvailableResourceProfile(
taskExecutorResourceSpec),
workingDirectory);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -132,6 +132,8 @@ public TaskExecutor build() throws Exception {
rpcService.getAddress(),
true,
taskExecutorResourceSpec,
resolvedTaskManagerConfiguration.getDefaultSlotResourceProfile(),
resolvedTaskManagerConfiguration.getTotalResourceProfile(),
workingDirectory);
resolvedTaskManagerServices =
TaskManagerServices.fromConfiguration(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
import org.apache.flink.runtime.rpc.AddressResolution;
import org.apache.flink.runtime.rpc.RpcService;
import org.apache.flink.runtime.rpc.RpcSystem;
import org.apache.flink.runtime.util.ConfigurationParserUtils;
import org.apache.flink.util.IOUtils;
import org.apache.flink.util.concurrent.Executors;

Expand Down Expand Up @@ -235,12 +236,18 @@ void testNodeIdShouldBeExternalAddressIfNotExplicitlySet() throws Exception {

private TaskManagerServicesConfiguration createTaskManagerServiceConfiguration(
Configuration config) throws Exception {
final TaskExecutorResourceSpec taskExecutorResourceSpec =
TaskExecutorResourceUtils.resourceSpecFromConfigForLocalExecution(config);
return TaskManagerServicesConfiguration.fromConfiguration(
config,
ResourceID.generate(),
InetAddress.getLocalHost().getHostName(),
true,
TaskExecutorResourceUtils.resourceSpecFromConfigForLocalExecution(config),
taskExecutorResourceSpec,
TaskExecutorResourceUtils.generateDefaultSlotResourceProfile(
taskExecutorResourceSpec, ConfigurationParserUtils.getSlot(config)),
TaskExecutorResourceUtils.generateTotalAvailableResourceProfile(
taskExecutorResourceSpec),
WorkingDirectory.create(
Files.createTempDirectory(temporaryFolder, UUID.randomUUID().toString())
.toFile()));
Expand Down