Synchronize with cancellation when removing a task from a task group

We were detaching the child by just modifying the list, but the cancellation path was assuming that that would not be done without holding the task status lock.

This patch just fixes the current runtime; the back-deployment side is complicated.

Fixes rdar://88398824
This commit is contained in:
John McCall
2022-10-28 18:18:51 -04:00
parent e79aefa39f
commit 7f737d235d
13 changed files with 331 additions and 103 deletions

View File

@@ -44,10 +44,14 @@ public:
/// Checks the cancellation status of the group.
bool isCancelled();
// Add a child task to the group. Always called with the status record lock of
// the parent task held
// Add a child task to the task group. Always called while holding the
// status record lock of the task group's owning task.
void addChildTask(AsyncTask *task);
// Remove a child task from the task group. Always called while holding
// the status record lock of the task group's owning task.
void removeChildTask(AsyncTask *task);
// Provide accessor for task group's status record
TaskGroupTaskStatusRecord *getTaskRecord();
};

View File

@@ -150,15 +150,24 @@ public:
///
/// A record always is a specific `TaskGroupImpl`.
///
/// This record holds references to all the non-completed children of
/// the task group. It may also hold references to completed children
/// which have not yet been found by `next()`.
///
/// The child tasks are stored as an invasive single-linked list, starting
/// from `FirstChild` and continuing through the `NextChild` pointers of all
/// the linked children.
///
/// All children of the specific `Group` are stored "by" this record,
/// so that they may be cancelled when this task becomes cancelled.
/// This list structure should only ever be modified:
/// - while holding the status record lock of the owning task, so that
/// asynchronous operations such as cancellation can walk the structure
/// without having to acquire a secondary lock, and
/// - synchronously with the owning task, so that the owning task doesn't
/// have to acquire the status record lock just to walk the structure
/// itself.
///
/// When the group exits, it may simply remove this single record from the task
/// running it. As it has guaranteed that the tasks have already completed.
/// running it, as it has guaranteed that the tasks have already completed.
///
/// Group child tasks DO NOT have their own `ChildTaskStatusRecord` entries,
/// and are only tracked by their respective `TaskGroupTaskStatusRecord`.

View File

@@ -34,11 +34,11 @@
/// entries, or define one or more of the more specific OVERRIDE_* variants to
/// get only those entries.
// NOTE: this file is used to build the definition of OverrideSection in
// CompatibilityOverride.cpp, which is part of the ABI. Moving or removing
// entries in this file will break the ABI. Additional entries can be added to
// the end. ABI breaks or version-specific changes can be accommodated by
// changing the name of the override section in that file.
// NOTE: the entries in this file are used to build the struct layout for
// OverrideSection in CompatibilityOverride.cpp, which is built into the
// concurrency runtime. The back-deployment concurrency runtime targets
// the same compatibility overrides as the OS-deployed runtime in Swift 5.6.
// This file should not be edited.
#ifdef OVERRIDE
# define OVERRIDE_ACTOR OVERRIDE

View File

