mirror of
https://github.com/apple/swift.git
synced 2025-12-21 12:14:44 +01:00
[Concurrency] Implement a builtin createAsyncTask() to create a new task.
`Builtin.createAsyncTask` takes flags, an optional parent task, and an async/throwing function to execute, and passes it along to the `swift_task_create_f` entry point to create a new (potentially child) task, returning the new task and its initial context.
This commit is contained in:
@@ -728,6 +728,13 @@ BUILTIN_MISC_OPERATION_WITH_SILGEN(GetCurrentAsyncTask, "getCurrentAsyncTask", "
|
||||
/// Cancel the given asynchronous task.
|
||||
BUILTIN_MISC_OPERATION_WITH_SILGEN(CancelAsyncTask, "cancelAsyncTask", "", Special)
|
||||
|
||||
/// createAsyncTask(): (
|
||||
/// Int, Builtin.NativeObject?, @escaping () async throws -> Void
|
||||
/// ) -> Builtin.NativeObject
|
||||
///
|
||||
/// Create a new asynchronous task, given flags, an (optional) parent task, and
|
||||
/// a function to execute.
|
||||
BUILTIN_MISC_OPERATION_WITH_SILGEN(CreateAsyncTask, "createAsyncTask", "", Special)
|
||||
|
||||
/// globalStringTablePointer has type String -> Builtin.RawPointer.
|
||||
/// It returns an immortal, global string table pointer for strings constructed
|
||||
|
||||
@@ -135,6 +135,11 @@ llvm::AtomicOrdering decodeLLVMAtomicOrdering(StringRef O);
|
||||
/// Returns true if the builtin with ID \p ID has a defined static overload for
|
||||
/// the type \p Ty.
|
||||
bool canBuiltinBeOverloadedForType(BuiltinValueKind ID, Type Ty);
|
||||
|
||||
/// Retrieve the AST-level AsyncTaskAndContext type, used for the
|
||||
/// createAsyncTask builtin.
|
||||
Type getAsyncTaskAndContextType(ASTContext &ctx);
|
||||
|
||||
}
|
||||
|
||||
#endif
|
||||
|
||||
@@ -1489,7 +1489,7 @@ FUNCTION(TaskDealloc,
|
||||
ARGS(SwiftTaskPtrTy, Int8PtrTy),
|
||||
ATTRS(NoUnwind, ArgMemOnly))
|
||||
|
||||
// void swift_task_dealloc(AsyncTask *task, void *ptr);
|
||||
// void swift_task_cancel(AsyncTask *task);
|
||||
FUNCTION(TaskCancel,
|
||||
swift_task_cancel, SwiftCC,
|
||||
ConcurrencyAvailability,
|
||||
@@ -1497,6 +1497,16 @@ FUNCTION(TaskCancel,
|
||||
ARGS(SwiftTaskPtrTy),
|
||||
ATTRS(NoUnwind, ArgMemOnly))
|
||||
|
||||
// AsyncTaskAndContext swift_task_create_f(
|
||||
// size_t flags, AsyncTask *task, AsyncFunctionType<void()> *function,
|
||||
// size_t initialContextSize);
|
||||
FUNCTION(TaskCreateFunc,
|
||||
swift_task_create_f, SwiftCC,
|
||||
ConcurrencyAvailability,
|
||||
RETURNS(AsyncTaskAndContextTy),
|
||||
ARGS(SizeTy, SwiftTaskPtrTy, TaskContinuationFunctionPtrTy, SizeTy),
|
||||
ATTRS(NoUnwind, ArgMemOnly))
|
||||
|
||||
#undef RETURNS
|
||||
#undef ARGS
|
||||
#undef ATTRS
|
||||
|
||||
@@ -1350,6 +1350,25 @@ static ValueDecl *getCancelAsyncTask(ASTContext &ctx, Identifier id) {
|
||||
id, { ctx.TheNativeObjectType }, ctx.TheEmptyTupleType);
|
||||
}
|
||||
|
||||
Type swift::getAsyncTaskAndContextType(ASTContext &ctx) {
|
||||
TupleTypeElt resultTupleElements[2] = {
|
||||
ctx.TheNativeObjectType, // task,
|
||||
ctx.TheRawPointerType // initial context
|
||||
};
|
||||
|
||||
return TupleType::get(resultTupleElements, ctx);
|
||||
}
|
||||
|
||||
static ValueDecl *getCreateAsyncTask(ASTContext &ctx, Identifier id) {
|
||||
auto extInfo = ASTExtInfoBuilder().withAsync().withThrows().build();
|
||||
return getBuiltinFunction(
|
||||
id,
|
||||
{ ctx.getIntDecl()->getDeclaredInterfaceType(),
|
||||
OptionalType::get(ctx.TheNativeObjectType),
|
||||
FunctionType::get({ }, ctx.TheEmptyTupleType, extInfo) },
|
||||
getAsyncTaskAndContextType(ctx));
|
||||
}
|
||||
|
||||
static ValueDecl *getPoundAssert(ASTContext &Context, Identifier Id) {
|
||||
auto int1Type = BuiltinIntegerType::get(1, Context);
|
||||
auto optionalRawPointerType = BoundGenericEnumType::get(
|
||||
@@ -2482,6 +2501,9 @@ ValueDecl *swift::getBuiltinValueDecl(ASTContext &Context, Identifier Id) {
|
||||
case BuiltinValueKind::CancelAsyncTask:
|
||||
return getCancelAsyncTask(Context, Id);
|
||||
|
||||
case BuiltinValueKind::CreateAsyncTask:
|
||||
return getCreateAsyncTask(Context, Id);
|
||||
|
||||
case BuiltinValueKind::PoundAssert:
|
||||
return getPoundAssert(Context, Id);
|
||||
|
||||
|
||||
@@ -225,6 +225,24 @@ void irgen::emitBuiltinCall(IRGenFunction &IGF, const BuiltinInfo &Builtin,
|
||||
return;
|
||||
}
|
||||
|
||||
if (Builtin.ID == BuiltinValueKind::CreateAsyncTask) {
|
||||
auto flags = args.claimNext();
|
||||
auto parentTask = args.claimNext();
|
||||
auto taskFunction = args.claimNext();
|
||||
auto taskContext = args.claimNext();
|
||||
auto newTaskAndContext = emitTaskCreate(
|
||||
IGF, flags, parentTask, taskFunction, taskContext);
|
||||
|
||||
// Cast back to NativeObject/RawPointer.
|
||||
auto newTask = IGF.Builder.CreateExtractValue(newTaskAndContext, { 0 });
|
||||
newTask = IGF.Builder.CreateBitCast(newTask, IGF.IGM.RefCountedPtrTy);
|
||||
auto newContext = IGF.Builder.CreateExtractValue(newTaskAndContext, { 1 });
|
||||
newContext = IGF.Builder.CreateBitCast(newContext, IGF.IGM.Int8PtrTy);
|
||||
out.add(newTask);
|
||||
out.add(newContext);
|
||||
return;
|
||||
}
|
||||
|
||||
// If this is an LLVM IR intrinsic, lower it to an intrinsic call.
|
||||
const IntrinsicInfo &IInfo = IGF.getSILModule().getIntrinsicInfo(FnId);
|
||||
llvm::Intrinsic::ID IID = IInfo.ID;
|
||||
|
||||
@@ -3466,8 +3466,52 @@ void irgen::emitTaskCancel(IRGenFunction &IGF, llvm::Value *task) {
|
||||
auto *call = IGF.Builder.CreateCall(IGF.IGM.getTaskCancelFn(), {task});
|
||||
call->setDoesNotThrow();
|
||||
call->setCallingConv(IGF.IGM.SwiftCC);
|
||||
call->addAttribute(llvm::AttributeList::FunctionIndex,
|
||||
llvm::Attribute::ReadNone);
|
||||
}
|
||||
|
||||
llvm::Value *irgen::emitTaskCreate(
|
||||
IRGenFunction &IGF, llvm::Value *flags, llvm::Value *parentTask,
|
||||
llvm::Value *taskFunction, llvm::Value *localContextInfo) {
|
||||
parentTask = IGF.Builder.CreateBitOrPointerCast(
|
||||
parentTask, IGF.IGM.SwiftTaskPtrTy);
|
||||
taskFunction = IGF.Builder.CreateBitOrPointerCast(
|
||||
taskFunction, IGF.IGM.TaskContinuationFunctionPtrTy);
|
||||
|
||||
// Determine the size of the async context for the closure.
|
||||
// FIXME: If the task function comes in as an AsyncFunctionPointer, we might
|
||||
// want to use swift_task_create instead of swift_task_create_f.
|
||||
ASTContext &ctx = IGF.IGM.IRGen.SIL.getASTContext();
|
||||
auto extInfo = ASTExtInfoBuilder().withAsync().withThrows().build();
|
||||
auto taskFunctionType = FunctionType::get(
|
||||
{ }, ctx.TheEmptyTupleType, extInfo);
|
||||
CanSILFunctionType taskFunctionCanSILType =
|
||||
IGF.IGM.getLoweredType(taskFunctionType).castTo<SILFunctionType>();
|
||||
auto layout = getAsyncContextLayout(
|
||||
IGF.IGM, taskFunctionCanSILType, taskFunctionCanSILType,
|
||||
SubstitutionMap());
|
||||
auto layoutSize = getAsyncContextSize(layout);
|
||||
auto layoutSizeVal = llvm::ConstantInt::get(
|
||||
IGF.IGM.SizeTy, layoutSize.getValue());
|
||||
|
||||
// Call the function.
|
||||
auto *result = IGF.Builder.CreateCall(
|
||||
IGF.IGM.getTaskCreateFuncFn(),
|
||||
{ flags, parentTask, taskFunction, layoutSizeVal });
|
||||
result->setDoesNotThrow();
|
||||
result->setCallingConv(IGF.IGM.SwiftCC);
|
||||
|
||||
// Write the local context information into the initial context for the task.
|
||||
if (layout.hasLocalContext()) {
|
||||
// Dig out the initial context returned from task creation.
|
||||
auto initialContext = IGF.Builder.CreateExtractValue(result, { 1 });
|
||||
Address initialContextAddr = layout.emitCastTo(IGF, initialContext);
|
||||
|
||||
auto localContextLayout = layout.getLocalContextLayout();
|
||||
auto localContextAddr = localContextLayout.project(
|
||||
IGF, initialContextAddr, llvm::None);
|
||||
IGF.Builder.CreateStore(localContextInfo, localContextAddr);
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
std::pair<Address, Size> irgen::emitAllocAsyncContext(IRGenFunction &IGF,
|
||||
|
||||
@@ -390,6 +390,12 @@ namespace irgen {
|
||||
void emitTaskDealloc(IRGenFunction &IGF, Address address, llvm::Value *size);
|
||||
void emitTaskCancel(IRGenFunction &IGF, llvm::Value *task);
|
||||
|
||||
/// Emit a class to swift_task_create[_f] with the given flags, parent task,
|
||||
/// and task function.
|
||||
llvm::Value *emitTaskCreate(
|
||||
IRGenFunction &IGF, llvm::Value *flags, llvm::Value *parentTask,
|
||||
llvm::Value *taskFunction, llvm::Value *localContextInfo);
|
||||
|
||||
/// Allocate task local storage for the specified layout but using the
|
||||
/// provided dynamic size. Allowing the size to be specified dynamically is
|
||||
/// necessary for applies of thick functions the sizes of whose async contexts
|
||||
|
||||
@@ -602,6 +602,10 @@ IRGenModule::IRGenModule(IRGenerator &irgen,
|
||||
/*isVarArg*/ false);
|
||||
TaskContinuationFunctionPtrTy = TaskContinuationFunctionTy->getPointerTo();
|
||||
|
||||
AsyncTaskAndContextTy = createStructType(
|
||||
*this, "swift.async_task_and_context",
|
||||
{ SwiftTaskPtrTy, SwiftContextPtrTy });
|
||||
|
||||
DifferentiabilityWitnessTy = createStructType(
|
||||
*this, "swift.differentiability_witness", {Int8PtrTy, Int8PtrTy});
|
||||
}
|
||||
|
||||
@@ -731,7 +731,7 @@ public:
|
||||
llvm::PointerType *SwiftExecutorPtrTy;
|
||||
llvm::FunctionType *TaskContinuationFunctionTy;
|
||||
llvm::PointerType *TaskContinuationFunctionPtrTy;
|
||||
|
||||
llvm::StructType *AsyncTaskAndContextTy;
|
||||
llvm::StructType *DifferentiabilityWitnessTy; // { i8*, i8* }
|
||||
|
||||
llvm::GlobalVariable *TheTrivialPropertyDescriptor = nullptr;
|
||||
|
||||
@@ -1020,6 +1020,7 @@ ANY_OWNERSHIP_BUILTIN(IntInstrprofIncrement)
|
||||
CONSTANT_OWNERSHIP_BUILTIN(Owned, MustBeInvalidated, COWBufferForReading)
|
||||
CONSTANT_OWNERSHIP_BUILTIN(Owned, MustBeInvalidated, UnsafeGuaranteed)
|
||||
CONSTANT_OWNERSHIP_BUILTIN(Guaranteed, MustBeLive, CancelAsyncTask)
|
||||
CONSTANT_OWNERSHIP_BUILTIN(Guaranteed, MustBeLive, CreateAsyncTask)
|
||||
|
||||
#undef CONSTANT_OWNERSHIP_BUILTIN
|
||||
|
||||
|
||||
@@ -543,6 +543,7 @@ CONSTANT_OWNERSHIP_BUILTIN(None, IntInstrprofIncrement)
|
||||
CONSTANT_OWNERSHIP_BUILTIN(None, GlobalStringTablePointer)
|
||||
CONSTANT_OWNERSHIP_BUILTIN(Owned, GetCurrentAsyncTask)
|
||||
CONSTANT_OWNERSHIP_BUILTIN(None, CancelAsyncTask)
|
||||
CONSTANT_OWNERSHIP_BUILTIN(Owned, CreateAsyncTask)
|
||||
|
||||
#undef CONSTANT_OWNERSHIP_BUILTIN
|
||||
|
||||
|
||||
@@ -1800,6 +1800,7 @@ static void visitBuiltinAddress(BuiltinInst *builtin,
|
||||
case BuiltinValueKind::IntInstrprofIncrement:
|
||||
case BuiltinValueKind::TSanInoutAccess:
|
||||
case BuiltinValueKind::CancelAsyncTask:
|
||||
case BuiltinValueKind::CreateAsyncTask:
|
||||
return;
|
||||
|
||||
// General memory access to a pointer in first operand position.
|
||||
|
||||
@@ -1405,7 +1405,7 @@ static ManagedValue emitBuiltinGetCurrentAsyncTask(
|
||||
return SGF.emitManagedRValueWithEndLifetimeCleanup(apply);
|
||||
}
|
||||
|
||||
// Emit SIL for the named builtin: getCurrentAsyncTask.
|
||||
// Emit SIL for the named builtin: cancelAsyncTask.
|
||||
static ManagedValue emitBuiltinCancelAsyncTask(
|
||||
SILGenFunction &SGF, SILLocation loc, SubstitutionMap subs,
|
||||
ArrayRef<ManagedValue> args, SGFContext C) {
|
||||
@@ -1419,6 +1419,22 @@ static ManagedValue emitBuiltinCancelAsyncTask(
|
||||
return ManagedValue::forUnmanaged(apply);
|
||||
}
|
||||
|
||||
// Emit SIL for the named builtin: createAsyncTask.
|
||||
static ManagedValue emitBuiltinCreateAsyncTask(
|
||||
SILGenFunction &SGF, SILLocation loc, SubstitutionMap subs,
|
||||
ArrayRef<ManagedValue> args, SGFContext C) {
|
||||
ASTContext &ctx = SGF.getASTContext();
|
||||
auto flags = args[0].forward(SGF);
|
||||
auto parentTask = args[1].borrow(SGF, loc).forward(SGF);
|
||||
auto function = args[2].borrow(SGF, loc).forward(SGF);
|
||||
auto apply = SGF.B.createBuiltin(
|
||||
loc,
|
||||
ctx.getIdentifier(getBuiltinName(BuiltinValueKind::CreateAsyncTask)),
|
||||
SGF.getLoweredType(getAsyncTaskAndContextType(ctx)), SubstitutionMap(),
|
||||
{ flags, parentTask, function });
|
||||
return SGF.emitManagedRValueWithCleanup(apply);
|
||||
}
|
||||
|
||||
Optional<SpecializedEmitter>
|
||||
SpecializedEmitter::forDecl(SILGenModule &SGM, SILDeclRef function) {
|
||||
// Only consider standalone declarations in the Builtin module.
|
||||
|
||||
@@ -166,6 +166,7 @@ static bool isBarrier(SILInstruction *inst) {
|
||||
case BuiltinValueKind::UnsafeGuaranteed:
|
||||
case BuiltinValueKind::UnsafeGuaranteedEnd:
|
||||
case BuiltinValueKind::CancelAsyncTask:
|
||||
case BuiltinValueKind::CreateAsyncTask:
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -6,6 +6,7 @@
|
||||
sil_stage canonical
|
||||
|
||||
import Builtin
|
||||
import Swift
|
||||
|
||||
// CHECK-LABEL: define hidden swiftcc void @get_task(%swift.task* %0, %swift.executor* %1, %swift.context* %2)
|
||||
sil hidden [ossa] @get_task : $@async @convention(thin) () -> @owned Builtin.NativeObject {
|
||||
@@ -21,10 +22,25 @@ bb0:
|
||||
// CHECK-LABEL: define hidden swiftcc void @cancel_task(%swift.refcounted* %0)
|
||||
// CHECK: [[TASK:%.*]] = bitcast %swift.refcounted* %0 to %swift.task*
|
||||
// CHECK-NEXT: call swiftcc void @swift_task_cancel(%swift.task* [[TASK]])
|
||||
|
||||
sil hidden [ossa] @cancel_task : $@convention(method) (@guaranteed Builtin.NativeObject) -> () {
|
||||
bb0(%0 : @guaranteed $Builtin.NativeObject):
|
||||
%4 = builtin "cancelAsyncTask"(%0 : $Builtin.NativeObject) : $()
|
||||
%5 = tuple ()
|
||||
return %5 : $()
|
||||
}
|
||||
|
||||
// CHECK-LABEL: define hidden swiftcc void @launch_task
|
||||
sil hidden [ossa] @launch_task : $@convention(method) @async (Int, Optional<Builtin.NativeObject>, @guaranteed @async @callee_guaranteed () -> (@error Error)) -> () {
|
||||
bb0(%0 : $Int, %1: @unowned $Optional<Builtin.NativeObject>, %2: @guaranteed $@async @callee_guaranteed () -> (@error Error)):
|
||||
%3 = begin_borrow %1 : $Optional<Builtin.NativeObject>
|
||||
// CHECK: [[NEW_TASK_AND_CONTEXT:%.*]] = call swiftcc %swift.async_task_and_context @swift_task_create_f
|
||||
// CHECK-NEXT: [[NEW_CONTEXT_RAW:%.*]] = extractvalue %swift.async_task_and_context [[NEW_TASK_AND_CONTEXT]], 1
|
||||
// CHECK-NEXT: [[NEW_CONTEXT:%.*]] = bitcast %swift.context* [[NEW_CONTEXT_RAW]] to
|
||||
// CHECK-NEXT: [[CONTEXT_INFO_LOC:%.*]] = getelementptr inbounds <{{.*}}>* [[NEW_CONTEXT]]
|
||||
// CHECK-NEXT: store %swift.refcounted* %{{.*}}, %swift.refcounted** [[CONTEXT_INFO_LOC]]
|
||||
%20 = builtin "createAsyncTask"(%0 : $Int, %3 : $Optional<Builtin.NativeObject>, %2 : $@async @callee_guaranteed () -> (@error Error)) : $(Builtin.NativeObject, Builtin.RawPointer)
|
||||
end_borrow %3 : $Optional<Builtin.NativeObject>
|
||||
destroy_value %20 : $(Builtin.NativeObject, Builtin.RawPointer)
|
||||
%21 = tuple ()
|
||||
return %21 : $()
|
||||
}
|
||||
@@ -1,7 +1,9 @@
|
||||
// RUN: %target-swift-frontend -emit-silgen %s -module-name test -swift-version 5 -enable-experimental-concurrency -parse-stdlib -sil-verify-all | %FileCheck %s
|
||||
// REQUIRES: concurrency
|
||||
|
||||
struct X {
|
||||
import Swift
|
||||
|
||||
public struct X {
|
||||
// CHECK-LABEL: sil hidden [ossa] @$s4test1XV14getCurrentTaskBoyYF
|
||||
func getCurrentTask() async -> Builtin.NativeObject {
|
||||
// CHECK: builtin "getCurrentAsyncTask"() : $Builtin.NativeObject
|
||||
@@ -13,4 +15,14 @@ struct X {
|
||||
// CHECK: builtin "cancelAsyncTask"(%0 : $Builtin.NativeObject) : $()
|
||||
Builtin.cancelAsyncTask(task)
|
||||
}
|
||||
|
||||
// CHECK-LABEL: sil hidden [ossa] @$s4test1XV10launchTaskyyYF : $@convention(method) @async (X) -> ()
|
||||
func launchTask() async {
|
||||
// CHECK: builtin "createAsyncTask"([[FLAGS:%.*]] : $Int, [[PARENT:%.*]] : $Optional<Builtin.NativeObject>, [[FN:%.*]] : $@async @callee_guaranteed () -> @error Error) : $(Builtin.NativeObject, Builtin.RawPointer)
|
||||
let task = Builtin.getCurrentAsyncTask()
|
||||
let childTask = Builtin.createAsyncTask(0, task) {
|
||||
await launchTask()
|
||||
print("child is done")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user