Skip to content
Merged
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 @@ -34,8 +34,10 @@
import org.apache.flink.runtime.clusterframework.types.ResourceID;
import org.apache.flink.runtime.clusterframework.types.SlotID;
import org.apache.flink.runtime.concurrent.FutureUtils;
import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor;
import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
import org.apache.flink.runtime.entrypoint.ClusterInformation;
import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager;
import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
Expand Down Expand Up @@ -75,6 +77,7 @@
import org.apache.flink.runtime.rpc.RpcEndpoint;
import org.apache.flink.runtime.rpc.RpcService;
import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils;
import org.apache.flink.runtime.shuffle.ShuffleDescriptor;
import org.apache.flink.runtime.shuffle.ShuffleEnvironment;
import org.apache.flink.runtime.state.TaskExecutorLocalStateStoresManager;
import org.apache.flink.runtime.state.TaskLocalStateStore;
Expand All @@ -86,6 +89,7 @@
import org.apache.flink.runtime.taskexecutor.exceptions.TaskException;
import org.apache.flink.runtime.taskexecutor.exceptions.TaskManagerException;
import org.apache.flink.runtime.taskexecutor.exceptions.TaskSubmissionException;
import org.apache.flink.runtime.taskexecutor.partition.PartitionTable;
import org.apache.flink.runtime.taskexecutor.rpc.RpcCheckpointResponder;
import org.apache.flink.runtime.taskexecutor.rpc.RpcGlobalAggregateManager;
import org.apache.flink.runtime.taskexecutor.rpc.RpcInputSplitProvider;
Expand Down Expand Up @@ -204,6 +208,8 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway {
/** The heartbeat manager for resource manager in the task manager. */
private HeartbeatManager<Void, SlotReport> resourceManagerHeartbeatManager;

private final PartitionTable partitionTable;

// --------- resource manager --------

@Nullable
Expand All @@ -220,6 +226,8 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway {

private final StackTraceSampleService stackTraceSampleService;

private Map<JobID, Collection<CompletableFuture<ExecutionState>>> taskResultPartitionCleanupFuturesPerJob = new HashMap<>(8);

public TaskExecutor(
RpcService rpcService,
TaskManagerConfiguration taskManagerConfiguration,
Expand All @@ -229,7 +237,8 @@ public TaskExecutor(
TaskManagerMetricGroup taskManagerMetricGroup,
@Nullable String metricQueryServiceAddress,
BlobCacheService blobCacheService,
FatalErrorHandler fatalErrorHandler) {
FatalErrorHandler fatalErrorHandler,
PartitionTable partitionTable) {

super(rpcService, AkkaRpcServiceUtils.createRandomName(TASK_MANAGER_NAME));

Expand All @@ -240,6 +249,7 @@ public TaskExecutor(
this.taskExecutorServices = checkNotNull(taskExecutorServices);
this.haServices = checkNotNull(haServices);
this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
this.partitionTable = partitionTable;
this.taskManagerMetricGroup = checkNotNull(taskManagerMetricGroup);
this.blobCacheService = checkNotNull(blobCacheService);
this.metricQueryServiceAddress = metricQueryServiceAddress;
Expand Down Expand Up @@ -587,6 +597,7 @@ public CompletableFuture<Acknowledge> submitTask(
if (taskAdded) {
task.startTaskThread();

setupResultPartitionBookkeeping(tdd, task.getTerminationFuture());
return CompletableFuture.completedFuture(Acknowledge.get());
} else {
final String message = "TaskManager already contains a task for id " +
Expand All @@ -600,6 +611,40 @@ public CompletableFuture<Acknowledge> submitTask(
}
}

private void setupResultPartitionBookkeeping(TaskDeploymentDescriptor tdd, CompletableFuture<ExecutionState> terminationFuture) {
final List<ResultPartitionID> partitionsRequiringRelease = tdd.getProducedPartitions().stream()
// partitions that are released on consumption don't require any explicit release call
.filter(d -> !d.isReleasedOnConsumption())
.map(ResultPartitionDeploymentDescriptor::getShuffleDescriptor)
// partitions without local resources don't store anything on the TaskExecutor
.filter(d -> d.storesLocalResourcesOn().isPresent())
.map(ShuffleDescriptor::getResultPartitionID)
.collect(Collectors.toList());

partitionTable.startTrackingPartitions(tdd.getJobId(), partitionsRequiringRelease);

final CompletableFuture<ExecutionState> taskTerminationWithResourceCleanupFuture =
terminationFuture.thenApplyAsync(
executionState -> {
if (executionState != ExecutionState.FINISHED) {
partitionTable.stopTrackingPartitions(tdd.getJobId(), partitionsRequiringRelease);
}
return executionState;
},
getMainThreadExecutor());

taskResultPartitionCleanupFuturesPerJob.compute(
tdd.getJobId(),
(jobID, completableFutures) -> {
if (completableFutures == null) {
completableFutures = new ArrayList<>(4);
}

completableFutures.add(taskTerminationWithResourceCleanupFuture);
return completableFutures;
});
}

@Override
public CompletableFuture<Acknowledge> cancelTask(ExecutionAttemptID executionAttemptID, Time timeout) {
final Task task = taskSlotTable.getTask(executionAttemptID);
Expand Down Expand Up @@ -666,7 +711,9 @@ public CompletableFuture<Acknowledge> updatePartitions(
@Override
public void releasePartitions(JobID jobId, Collection<ResultPartitionID> partitionIds) {
try {
partitionTable.stopTrackingPartitions(jobId, partitionIds);
shuffleEnvironment.releasePartitionsLocally(partitionIds);
closeJobManagerConnectionIfNoAllocatedResources(jobId);
} catch (Throwable t) {
// TODO: Do we still need this catch branch?
onFatalError(t);
Expand Down Expand Up @@ -1278,10 +1325,15 @@ private JobManagerConnection associateWithJobManager(
private void disassociateFromJobManager(JobManagerConnection jobManagerConnection, Exception cause) throws IOException {
checkNotNull(jobManagerConnection);

final JobID jobId = jobManagerConnection.getJobID();

// cleanup remaining partitions once all tasks for this job have completed
scheduleResultPartitionCleanup(jobId);

final KvStateRegistry kvStateRegistry = kvStateService.getKvStateRegistry();

if (kvStateRegistry != null) {
kvStateRegistry.unregisterListener(jobManagerConnection.getJobID());
kvStateRegistry.unregisterListener(jobId);
}

final KvStateClientProxy kvStateClientProxy = kvStateService.getKvStateClientProxy();
Expand All @@ -1295,6 +1347,17 @@ private void disassociateFromJobManager(JobManagerConnection jobManagerConnectio
jobManagerConnection.getLibraryCacheManager().shutdown();
}

private void scheduleResultPartitionCleanup(JobID jobId) {
final Collection<CompletableFuture<ExecutionState>> taskTerminationFutures = taskResultPartitionCleanupFuturesPerJob.remove(jobId);
if (taskTerminationFutures != null) {
FutureUtils.waitForAll(taskTerminationFutures)
.thenRunAsync(() -> {
Collection<ResultPartitionID> partitionsForJob = partitionTable.stopTrackingPartitions(jobId);
shuffleEnvironment.releasePartitionsLocally(partitionsForJob);
}, getMainThreadExecutor());
}
}

private void registerQueryableState(JobID jobId, JobMasterGateway jobMasterGateway) {
final KvStateServer kvStateServer = kvStateService.getKvStateServer();
final KvStateRegistry kvStateRegistry = kvStateService.getKvStateRegistry();
Expand Down Expand Up @@ -1404,20 +1467,7 @@ private void freeSlotInternal(AllocationID allocationId, Throwable cause) {
}

if (jobId != null) {
// check whether we still have allocated slots for the same job
if (taskSlotTable.getAllocationIdsPerJob(jobId).isEmpty()) {
// we can remove the job from the job leader service
try {
jobLeaderService.removeJob(jobId);
} catch (Exception e) {
log.info("Could not remove job {} from JobLeaderService.", jobId, e);
}

closeJobManagerConnection(
jobId,
new FlinkException("TaskExecutor " + getAddress() +
" has no more allocated slots for job " + jobId + '.'));
}
closeJobManagerConnectionIfNoAllocatedResources(jobId);
}
}
} catch (SlotNotFoundException e) {
Expand All @@ -1427,6 +1477,23 @@ private void freeSlotInternal(AllocationID allocationId, Throwable cause) {
localStateStoresManager.releaseLocalStateForAllocationId(allocationId);
}

private void closeJobManagerConnectionIfNoAllocatedResources(JobID jobId) {
// check whether we still have allocated slots for the same job
if (taskSlotTable.getAllocationIdsPerJob(jobId).isEmpty() && !partitionTable.hasTrackedPartitions(jobId)) {
// we can remove the job from the job leader service
try {
jobLeaderService.removeJob(jobId);
Copy link
Contributor

Choose a reason for hiding this comment

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

Why do we need the jobLeaderService if there are only partitions but no slots allocated to a given job?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I'm not sure; I'm just delayed all operations that were done before.

For the time being we could probably remove the job from the jobLeaderSerbice once the last slot was freed. After all, if a jobmaster failover occurs we remove all partitions anyway.

Long-term though, once we introduce a grace-period for reconnects and/or failovers, I would've expected that we want to continue to observe for leader changes.

} catch (Exception e) {
log.info("Could not remove job {} from JobLeaderService.", jobId, e);
}

closeJobManagerConnection(
jobId,
new FlinkException("TaskExecutor " + getAddress() +
" has no more allocated slots for job " + jobId + '.'));
}
}

private void timeoutSlot(AllocationID allocationId, UUID ticket) {
checkNotNull(allocationId);
checkNotNull(ticket);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@
import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils;
import org.apache.flink.runtime.security.SecurityConfiguration;
import org.apache.flink.runtime.security.SecurityUtils;
import org.apache.flink.runtime.taskexecutor.partition.PartitionTable;
import org.apache.flink.runtime.taskmanager.MemoryLogger;
import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
import org.apache.flink.runtime.util.EnvironmentInformation;
Expand Down Expand Up @@ -384,7 +385,8 @@ public static TaskExecutor startTaskManager(
taskManagerMetricGroup.f0,
metricQueryServiceAddress,
blobCacheService,
fatalErrorHandler);
fatalErrorHandler,
new PartitionTable());
}

/**
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,93 @@
/*
* 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.runtime.taskexecutor.partition;

import org.apache.flink.api.common.JobID;
import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
import org.apache.flink.util.Preconditions;

import javax.annotation.concurrent.ThreadSafe;

import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;

/**
* Thread-safe Utility for tracking partitions.
*/
@ThreadSafe
public class PartitionTable {

private final Map<JobID, Set<ResultPartitionID>> trackedPartitionsPerJob = new ConcurrentHashMap<>(8);
Copy link
Contributor

Choose a reason for hiding this comment

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

We can also consider per task ExecutionAttemptID mapping to support release by task id, then external from JM and connection loss release could be unified with scheduling the release per task the same way.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This could be done yes. I'd revisit this once everything is in place and we can better gauge what would be the most appropriate mappings.


/**
* Returns whether any partitions are being tracked for the given job.
*/
public boolean hasTrackedPartitions(JobID jobId) {
return trackedPartitionsPerJob.containsKey(jobId);
}

/**
* Starts the tracking of the given partition for the given job.
*/
public void startTrackingPartitions(JobID jobId, Collection<ResultPartitionID> newPartitionIds) {
Preconditions.checkNotNull(jobId);
Preconditions.checkNotNull(newPartitionIds);

trackedPartitionsPerJob.compute(jobId, (ignored, partitionIds) -> {
if (partitionIds == null) {
partitionIds = new HashSet<>(8);
}
partitionIds.addAll(newPartitionIds);
return partitionIds;
});
}

/**
* Stops the tracking of all partition for the given job.
*/
public Collection<ResultPartitionID> stopTrackingPartitions(JobID jobId) {
Preconditions.checkNotNull(jobId);

Set<ResultPartitionID> storedPartitions = trackedPartitionsPerJob.remove(jobId);
return storedPartitions == null
? Collections.emptyList()
: storedPartitions;
}

/**
* Stops the tracking of the given set of partitions for the given job.
*/
public void stopTrackingPartitions(JobID jobId, Collection<ResultPartitionID> partitionIds) {
Preconditions.checkNotNull(jobId);
Preconditions.checkNotNull(partitionIds);

// If the JobID is unknown we do not fail here, in line with ShuffleEnvironment#releaseFinishedPartitions
trackedPartitionsPerJob.computeIfPresent(
jobId,
(key, resultPartitionIDS) -> {
resultPartitionIDS.removeAll(partitionIds);
return resultPartitionIDS.isEmpty()
? null
: resultPartitionIDS;
});
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,12 @@

package org.apache.flink.runtime.io.network.partition;

import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor;
import org.apache.flink.runtime.io.network.NettyShuffleEnvironment;
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
import org.apache.flink.runtime.shuffle.PartitionDescriptor;
import org.apache.flink.runtime.shuffle.ShuffleDescriptor;
import org.apache.flink.runtime.util.NettyShuffleDescriptorBuilder;

import org.hamcrest.Matchers;

Expand Down Expand Up @@ -64,4 +69,19 @@ static void verifyCreateSubpartitionViewThrowsException(
assertThat(partitionId, Matchers.is(notFound.getPartitionId()));
}
}

public static ResultPartitionDeploymentDescriptor createPartitionDeploymentDescriptor(ResultPartitionType partitionType) {
ShuffleDescriptor shuffleDescriptor = NettyShuffleDescriptorBuilder.newBuilder().buildLocal();
PartitionDescriptor partitionDescriptor = new PartitionDescriptor(
new IntermediateDataSetID(),
shuffleDescriptor.getResultPartitionID().getPartitionId(),
partitionType,
1,
0);
return new ResultPartitionDeploymentDescriptor(
partitionDescriptor,
shuffleDescriptor,
1,
true);
}
}
Loading