[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:
Doug Gregor
2020-11-07 21:21:16 -08:00
parent b43eab419a
commit 4c2c2f32e9
16 changed files with 171 additions and 7 deletions

View File

@@ -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

View File

@@ -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

View File

@@ -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

View File

@@ -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);

View File

@@ -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;

View File

@@ -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,

View File

@@ -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

View File

@@ -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});
}

View File

@@ -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;

View File

@@ -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

View File

@@ -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

View File

@@ -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.

View File

@@ -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.

View File

@@ -166,6 +166,7 @@ static bool isBarrier(SILInstruction *inst) {
case BuiltinValueKind::UnsafeGuaranteed:
case BuiltinValueKind::UnsafeGuaranteedEnd:
case BuiltinValueKind::CancelAsyncTask:
case BuiltinValueKind::CreateAsyncTask:
return true;
}
}

View File

@@ -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 : $()
}

View File

@@ -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")
}
}
}