Remove the unused swift_asyncLet_{start,end,wait,wait_throwing} runtime

functions.

These were introduced in an early draft implementation of async let, but
never used by a released compiler. They are not used as symbols by any
app binaries. There's no reason to keep carrying them.

While I'm at it, dramatically improve the documentation of the remaining
async let API functions.
This commit is contained in:
John McCall
2025-09-24 18:53:51 -04:00
parent cd67912a50
commit 3cfda35b7c
17 changed files with 188 additions and 463 deletions

View File

@@ -2910,9 +2910,9 @@ enum class TaskOptionRecordKind : uint8_t {
InitialSerialExecutor = 0,
/// Request a child task to be part of a specific task group.
TaskGroup = 1,
/// DEPRECATED. AsyncLetWithBuffer is used instead.
/// UNUSED. AsyncLetWithBuffer is used instead.
/// Request a child task for an 'async let'.
AsyncLet = 2,
// AsyncLet = 2,
/// Request a child task for an 'async let'.
AsyncLetWithBuffer = 3,
/// Information about the result type of the task, used in embedded Swift.

View File

@@ -166,25 +166,6 @@ public:
}
};
/// DEPRECATED. AsyncLetWithBufferTaskOptionRecord is used instead.
/// Task option to specify that the created task is for an 'async let'.
class AsyncLetTaskOptionRecord : public TaskOptionRecord {
AsyncLet *asyncLet;
public:
AsyncLetTaskOptionRecord(AsyncLet *asyncLet)
: TaskOptionRecord(TaskOptionRecordKind::AsyncLet),
asyncLet(asyncLet) {}
AsyncLet *getAsyncLet() const {
return asyncLet;
}
static bool classof(const TaskOptionRecord *record) {
return record->getKind() == TaskOptionRecordKind::AsyncLet;
}
};
class AsyncLetWithBufferTaskOptionRecord : public TaskOptionRecord {
AsyncLet *asyncLet;
void *resultBuffer;

View File

@@ -351,24 +351,11 @@ bool swift_taskGroup_isCancelled(TaskGroup *group);
SWIFT_EXPORT_FROM(swift_Concurrency) SWIFT_CC(swift)
bool swift_taskGroup_isEmpty(TaskGroup *group);
/// DEPRECATED. swift_asyncLet_begin is used instead.
/// Its Swift signature is
/// Enter the scope of an async let binding, creating a child task of the
/// current task to run the given function.
///
/// \code
/// func swift_asyncLet_start<T>(
/// asyncLet: Builtin.RawPointer,
/// options: Builtin.RawPointer?,
/// operation: __owned @Sendable () async throws -> T
/// )
/// \endcode
SWIFT_EXPORT_FROM(swift_Concurrency) SWIFT_CC(swift)
void swift_asyncLet_start(AsyncLet *alet,
TaskOptionRecord *options,
const Metadata *futureResultType,
void *closureEntryPoint, HeapObject *closureContext);
/// Begin an async let child task.
/// Its Swift signature is
/// Behaves approximately like a Swift function with the following
/// signature, except the generic argument is in a different position:
///
/// \code
/// func swift_asyncLet_start<T>(
@@ -378,6 +365,11 @@ void swift_asyncLet_start(AsyncLet *alet,
/// resultBuffer: Builtin.RawPointer
/// )
/// \endcode
///
/// Previous versions of the concurrency runtime also provided a
/// \c swift_asyncLet_start which did not take a result buffer. This
/// function was never used by a released version of the compiler,
/// and it has been removed.
SWIFT_EXPORT_FROM(swift_Concurrency) SWIFT_CC(swift)
void swift_asyncLet_begin(AsyncLet *alet,
TaskOptionRecord *options,
@@ -385,57 +377,12 @@ void swift_asyncLet_begin(AsyncLet *alet,
void *closureEntryPoint, HeapObject *closureContext,
void *resultBuffer);
/// This matches the ABI of a closure `<T>(Builtin.RawPointer) async -> T`
using AsyncLetWaitSignature =
SWIFT_CC(swiftasync)
void(OpaqueValue *,
SWIFT_ASYNC_CONTEXT AsyncContext *, AsyncTask *, Metadata *);
/// DEPRECATED. swift_asyncLet_get is used instead.
/// Wait for a non-throwing async-let to complete.
/// Get the value of a non-throwing async let, awaiting the result
/// if necessary.
///
/// This can be called from any thread. Its Swift signature is
/// This must be called from the parent task of the async let.
///
/// \code
/// func swift_asyncLet_wait(
/// _ asyncLet: _owned Builtin.RawPointer
/// ) async -> Success
/// \endcode
SWIFT_EXPORT_FROM(swift_Concurrency) SWIFT_CC(swiftasync)
void swift_asyncLet_wait(OpaqueValue *,
SWIFT_ASYNC_CONTEXT AsyncContext *,
AsyncLet *, TaskContinuationFunction *,
AsyncContext *);
/// DEPRECATED. swift_asyncLet_get_throwing is used instead.
/// Wait for a potentially-throwing async-let to complete.
///
/// This can be called from any thread. Its Swift signature is
///
/// \code
/// func swift_asyncLet_wait_throwing(
/// _ asyncLet: _owned Builtin.RawPointer
/// ) async throws -> Success
/// \endcode
SWIFT_EXPORT_FROM(swift_Concurrency) SWIFT_CC(swiftasync)
void swift_asyncLet_wait_throwing(OpaqueValue *,
SWIFT_ASYNC_CONTEXT AsyncContext *,
AsyncLet *,
ThrowingTaskFutureWaitContinuationFunction *,
AsyncContext *);
/// DEPRECATED. swift_asyncLet_finish is used instead.
/// Its Swift signature is
///
/// \code
/// func swift_asyncLet_end(_ alet: Builtin.RawPointer)
/// \endcode
SWIFT_EXPORT_FROM(swift_Concurrency) SWIFT_CC(swift)
void swift_asyncLet_end(AsyncLet *alet);
/// Get the value of a non-throwing async-let, awaiting the result if necessary.
///
/// This can be called from any thread. Its Swift signature is
/// Behaves like a Swift async function with the signature:
///
/// \code
/// func swift_asyncLet_get(
@@ -444,21 +391,31 @@ void swift_asyncLet_end(AsyncLet *alet);
/// ) async
/// \endcode
///
/// \c result points at the variable storage for the binding. It is
/// uninitialized until the first call to \c swift_asyncLet_get or
/// \c swift_asyncLet_get_throwing. That first call initializes the storage
/// with the result of the child task. Subsequent calls do nothing and leave
/// the value in place.
SWIFT_EXPORT_FROM(swift_Concurrency) SWIFT_CC(swiftasync)
void swift_asyncLet_get(SWIFT_ASYNC_CONTEXT AsyncContext *,
AsyncLet *,
void *,
TaskContinuationFunction *,
AsyncContext *);
/// Get the value of a throwing async-let, awaiting the result if necessary.
/// except that it uses the special calling convention in which the
/// continuation function, parent context, and child context are all
/// passed in separately rather than being stored in the child context.
/// The child async context has a fixed size of 5 * sizeof(void*).
///
/// This can be called from any thread. Its Swift signature is
/// \c resultBuffer must be the same result buffer that was passed to
/// swift_asyncLet_begin. After asynchronous return, it is guaranteed
/// to be initialized.
///
/// Previous versions of the concurrency runtime also provided a
/// \c swift_asyncLet_wait which was meant to be paired with
/// \c swift_asyncLet_start. This function was never used by
/// a released version of the compiler, and it has been removed.
SWIFT_EXPORT_FROM(swift_Concurrency) SWIFT_CC(swiftasync)
void swift_asyncLet_get(SWIFT_ASYNC_CONTEXT AsyncContext *parentContext,
AsyncLet *alet,
void *resultBuffer,
TaskContinuationFunction *continuation,
AsyncContext *childContext);
/// Get the value of a throwing async let, awaiting the result if necessary.
///
/// This must be called from the parent task of the async let.
///
/// Behaves like a Swift async function with the signature:
///
/// \code
/// func swift_asyncLet_get_throwing(
@@ -467,30 +424,61 @@ void swift_asyncLet_get(SWIFT_ASYNC_CONTEXT AsyncContext *,
/// ) async throws
/// \endcode
///
/// \c result points at the variable storage for the binding. It is
/// uninitialized until the first call to \c swift_asyncLet_get or
/// \c swift_asyncLet_get_throwing. That first call initializes the storage
/// with the result of the child task. Subsequent calls do nothing and leave
/// the value in place. A pointer to the storage inside the child task is
/// returned if the task completes successfully, otherwise the error from the
/// child task is thrown.
SWIFT_EXPORT_FROM(swift_Concurrency) SWIFT_CC(swiftasync)
void swift_asyncLet_get_throwing(SWIFT_ASYNC_CONTEXT AsyncContext *,
AsyncLet *,
void *,
ThrowingTaskFutureWaitContinuationFunction *,
AsyncContext *);
/// Exit the scope of an async-let binding. If the task is still running, it
/// is cancelled, and we await its completion; otherwise, we destroy the
/// value in the variable storage.
/// except that it uses the special calling convention in which the
/// continuation function, parent context, and child context are all
/// passed in separately rather than being stored in the child context.
/// The child async context has a fixed size of 5 * sizeof(void*).
///
/// Its Swift signature is
/// \c resultBuffer must be the same result buffer that was passed to
/// swift_asyncLet_begin. After asynchronous return, it is guaranteed
/// to be initialized unless the operation throws by passing an error
/// to the continuation function.
///
/// Previous versions of the concurrency runtime also provided a
/// \c swift_asyncLet_wait_throwing which was meant to be paired with
/// \c swift_asyncLet_start. This function was never used by
/// a released version of the compiler, and it has been removed.
SWIFT_EXPORT_FROM(swift_Concurrency) SWIFT_CC(swiftasync)
void swift_asyncLet_get_throwing(SWIFT_ASYNC_CONTEXT AsyncContext *parentContext,
AsyncLet *alet,
void *resultBuffer,
ThrowingTaskFutureWaitContinuationFunction *
continuation,
AsyncContext *childContext);
/// Exit the scope of an async let binding that was created with
/// swift_asyncLet_begin. If the child task is still running, it is
/// cancelled, and the current task is suspended to await its completion.
///
/// This must be called from the parent task of the async let.
///
/// Behaves like a Swift async function with the signature:
///
/// \code
/// func swift_asyncLet_finish(_ asyncLet: Builtin.RawPointer,
/// _ resultBuffer: Builtin.RawPointer) async
/// \endcode
///
/// except that it uses the special calling convention in which the
/// continuation function, parent context, and child context are all
/// passed in separately rather than being stored in the child context.
/// The child async context has a fixed size of 5 * sizeof(void*).
///
/// \c resultBuffer must be the same result buffer that was passed to
/// swift_asyncLet_begin.
///
/// Prior to asynchronous return, the value in the result buffer is
/// destroyed (if present) and any memory initially allocated for the
/// task is deallocated. Because swift_asyncLet_begin potentially
/// allocates memory on the parent task's task allocator, the call to
/// this function must be properly paired with the begin.
///
/// The async let is invalid after this call and cannot be used again.
///
/// Previous versions of the concurrency runtime also provided a
/// \c swift_asyncLet_end which was not asynchronous and was meant
/// to be paired with \c swift_asyncLet_start. This function was never
/// used by a released version of the compiler, and it has been removed.
SWIFT_EXPORT_FROM(swift_Concurrency) SWIFT_CC(swiftasync)
void swift_asyncLet_finish(SWIFT_ASYNC_CONTEXT AsyncContext *,
AsyncLet *,
@@ -498,11 +486,13 @@ void swift_asyncLet_finish(SWIFT_ASYNC_CONTEXT AsyncContext *,
TaskContinuationFunction *,
AsyncContext *);
/// Get the value of a non-throwing async-let, awaiting the result if necessary,
/// and then destroy the child task. The result buffer is left initialized after
/// returning.
/// Get the value of a non-throwing async let binding that was created
/// with \c swift_asyncLet_begin, awaiting the task's completion if
/// necessary, and then destroy the task and the binding.
///
/// This can be called from any thread. Its Swift signature is
/// This must be called from the parent task of the async let.
///
/// Behaves like a Swift async function with the signature:
///
/// \code
/// func swift_asyncLet_get(
@@ -511,22 +501,34 @@ void swift_asyncLet_finish(SWIFT_ASYNC_CONTEXT AsyncContext *,
/// ) async
/// \endcode
///
/// \c result points at the variable storage for the binding. It is
/// uninitialized until the first call to \c swift_asyncLet_get or
/// \c swift_asyncLet_get_throwing. The child task will be invalidated after
/// this call, so the `async let` can not be gotten or finished afterward.
SWIFT_EXPORT_FROM(swift_Concurrency) SWIFT_CC(swiftasync)
void swift_asyncLet_consume(SWIFT_ASYNC_CONTEXT AsyncContext *,
AsyncLet *,
void *,
TaskContinuationFunction *,
AsyncContext *);
/// Get the value of a throwing async-let, awaiting the result if necessary,
/// and then destroy the child task. The result buffer is left initialized after
/// returning.
/// except that it uses the special calling convention in which the
/// continuation function, parent context, and child context are all
/// passed in separately rather than being stored in the child context.
/// The child async context has a fixed size of 5 * sizeof(void*).
///
/// This can be called from any thread. Its Swift signature is
/// \c resultBuffer must be the same result buffer that was passed to
/// swift_asyncLet_begin. After asynchronous return, this is guaranteed
/// to be initialized.
///
/// The async let is invalid after this call and cannot be used again.
/// Any memory initially allocated for the task is deallocated. Because
/// \c swift_asyncLet_begin potentially allocates memory on the parent
/// task's task allocator, the call to this function must be properly
/// paired with the begin.
SWIFT_EXPORT_FROM(swift_Concurrency) SWIFT_CC(swiftasync)
void swift_asyncLet_consume(SWIFT_ASYNC_CONTEXT AsyncContext *callerContext,
AsyncLet *alet,
void *resultBuffer,
TaskContinuationFunction *continuation,
AsyncContext *callContext);
/// Get the value of a throwing async let binding that was created
/// with \c swift_asyncLet_begin, awaiting the task's completion if
/// necessary, and then destroy the task and the binding.
///
/// This must be called from the parent task of the async let.
///
/// Behaves like a Swift async function with the signature:
///
/// \code
/// func swift_asyncLet_get_throwing(
@@ -535,18 +537,28 @@ void swift_asyncLet_consume(SWIFT_ASYNC_CONTEXT AsyncContext *,
/// ) async throws
/// \endcode
///
/// \c result points at the variable storage for the binding. It is
/// uninitialized until the first call to \c swift_asyncLet_get or
/// \c swift_asyncLet_get_throwing. That first call initializes the storage
/// with the result of the child task. Subsequent calls do nothing and leave
/// the value in place. The child task will be invalidated after
/// this call, so the `async let` can not be gotten or finished afterward.
/// except that it uses the special calling convention in which the
/// continuation function, parent context, and child context are all
/// passed in separately rather than being stored in the child context.
/// The child async context has a fixed size of 5 * sizeof(void*).
///
/// \c resultBuffer must be the same result buffer that was passed to
/// swift_asyncLet_begin. After asynchronous return, it is guaranteed
/// to be initialized unless the operation throws by passing an error
/// to the continuation function.
///
/// The async let is invalid after this call and cannot be used again.
/// Any memory initially allocated for the task is deallocated. Because
/// \c swift_asyncLet_begin potentially allocates memory on the parent
/// task's task allocator, the call to this function must be properly
/// paired with the begin.
SWIFT_EXPORT_FROM(swift_Concurrency) SWIFT_CC(swiftasync)
void swift_asyncLet_consume_throwing(SWIFT_ASYNC_CONTEXT AsyncContext *,
AsyncLet *,
void *,
ThrowingTaskFutureWaitContinuationFunction *,
AsyncContext *);
void swift_asyncLet_consume_throwing(SWIFT_ASYNC_CONTEXT AsyncContext *callerContext,
AsyncLet *alet,
void *resultBuffer,
ThrowingTaskFutureWaitContinuationFunction *
continuation,
AsyncContext *callContext);
/// Returns true if the currently executing AsyncTask has a
/// 'TaskGroupTaskStatusRecord' present.

View File

@@ -2555,27 +2555,6 @@ FUNCTION(DistributedActorInitializeRemote,
EFFECT(RuntimeEffect::Concurrency),
UNKNOWN_MEMEFFECTS)
/// void swift_asyncLet_start(
/// AsyncLet *alet,
/// TaskOptionRecord *options,
/// const Metadata *futureResultType,
/// void *closureEntryPoint,
/// HeapObject *closureContext
/// );
FUNCTION(AsyncLetStart,
_Concurrency, swift_asyncLet_start, SwiftCC,
ConcurrencyAvailability,
RETURNS(VoidTy),
ARGS(SwiftAsyncLetPtrTy, // AsyncLet*
SwiftTaskOptionRecordPtrTy, // options
TypeMetadataPtrTy, // futureResultType
Int8PtrTy, // closureEntry
OpaquePtrTy // closureContext
),
ATTRS(NoUnwind),
EFFECT(RuntimeEffect::Concurrency),
MEMEFFECTS(ArgMemOnly))
/// void swift_asyncLet_begin(
/// AsyncLet *alet,
/// TaskOptionRecord *options,
@@ -2599,16 +2578,6 @@ FUNCTION(AsyncLetBegin,
EFFECT(RuntimeEffect::Concurrency),
MEMEFFECTS(ArgMemOnly))
// void swift_asyncLet_end(AsyncLet *alet);
FUNCTION(EndAsyncLet,
_Concurrency, swift_asyncLet_end, SwiftCC,
ConcurrencyAvailability,
RETURNS(VoidTy),
ARGS(SwiftAsyncLetPtrTy),
ATTRS(NoUnwind),
EFFECT(RuntimeEffect::Concurrency),
UNKNOWN_MEMEFFECTS)
/// void swift_task_run_inline(
/// OpaqueValue *result,
/// void *closureAFP,

View File

@@ -234,7 +234,8 @@ llvm::Value *irgen::emitBuiltinStartAsyncLet(IRGenFunction &IGF,
localContextInfo = IGF.Builder.CreateBitCast(localContextInfo,
IGF.IGM.OpaquePtrTy);
// stack allocate AsyncLet, and begin lifetime for it (until EndAsyncLet)
// Stack allocate the AsyncLet structure and begin lifetime for it.
// This will be balanced in EndAsyncLetLifetime.
auto ty = llvm::ArrayType::get(IGF.IGM.Int8PtrTy, NumWords_AsyncLet);
auto address = IGF.createAlloca(ty, Alignment(Alignment_AsyncLet));
auto alet = IGF.Builder.CreateBitCast(address.getAddress(),
@@ -298,19 +299,13 @@ llvm::Value *irgen::emitBuiltinStartAsyncLet(IRGenFunction &IGF,
taskOptions =
maybeAddEmbeddedSwiftResultTypeInfo(IGF, taskOptions, futureResultType);
llvm::CallInst *call;
if (localResultBuffer) {
// This is @_silgen_name("swift_asyncLet_begin")
call = IGF.Builder.CreateCall(IGF.IGM.getAsyncLetBeginFunctionPointer(),
{alet, taskOptions, futureResultTypeMetadata,
taskFunction, localContextInfo,
localResultBuffer});
} else {
// This is @_silgen_name("swift_asyncLet_start")
call = IGF.Builder.CreateCall(IGF.IGM.getAsyncLetStartFunctionPointer(),
{alet, taskOptions, futureResultTypeMetadata,
taskFunction, localContextInfo});
}
// Call swift_asyncLet_begin. We no longer use swift_asyncLet_start.
llvm::CallInst *call =
IGF.Builder.CreateCall(IGF.IGM.getAsyncLetBeginFunctionPointer(),
{alet, taskOptions, futureResultTypeMetadata,
taskFunction, localContextInfo,
localResultBuffer});
call->setDoesNotThrow();
call->setCallingConv(IGF.IGM.SwiftCC);

View File

@@ -3055,11 +3055,6 @@ FunctionPointer::Kind irgen::classifyFunctionPointerKind(SILFunction *fn) {
if (name == "swift_task_future_wait_throwing")
return SpecialKind::TaskFutureWaitThrowing;
if (name == "swift_asyncLet_wait")
return SpecialKind::AsyncLetWait;
if (name == "swift_asyncLet_wait_throwing")
return SpecialKind::AsyncLetWaitThrowing;
if (name == "swift_asyncLet_get")
return SpecialKind::AsyncLetGet;
if (name == "swift_asyncLet_get_throwing")

View File

@@ -246,28 +246,6 @@ OVERRIDE_TASK(continuation_await, void,
swift::, (ContinuationAsyncContext *context),
(context))
OVERRIDE_ASYNC_LET(asyncLet_wait, void, SWIFT_EXPORT_FROM(swift_Concurrency),
SWIFT_CC(swiftasync), swift::,
(OpaqueValue *result,
SWIFT_ASYNC_CONTEXT AsyncContext *callerContext,
AsyncLet *alet, TaskContinuationFunction *resumeFn,
AsyncContext *callContext),
(result, callerContext, alet, resumeFn, callContext))
OVERRIDE_ASYNC_LET(asyncLet_wait_throwing, void,
SWIFT_EXPORT_FROM(swift_Concurrency), SWIFT_CC(swiftasync),
swift::,
(OpaqueValue *result,
SWIFT_ASYNC_CONTEXT AsyncContext *callerContext,
AsyncLet *alet,
ThrowingTaskFutureWaitContinuationFunction *resume,
AsyncContext *callContext),
(result, callerContext, alet, resume, callContext))
OVERRIDE_ASYNC_LET(asyncLet_end, void,
SWIFT_EXPORT_FROM(swift_Concurrency), SWIFT_CC(swift),
swift::, (AsyncLet *alet), (alet))
OVERRIDE_ASYNC_LET(asyncLet_get, void,
SWIFT_EXPORT_FROM(swift_Concurrency), SWIFT_CC(swiftasync),
swift::,

View File

@@ -46,7 +46,23 @@ public:
private:
// Flags stored in the low bits of the task pointer.
enum {
/// Whether the result buffer (the `void*` passed to all the various
/// runtime functions) has already been initialized. This implies that
/// the async let task has completed (and without throwing).
///
/// Note that the result buffer is currently a *copy* of the actual
/// return value: we currently set up the task so that it evaluates
/// into a future, then wait on that future the same way we would
/// wait for an unstructured task. This is wasteful, since async let
/// tasks always have a single waiter; there's no good reason not to
/// evaluate the task directly into the result buffer and avoid the
/// copy.
HasResult = 1 << 0,
/// Whether the task was allocated with the parent task's stack
/// allocator. We normally try to use the allocation given to us by
/// the compiler, but if that's not big enough, the runtime must
/// allocate more memory.
DidAllocateFromParentTask = 1 << 1,
};
@@ -160,32 +176,7 @@ void swift::asyncLet_addImpl(AsyncTask *task, AsyncLet *asyncLet,
}
// =============================================================================
// ==== start ------------------------------------------------------------------
SWIFT_CC(swift)
void swift::swift_asyncLet_start(AsyncLet *alet,
TaskOptionRecord *options,
const Metadata *futureResultType,
void *closureEntryPoint,
HeapObject *closureContext) {
auto flags = TaskCreateFlags();
#if SWIFT_CONCURRENCY_TASK_TO_THREAD_MODEL
// In the task to thread model, we don't want tasks to start running on
// separate threads - they will run in the context of the parent
flags.setEnqueueJob(false);
#else
flags.setEnqueueJob(true);
#endif
AsyncLetTaskOptionRecord asyncLetOptionRecord(alet);
asyncLetOptionRecord.Parent = options;
swift_task_create(
flags.getOpaqueValue(),
&asyncLetOptionRecord,
futureResultType,
closureEntryPoint, closureContext);
}
// ==== begin ------------------------------------------------------------------
SWIFT_CC(swift)
void swift::swift_asyncLet_begin(AsyncLet *alet,
@@ -218,30 +209,6 @@ void swift::swift_asyncLet_begin(AsyncLet *alet,
closureEntryPoint, closureContext);
}
// =============================================================================
// ==== wait -------------------------------------------------------------------
SWIFT_CC(swiftasync)
static void swift_asyncLet_waitImpl(
OpaqueValue *result, SWIFT_ASYNC_CONTEXT AsyncContext *callerContext,
AsyncLet *alet, TaskContinuationFunction *resumeFunction,
AsyncContext *callContext) {
auto task = alet->getTask();
swift_task_future_wait(result, callerContext, task, resumeFunction,
callContext);
}
SWIFT_CC(swiftasync)
static void swift_asyncLet_wait_throwingImpl(
OpaqueValue *result, SWIFT_ASYNC_CONTEXT AsyncContext *callerContext,
AsyncLet *alet,
ThrowingTaskFutureWaitContinuationFunction *resumeFunction,
AsyncContext * callContext) {
auto task = alet->getTask();
swift_task_future_wait_throwing(result, callerContext, task, resumeFunction,
callerContext);
}
// =============================================================================
// ==== get -------------------------------------------------------------------
@@ -323,30 +290,6 @@ static void swift_asyncLet_get_throwingImpl(
futureContext);
}
// =============================================================================
// ==== end --------------------------------------------------------------------
SWIFT_CC(swift)
static void swift_asyncLet_endImpl(AsyncLet *alet) {
auto task = alet->getTask();
// Cancel the task as we exit the scope
swift_task_cancel(task);
// Remove the child record from the parent task
auto record = asImpl(alet)->getTaskRecord();
removeStatusRecordFromSelf(record);
// TODO: we need to implicitly await either before the end or here somehow.
// and finally, release the task and free the async-let
AsyncTask *parent = swift_task_getCurrent();
assert(parent && "async-let must have a parent task");
SWIFT_TASK_DEBUG_LOG("async let end of task %p, parent: %p", task, parent);
_swift_task_dealloc_specific(parent, task);
}
// =============================================================================
// ==== finish -----------------------------------------------------------------
@@ -359,24 +302,27 @@ static void asyncLet_finish_after_task_completion(SWIFT_ASYNC_CONTEXT AsyncConte
TaskContinuationFunction *resumeFunction,
AsyncContext *callContext,
SWIFT_CONTEXT void *error) {
auto parentTask = swift_task_getCurrent();
assert(parentTask && "async-let must have a parent task");
auto task = alet->getTask();
// Remove the child record from the parent task
auto record = asImpl(alet)->getTaskRecord();
removeStatusRecordFromSelf(record);
// and finally, release the task and destroy the async-let
assert(swift_task_getCurrent() && "async-let must have a parent task");
SWIFT_TASK_DEBUG_LOG("async let end of task %p, parent: %p", task,
swift_task_getCurrent());
// Destruct the task.
// Destroy the task. Note that this destroys the copy of the result
// (error or normal) in the task's future fragment.
SWIFT_TASK_DEBUG_LOG("async let end of task %p, parent: %p", task, parentTask);
task->~AsyncTask();
// Deallocate it out of the parent, if it was allocated there.
// Deallocate the memory for the child task, if it was allocated with
// the parent's stack allocator.
if (alet->didAllocateFromParentTask()) {
swift_task_dealloc(task);
_swift_task_dealloc_specific(parentTask, (void*) task);
}
// Call the continuation function.
return function_cast<ThrowingTaskFutureWaitContinuationFunction*>(resumeFunction)
(callerContext, error);
}
@@ -392,6 +338,11 @@ static void _asyncLet_finish_continuation(
auto alet = continuationContext->alet;
auto resultBuffer = continuationContext->resultBuffer;
// We waited for the task using swift_task_future_wait_throwing,
// which means we've been passed a copy of the result in the future
// fragment (either an error, if the task threw, or the result
// in the result buffer). Destroy that copy now.
// Destroy the error, or the result that was stored to the buffer.
if (error) {
#if SWIFT_CONCURRENCY_EMBEDDED

View File

@@ -23,23 +23,6 @@ public func _asyncLetStart<T>(
operation: @Sendable () async throws -> T
)
/// DEPRECATED. use _asyncLet_get instead
@available(SwiftStdlib 5.1, *)
@_silgen_name("swift_asyncLet_wait")
public func _asyncLetGet<T>(asyncLet: Builtin.RawPointer) async -> T
/// DEPRECATED. use _asyncLet_get_throwing instead
@available(SwiftStdlib 5.1, *)
@_silgen_name("swift_asyncLet_wait_throwing")
public func _asyncLetGetThrowing<T>(asyncLet: Builtin.RawPointer) async throws -> T
/// DEPRECATED. use _asyncLet_finish instead
@available(SwiftStdlib 5.1, *)
@_silgen_name("swift_asyncLet_end")
public func _asyncLetEnd(
asyncLet: Builtin.RawPointer // TODO: should this take __owned?
)
/// Wait if necessary and then project the result value of an async let
@available(SwiftStdlib 5.1, *)
@_silgen_name("swift_asyncLet_get")

View File

@@ -773,13 +773,6 @@ swift_task_create_commonImpl(size_t rawTaskCreateFlags,
jobFlags.task_setIsGroupChildTask(true);
break;
case TaskOptionRecordKind::AsyncLet:
asyncLet = cast<AsyncLetTaskOptionRecord>(option)->getAsyncLet();
assert(asyncLet && "Missing async let storage");
jobFlags.task_setIsAsyncLetTask(true);
jobFlags.task_setIsChildTask(true);
break;
case TaskOptionRecordKind::AsyncLetWithBuffer: {
auto *aletRecord = cast<AsyncLetWithBufferTaskOptionRecord>(option);
asyncLet = aletRecord->getAsyncLet();

View File

@@ -301,11 +301,11 @@ namespace {
/// @_silgen_name("swift_task_future_wait_throwing")
/// func _taskFutureGetThrowing<T>(_ task: Builtin.NativeObject) async throws -> T
///
/// @_silgen_name("swift_asyncLet_wait")
/// func _asyncLetGet<T>(_ task: Builtin.RawPointer) async -> T
/// @_silgen_name("swift_asyncLet_get")
/// func _asyncLet_get<T>(_ task: Builtin.RawPointer) async -> T
///
/// @_silgen_name("swift_asyncLet_waitThrowing")
/// func _asyncLetGetThrowing<T>(_ task: Builtin.RawPointer) async throws -> T
/// @_silgen_name("swift_asyncLet_get_throwing")
/// func _asyncLet_get_throwing<T>(_ task: Builtin.RawPointer) async throws -> T
///
/// @_silgen_name("swift_taskGroup_wait_next_throwing")
/// func _taskGroupWaitNext<T>(group: Builtin.RawPointer) async throws -> T?

View File

@@ -1088,13 +1088,9 @@ __swift_createJobForTestingOnly
_swift_asyncLet_begin
_swift_asyncLet_consume
_swift_asyncLet_consume_throwing
_swift_asyncLet_end
_swift_asyncLet_finish
_swift_asyncLet_get
_swift_asyncLet_get_throwing
_swift_asyncLet_start
_swift_asyncLet_wait
_swift_asyncLet_wait_throwing
_swift_async_extendedFramePointerFlags
_swift_concurrency_jobPriority
_swift_continuation_await

View File

@@ -1088,13 +1088,9 @@ __swift_createJobForTestingOnly
_swift_asyncLet_begin
_swift_asyncLet_consume
_swift_asyncLet_consume_throwing
_swift_asyncLet_end
_swift_asyncLet_finish
_swift_asyncLet_get
_swift_asyncLet_get_throwing
_swift_asyncLet_start
_swift_asyncLet_wait
_swift_asyncLet_wait_throwing
_swift_async_extendedFramePointerFlags
_swift_concurrency_jobPriority
_swift_continuation_await

View File

@@ -1088,13 +1088,9 @@ __swift_createJobForTestingOnly
_swift_asyncLet_begin
_swift_asyncLet_consume
_swift_asyncLet_consume_throwing
_swift_asyncLet_end
_swift_asyncLet_finish
_swift_asyncLet_get
_swift_asyncLet_get_throwing
_swift_asyncLet_start
_swift_asyncLet_wait
_swift_asyncLet_wait_throwing
_swift_async_extendedFramePointerFlags
_swift_concurrency_jobPriority
_swift_continuation_await

View File

@@ -1088,13 +1088,9 @@ __swift_createJobForTestingOnly
_swift_asyncLet_begin
_swift_asyncLet_consume
_swift_asyncLet_consume_throwing
_swift_asyncLet_end
_swift_asyncLet_finish
_swift_asyncLet_get
_swift_asyncLet_get_throwing
_swift_asyncLet_start
_swift_asyncLet_wait
_swift_asyncLet_wait_throwing
_swift_async_extendedFramePointerFlags
_swift_concurrency_jobPriority
_swift_continuation_await

View File

@@ -248,18 +248,6 @@ TEST_F(CompatibilityOverrideConcurrencyTest,
swift_continuation_throwingResumeWithError(nullptr, nullptr);
}
TEST_F(CompatibilityOverrideConcurrencyTest, test_swift_asyncLet_wait) {
swift_asyncLet_wait(nullptr, nullptr, nullptr, nullptr, nullptr);
}
TEST_F(CompatibilityOverrideConcurrencyTest, test_swift_asyncLet_wait_throwing) {
swift_asyncLet_wait(nullptr, nullptr, nullptr, nullptr, nullptr);
}
TEST_F(CompatibilityOverrideConcurrencyTest, test_swift_asyncLet_end) {
swift_asyncLet_end(nullptr);
}
TEST_F(CompatibilityOverrideConcurrencyTest, test_swift_taskGroup_initialize) {
swift_taskGroup_initialize(nullptr, nullptr);
}

View File

@@ -259,110 +259,6 @@
],
"funcSelfKind": "NonMutating"
},
{
"kind": "Function",
"name": "_asyncLetGet",
"printedName": "_asyncLetGet(asyncLet:)",
"children": [
{
"kind": "TypeNominal",
"name": "GenericTypeParam",
"printedName": "τ_0_0"
},
{
"kind": "TypeNominal",
"name": "BuiltinRawPointer",
"printedName": "Builtin.RawPointer"
}
],
"declKind": "Func",
"usr": "s:12_Concurrency12_asyncLetGet0bC0xBp_tYalF",
"mangledName": "swift_asyncLet_wait",
"moduleName": "_Concurrency",
"genericSig": "<τ_0_0>",
"sugared_genericSig": "<T>",
"intro_Macosx": "10.15",
"intro_iOS": "13.0",
"intro_tvOS": "13.0",
"intro_watchOS": "6.0",
"declAttributes": [
"SILGenName",
"Available",
"Available",
"Available",
"Available"
],
"funcSelfKind": "NonMutating"
},
{
"kind": "Function",
"name": "_asyncLetGetThrowing",
"printedName": "_asyncLetGetThrowing(asyncLet:)",
"children": [
{
"kind": "TypeNominal",
"name": "GenericTypeParam",
"printedName": "τ_0_0"
},
{
"kind": "TypeNominal",
"name": "BuiltinRawPointer",
"printedName": "Builtin.RawPointer"
}
],
"declKind": "Func",
"usr": "s:12_Concurrency20_asyncLetGetThrowing0bC0xBp_tYaKlF",
"mangledName": "swift_asyncLet_wait_throwing",
"moduleName": "_Concurrency",
"genericSig": "<τ_0_0>",
"sugared_genericSig": "<T>",
"intro_Macosx": "10.15",
"intro_iOS": "13.0",
"intro_tvOS": "13.0",
"intro_watchOS": "6.0",
"declAttributes": [
"SILGenName",
"Available",
"Available",
"Available",
"Available"
],
"throwing": true,
"funcSelfKind": "NonMutating"
},
{
"kind": "Function",
"name": "_asyncLetEnd",
"printedName": "_asyncLetEnd(asyncLet:)",
"children": [
{
"kind": "TypeNominal",
"name": "Void",
"printedName": "()"
},
{
"kind": "TypeNominal",
"name": "BuiltinRawPointer",
"printedName": "Builtin.RawPointer"
}
],
"declKind": "Func",
"usr": "s:12_Concurrency12_asyncLetEnd0bC0yBp_tF",
"mangledName": "swift_asyncLet_end",
"moduleName": "_Concurrency",
"intro_Macosx": "10.15",
"intro_iOS": "13.0",
"intro_tvOS": "13.0",
"intro_watchOS": "6.0",
"declAttributes": [
"SILGenName",
"Available",
"Available",
"Available",
"Available"
],
"funcSelfKind": "NonMutating"
},
{
"kind": "Function",
"name": "_asyncLet_get",