mirror of
https://github.com/apple/swift.git
synced 2025-12-14 20:36:38 +01:00
Change the logic for adding new task status records to a task
This change has two parts to it: 1. Add in a new interface (addStatusRecordWithChecks) for adding task status records that also takes in a function ref. This function ref will be used to evaluate if current state of the parent task has any changes that need to be propagated to the child task that has been created. This is necessary to prevent the following race between task creation and concurrent cancellation and escalation: a. Parent task create child task. It does lazy relaxed loads on its own state while doing so and propagates this state to the child. b. Child task is created but has not been attached to the parent task/task group. c. Parent task gets cancelled by another thread. d. Child task gets linked into the parent’s task status records but no reevaluation has happened to account for changes that might have happened to the parent after (a). 2. Move status record management functions from the Runtime/Concurrency.h to TaskPrivate.h. Remove any corresponding overrides that are no longer needed. Remove unused tryAddStatusRecord method whose functionality is provided by addStatusRecordWithChecks. Radar-Id: rdar://problem/86347801
This commit is contained in:
@@ -35,7 +35,7 @@ namespace swift {
|
|||||||
/// access by a cancelling thread. In particular, the chain of
|
/// access by a cancelling thread. In particular, the chain of
|
||||||
/// status records must not be disturbed. When the task leaves
|
/// status records must not be disturbed. When the task leaves
|
||||||
/// the scope that requires the status record, the record can
|
/// the scope that requires the status record, the record can
|
||||||
/// be unregistered from the task with `swift_task_removeStatusRecord`,
|
/// be unregistered from the task with `removeStatusRecord`,
|
||||||
/// at which point the memory can be returned to the system.
|
/// at which point the memory can be returned to the system.
|
||||||
class TaskStatusRecord {
|
class TaskStatusRecord {
|
||||||
public:
|
public:
|
||||||
@@ -256,7 +256,7 @@ public:
|
|||||||
///
|
///
|
||||||
/// The end of any call to the function will be ordered before the
|
/// The end of any call to the function will be ordered before the
|
||||||
/// end of a call to unregister this record from the task. That is,
|
/// end of a call to unregister this record from the task. That is,
|
||||||
/// code may call `swift_task_removeStatusRecord` and freely
|
/// code may call `removeStatusRecord` and freely
|
||||||
/// assume after it returns that this function will not be
|
/// assume after it returns that this function will not be
|
||||||
/// subsequently used.
|
/// subsequently used.
|
||||||
class CancellationNotificationStatusRecord : public TaskStatusRecord {
|
class CancellationNotificationStatusRecord : public TaskStatusRecord {
|
||||||
@@ -284,7 +284,7 @@ public:
|
|||||||
///
|
///
|
||||||
/// The end of any call to the function will be ordered before the
|
/// The end of any call to the function will be ordered before the
|
||||||
/// end of a call to unregister this record from the task. That is,
|
/// end of a call to unregister this record from the task. That is,
|
||||||
/// code may call `swift_task_removeStatusRecord` and freely
|
/// code may call `removeStatusRecord` and freely
|
||||||
/// assume after it returns that this function will not be
|
/// assume after it returns that this function will not be
|
||||||
/// subsequently used.
|
/// subsequently used.
|
||||||
class EscalationNotificationStatusRecord : public TaskStatusRecord {
|
class EscalationNotificationStatusRecord : public TaskStatusRecord {
|
||||||
|
|||||||
@@ -17,9 +17,9 @@
|
|||||||
#ifndef SWIFT_RUNTIME_CONCURRENCY_H
|
#ifndef SWIFT_RUNTIME_CONCURRENCY_H
|
||||||
#define SWIFT_RUNTIME_CONCURRENCY_H
|
#define SWIFT_RUNTIME_CONCURRENCY_H
|
||||||
|
|
||||||
|
#include "swift/ABI/AsyncLet.h"
|
||||||
#include "swift/ABI/Task.h"
|
#include "swift/ABI/Task.h"
|
||||||
#include "swift/ABI/TaskGroup.h"
|
#include "swift/ABI/TaskGroup.h"
|
||||||
#include "swift/ABI/AsyncLet.h"
|
|
||||||
#include "swift/ABI/TaskStatus.h"
|
#include "swift/ABI/TaskStatus.h"
|
||||||
|
|
||||||
#pragma clang diagnostic push
|
#pragma clang diagnostic push
|
||||||
@@ -466,40 +466,6 @@ void swift_asyncLet_consume_throwing(SWIFT_ASYNC_CONTEXT AsyncContext *,
|
|||||||
SWIFT_EXPORT_FROM(swift_Concurrency) SWIFT_CC(swift)
|
SWIFT_EXPORT_FROM(swift_Concurrency) SWIFT_CC(swift)
|
||||||
bool swift_taskGroup_hasTaskGroupRecord();
|
bool swift_taskGroup_hasTaskGroupRecord();
|
||||||
|
|
||||||
/// Add a status record to a task. The record should not be
|
|
||||||
/// modified while it is registered with a task.
|
|
||||||
///
|
|
||||||
/// This must be called synchronously with the task.
|
|
||||||
///
|
|
||||||
/// If the task is already cancelled, returns `false` but still adds
|
|
||||||
/// the status record.
|
|
||||||
SWIFT_EXPORT_FROM(swift_Concurrency) SWIFT_CC(swift)
|
|
||||||
bool swift_task_addStatusRecord(TaskStatusRecord *record);
|
|
||||||
|
|
||||||
/// Add a status record to a task if the task has not already
|
|
||||||
/// been cancelled. The record should not be modified while it is
|
|
||||||
/// registered with a task.
|
|
||||||
///
|
|
||||||
/// This must be called synchronously with the task.
|
|
||||||
///
|
|
||||||
/// If the task is already cancelled, returns `false` and does not
|
|
||||||
/// add the status record.
|
|
||||||
SWIFT_EXPORT_FROM(swift_Concurrency) SWIFT_CC(swift)
|
|
||||||
bool swift_task_tryAddStatusRecord(TaskStatusRecord *record);
|
|
||||||
|
|
||||||
/// Remove a status record from a task. After this call returns,
|
|
||||||
/// the record's memory can be freely modified or deallocated.
|
|
||||||
///
|
|
||||||
/// This must be called synchronously with the task. The record must
|
|
||||||
/// be registered with the task or else this may crash.
|
|
||||||
///
|
|
||||||
/// The given record need not be the last record added to
|
|
||||||
/// the task, but the operation may be less efficient if not.
|
|
||||||
///
|
|
||||||
/// Returns false if the task has been cancelled.
|
|
||||||
SWIFT_EXPORT_FROM(swift_Concurrency) SWIFT_CC(swift)
|
|
||||||
bool swift_task_removeStatusRecord(TaskStatusRecord *record);
|
|
||||||
|
|
||||||
/// Signifies whether the current task is in the middle of executing the
|
/// Signifies whether the current task is in the middle of executing the
|
||||||
/// operation block of a `with(Throwing)TaskGroup(...) { <operation> }`.
|
/// operation block of a `with(Throwing)TaskGroup(...) { <operation> }`.
|
||||||
///
|
///
|
||||||
|
|||||||
@@ -308,18 +308,6 @@ OVERRIDE_TASK_LOCAL(task_localsCopyTo, void,
|
|||||||
(AsyncTask *target),
|
(AsyncTask *target),
|
||||||
(target))
|
(target))
|
||||||
|
|
||||||
OVERRIDE_TASK_STATUS(task_addStatusRecord, bool,
|
|
||||||
SWIFT_EXPORT_FROM(swift_Concurrency), SWIFT_CC(swift),
|
|
||||||
swift::, (TaskStatusRecord *newRecord), (newRecord))
|
|
||||||
|
|
||||||
OVERRIDE_TASK_STATUS(task_tryAddStatusRecord, bool,
|
|
||||||
SWIFT_EXPORT_FROM(swift_Concurrency), SWIFT_CC(swift),
|
|
||||||
swift::, (TaskStatusRecord *newRecord), (newRecord))
|
|
||||||
|
|
||||||
OVERRIDE_TASK_STATUS(task_removeStatusRecord, bool,
|
|
||||||
SWIFT_EXPORT_FROM(swift_Concurrency), SWIFT_CC(swift),
|
|
||||||
swift::, (TaskStatusRecord *record), (record))
|
|
||||||
|
|
||||||
OVERRIDE_TASK_STATUS(task_hasTaskGroupStatusRecord, bool,
|
OVERRIDE_TASK_STATUS(task_hasTaskGroupStatusRecord, bool,
|
||||||
SWIFT_EXPORT_FROM(swift_Concurrency), SWIFT_CC(swift),
|
SWIFT_EXPORT_FROM(swift_Concurrency), SWIFT_CC(swift),
|
||||||
swift::, , )
|
swift::, , )
|
||||||
|
|||||||
@@ -142,8 +142,14 @@ void swift::asyncLet_addImpl(AsyncTask *task, AsyncLet *asyncLet,
|
|||||||
auto record = impl->getTaskRecord();
|
auto record = impl->getTaskRecord();
|
||||||
assert(impl == record && "the async-let IS the task record");
|
assert(impl == record && "the async-let IS the task record");
|
||||||
|
|
||||||
// ok, now that the group actually is initialized: attach it to the task
|
// ok, now that the async let task actually is initialized: attach it to the
|
||||||
swift_task_addStatusRecord(record);
|
// current task
|
||||||
|
bool addedRecord =
|
||||||
|
addStatusRecord(record, [&](ActiveTaskStatus parentStatus) {
|
||||||
|
updateNewChildWithParentAndGroupState(task, parentStatus, NULL);
|
||||||
|
return true;
|
||||||
|
});
|
||||||
|
assert(addedRecord);
|
||||||
}
|
}
|
||||||
|
|
||||||
// =============================================================================
|
// =============================================================================
|
||||||
@@ -309,7 +315,7 @@ static void swift_asyncLet_endImpl(AsyncLet *alet) {
|
|||||||
|
|
||||||
// Remove the child record from the parent task
|
// Remove the child record from the parent task
|
||||||
auto record = asImpl(alet)->getTaskRecord();
|
auto record = asImpl(alet)->getTaskRecord();
|
||||||
swift_task_removeStatusRecord(record);
|
removeStatusRecord(record);
|
||||||
|
|
||||||
// TODO: we need to implicitly await either before the end or here somehow.
|
// TODO: we need to implicitly await either before the end or here somehow.
|
||||||
|
|
||||||
@@ -337,7 +343,7 @@ static void asyncLet_finish_after_task_completion(SWIFT_ASYNC_CONTEXT AsyncConte
|
|||||||
|
|
||||||
// Remove the child record from the parent task
|
// Remove the child record from the parent task
|
||||||
auto record = asImpl(alet)->getTaskRecord();
|
auto record = asImpl(alet)->getTaskRecord();
|
||||||
swift_task_removeStatusRecord(record);
|
removeStatusRecord(record);
|
||||||
|
|
||||||
// and finally, release the task and destroy the async-let
|
// and finally, release the task and destroy the async-let
|
||||||
assert(swift_task_getCurrent() && "async-let must have a parent task");
|
assert(swift_task_getCurrent() && "async-let must have a parent task");
|
||||||
|
|||||||
@@ -1056,19 +1056,27 @@ swift_task_addCancellationHandlerImpl(
|
|||||||
auto *record = new (allocation)
|
auto *record = new (allocation)
|
||||||
CancellationNotificationStatusRecord(unsigned_handler, context);
|
CancellationNotificationStatusRecord(unsigned_handler, context);
|
||||||
|
|
||||||
if (swift_task_addStatusRecord(record))
|
bool fireHandlerNow = false;
|
||||||
return record;
|
|
||||||
|
|
||||||
// else, the task was already cancelled, so while the record was added,
|
addStatusRecord(record, [&](ActiveTaskStatus parentStatus) {
|
||||||
// we must run it immediately here since no other task will trigger it.
|
if (parentStatus.isCancelled()) {
|
||||||
|
fireHandlerNow = true;
|
||||||
|
// We don't fire the cancellation handler here since this function needs
|
||||||
|
// to be idempotent
|
||||||
|
}
|
||||||
|
return true;
|
||||||
|
});
|
||||||
|
|
||||||
|
if (fireHandlerNow) {
|
||||||
record->run();
|
record->run();
|
||||||
|
}
|
||||||
return record;
|
return record;
|
||||||
}
|
}
|
||||||
|
|
||||||
SWIFT_CC(swift)
|
SWIFT_CC(swift)
|
||||||
static void swift_task_removeCancellationHandlerImpl(
|
static void swift_task_removeCancellationHandlerImpl(
|
||||||
CancellationNotificationStatusRecord *record) {
|
CancellationNotificationStatusRecord *record) {
|
||||||
swift_task_removeStatusRecord(record);
|
removeStatusRecord(record);
|
||||||
swift_task_dealloc(record);
|
swift_task_dealloc(record);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -474,11 +474,14 @@ static void swift_taskGroup_initializeImpl(TaskGroup *group, const Metadata *T)
|
|||||||
assert(impl == record && "the group IS the task record");
|
assert(impl == record && "the group IS the task record");
|
||||||
|
|
||||||
// ok, now that the group actually is initialized: attach it to the task
|
// ok, now that the group actually is initialized: attach it to the task
|
||||||
bool notCancelled = swift_task_addStatusRecord(record);
|
addStatusRecord(record, [&](ActiveTaskStatus parentStatus) {
|
||||||
|
|
||||||
// If the task has already been cancelled, reflect that immediately in
|
// If the task has already been cancelled, reflect that immediately in
|
||||||
// the group status.
|
// the group's status.
|
||||||
if (!notCancelled) impl->statusCancel();
|
if (parentStatus.isCancelled()) {
|
||||||
|
impl->statusCancel();
|
||||||
|
}
|
||||||
|
return true;
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
// =============================================================================
|
// =============================================================================
|
||||||
@@ -505,7 +508,7 @@ void TaskGroupImpl::destroy() {
|
|||||||
SWIFT_TASK_DEBUG_LOG("destroying task group = %p", this);
|
SWIFT_TASK_DEBUG_LOG("destroying task group = %p", this);
|
||||||
|
|
||||||
// First, remove the group from the task and deallocate the record
|
// First, remove the group from the task and deallocate the record
|
||||||
swift_task_removeStatusRecord(getTaskRecord());
|
removeStatusRecord(getTaskRecord());
|
||||||
|
|
||||||
// No need to drain our queue here, as by the time we call destroy,
|
// No need to drain our queue here, as by the time we call destroy,
|
||||||
// all tasks inside the group must have been awaited on already.
|
// all tasks inside the group must have been awaited on already.
|
||||||
|
|||||||
@@ -238,12 +238,23 @@ public:
|
|||||||
bool isStoredPriorityEscalated() const {
|
bool isStoredPriorityEscalated() const {
|
||||||
return Flags & IsEscalated;
|
return Flags & IsEscalated;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Creates a new active task status for a task with the specified priority
|
||||||
|
/// and masks away any existing priority related flags on the task status. All
|
||||||
|
/// other flags about the task are unmodified. This is only safe to use to
|
||||||
|
/// generate an initial task status for a new task that is not yet running.
|
||||||
|
ActiveTaskStatus withNewPriority(JobPriority priority) const {
|
||||||
|
return ActiveTaskStatus(Record,
|
||||||
|
(Flags & ~PriorityMask) | uintptr_t(priority));
|
||||||
|
}
|
||||||
|
|
||||||
ActiveTaskStatus withEscalatedPriority(JobPriority priority) const {
|
ActiveTaskStatus withEscalatedPriority(JobPriority priority) const {
|
||||||
assert(priority > getStoredPriority());
|
assert(priority > getStoredPriority());
|
||||||
return ActiveTaskStatus(Record,
|
return ActiveTaskStatus(Record,
|
||||||
(Flags & ~PriorityMask)
|
(Flags & ~PriorityMask)
|
||||||
| IsEscalated | uintptr_t(priority));
|
| IsEscalated | uintptr_t(priority));
|
||||||
}
|
}
|
||||||
|
|
||||||
ActiveTaskStatus withoutStoredPriorityEscalation() const {
|
ActiveTaskStatus withoutStoredPriorityEscalation() const {
|
||||||
assert(isStoredPriorityEscalated());
|
assert(isStoredPriorityEscalated());
|
||||||
return ActiveTaskStatus(Record, Flags & ~IsEscalated);
|
return ActiveTaskStatus(Record, Flags & ~IsEscalated);
|
||||||
@@ -438,6 +449,39 @@ inline bool AsyncTask::localValuePop() {
|
|||||||
return _private().Local.popValue(this);
|
return _private().Local.popValue(this);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/*************** Methods for Status records manipulation ******************/
|
||||||
|
|
||||||
|
/// Remove a status record from a task. After this call returns,
|
||||||
|
/// the record's memory can be freely modified or deallocated.
|
||||||
|
///
|
||||||
|
/// This must be called synchronously with the task. The record must
|
||||||
|
/// be registered with the task or else this may crash.
|
||||||
|
///
|
||||||
|
/// The given record need not be the last record added to
|
||||||
|
/// the task, but the operation may be less efficient if not.
|
||||||
|
///
|
||||||
|
/// Returns false if the task has been cancelled.
|
||||||
|
SWIFT_CC(swift)
|
||||||
|
bool removeStatusRecord(TaskStatusRecord *record);
|
||||||
|
|
||||||
|
/// Add a status record to a task. This must be called synchronously with the
|
||||||
|
/// task.
|
||||||
|
///
|
||||||
|
/// This function also takes in a function_ref which is given the task status of
|
||||||
|
/// the task we're adding the record to, to determine if the current status of
|
||||||
|
/// the task permits adding the status record. This function_ref may be called
|
||||||
|
/// multiple times and must be idempotent.
|
||||||
|
SWIFT_CC(swift)
|
||||||
|
bool addStatusRecord(TaskStatusRecord *record,
|
||||||
|
llvm::function_ref<bool(ActiveTaskStatus)> testAddRecord);
|
||||||
|
|
||||||
|
/// A helper function for updating a new child task that is created with
|
||||||
|
/// information from the parent or the group that it was going to be added to.
|
||||||
|
SWIFT_CC(swift)
|
||||||
|
void updateNewChildWithParentAndGroupState(AsyncTask *child,
|
||||||
|
ActiveTaskStatus parentStatus,
|
||||||
|
TaskGroup *group);
|
||||||
|
|
||||||
} // end namespace swift
|
} // end namespace swift
|
||||||
|
|
||||||
#endif
|
#endif
|
||||||
|
|||||||
@@ -257,9 +257,11 @@ static bool withStatusRecordLock(AsyncTask *task,
|
|||||||
/**************************************************************************/
|
/**************************************************************************/
|
||||||
|
|
||||||
SWIFT_CC(swift)
|
SWIFT_CC(swift)
|
||||||
static bool swift_task_addStatusRecordImpl(TaskStatusRecord *newRecord) {
|
bool swift::addStatusRecord(
|
||||||
auto task = swift_task_getCurrent();
|
TaskStatusRecord *newRecord,
|
||||||
|
llvm::function_ref<bool(ActiveTaskStatus status)> shouldAddRecord) {
|
||||||
|
|
||||||
|
auto task = swift_task_getCurrent();
|
||||||
// Load the current state. We can use a relaxed load because we're
|
// Load the current state. We can use a relaxed load because we're
|
||||||
// synchronous with the task.
|
// synchronous with the task.
|
||||||
auto oldStatus = task->_private().Status.load(std::memory_order_relaxed);
|
auto oldStatus = task->_private().Status.load(std::memory_order_relaxed);
|
||||||
@@ -273,53 +275,27 @@ static bool swift_task_addStatusRecordImpl(TaskStatusRecord *newRecord) {
|
|||||||
newRecord->resetParent(oldStatus.getInnermostRecord());
|
newRecord->resetParent(oldStatus.getInnermostRecord());
|
||||||
|
|
||||||
// Set the record as the new innermost record.
|
// Set the record as the new innermost record.
|
||||||
|
ActiveTaskStatus newStatus = oldStatus.withInnermostRecord(newRecord);
|
||||||
|
|
||||||
|
if (shouldAddRecord(newStatus)) {
|
||||||
// We have to use a release on success to make the initialization of
|
// We have to use a release on success to make the initialization of
|
||||||
// the new record visible to the cancelling thread.
|
// the new record visible to the cancelling thread.
|
||||||
ActiveTaskStatus newStatus = oldStatus.withInnermostRecord(newRecord);
|
if (task->_private().Status.compare_exchange_weak(
|
||||||
if (task->_private().Status.compare_exchange_weak(oldStatus, newStatus,
|
oldStatus, newStatus,
|
||||||
/*success*/ std::memory_order_release,
|
/*success*/ std::memory_order_release,
|
||||||
/*failure*/ std::memory_order_relaxed))
|
/*failure*/ std::memory_order_relaxed)) {
|
||||||
return !oldStatus.isCancelled();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
SWIFT_CC(swift)
|
|
||||||
static bool swift_task_tryAddStatusRecordImpl(TaskStatusRecord *newRecord) {
|
|
||||||
auto task = swift_task_getCurrent();
|
|
||||||
|
|
||||||
// Load the current state. We can use a relaxed load because we're
|
|
||||||
// synchronous with the task.
|
|
||||||
auto oldStatus = task->_private().Status.load(std::memory_order_relaxed);
|
|
||||||
|
|
||||||
while (true) {
|
|
||||||
// If the old info is already cancelled, do nothing.
|
|
||||||
if (oldStatus.isCancelled())
|
|
||||||
return false;
|
|
||||||
|
|
||||||
// Wait for any active lock to be released.
|
|
||||||
if (oldStatus.isLocked()) {
|
|
||||||
waitForStatusRecordUnlock(task, oldStatus);
|
|
||||||
|
|
||||||
if (oldStatus.isCancelled())
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
// Reset the parent of the new record.
|
|
||||||
newRecord->resetParent(oldStatus.getInnermostRecord());
|
|
||||||
|
|
||||||
// Set the record as the new innermost record.
|
|
||||||
// We have to use a release on success to make the initialization of
|
|
||||||
// the new record visible to the cancelling thread.
|
|
||||||
ActiveTaskStatus newStatus = oldStatus.withInnermostRecord(newRecord);
|
|
||||||
if (task->_private().Status.compare_exchange_weak(oldStatus, newStatus,
|
|
||||||
/*success*/ std::memory_order_release,
|
|
||||||
/*failure*/ std::memory_order_relaxed))
|
|
||||||
return true;
|
return true;
|
||||||
|
} else {
|
||||||
|
// Retry
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
SWIFT_CC(swift)
|
SWIFT_CC(swift)
|
||||||
static bool swift_task_removeStatusRecordImpl(TaskStatusRecord *record) {
|
bool swift::removeStatusRecord(TaskStatusRecord *record) {
|
||||||
auto task = swift_task_getCurrent();
|
auto task = swift_task_getCurrent();
|
||||||
SWIFT_TASK_DEBUG_LOG("remove status record = %p, from current task = %p",
|
SWIFT_TASK_DEBUG_LOG("remove status record = %p, from current task = %p",
|
||||||
record, task);
|
record, task);
|
||||||
@@ -411,7 +387,14 @@ swift_task_attachChildImpl(AsyncTask *child) {
|
|||||||
auto record = new (allocation) swift::ChildTaskStatusRecord(child);
|
auto record = new (allocation) swift::ChildTaskStatusRecord(child);
|
||||||
SWIFT_TASK_DEBUG_LOG("attach child task = %p, record = %p, to current task = %p",
|
SWIFT_TASK_DEBUG_LOG("attach child task = %p, record = %p, to current task = %p",
|
||||||
child, record, swift_task_getCurrent());
|
child, record, swift_task_getCurrent());
|
||||||
swift_task_addStatusRecord(record);
|
|
||||||
|
bool added_record = swift_task_addStatusRecordWithChecks(
|
||||||
|
record, [&](ActiveTaskStatus parentStatus) {
|
||||||
|
swift_task_updateNewChildWithParentAndGroupState(child, parentStatus,
|
||||||
|
NULL);
|
||||||
|
return true;
|
||||||
|
});
|
||||||
|
assert(added_record);
|
||||||
return record;
|
return record;
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -421,6 +404,39 @@ swift_task_detachChildImpl(ChildTaskStatusRecord *record) {
|
|||||||
swift_task_removeStatusRecord(record);
|
swift_task_removeStatusRecord(record);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Called in the path of linking a child into a parent/group synchronously with
|
||||||
|
/// the parent task.
|
||||||
|
//
|
||||||
|
/// When called to link a child into a parent directly, this does not hold the
|
||||||
|
/// parent's task status record lock. When called to link a child into a task
|
||||||
|
/// group, this holds the parent's task status record lock.
|
||||||
|
SWIFT_CC(swift)
|
||||||
|
void swift::updateNewChildWithParentAndGroupState(AsyncTask *child,
|
||||||
|
ActiveTaskStatus parentStatus,
|
||||||
|
TaskGroup *group) {
|
||||||
|
// We can take the fast path of just modifying the ActiveTaskStatus in the
|
||||||
|
// child task since we know that it won't have any task status records and
|
||||||
|
// cannot be accessed by anyone else since it hasn't been linked in yet.
|
||||||
|
// Avoids the extra logic in `swift_task_cancel` and `swift_task_escalate`
|
||||||
|
auto oldChildTaskStatus =
|
||||||
|
child->_private().Status.load(std::memory_order_relaxed);
|
||||||
|
assert(oldChildTaskStatus.getInnermostRecord() == NULL);
|
||||||
|
|
||||||
|
auto newChildTaskStatus = oldChildTaskStatus;
|
||||||
|
|
||||||
|
if (parentStatus.isCancelled() || (group && group->isCancelled())) {
|
||||||
|
newChildTaskStatus = newChildTaskStatus.withCancelled();
|
||||||
|
}
|
||||||
|
|
||||||
|
// Propagate max priority of parent to child task's active status and the Job
|
||||||
|
// header
|
||||||
|
JobPriority pri = parentStatus.getStoredPriority();
|
||||||
|
newChildTaskStatus = newChildTaskStatus.withNewPriority(pri);
|
||||||
|
child->Flags.setPriority(pri);
|
||||||
|
|
||||||
|
child->_private().Status.store(newChildTaskStatus, std::memory_order_relaxed);
|
||||||
|
}
|
||||||
|
|
||||||
SWIFT_CC(swift)
|
SWIFT_CC(swift)
|
||||||
static void swift_taskGroup_attachChildImpl(TaskGroup *group,
|
static void swift_taskGroup_attachChildImpl(TaskGroup *group,
|
||||||
AsyncTask *child) {
|
AsyncTask *child) {
|
||||||
@@ -430,11 +446,19 @@ static void swift_taskGroup_attachChildImpl(TaskGroup *group,
|
|||||||
// Acquire the status record lock of parent - we want to synchronize with
|
// Acquire the status record lock of parent - we want to synchronize with
|
||||||
// concurrent cancellation or escalation as we're adding new tasks to the
|
// concurrent cancellation or escalation as we're adding new tasks to the
|
||||||
// group.
|
// group.
|
||||||
|
|
||||||
auto parent = swift_task_getCurrent();
|
auto parent = swift_task_getCurrent();
|
||||||
withStatusRecordLock(parent, LockContext::OnTask,
|
withStatusRecordLock(parent, LockContext::OnTask, [&](ActiveTaskStatus &parentStatus) {
|
||||||
[&](ActiveTaskStatus &status) {
|
|
||||||
group->addChildTask(child);
|
group->addChildTask(child);
|
||||||
|
|
||||||
|
// After getting parent's status record lock, do some sanity checks to
|
||||||
|
// see if parent task or group has state changes that need to be
|
||||||
|
// propagated to the child.
|
||||||
|
//
|
||||||
|
// This is the same logic that we would do if we were adding a child
|
||||||
|
// task status record - see also asyncLet_addImpl. Since we attach a
|
||||||
|
// child task to a TaskGroupRecord instead, we synchronize on the
|
||||||
|
// parent's task status and then update the child.
|
||||||
|
updateNewChildWithParentAndGroupState(child, parentStatus, group);
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -140,6 +140,8 @@ if(("${SWIFT_HOST_VARIANT_SDK}" STREQUAL "${SWIFT_PRIMARY_VARIANT_SDK}") AND
|
|||||||
|
|
||||||
target_include_directories(SwiftRuntimeTests BEFORE PRIVATE
|
target_include_directories(SwiftRuntimeTests BEFORE PRIVATE
|
||||||
${SWIFT_SOURCE_DIR}/stdlib/include)
|
${SWIFT_SOURCE_DIR}/stdlib/include)
|
||||||
|
target_include_directories(SwiftRuntimeTests BEFORE PRIVATE
|
||||||
|
${SWIFT_SOURCE_DIR}/stdlib/public)
|
||||||
|
|
||||||
# FIXME: cross-compile for all variants.
|
# FIXME: cross-compile for all variants.
|
||||||
target_link_libraries(SwiftRuntimeTests
|
target_link_libraries(SwiftRuntimeTests
|
||||||
|
|||||||
@@ -227,18 +227,6 @@ TEST_F(CompatibilityOverrideConcurrencyTest, test_swift_task_localsCopyTo) {
|
|||||||
swift_task_localsCopyTo(nullptr);
|
swift_task_localsCopyTo(nullptr);
|
||||||
}
|
}
|
||||||
|
|
||||||
TEST_F(CompatibilityOverrideConcurrencyTest, test_swift_task_addStatusRecord) {
|
|
||||||
swift_task_addStatusRecord(nullptr);
|
|
||||||
}
|
|
||||||
|
|
||||||
TEST_F(CompatibilityOverrideConcurrencyTest, test_swift_task_tryAddStatusRecord) {
|
|
||||||
swift_task_tryAddStatusRecord(nullptr);
|
|
||||||
}
|
|
||||||
|
|
||||||
TEST_F(CompatibilityOverrideConcurrencyTest, test_swift_task_removeStatusRecord) {
|
|
||||||
swift_task_removeStatusRecord(nullptr);
|
|
||||||
}
|
|
||||||
|
|
||||||
TEST_F(CompatibilityOverrideConcurrencyTest, task_hasTaskGroupStatusRecord) {
|
TEST_F(CompatibilityOverrideConcurrencyTest, task_hasTaskGroupStatusRecord) {
|
||||||
swift_task_hasTaskGroupStatusRecord();
|
swift_task_hasTaskGroupStatusRecord();
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -10,8 +10,8 @@
|
|||||||
//
|
//
|
||||||
//===----------------------------------------------------------------------===//
|
//===----------------------------------------------------------------------===//
|
||||||
|
|
||||||
#include "swift/Runtime/Concurrency.h"
|
|
||||||
#include "swift/Basic/STLExtras.h"
|
#include "swift/Basic/STLExtras.h"
|
||||||
|
#include "swift/Runtime/Concurrency.h"
|
||||||
#include "gtest/gtest.h"
|
#include "gtest/gtest.h"
|
||||||
|
|
||||||
using namespace swift;
|
using namespace swift;
|
||||||
@@ -137,149 +137,3 @@ TEST(TaskStatusTest, cancellation_simple) {
|
|||||||
swift_job_run(task, createFakeExecutor(1234));
|
swift_job_run(task, createFakeExecutor(1234));
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
// Test basic deadline mechanics (other than actually setting up
|
|
||||||
// something to cancel the task). Also tests adding and removing
|
|
||||||
// records quite a bit.
|
|
||||||
TEST(TaskStatusTest, deadline) {
|
|
||||||
struct Storage { int value; };
|
|
||||||
withSimpleTask(Storage{47},
|
|
||||||
[&](ValueContext<Storage> *context) {
|
|
||||||
auto task = swift_task_getCurrent();
|
|
||||||
EXPECT_FALSE(swift_task_isCancelled(task));
|
|
||||||
|
|
||||||
TaskDeadline deadlineOne = { 1234 };
|
|
||||||
TaskDeadline deadlineTwo = { 2345 };
|
|
||||||
DeadlineStatusRecord recordOne(deadlineOne);
|
|
||||||
DeadlineStatusRecord recordTwo(deadlineTwo);
|
|
||||||
bool result;
|
|
||||||
|
|
||||||
NearestTaskDeadline nearest = swift_task_getNearestDeadline(task);
|
|
||||||
EXPECT_EQ(NearestTaskDeadline::None, nearest.ValueKind);
|
|
||||||
|
|
||||||
// Add deadline 1. Check that we haven't been cancelled yet.
|
|
||||||
result = swift_task_addStatusRecord(&recordOne);
|
|
||||||
EXPECT_TRUE(result);
|
|
||||||
|
|
||||||
// There should now be an active deadline.
|
|
||||||
nearest = swift_task_getNearestDeadline(task);
|
|
||||||
EXPECT_EQ(NearestTaskDeadline::Active, nearest.ValueKind);
|
|
||||||
EXPECT_EQ(deadlineOne, nearest.Value);
|
|
||||||
|
|
||||||
// Remove deadline 1. Check that we haven't been cancelled yet.
|
|
||||||
result = swift_task_removeStatusRecord(&recordOne);
|
|
||||||
EXPECT_TRUE(result);
|
|
||||||
|
|
||||||
// There shouldn't be an active deadline anymore.
|
|
||||||
nearest = swift_task_getNearestDeadline(task);
|
|
||||||
EXPECT_EQ(NearestTaskDeadline::None, nearest.ValueKind);
|
|
||||||
|
|
||||||
// Add deadline 1, then 2.
|
|
||||||
result = swift_task_addStatusRecord(&recordOne);
|
|
||||||
EXPECT_TRUE(result);
|
|
||||||
result = swift_task_addStatusRecord(&recordTwo);
|
|
||||||
EXPECT_TRUE(result);
|
|
||||||
|
|
||||||
// The nearest deadline should be deadline 1.
|
|
||||||
nearest = swift_task_getNearestDeadline(task);
|
|
||||||
EXPECT_EQ(NearestTaskDeadline::Active, nearest.ValueKind);
|
|
||||||
EXPECT_EQ(deadlineOne, nearest.Value);
|
|
||||||
|
|
||||||
// Remove the deadlines.
|
|
||||||
result = swift_task_removeStatusRecord(&recordTwo);
|
|
||||||
EXPECT_TRUE(result);
|
|
||||||
result = swift_task_removeStatusRecord(&recordOne);
|
|
||||||
EXPECT_TRUE(result);
|
|
||||||
|
|
||||||
// Add deadline 2, then 1s.
|
|
||||||
result = swift_task_addStatusRecord(&recordTwo);
|
|
||||||
EXPECT_TRUE(result);
|
|
||||||
|
|
||||||
// In the middle, the nearest deadline should be deadline 2.
|
|
||||||
nearest = swift_task_getNearestDeadline(task);
|
|
||||||
EXPECT_EQ(NearestTaskDeadline::Active, nearest.ValueKind);
|
|
||||||
EXPECT_EQ(deadlineTwo, nearest.Value);
|
|
||||||
|
|
||||||
result = swift_task_addStatusRecord(&recordOne);
|
|
||||||
EXPECT_TRUE(result);
|
|
||||||
|
|
||||||
// The nearest deadline should be deadline 1.
|
|
||||||
nearest = swift_task_getNearestDeadline(task);
|
|
||||||
EXPECT_EQ(NearestTaskDeadline::Active, nearest.ValueKind);
|
|
||||||
EXPECT_EQ(deadlineOne, nearest.Value);
|
|
||||||
|
|
||||||
// Remove the deadlines.
|
|
||||||
result = swift_task_removeStatusRecord(&recordOne);
|
|
||||||
EXPECT_TRUE(result);
|
|
||||||
result = swift_task_removeStatusRecord(&recordTwo);
|
|
||||||
EXPECT_TRUE(result);
|
|
||||||
|
|
||||||
// Do the same thing with tryAddStatus.
|
|
||||||
result = swift_task_tryAddStatusRecord(&recordTwo);
|
|
||||||
EXPECT_TRUE(result);
|
|
||||||
result = swift_task_tryAddStatusRecord(&recordOne);
|
|
||||||
EXPECT_TRUE(result);
|
|
||||||
// The nearest deadline should be deadline 1.
|
|
||||||
nearest = swift_task_getNearestDeadline(task);
|
|
||||||
EXPECT_EQ(NearestTaskDeadline::Active, nearest.ValueKind);
|
|
||||||
EXPECT_EQ(deadlineOne, nearest.Value);
|
|
||||||
result = swift_task_removeStatusRecord(&recordOne);
|
|
||||||
EXPECT_TRUE(result);
|
|
||||||
result = swift_task_removeStatusRecord(&recordTwo);
|
|
||||||
EXPECT_TRUE(result);
|
|
||||||
|
|
||||||
// Remove out of order.
|
|
||||||
result = swift_task_addStatusRecord(&recordTwo);
|
|
||||||
EXPECT_TRUE(result);
|
|
||||||
result = swift_task_addStatusRecord(&recordOne);
|
|
||||||
EXPECT_TRUE(result);
|
|
||||||
// The nearest deadline should be deadline 1.
|
|
||||||
nearest = swift_task_getNearestDeadline(task);
|
|
||||||
EXPECT_EQ(NearestTaskDeadline::Active, nearest.ValueKind);
|
|
||||||
EXPECT_EQ(deadlineOne, nearest.Value);
|
|
||||||
result = swift_task_removeStatusRecord(&recordTwo);
|
|
||||||
EXPECT_TRUE(result);
|
|
||||||
result = swift_task_removeStatusRecord(&recordOne);
|
|
||||||
EXPECT_TRUE(result);
|
|
||||||
|
|
||||||
// Add deadline 2, then cancel.
|
|
||||||
result = swift_task_addStatusRecord(&recordTwo);
|
|
||||||
EXPECT_TRUE(result);
|
|
||||||
|
|
||||||
// The nearest deadline should be deadline 2.
|
|
||||||
nearest = swift_task_getNearestDeadline(task);
|
|
||||||
EXPECT_EQ(NearestTaskDeadline::Active, nearest.ValueKind);
|
|
||||||
EXPECT_EQ(deadlineTwo, nearest.Value);
|
|
||||||
|
|
||||||
// Cancel.
|
|
||||||
swift_task_cancel(task);
|
|
||||||
EXPECT_TRUE(swift_task_isCancelled(task));
|
|
||||||
|
|
||||||
// We should report already cancelled now.
|
|
||||||
nearest = swift_task_getNearestDeadline(task);
|
|
||||||
EXPECT_EQ(NearestTaskDeadline::AlreadyCancelled, nearest.ValueKind);
|
|
||||||
|
|
||||||
// Add deadline 1.
|
|
||||||
result = swift_task_addStatusRecord(&recordOne);
|
|
||||||
EXPECT_FALSE(result);
|
|
||||||
|
|
||||||
nearest = swift_task_getNearestDeadline(task);
|
|
||||||
EXPECT_EQ(NearestTaskDeadline::AlreadyCancelled, nearest.ValueKind);
|
|
||||||
|
|
||||||
result = swift_task_removeStatusRecord(&recordOne);
|
|
||||||
EXPECT_FALSE(result);
|
|
||||||
|
|
||||||
result = swift_task_tryAddStatusRecord(&recordOne);
|
|
||||||
EXPECT_FALSE(result);
|
|
||||||
|
|
||||||
result = swift_task_removeStatusRecord(&recordTwo);
|
|
||||||
EXPECT_FALSE(result);
|
|
||||||
|
|
||||||
nearest = swift_task_getNearestDeadline(task);
|
|
||||||
EXPECT_EQ(NearestTaskDeadline::AlreadyCancelled, nearest.ValueKind);
|
|
||||||
|
|
||||||
EXPECT_TRUE(swift_task_isCancelled(task));
|
|
||||||
}, [&](AsyncTask *task) {
|
|
||||||
swift_job_run(task, createFakeExecutor(1234));
|
|
||||||
});
|
|
||||||
}
|
|
||||||
|
|||||||
Reference in New Issue
Block a user