@@ -1,4 +1,4 @@
//===--- CompatibilityOverride.h - Back-deploying compatibility fixes --*- C++ -*-===//
//===--- CompatibilityOverride.h - Back-deployment patches ------*- C++ -*-===//
//
// This source file is part of the Swift.org open source project
//
@@ -10,7 +10,71 @@
//
//===----------------------------------------------------------------------===//
//
// Support back-deploying compatibility fixes for newer apps running on older runtimes.
// The compatibility override system supports the back-deployment of
// bug fixes and ABI additions to existing Swift systems. This is
// primarily of interest on Apple platforms, since other platforms don't
// ship with a stable Swift runtime that cannot simply be replaced.
//
// The compatibility override system works as follows:
//
// 1. Certain runtime functions are "hooked" so that they can be replaced
// by the program at launch time. The function at the public symbol
// (we will use `swift_task_cancel` as a consistent example) is a
// thunk that either calls the standard implementation or its dynamic
// replacement. If a dynamic replacement is called, it is passed
// the standard implementation as an extra argument.
//
// 2. The public thunks are defined in different .cpp files throughout
// the runtime by the COMPATIBILITY_OVERRIDE macro below, triggered
// by an #include at the bottom of the file. The list of definitions
// to expand in a particular file is defined by the appropriate
// CompatibilityOverride*.def file for the current runtime component
// (i.e. either the main runtime or the concurrency runtime).
// The standard implementation must be defined in that file as a
// function with the suffix `Impl`, e.g. `swift_task_cancelImpl`.
// Usually the standard implementation should be static, and
// everywhere else in the runtime should call the public symbol.
//
// 3. The public thunks determine their replacement by calling an
// override accessor for the symbol the first time they are called.
// These accessors are named e.g. `swift::getOverride_swift_task_cancel`
// and are defined by CompatibilityOverride.cpp, which is built
// separately for each runtime component using different build
// settings.
//
// 4. The override accessors check for a Mach-O section with a specific
// name, and if it exists, they interpret the section as a struct
// with one field per replaceable function. The order of fields is
// determined by the appropriate CompatibilityOverride*.def file.
// The section name, the struct layout, and the function signatures of
// the replacement functions are the only parts of this which are ABI;
// everything else is an internal detail of the runtime which can be
// changed if useful.
//
// 5. The name of the Mach-O section is specific to both the current
// runtime component and the current version of the Swift runtime.
// Therefore, a compatibility override library always targets a
// specific runtime version and implicitly does nothing on other
// versions.
//
// 6. Compatibility override libraries define a Mach-O section with the
// appropriate name and layout for their target component and version
// and initialize the appropriate fields within it to the replacement
// functions. This occurs in the Overrides.cpp file in the library.
// Compatibility override libraries are linked against later override
// libraries for the same component, so if a patch needs to be applied
// to multiple versions, the last version can define public symbols
// that the other versions can use (assuming that any internal runtime
// structures are roughly compatible).
//
// 7. Compatibility override libraries are rebuilt with every Swift
// release in case that release requires new patches to the target
// runtime. They are therefore live code, unlike e.g. the
// back-deployment concurrency runtime.
//
// 8. The back-deployment concurrency runtime looks for the same section
// name as the OS-installed 5.6 runtime and therefore will be patched
// by the 5.6 compatibility override library.
//
//===----------------------------------------------------------------------===//

View File

@@ -11,7 +11,8 @@
//===----------------------------------------------------------------------===//
//
// This file defines x-macros used for metaprogramming with the set of
// compatibility override functions.
// compatibility override functions. See CompatibilityOverride.h for
// a detailed explanation of how this system works.
//
//===----------------------------------------------------------------------===//
@@ -32,13 +33,42 @@
///
/// The entries are organized by group. A user may define OVERRIDE to get all
/// entries, or define one or more of the more specific OVERRIDE_* variants to
/// get only those entries.
/// get only those entries. The more specific OVERRIDE_* variants group
/// entries into the functions that are emitted in the specified file;
/// for example, OVERRIDE_ACTOR identifies the functions that are defined
/// in Actor.cpp.
// NOTE: this file is used to build the definition of OverrideSection in
// CompatibilityOverride.cpp, which is part of the ABI. Moving or removing
// entries in this file will break the ABI. Additional entries can be added to
// the end. ABI breaks or version-specific changes can be accommodated by
// changing the name of the override section in that file.
// NOTE: the entries in this file are used to build the struct layout for
// the OverrideSection in the CompatibilityOverride.cpp that is built into
// the concurrency runtime. A matching file must be used to build the
// ConcurrencyOverrideSection in Overrides.cpp for future compatibility
// override libraries that target this release.
//
// Because compatibility override libraries target a specific release of
// Swift, there is no inherent reason the entries in this file cannot be
// arbitrarily rearranged between release cycles, as long as a matching
// file is used to build any future compatibility override library
// targeting this release. However, the targeting of compatibility
// override libraries is precise only to a specific major+minor release
// number (e.g. 5.6). Therefore, care must be taken to avoid ABI breaks
// in this file between patch releases, or else it will become impossible
// to create a compatibility override library for this release:
//
// - Moving or removing entries in this file will break the ABI.
//
// - Changing an entry to use a different implementation file is allowed,
// but do not move the entry to be grouped with the other entries for
// the implementation file, as this will break the ABI.
//
// - New entries can be added to the end without breaking the ABI. This
// is possible even if there have already been patch releases for this
// major+minor release, since older patch releases of the runtime will
// simply not read the new fields. It is not possible if a compatibility
// override library has already been released for this major+minor
// release, but that is unlikely for releases under active development.
//
// When creating a new compatibility override library, always clone the
// last .def files from the appropriate release branch and edit this comment.
#ifdef OVERRIDE
# define OVERRIDE_ACTOR OVERRIDE
@@ -319,9 +349,9 @@ OVERRIDE_TASK_STATUS(task_hasTaskGroupStatusRecord, bool,
OVERRIDE_TASK_STATUS(task_cancel, void, SWIFT_EXPORT_FROM(swift_Concurrency),
SWIFT_CC(swift), swift::, (AsyncTask *task), (task))
OVERRIDE_TASK_STATUS(task_cancel_group_child_tasks, void,
SWIFT_EXPORT_FROM(swift_Concurrency), SWIFT_CC(swift),
swift::, (TaskGroup *group), (group))
OVERRIDE_TASK_GROUP(task_cancel_group_child_tasks, void,
SWIFT_EXPORT_FROM(swift_Concurrency), SWIFT_CC(swift),
swift::, (TaskGroup *group), (group))
OVERRIDE_TASK_STATUS(task_escalate, JobPriority,
SWIFT_EXPORT_FROM(swift_Concurrency), SWIFT_CC(swift),

View File

@@ -11,7 +11,8 @@
//===----------------------------------------------------------------------===//
//
// This file defines x-macros used for metaprogramming with the set of
// compatibility override functions.
// compatibility override functions. See CompatibilityOverride.h for
// a detailed explanation of how this system works.
//
//===----------------------------------------------------------------------===//
@@ -31,15 +32,43 @@
/// parentheses
///
/// The entries are organized by group. A user may define OVERRIDE to get all
/// entries, or define one or more of OVERRIDE_METADATALOOKUP, OVERRIDE_CASTING,
/// OVERRIDE_OBJC, OVERRIDE_FOREIGN, OVERRIDE_PROTOCOLCONFORMANCE,
/// and OVERRIDE_KEYPATH to get only those entries.
/// entries, or define one or more of the more specific OVERRIDE_* variants to
/// get only those entries. The more specific OVERRIDE_* variants group
/// entries into the functions that are emitted in the specified file;
/// for example, OVERRIDE_CASTING identifies the functions that are defined
/// in Casting.cpp.
// NOTE: this file is used to build the definition of OverrideSection in
// CompatibilityOverride.cpp, which is part of the ABI. Moving or removing
// entries in this file will break the ABI. Additional entries can be added to
// the end. ABI breaks or version-specific changes can be accommodated by
// changing the name of the override section in that file.
// NOTE: the entries in this file are used to build the struct layout for
// the OverrideSection in the CompatibilityOverride.cpp that is built into
// the primary runtime. A matching file must be used to build the
// RuntimeOverrideSection in Overrides.cpp for future compatibility
// override libraries that target this release.
//
// Because compatibility override libraries target a specific release of
// Swift, there is no inherent reason the entries in this file cannot be
// arbitrarily rearranged between release cycles, as long as a matching
// file is used to build any future compatibility override library
// targeting this release. However, the targeting of compatibility
// override libraries is precise only to a specific major+minor release
// number (e.g. 5.6). Therefore, care must be taken to avoid ABI breaks
// in this file between patch releases, or else it will become impossible
// to create a compatibility override library for this release:
//
// - Moving or removing entries in this file will break the ABI.
//
// - Changing an entry to use a different implementation file is allowed,
// but do not move the entry to be grouped with the other entries for
// the implementation file, as this will break the ABI.
//
// - New entries can be added to the end without breaking the ABI. This
// is possible even if there have already been patch releases for this
// major+minor release, since older patch releases of the runtime will
// simply not read the new fields. It is not possible if a compatibility
// override library has already been released for this major+minor
// release, but that is unlikely for releases under active development.
//
// When creating a new compatibility override library, always clone the
// last .def files from the appropriate release branch and edit this comment.
#ifdef OVERRIDE
# define OVERRIDE_METADATALOOKUP OVERRIDE

View File

@@ -313,8 +313,9 @@ private:
/// AsyncTask.
NaiveQueue<ReadyQueueItem> readyQueue;
/// Single waiting `AsyncTask` currently waiting on `group.next()`,
/// or `nullptr` if no task is currently waiting.
/// The task currently waiting on `group.next()`. Since only the owning
/// task can ever be waiting on a group, this is just either a reference
/// to that task or null.
std::atomic<AsyncTask *> waitQueue;
const Metadata *successType;
@@ -504,18 +505,37 @@ static void swift_taskGroup_initializeImpl(TaskGroup *group, const Metadata *T)
}
// =============================================================================
// ==== add / attachChild ------------------------------------------------------
// ==== child task management --------------------------------------------------
void TaskGroup::addChildTask(AsyncTask *child) {
SWIFT_TASK_DEBUG_LOG("attach child task = %p to group = %p", child, this);
// The counterpart of this (detachChild) is performed by the group itself,
// when it offers the completed (child) task's value to a waiting task -
// during the implementation of `await group.next()`.
// Add the child task to this task group. The corresponding removal
// won't happen until the parent task successfully polls for this child
// task, either synchronously in poll (if a task is available
// synchronously) or asynchronously in offer (otherwise). In either
// case, the work ends up being non-concurrent with the parent task.
// The task status record lock is held during this operation, which
// prevents us from racing with cancellation or escalation. We don't
// need to acquire the task group lock because the child list is only
// accessed under the task status record lock.
auto groupRecord = asImpl(this)->getTaskRecord();
groupRecord->attachChild(child);
}
void TaskGroup::removeChildTask(AsyncTask *child) {
SWIFT_TASK_DEBUG_LOG("detach child task = %p from group = %p", child, this);
auto groupRecord = asImpl(this)->getTaskRecord();
// The task status record lock is held during this operation, which
// prevents us from racing with cancellation or escalation. We don't
// need to acquire the task group lock because the child list is only
// accessed under the task status record lock.
groupRecord->detachChild(child);
}
// =============================================================================
// ==== destroy ----------------------------------------------------------------
SWIFT_CC(swift)
@@ -585,18 +605,18 @@ static void fillGroupNextResult(TaskFutureWaitAsyncContext *context,
// TaskGroup is locked upon entry and exit
void TaskGroupImpl::enqueueCompletedTask(AsyncTask *completedTask, bool hadErrorResult) {
// Retain the task while it is in the queue;
// it must remain alive until the task group is alive.
swift_retain(completedTask);
// Retain the task while it is in the queue; it must remain alive until
// it is found by poll. This retain will balanced by the release in poll.
swift_retain(completedTask);
auto readyItem = ReadyQueueItem::get(
hadErrorResult ? ReadyStatus::Error : ReadyStatus::Success,
completedTask
);
auto readyItem = ReadyQueueItem::get(
hadErrorResult ? ReadyStatus::Error : ReadyStatus::Success,
completedTask
);
assert(completedTask == readyItem.getTask());
assert(readyItem.getTask()->isFuture());
readyQueue.enqueue(readyItem);
assert(completedTask == readyItem.getTask());
assert(readyItem.getTask()->isFuture());
readyQueue.enqueue(readyItem);
}
void TaskGroupImpl::offer(AsyncTask *completedTask, AsyncContext *context) {
@@ -607,6 +627,15 @@ void TaskGroupImpl::offer(AsyncTask *completedTask, AsyncContext *context) {
assert(completedTask->groupChildFragment()->getGroup() == asAbstract(this));
SWIFT_TASK_DEBUG_LOG("offer task %p to group %p", completedTask, this);
// The current ownership convention is that we are *not* given ownership
// of a retain on completedTask; we're called from the task completion
// handler, and the task will release itself. So if we need the task
// to survive this call (e.g. because there isn't an immediate waiting
// task), we will need to retain it, which we do in enqueueCompletedTask.
// This is wasteful, and the task completion function should be fixed to
// transfer ownership of a retain into this function, in which case we
// will need to release in the other path.
lock(); // TODO: remove fragment lock, and use status for synchronization
// Immediately increment ready count and acquire the status
@@ -641,20 +670,25 @@ void TaskGroupImpl::offer(AsyncTask *completedTask, AsyncContext *context) {
waitingTask, nullptr,
/*success*/ std::memory_order_release,
/*failure*/ std::memory_order_acquire)) {
#if SWIFT_CONCURRENCY_TASK_TO_THREAD_MODEL
// We have completed a child task in a task group task and we know
// there is a waiting task who will reevaluate TaskGroupImpl::poll once
// we return, by virtue of being in the task-to-thread model.
// We want poll() to then satisfy the condition of having readyTasks()
// that it can dequeue from the readyQueue so we need to enqueue our
// completion.
// TODO (rokhinip): There's probably a more efficient way to deal with
// this since the child task can directly offer the result to the
// parent who will run next but that requires a fair bit of plumbing
enqueueCompletedTask(completedTask, hadErrorResult);
unlock(); // TODO: remove fragment lock, and use status for synchronization
return;
#if SWIFT_CONCURRENCY_TASK_TO_THREAD_MODEL
// In the task-to-thread model, child tasks are always actually
// run synchronously on the parent task's thread. For task groups
// specifically, this means that poll() will pick a child task
// that was added to the group and run it to completion as a
// subroutine. Therefore, when we enter offer(), we know that
// the parent task is waiting and we can just return to it.
// The task-to-thread logic in poll() currently expects the child
// task to enqueue itself instead of just filling in the result in
// the waiting task. This is a little wasteful; there's no reason
// we can't just have the parent task set itself up as a waiter.
// But since it's what we're doing, we basically take the same
// path as we would if there wasn't a waiter.
enqueueCompletedTask(completedTask, hadErrorResult);
unlock(); // TODO: remove fragment lock, and use status for synchronization
return;
#else /* SWIFT_CONCURRENCY_TASK_TO_THREAD_MODEL */
if (statusCompletePendingReadyWaiting(assumed)) {
// Run the task.
@@ -662,16 +696,28 @@ void TaskGroupImpl::offer(AsyncTask *completedTask, AsyncContext *context) {
unlock(); // TODO: remove fragment lock, and use status for synchronization
// Remove the child from the task group's running tasks list.
// The parent task isn't currently running (we're about to wake
// it up), so we're still synchronous with it. We can safely
// acquire our parent's status record lock here (which would
// ordinarily run the risk of deadlock, since e.g. cancellation
// does a parent -> child traversal while recursively holding
// locks) because we know that the child task is completed and
// we can't be holding its locks ourselves.
_swift_taskGroup_detachChild(asAbstract(this), completedTask);
auto waitingContext =
static_cast<TaskFutureWaitAsyncContext *>(
waitingTask->ResumeContext);
fillGroupNextResult(waitingContext, result);
detachChild(result.retainedTask);
_swift_tsan_acquire(static_cast<Job *>(waitingTask));
// TODO: allow the caller to suggest an executor
waitingTask->flagAsAndEnqueueOnExecutor(ExecutorRef::generic());
// completedTask will be released by the remainder of its
// completion function.
return;
} // else, try again
#endif
@@ -766,9 +812,11 @@ static void swift_taskGroup_wait_next_throwingImpl(
group, waitingTask, polled.retainedTask);
fillGroupNextResult(context, polled);
if (auto completedTask = polled.retainedTask) {
// it would be null for PollStatus::Empty, then we don't need to release
group->detachChild(polled.retainedTask);
swift_release(polled.retainedTask);
// Remove the child from the task group's running tasks list.
_swift_taskGroup_detachChild(asAbstract(group), completedTask);
// Balance the retain done by enqueueCompletedTask.
swift_release(completedTask);
}
return waitingTask->runInFullyEstablishedContext();
@@ -839,6 +887,10 @@ reevaluate_if_taskgroup_has_results:;
// be swift_release'd; we kept it alive while it was in the readyQueue by
// an additional retain issued as we enqueued it there.
result.retainedTask = item.getTask();
// Note that the task was detached from the task group when it
// completed, so we don't need to do that bit of record-keeping here.
switch (item.getStatus()) {
case ReadyStatus::Success:
// Immediately return the polled value
@@ -933,6 +985,7 @@ static bool swift_taskGroup_isCancelledImpl(TaskGroup *group) {
// =============================================================================
// ==== cancelAll --------------------------------------------------------------
SWIFT_CC(swift)
static void swift_taskGroup_cancelAllImpl(TaskGroup *group) {
asImpl(group)->cancelAll();
@@ -941,19 +994,45 @@ static void swift_taskGroup_cancelAllImpl(TaskGroup *group) {
bool TaskGroupImpl::cancelAll() {
SWIFT_TASK_DEBUG_LOG("cancel all tasks in group = %p", this);
// store the cancelled bit
// Flag the task group itself as cancelled. If this was already
// done, any existing child tasks should already have been cancelled,
// and cancellation should automatically flow to any new child tasks,
// so there's nothing else for us to do.
auto old = statusCancel();
if (old.isCancelled()) {
// already was cancelled previously, nothing to do?
return false;
}
// FIXME: must also remove the records!!!!
// cancel all existing tasks within the group
swift_task_cancel_group_child_tasks(asAbstract(this));
// Cancel all the child tasks. TaskGroup is not a Sendable type,
// so cancelAll() can only be called from the owning task. This
// satisfies the precondition on cancelAllChildren().
_swift_taskGroup_cancelAllChildren(asAbstract(this));
return true;
}
SWIFT_CC(swift)
static void swift_task_cancel_group_child_tasksImpl(TaskGroup *group) {
// TaskGroup is not a Sendable type, and so this operation (which is not
// currently exposed in the API) can only be called from the owning
// task. This satisfies the precondition on cancelAllChildren().
_swift_taskGroup_cancelAllChildren(group);
}
/// Cancel all the children of the given task group.
///
/// The caller must guarantee that this is either called from the
/// owning task of the task group or while holding the owning task's
/// status record lock.
void swift::_swift_taskGroup_cancelAllChildren(TaskGroup *group) {
// Because only the owning task of the task group can modify the
// child list of a task group status record, and it can only do so
// while holding the owning task's status record lock, we do not need
// any additional synchronization within this function.
for (auto childTask: group->getTaskRecord()->children())
swift_task_cancel(childTask);
}
// =============================================================================
// ==== addPending -------------------------------------------------------------
SWIFT_CC(swift)

View File

@@ -82,6 +82,19 @@ AsyncTask *_swift_task_clearCurrent();
/// Set the active task reference for the current thread.
AsyncTask *_swift_task_setCurrent(AsyncTask *newTask);
/// Cancel all the child tasks that belong to `group`.
///
/// The caller must guarantee that this is either called from the
/// owning task of the task group or while holding the owning task's
/// status record lock.
void _swift_taskGroup_cancelAllChildren(TaskGroup *group);
/// Remove the given task from the given task group.
///
/// This is an internal API; clients outside of the TaskGroup implementation
/// should generally use a higher-level function.
void _swift_taskGroup_detachChild(TaskGroup *group, AsyncTask *child);
/// release() establishes a happens-before relation with a preceding acquire()
/// on the same address.
void _swift_tsan_acquire(void *addr);

View File

@@ -387,7 +387,9 @@ static void swift_taskGroup_attachChildImpl(TaskGroup *group,
// Acquire the status record lock of parent - we want to synchronize with
// concurrent cancellation or escalation as we're adding new tasks to the
// group.
auto parent = swift_task_getCurrent();
auto parent = child->childFragment()->getParent();
assert(parent == swift_task_getCurrent());
withStatusRecordLock(parent, LockContext::OnTask, [&](ActiveTaskStatus &parentStatus) {
group->addChildTask(child);
@@ -403,6 +405,18 @@ static void swift_taskGroup_attachChildImpl(TaskGroup *group,
});
}
void swift::_swift_taskGroup_detachChild(TaskGroup *group,
AsyncTask *child) {
// We are called synchronously from the perspective of the owning task.
// That doesn't necessarily mean the owning task *is* the current task,
// though, just that it's not concurrently running.
auto parent = child->childFragment()->getParent();
withStatusRecordLock(parent, LockContext::OnTask, [&](ActiveTaskStatus &parentStatus) {
group->removeChildTask(child);
});
}
/****************************** CANCELLATION ******************************/
/**************************************************************************/
@@ -421,10 +435,12 @@ static void performCancellationAction(TaskStatusRecord *record) {
return;
}
// Task groups need their children to be cancelled. Note that we do
// not want to formally cancel the task group itself; that property is
// under the synchronous control of the task that owns the group.
case TaskStatusRecordKind::TaskGroup: {
auto childRecord = cast<TaskGroupTaskStatusRecord>(record);
for (AsyncTask *child: childRecord->children())
swift_task_cancel(child);
auto groupRecord = cast<TaskGroupTaskStatusRecord>(record);
_swift_taskGroup_cancelAllChildren(groupRecord->getGroup());
return;
}
@@ -494,22 +510,6 @@ static void swift_task_cancelImpl(AsyncTask *task) {
});
}
SWIFT_CC(swift)
static void swift_task_cancel_group_child_tasksImpl(TaskGroup *group) {
// Acquire the status record lock.
//
// Guaranteed to be called from the context of the parent task that created
// the task group once we have #40616
auto task = swift_task_getCurrent();
withStatusRecordLock(task, LockContext::OnTask,
[&](ActiveTaskStatus &status) {
// We purposefully DO NOT make this a cancellation by itself.
// We are cancelling the task group, and all tasks it contains.
// We are NOT cancelling the entire parent task though.
performCancellationAction(group->getTaskRecord());
});
}
/**************************************************************************/
/******************************* ESCALATION *******************************/
/**************************************************************************/

View File

@@ -34,8 +34,9 @@
/// and OVERRIDE_KEYPATH to get only those entries.
// NOTE: this file is used to build the definition of OverrideSection in
// CompatibilityOverride.cpp, which is part of the ABI. Do not move or remove entries
// in this file after ABI stability. Additional entries can be added to the end.
// Overrides.cpp for the 5.0 compatibility library. It matches the
// definition of OverrideSection for the 5.0 release and must not be
// edited.
#ifdef OVERRIDE
# define OVERRIDE_METADATALOOKUP OVERRIDE

View File

@@ -34,8 +34,9 @@
/// and OVERRIDE_KEYPATH to get only those entries.
// NOTE: this file is used to build the definition of OverrideSection in
// CompatibilityOverride.cpp, which is part of the ABI. Do not move or remove entries
// in this file after ABI stability. Additional entries can be added to the end.
// Overrides.cpp for the 5.1 compatibility library. It matches the
// definition of OverrideSection for the 5.1 release and must not be
// edited.
#ifdef OVERRIDE
# define OVERRIDE_METADATALOOKUP OVERRIDE

View File

@@ -34,11 +34,10 @@
/// entries, or define one or more of the more specific OVERRIDE_* variants to
/// get only those entries.
// NOTE: this file is used to build the definition of OverrideSection in
// CompatibilityOverride.cpp, which is part of the ABI. Moving or removing
// entries in this file will break the ABI. Additional entries can be added to
// the end. ABI breaks or version-specific changes can be accommodated by
// changing the name of the override section in that file.
// NOTE: this file is used to build the definition of
// ConcurrencyOverrideSection in Overrides.cpp for the 5.6 compatibility
// library. It matches the definition of the concurrency OverrideSection
// for the 5.6 release and must not be edited.
#ifdef OVERRIDE
# define OVERRIDE_ACTOR OVERRIDE

View File

@@ -35,11 +35,10 @@
/// OVERRIDE_OBJC, OVERRIDE_FOREIGN, OVERRIDE_PROTOCOLCONFORMANCE,
/// and OVERRIDE_KEYPATH to get only those entries.
// NOTE: this file is used to build the definition of OverrideSection in
// CompatibilityOverride.cpp, which is part of the ABI. Moving or removing
// entries in this file will break the ABI. Additional entries can be added to
// the end. ABI breaks or version-specific changes can be accommodated by
// changing the name of the override section in that file.
// NOTE: this file is used to build the definition of RuntimeOverrideSection
// in Overrides.cpp for the 5.6 compatibility library. It matches the
// definition of the primary-runtime OverrideSection for the 5.6 release
// and must not be edited.
#ifdef OVERRIDE
# define OVERRIDE_METADATALOOKUP OVERRIDE