Runtime: Allocation-free awaitable async operations with ValueTask<T> and ValueTask

Created on 25 Feb 2018  Ā·  117Comments  Ā·  Source: dotnet/runtime

Background

ValueTask<T> is currently a discriminated union of a T and a Task<T>. This lets APIs that are likely to complete synchronously and return a value do so without allocating a Task<T> object to carry the result value. However, operations that complete asynchronously still need to allocate a Task<T>. There is no non-generic ValueTask counterpart today because if you have an operation that completes synchronously and successfully, you can just return Task.CompletedTask, no allocation.

That addresses the 80% case where synchronously completing operations no longer allocate. But for cases where you want to strive to address the 20% case of operations completing asynchronously and still not allocating, youā€™re forced to play tricks with custom awaitables, which are one-offs, donā€™t compose well, and generally arenā€™t appropriate for public surface area. Task and Task<T>, by design, never go from a completed to incomplete state, meaning you canā€™t reuse the same object; this has many usability benefits, but for APIs that really care about that last pound of performance, in particular around allocations, it can get in the way.

We have a bunch of new APIs in .NET Core 2.1 that return ValueTask<T>s, e.g. Stream.ReadAsync, ChannelReader.ReadAsync, PipeReader.ReadAsync, etc. In many of these cases, weā€™ve simply accepted that they might allocate; in others, custom APIs have been introduced specific to that method. Neither of these is a good place to be.

Proposal

I have implemented a new feature in ValueTask<T> and a counterpart non-generic ValueTask that lets these not only wrap a T result or a Task<T>, but also another arbitrary object that implements the IValueTaskSource<T> interface (or IValueTaskSource for the non-generic ValueTask). An implementation of that interface can be reused, pooled, etc., allowing for an implementation that returns a ValueTask<T> or ValueTask to have amortized non-allocating operations, both synchronously completing and asynchronously completing.

The enabling APIs

First, we need to add these interfaces:
```C#
namespace System.Threading.Tasks
{
public interface IValueTaskSource
{
bool IsCompleted { get; }
bool IsCompletedSuccessfully { get; }
void OnCompleted(Action continuation, object state, ValueTaskSourceOnCompletedFlags flags);
void GetResult();
}

public interface IValueTaskSource<out TResult>
{
    bool IsCompleted { get; }
    bool IsCompletedSuccessfully { get; }
    void OnCompleted(Action<object> continuation, object state, ValueTaskSourceOnCompletedFlags flags);
    TResult GetResult();
}

[Flags]
public enum ValueTaskSourceOnCompletedFlags
{
    None = 0x0,
    UseSchedulingContext = 0x1,
    FlowExecutionContext = 0x2,
}

}

An object implements `IValueTaskSource` to be wrappable by `ValueTask`, and `IValueTaskSource<TResult>` to be wrappable by `ValueTask<TResult>`.

Then we add this ctor to `ValueTask<TResult>`:
```C#
namespace System.Threading.Tasks
{
    public struct ValueTask<TResult>
    {
        public ValueTask(IValueTaskSource<TResult> source);
        ...
    }
}

Then we add a non-generic ValueTask counterpart to ValueTask<TResult>. This mirrors the ValueTask<TResult> surface area, except that it doesnā€™t have a Result property, doesnā€™t have a ctor that takes a TResult, uses Task in places where Task<TResult> was used, etc.
```C#
namespace System.Threading.Tasks
{
[AsyncMethodBuilder(typeof(AsyncValueTaskMethodBuilder))]
public readonly partial struct ValueTask : IEquatable
{
public ValueTask(Task task);
public ValueTask(IValueTaskSource source);
public bool IsCanceled { get; }
public bool IsCompleted { get; }
public bool IsCompletedSuccessfully { get; }
public bool IsFaulted { get; }
public Task AsTask();
public ConfiguredValueTaskAwaitable ConfigureAwait(bool continueOnCapturedContext);
public override bool Equals(object obj);
public bool Equals(ValueTask other);
public ValueTaskAwaiter GetAwaiter();
public override int GetHashCode();
public static bool operator ==(ValueTask left, ValueTask right);
public static bool operator !=(ValueTask left, ValueTask right);
}
}

And finally we add the System.Runtime.CompilerServices goo that allows `ValueTask` to be awaited and used as the return type of an async method:
```C#
namespace System.Runtime.CompilerServices
{
    public struct AsyncValueTaskMethodBuilder
    {
        public static AsyncValueTaskMethodBuilder Create();

        public void Start<TStateMachine>(ref TStateMachine stateMachine) where TStateMachine : IAsyncStateMachine;
        public void SetStateMachine(IAsyncStateMachine stateMachine);
        public void AwaitOnCompleted<TAwaiter, TStateMachine>(ref TAwaiter awaiter, ref TStateMachine stateMachine) where TAwaiter : INotifyCompletion where TStateMachine : IAsyncStateMachine;
        public void AwaitUnsafeOnCompleted<TAwaiter, TStateMachine>(ref TAwaiter awaiter, ref TStateMachine stateMachine) where TAwaiter : ICriticalNotifyCompletion where TStateMachine : IAsyncStateMachine;

        public void SetResult();
        public void SetException(Exception exception);
        public ValueTask Task;
    }

    public readonly struct ValueTaskAwaiter : ICriticalNotifyCompletion
    {
        public bool IsCompleted { get; }
        public void GetResult() { }
        public void OnCompleted(Action continuation);
        public void UnsafeOnCompleted(Action continuation);
   }

    public readonly struct ConfiguredValueTaskAwaitable
    {
        public ConfiguredValueTaskAwaiter GetAwaiter();
        public readonly struct ConfiguredValueTaskAwaiter : ICriticalNotifyCompletion
        {
            public bool IsCompleted { get; }
            public void GetResult();
            public void OnCompleted(Action continuation);
            public void UnsafeOnCompleted(Action continuation);
        }
    }
}

Changes to Previously Accepted APIs

At the very least, we would use the ValueTask and ValueTask<T> types in the following previously accepted/implemented APIs that are shipping in 2.1:

  • Pipelines. Instead of pipelines having a custom PipeAwaiter<T> type, it will return ValueTask<T> from the ReadAsync and FlushAsync methods that currently return PipeAwaiter. PipeAwaiter<T> will be deleted. Pipe uses this to reuse the same pipe object over and over so that reads and flushes are allocation-free.
  • Channels. The WaitToReadAsync and WaitToWriteAsync methods will return ValueTask<bool> instead of Task<bool>. The WriteAsync method will return ValueTask instead of Task. At least some of the channel implementations, if not all, will pool and reuse objects backing these value tasks.
  • Streams. The new WriteAsync(ReadOnlyMemory<byte>, CancellationToken) overload will return ValueTask instead of Task. Socketā€™s new ReceiveAsync/SendAsync methods that are already defined to return ValueTask<int> will take advantage of this support, making sending and receiving on a socket allocation free. NetworkStream will then expose that functionality via ReadAsync/WriteAsync. FileStream will potentially also pool so as to make synchronous and asynchronous reads/writes allocation-free.
  • WebSockets. The new SendAsync(ReadOnlyMemory<byte>, ā€¦) overload will return ValueTask instead of Task. Many SendAsync calls just pass back the result from the underlying NetworkStream, so this will incur the benefits mentioned above.

There are likely to be other opportunities in the future as well. And we could re-review some of the other newly added APIs in .NET Core 2.1, e.g. TextWriter.WriteLineAsync(ReadOnlyMemory<char>, ...), to determine if we want to change those from returning Task to ValueTask. The tradeoff is one of Task's usability vs the future potential for additional optimization.

Limitations

Task is powerful, in large part due to its ā€œonce completed, never go backā€ design. As a result, a ValueTask<T> that wraps either a T or a Task<T> has similar power. A ValueTask<T> that wraps an IValueTaskSource<T> can be used only in much more limited ways:

  • The 99.9% use case: either directly await the operation (e.g. await SomethingAsync();), await it with configuration (e.g. await SomethingAsync().ConfigureAwait(false);), or get a Task out (e.g. Task t = SomethingAsync().AsTask();). Using AsTask() incurs allocation if the ValueTask/ValueTask<T> wraps something other than a Task/Task<T>.
  • Once youā€™ve either awaited the ValueTask/ValueTask<T> or called AsTask, you must never touch it again.
  • With a ValueTask<T> that wraps a Task<T>, today you can call GetAwaiter().GetResult(), and if it hasnā€™t completed yet, it will block. That is unsupported for a ValueTask<T> wrapping an IValueTaskSource<T>, and thus should be generally discouraged unless you're sure of what it's wrapping. GetResult must only be used once the operation has completed, as is guaranteed by the await pattern.
  • With a ValueTask<T> that wraps a Task<T>, you can await it an unlimited number of times, both serially and in parallel. That is unsupported for a ValueTask<T> wrapping an IValueTaskSource<T>; it can be awaited/AsTask'd once and only once.
  • With a ValueTask<T> that wraps a Task<T>, you can call any other operations in the interim and then await the ValueTask<T>. That is unsupported for a ValueTask<T> wrapping an IValueTaskSource<T>; it should be awaited/AsTaskā€™d immediately, as the underlying implementation may be used for other operation, subject to whatever the library author chose to do.
  • You can choose to explicitly call IsCompletedSuccessfully and then use Result or GetAwaiter().GetResult(), but that is the only coding pattern outside of await/AsTask thatā€™s supported.
    We will need to document that ValueTask/ValueTask<T> should only be used in these limited patterns unless you know for sure what it wraps and that the wrapped object supports what's being done. And APIs that return a ValueTask/ValueTask<T> will need to be clear on the limitations, in hopes of preserving our ability to change the backing store behind ValueTask<T> in the future, e.g. an API that we ship in 2.1 that returns ValueTask<T> around a Task<T> then in the future instead wrapping an IValueTaskSource<T>.

Finally, note that as with any solution that involves object reuse and pooling, usability/diagnostics/debuggability are impacted. If an object is used after it's already been effectively freed, strange/bad behaviors can result.

Why now?

If we donā€™t ship this in 2.1, we will be unable to do so as effectively in the future:

  • Some methods (e.g. the new Stream.WriteAsync overload) are currently defined to return Task but should be changed to return ValueTask.
  • Some methods return ValueTask<T>, but if weā€™re not explicit about the limitations of how it should be used, itā€™ll be a breaking change to modify what it backs in the future.
  • Various types (e.g. PipeAwaiter<T>) will be instant legacy.
  • Prior to .NET Core 2.1, ValueTask<T> was just OOB. Itā€™s now also in System.Private.CoreLib, with core types like Stream depending on it.

Implementation Status

With the exception of pipelines, I have these changes implemented across coreclr and corefx. I can respond to any changes from API review, clean things up, and get it submitted as PRs across coreclr and corefx. Due to the breaking changes in existing APIs, it will require some coordination across the repos.

(EDIT stephentoub 2/25: Renamed IValueTaskObject to IValueTaskSource.)
(EDIT stephentoub 2/25: Changed OnCompleted to accept object state.)

area-System.Threading

Most helpful comment

The feature has been merged.

All 117 comments

cc: @davidfowl, @geoffkizer, @MadsTorgersen, @jaredpar, @kouvel, @tarekgh, @benaadams, @pakrym, @KrzysztofCwalina

IValueTaskObject =>IValueTaskAwaitable? e.g.

public interface IValueTaskAwaitable
{
    bool IsCompleted { get; }
    bool IsCompletedSuccessfully { get; }
    void OnCompleted(Action continuation, ValueTaskAwaitableOnCompletedFlags flags);
    void GetResult();
}

public interface IValueTaskAwaitable<out TResult>
{
    bool IsCompleted { get; }
    bool IsCompletedSuccessfully { get; }
    void OnCompleted(Action continuation, ValueTaskAwaitableOnCompletedFlags flags);
    TResult GetResult();
}

[Flags]
public enum ValueTaskAwaitableOnCompletedFlags
{
    None = 0x0,
    UseSchedulingContext = 0x1,
    FlowExecutionContext = 0x2,
}

Similar to the ConfiguredValueTaskAwaitable in the proposal

How are

bool IsCanceled { get; }
bool IsFaulted { get; }
AggregateException Exception { get; }

communicated from the IValueTaskObject/IValueTaskAwaitable as they aren't on the interface?

Do they come via throwing Cancelled vs OtherException on GetResult?

Ripple effects; would go via the statemachine and SetResult()/SetException(Exception exception); but I'm wondering how an exception/cancellation in the IValueTaskObject/IValueTaskAwaitable itself is communicated?

i.e Should they throw from GetResult and the state machine catch it?

e.g.

try
    SetResult(GetResult);
catch Exception
    SetException(ex)

@stephentoub does all of this work on .NET Standard 2.0? I'm assuming yes?

Can you give some details on what the ValueTaskAwaitableOnCompletedFlags do? I assume this is at least partly related to ConfigureAwait?

With a ValueTask that wraps a Task, you can await it an unlimited number of times, both serially and in parallel. That is unsupported for a ValueTask wrapping an IValueTaskObject; it can be awaited/AsTask'd once and only once.

Will it make impossible/complicated to use AsTask and then WhenAll/WhenAny?

With a ValueTask that wraps a Task, you can call any other operations in the interim and then await the ValueTask. That is unsupported for a ValueTask wrapping an IValueTaskObject; it should be awaited/AsTaskā€™d immediately, as the underlying implementation may be used for other operation, subject to whatever the library author chose to do.

So the following will be unsupported?

C# var read = pipe.Reader.ReadAsync(); var write = pipe.Writer.WriteAsync(); await read; await write; // or var write = pipe.Writer.WriteAsync(); var flush= pipe.Writer.FlushAsync(); await write; await flush;

On one hand, I've been wondering if the new interfaces could be made more general, so that they can serve as the solution to "make Task<T> covariant" (https://github.com/dotnet/roslyn/issues/2981) and "have a generic IAwaitable" (https://github.com/dotnet/corefx/issues/15469).

On the other hand, all those limitations make using such APIs error-prone and unpleasant, so I've thought if it would make sense to have a separate type for this (e.g. ReusableValueTask/<T>). But then you can't just change the implementation to start using reusable tasks, you have to change the API.

Maybe there is a way to avoid or diminish the limitations? Some options I considered:

  1. ValueTask keeps a version of the IValueTaskObject. If you reuse IValueTaskObject, its version changes. If you then attempt to access the value from ValueTask, you get an exception. (Though this would probably increase the size of ValueTask, which is not good.)
  2. ValueTask has a way to tell IValueTaskObject that it can be reused and ValueTask itself can be Disposed. If you don't Dispose the ValueTask, the IValueTaskObject can't be reused. This will make using ValueTask in a way that reuses IValueTaskObjects harder, but maybe that's acceptable, since it's only for users that need the "last pound of performance"?
  3. Provide a Roslyn analyzer that ensures you're not using ValueTask incorrectly.

IValueTaskObject =>IValueTaskAwaitable

But it's not awaitable (by design). From a technical perspective, for it to be awaitable, it would need to expose slightly different surface area, and it would need to implement ICriticalNotifyCompletion, which then makes it difficult or impossible to have the same object implement both IValueTaskObject<X> and IValueTaskObject<Y>, which for example pipelines wants/needs to do. From a purity perspective, awaitables are things that expose GetAwaiter, and awaiters provide the IsCompleted/OnCompleted/GetResult surface area. And from an ideological/perf perspective, I really don't want APIs returning an instance of this interface directly. The vast number of calls to the vast number of async APIs actually do complete synchronously, and for such operations it would be a terrible shame if accessing the result incurred a minimum of three interface calls: GetAwaiter(), IsCompleted, GetResult().

How are IsCanceled / Faulted / Exception communicated from the IValueTaskObject/IValueTaskAwaitable as they aren't on the interface?

Exception isn't on ValueTask<T>, it's on Task<T> (or Task rather), so that's not an issue.

I've implemented IsFaulted as IsCompleted && !IsCompletedSuccessfully.

I've made IsCanceled always return false for an IValueTaskObject/IValueTaskObject<T>. That's not how I started this effort, though. I started with there being an IsCanceled on the interface. The problem though is one of performance. It became clear very quickly as I implemented this in various places that calling GetResult is the signal that the object is no longer in use and can be reused, and that means the holder of the ValueTask/ValueTask<T> _must_ not touch the ValueTask again after GetResult has been called. If we need to then interpret an exception that emerges from GetResult as being for either cancellation or failure, as we do for example in ValueTask.AsTask, then we can't call IsCanceled after calling GetResult, which means we need to call it before, which means we need to call it even for a synchronously completed operation, which introduces another interface call on that faster path. Instead, I implemented the same semantics in AsTask that async methods have: exc is OperationCanceledException means the Task is canceled, everything else means faulted.

To me, the design I have seemed like the lesser of two evils. It does lead to an inconsistency, in that with code like:
```C#
ValueTask vt = SomeAsync();
bool faulted = vt.IsFaulted;
Task t = vt.AsTask();
bool canceled = t.IsCanceled;

could result in both `faulted` and `canceled` being true.  But for the 90% use case of:
```C#
int i = await SomeAsync();

whether the thing returned from SomeAsync is canceled or faulted is indistinguishable, and for the 9% use case of:
```C#
Task t = SomeAsync().AsTask();

you'll never look at the `ValueTask` and so it won't matter that its view of `IsFaulted` differs slightly from that of `Task`, and for the 0.9% optimization use case of:
```C#
ValueTask<int> vt = SomeAsync();
int i = vt.IsCompletedSuccessfully ? vt.Result : await vt;

you're not looking at IsFaulted or IsCanceled, so it also doesn't matter. It's only for that 0.1% use case (obviously I'm making up these numbers, but they seem accurate :smile:) that you'd see a potential discrepancy. And that didn't seem worth the extra perf hit to me.

(Honestly, I wish we hadn't added IsCanceled to Task or ValueTask... the only reason it ended up there is effectively a legacy reason. Because in .NET 4 we crashed the process if a failed Task had its exception go unobserved, we wanted to special-case cancellation so you didn't have to observe it, but then post-.NET 4 we stopped crashing by default for that anyway, and with async methods, they really end up recombining them into just success or exception.)

Anyway, that's how I ended up here. Do you disagree with the approach or see a flaw in my reasoning?

does all of this work on .NET Standard 2.0? I'm assuming yes?

Yes. Prior to .NET Core 2.1, ValueTask<T> was in the System.Threading.Tasks.Extensions.dll NuGet package, which has a netstandard1.0 build. In .NET Core 2.1, these types also move into System.Private.CoreLib and it's those types that are used for netcoreapp, but System.Threading.Tasks.Extensions retains its netstandard1.0 build. For this proposal, I've added the new types into both System.Private.CoreLib for netcoreapp and System.Threading.Tasks.Extensions for netstandard1.0, so the functionality will be available everywhere. That said, there are optimizations in the netcoreapp implementation not possible in the netstandard1.0 implementation; for example, AsTask is less efficient in the netstandard1.0 implementation.

Can you give some details on what the ValueTaskAwaitableOnCompletedFlags do? I assume this is at least partly related to ConfigureAwait?

Awaiters can implement the INotifyCompletion interface, which provides OnCompleted, or the ICriticalNotifyCompletion interface, which inherits INotifyCompletion and then also provides UnsafeOnCompleted. The only difference between the two is whether they flow ExecutionContext or not, e.g. if you put something into an AsyncLocal<T> before calling OnCompleted, that value will be available in the continuation callback, where if you do so before calling UnsafeOnCompleted, it may not be there because the implementation need not flow the ExecutionContext that contains that AsyncLocal<T> data. I've collapsed that distinction into the FlowExecutionContext flag, so if OnCompleted gets (flags & FlowExecutionContext) != 0, it must flow the context (e.g. use ExecutionContext.Capture() in OnCompleted and then use ExecutionContext.Run to invoke the callback), and if it gets (flags & FlowExecutionContext) == 0, it need not. The 99% case is that the flag won't be set, because ValueTask<T>'s awaiters implement ICriticalNotifyCompletion, and the compiler-generated state machine will call to the async method builder's AwaitUnsafeOnCompleted method if it does. The runtime always flows ExecutionContext across awaits (unless ExecutionContext.SuppressFlow() is called), so the only reason this exists on the awaiter as well is because it's public API surface area that can be called directly rather than only being usable via await. It's effectively legacy that both APIs are exposed rather than being collapsed as I've done, due to a now defunct security model.

UseSchedulingContext is just the bool continueOnCapturedContext value that's passed to ValueTask<T>.ConfigureAwait(bool); it's set if continueOnCapturedContext is true (or if ConfigureAwait wasn't used), and not set if it's false. If it's set, the implementation needs to query for the current scheduling context, and if it exists, use it when executing the callback. Generally this means at least looking at SynchronizationContext.Current, but Task also looks at TaskScheduler.Current, so those are the semantics I've implemented. Another implementation, like in pipelines, might choose to also factor in its pipelines scheduler. So, for example, when OnCompleted is called the implementation will check SynchronizationContext.Current, and if it's non-null/non-default, it'll hold on to it, and then when it's time to invoke the continuation, it'll use that context's Post method to queue the callback back to that SynchronizationContext. This is how work that runs on the UI thread, for example, gets back to the UI thread.

Will it make impossible/complicated to use AsTask and then WhenAll/WhenAny?

No. It'd be perfectly fine to do:
```C#
await Task.WhenAll(
SomeAsync().AsTask(),
SomeAsync().AsTask(),
SomeAsync().AsTask());

> So the following will be unsupported?

It really depends on the implementation of `IValueTaskObject`/`IValueTaskObject<T>`, and APIs that return `ValueTask`/`ValueTask<T>` will need to be clear on what's allowed and what's not.

For example, today it's perfectly acceptable to have multiple operations outstanding on a `Socket` at time, both sends and receives.  But while it's allowed, it's quite rare to have multiple reads outstanding or multiple writes outstanding, while it's very common to have a single read outstanding with a single write outstanding.  Thus the implementation I've provided of `ReceiveAsync` and `SendAsync` that return `ValueTask<int>` allow for any number of outstanding operations, but only optimize for the single outstanding read / single outstanding write case.  The implementation maintains a single reusable `IValueTaskObject<int>` for receives and a single reusable `IValueTaskObject<int>` for sends, and the implementation tracks whether `GetResult` has been called.  When a new receive/send operation is issued, the implementation tries to reserve the corresponding singleton, which it can only do if no one else is currently using it; if no one else is using it, then we reuse that singleton for that operation, but if someone else is using it, you get a slower implementation that does allocate.  Thus, with code like:
```C#
int bytesReceived = await socket.ReceiveAsync(memory, cancellationToken);
bytesReceived += await socket.ReceiveAsync(memory, cancellationToken);
bytesReceived += await socket.ReceiveAsync(memory, cancellationToken);

each of those operations will take the fast, non-allocating path, but with code like:
```C#
ValueTask vt1 = socket.ReceiveAsync(memory1, cancellationToken);
ValueTask vt2 = socket.ReceiveAsync(memory2, cancellationToken);
ValueTask vt3 = socket.ReceiveAsync(memory3, cancellationToken);
int bytesReceived = await vt1;
bytesReceived += await vt2;
bytesReceived += await vt3;

the `vt1` operation will be non-allocating but the `vt2` and `vt3` operations both will be.  What's explicitly not supported and will very much be a bug on the developer's part is if they touch one of these `ValueTask<int>` instances after it's already been awaited, e.g. this code is bad:
```C#
ValueTask<int> vt1 = socket.ReceiveAsync(memory1, cancellationToken);
await vt1;
await vt1; // BUG BUG BUG

In contrast, for example, System.Threading.Channels has a single-consumer specialized unbounded channel, e.g.
```C#
Channel c = Channel.CreateUnbounded(new UnboundedChannelOptions { SingleReader = true });

That implementation explicitly only supports a single reader at a time (with any number of writers), and thus it caches a singleton `IValueTaskObject<T>` that's reused for every `ReadAsync` on the channel.  It's thus fine to do:
```C#
T item1 = await c.Reader.ReadAsync();
T item2 = await c.Reader.ReadAsync();

and fine to do:
```C#
ValueTask vt = c.Reader.ReadAsync();
await c.Writer.WriteAsync(producedItem);
T consumedItem = await vt;

but it's very much an error on the developer's part to do:
```C#
ValueTask<T> vt1 = c.Reader.ReadAsync();
ValueTask<T> vt2 = c.Reader.ReadAsync(); // BUG BUG BUG

and an error to do:
```C#
ValueTask vt = c.Reader.ReadAsync();
await vt;
await vt; // BUG BUG BUG

and an error to do:
```C#
ValueTask<T> vt = c.Reader.ReadAsync();
await c.Reader.ReadAsync(); // BUG BUG BUG
await vt; // BUG BUG BUG

In other words, it's really up to the API returning the ValueTask or ValueTask<T> what semantics it wants to provide.

so that they can serve as the solution to "make Task covariant" (dotnet/roslyn#2981) and "have a generic IAwaitable" (#15469).

I explicitly opted away from that, for the reasons outlined earlier in this response.

ValueTask keeps a version of the IValueTaskObject. If you reuse IValueTaskObject, its version changes. If you then attempt to access the value from ValueTask, you get an exception. (Though this would probably increase the size of ValueTask, which is not good.)

I considered the cookie approach. Basically ValueTask/ValueTask<int> would store an int or a long version number, populated into the struct when constructed with an IValueTaskObject, e.g.
```C#
public struct ValueTask
{
public ValueTask(IValueTaskObject obj, long version);
...
}

and then all of `IValueTaskObject<T>`'s members would also take a version value:
```C#
public interface IValueTaskObject<out T>
{
    public bool IsCompleted(long version);
    public bool IsCompletedSuccessfully(long version);
    public T GetResult(long version);
    public void OnCompleted(Action continuation, ValueTaskObjectOnCompletedFlags flags, long version);
}

and it would be up to the IValueTaskObject<T> implementation to store a corresponding version and fail the operation if the two cookies didn't match. That's feasible. But as you say, it increases the size of ValueTask and ValueTask<T>, purely for a diagnostic benefit in the case of misuse. Generally I prefer to avoid such overheads, especially since these types end up not only getting returned out of methods, but often stored into the state machines of the async methods consuming them, thus increasing the size of those state machines.

I know @KrzysztofCwalina was a fan of this approach, though, at least in principle.

ValueTask has a way to tell IValueTaskObject that it can be reused and ValueTask itself can be Disposed

I'm not understanding this... wouldn't it be the other way around, the IValueTaskObject<T> telling the ValueTask<T> whether the IValueTaskObject<T> can be reused? (I don't know what it means for a ValueTask<T> to know whether it itself is reusable or not. Regardless, I don't think that really addresses the issue. The 99% case here is:
```C#
await SomeAsync();

and `await` doesn't `Dispose` of the thing it's given.  Rather, it calls `GetResult` as the last step, so effectively `GetResult` is the signal that the object can then be reused.  If a `Dispose` were required, that would effectively nullify any benefits here, as to enable reuse you'd need to change all such awaits to instead be:
```C#
using (ValueTask<int> vt = SomeAsync())
{
    await vt;
}

and not only is that klunky and more expensive, I actually would expect it would lead to more errors, as it promotes storing the ValueTask<T> into a local, making it more likely you'll accidentally reuse it.

Provide a Roslyn analyzer that ensures you're not using ValueTask incorrectly.

That might be reasonable, though I expect it would likely have both false negatives and false positives. Happy to be proven wrong, though.

I'm not understanding this... wouldn't it be the other way around, the IValueTaskObject<T> telling the ValueTask<T> whether the IValueTaskObject<T> can be reused?

What I meant is that the ValueTask<T> would tell IValueTaskObject<T> whether the IValueTaskObject<T> can be reused. The code would effectively be something like:

```c#
struct ValueTask
{
public void Dispose() => valueTaskObject?.Release();
}

> The 99% case here is: `await SomeAsync();` [ā€¦] If a Dispose were required, that would effectively nullify any benefits here

The proposed design means that the 99% case is efficient, but also makes it easy to write buggy code.

I don't like to sacrifice safety for performance, because performance often doesn't matter much, while safety always matters. And with this proposal, any API that returns `ValueTask<T>` becomes dangerous.

What if it wasn't `Dispose()`, but instead a method on `ValueTask<T>` that returns another awaitable?

```c#
await SomeAsync(); // allocates

var vt1 = SomeAsync();
await vt1;
await vt1; // ok

await SomeAsync().IKnowICantReuseTheReturnedValueISwear(); // does not allocate

var vt2 = SomeAsync().IKnowICantReuseTheReturnedValueISwear();
await vt2;
await vt2; // bug

(With IKnowICantReuseTheReturnedValueISwear obviously having a different name.)

This makes it much easier to use than Dispose(), but only becomes dangerous if you explicitly opt-in.


I guess the important question here is: will almost all code that uses ValueTask<T> be performance critical (to the point that allocations matter)? If ValueTask<T> will always be only part of APIs that are specifically designed for performance, then it's okay if it's a bit dangerous.

But if ValueTask<T> will become part of general-purpose APIs (that won't have Task<T>-returning alternatives), then I think it's important to make sure it's not dangerous.

Anyway, that's how I ended up here. Do you disagree with the approach or see a flaw in my reasoning?

No; just checking :)

ValueTask<T> vt = c.Reader.ReadAsync();
await vt;
await vt; // BUG BUG BUG

Could ValueTask both clear its IValueTaskObject on return from first await (so second await failed) and throw if a second action is queued to OnCompleted if an action is already present?

It wouldn't cater for struct copies; but would that reduce the 0.1% to 0.001%? šŸ˜‰

Could ValueTask both clear its IValueTaskObject on return from first await (so second await failed) and throw if a second action is queued to OnCompleted if an action is already present?

It's a readonly struct, and even if it weren't, by definition there's a copy when getting the awaiter from it, so each time you await it you're seeing a different copy of the struct. An implementation of IValueTaskObject could try to put some safeguards in place, but what it can detect would be limited by how much reuse it wants to allow.

ValueTask would tell IValueTaskObject whether the IValueTaskObject can be reused

It already does that: if GetResult hasn't been called, the implementation must be considered still in use, and an implementation could choose to throw or take a slower path or some such thing.

The proposed design means that the 99% case is efficient, but also makes it easy to write buggy code.

I disagree. Look at all of the code that uses tasks that's been written in the last few years; 99% of it just awaits the operation directly... it's fairly rare to get a handle to the task and do something other than await it. Sometimes you use operations with combinators, but notice combinators aren't exposed here for ValueTask and ValueTask<T>... you need to use AsTask to then use Task.WhenAll/Any, etc. Storing the operation into a local and doing something other than awaiting it is generally only done when you're trying to do something special, often for performance reasons, and then you already need to know what you're doing.

And with this proposal, any API that returns ValueTask becomes dangerous.

Many APIs in .NET (and any programming language for that matter) can be misused in a dangerous way. Access a Dictionary<TKey,TValue> from multiple concurrent operations (which is really easy to accidentally do, if for example you put one in a static and access it from multiple web requests) and you can corrupt it easily, in ways that can lead to corrupted data, infinite loops, and other messes. Issue multiple ReadAsync calls on an arbitrary Stream implementation without waiting for the previous one to complete, and you'll likely corrupt data (a few streams allow this, but most don't, and many don't implement any safeties). Using an object after it's been Disposed; some objects detect and throw for this, some don't. Using ArrayPool and accidentally returning the same array multiple times, or forgetting to return the arrays, or continuing to use an array after it's already been returned. Etc.

because performance often doesn't matter much

If performance doesn't matter for a method, it can simply return a Task or Task<T>. Performance with operations like Stream.Read/WriteAsync often really does matter. Notice I've explicitly _not_ said that all methods should return ValueTask and ValueTask<T> moving forward, as I don't believe they should.

This makes it much easier to use than Dispose(), but only becomes dangerous if you explicitly opt-in.

I don't see how that's a feasible design. By the time SomeAsync returns to the caller, it's already scheduled the asynchronous operation, and already needs to know what object it's talking to upon completion.

Doesn't this create the situation in which the consumer of your API now needs to understand how to properly await based on the kind of ValueTask you're returning? I'm seeing this have huge benefits for code that, say, deserializes data off the network, but if I return a ValueTask that wraps an IValueTaskObject, how do I indicate to my consumers to avoid the various pitfalls, like double awaiting?

With custom awaitables like PipeAwaiter, it just isn't as easy for API consumers to do the wrong thing. Definitely not as easy to compose, I agree, and writing custom awaiters has its own share of pitfalls, but again, those aren't exposed to the consumer.

Doesn't this create the situation in which the consumer of your API now needs to understand how to properly await based on the kind of ValueTask you're returning?

A consumer of an API needs to know the semantics of that API, including details about how its return type behaves. By default, you need to assume if you get a ValueTask/ValueTask<T> from an arbitrary method that you know nothing else about that you can either await it once or use AsTask once, before doing other arbitrary things with that same API. If you know more about the semantics of the method from which you got it, you may be able to get away with more.

With custom awaitables like PipeAwaiter, it just isn't as easy for API consumers to do the wrong thing.

Why not?
C# var result1 = pipeReader.ReadAsync(); await result1; var result2 = pipeReader.ReadAsync(); await result1;
How does misuse difficulty change here based on the concrete type of the var?

This is a really great work @stephentoub !
I recently had a performance bottleneck on an application that this would allow to solve in a much cleaner way.

With the increase in focus on performance in the .NET ecosystem I can see this used not only in core pieces of the stack or performance critical business applications, but also to a wider set of library and applications.
However the drawbacks and risk of run-time errors due to misuses will either discourage the use of this construct or risk affecting the stability/correctness lot of applications and, although it will be a developer error, I believe this can be bad for reputation.

I believe the semantic of ValueTask/ValueTask<T> should be defined with certainty (await once or AsTask once) and enforced by the compiler. Maybe also the fact that only one operation should be in-flight should be a common convention and lead to a warning otherwise.

Have you also considered not making it compatible with Task (given also some other differences like the IsCancelled behaviour) and make this a first class concept, eventually with is own keywords (e.g. asynconce/awaitonce)?

This is a really great work

Thanks.

I believe the semantic of ValueTask/ValueTask should be defined with certainty (await once or AsTask once)

Essentially that's what I'm saying. If you happen to know more about the implementation, you might be able to get away with more.

Have you also considered not making it compatible with Task

ValueTask<T> already shipped. We can't change that (nor would I want to).

In this particular case, because PipeAwaiter has a specific set of usage semantics. It isn't a Task or ValueTask, so that gives me heads up that I need to understand what it supports.

I'm not at all against the idea, it's just that my consumer has to understand how I'm using ValueTask internally and then make decisions based on that. It isn't intuitive that there are situations which are totally fine with a normally wrapped ValueTask but not with one that wraps IValueTaskObject. If the usages are distinct, it feels like the types should also be distinct. AsyncValueTask, if you will.

Now, that may be non-trivial for a myriad of other reasons.

In this particular case, because PipeAwaiter has a specific set of usage semantics. It isn't a Task or ValueTask, so that gives me heads up that I need to understand what it supports.

I'd argue this isn't about PipeAwaiter, it's about the API that returned it, and that's the case regardless of the return type. Further, I'd argue that the 99% case (and it's really probably more the 99.99% case) is no one knows or cares what the return type is because they simply await it. It's just compiler goo to let you write await SomethingAsync(), just like you generally don't care whether an enumerable type exposes a struct enumerable and instead just foreach (var item in enumerable)... that you might be using a nested struct type isn't relevant, other than for its perf implications.

BTW Stephen, despite my hesitations (I write a lot of low level stuff that junior devs need to be able to easily consume for our system), this is a really great idea and awesome stuff which could solve a lot of the "data coming off the wire" async allocation scenarios.

Have you also considered not making it compatible with Task

ValueTask already shipped. We can't change that (nor would I want to).

Sorry, I meant this new feature.
So not making it an extension of ValueTask<T> but a new concept (as @mattnischan is suggesting) and even more obvious with different keywords (which eventually behind the scene can do something similar to Dispose as @svick was suggesting)

So not making it an extension of ValueTask but a new concept (as @mattnischan is suggesting)

Whatever this thing is, it's going to need to wrap Tasks, as the vast majority of these async APIs are going to still return tasks under the covers. And it's going to need to be as efficient as possible with Task, which means it shouldn't go through an interface to get to it, plus the fact that there'd be no way to make it work with netstandard2.0 if Task needed to implement another interface to make that work.

At that point, you're just introducing another type with the exact same support as ValueTask<T> and just calling it something else. ValueTask<T> has only ever been released in an out-of-band NuGet package, and use is limited. I think it's perfectly reasonable to say now that you need to assume this conservative view of what you can do with a ValueTask<T>.

Further, even with something that was given a different name, there are still going to be differences between APIs that return one, as it's up to the implementation what level of reuse is possible. As I noted earlier, Socket for example lets you make any number of calls to receive/send data, and doing so won't invalidate a previously returned ValueTask<T>, but doing so with PipeReader will. NetworkStream's override of ReadAsync supports one thing, but DeflateStream's override of ReadAsync supports another (and that's true today even with them both returning the same type), and they necessarily need to return the same type. You simply need to understand the semantics of the method you're using; that's always been true and will continue to be true; the return type is largely irrelevant.

I do not see how shipping a different type addresses safety concerns here. It might address the small inconsistency around IsCanceled, but I don't believe that's an important issue necessitating introducing another type.

If we introduced a differently named shared type for this, ValueTask<T> will become completely defunct, with no reason to ever use it, and all of the same issues will exist with the newly named type.

I'm happy with it; your initial warnings were more scary than how it would match to use cases.

i.e. normally you await an operation prior to initializing the same operation, and with Task.When requiring the .AsTask() cast it should cover most scenarios (with docs).

IValueTaskObject sounds overly generic though; mainly due to Object which doesn't suggest the methods. IValueTaskResultSource or similar would be better?

Bikeshedding on names aside, thank you for taking time to explain šŸ˜„

Thanks @stephentoub for the comprehensive explanation.
I personally would prefer if the semantic was on the safe side, discouraging reuse so that it could be eventually enforced by the compiler in future (as you explained having different types for the scenarios will not help, unless you have completely different APIs/keywords, but as you said then it can't be delivered to the current ecosystem)
I agree that you should understand the semantic of the operation you invoke, but its generally better if the semantic can be made explicit in the contract, similar to the evolution of the language to have nullable reference types.
I'm probably just worrying too much, sorry for polluting this thread.

IValueTaskResultSource or similar would be better

I like IValueTaskSource.

your initial warnings were more scary than how it would match to use cases.

Yeah :smile: I just wanted to be upfront about potential concerns. If I was actually scared of it, we wouldn't be having this conversation as I wouldn't have opened the issue. :wink:

Thanks @stephentoub for the comprehensive explanation

You're very welcome. Thank you for participating!

eventually enforced by the compiler

What does that look like? How does an awaitonce keyword prevent:
C# var t = SomeAsync(); var s = t; awaitonce t; awaitonce s;
?

Does it also make sense to provide a way to create ValueTask() from Exception instance to have allocation-free equivalent for Task.FromException(...). This might be useful for cases where operation should fail synchronously (so no IValueTaskObject instance allocation or acquire from reusable pool is required) but for some reason it's inconvenient/impossible to just throw and force all calling parties to wrap the call with try/catch. Basically same reason to use as for Task.FromException.

@stephentoub Nice! You the man!!! On the high performance side, this will open up all sorts of opportunities.

Does it also make sense to provide a way to create ValueTask() from Exception instance to have allocation-free equivalent for Task.FromException(...)

The only way I know of to do that would be to make ValueTask bigger, which would incur cost for all uses. I don't think that's a desirable trade-off. Exceptions are already very expensive.

Will there be an equivalent of TaskCompletionSource? Will it be resettable/reusable? What would this look like?

Is it too late to simply remove IsCancelled from ValueTask?

Could we add an overload of OnComplete with a state argument, to avoid delegate allocation? Seems like the compiler could be modified to use this in the future.

Could the compiler simply disallow local vars of type ValueTask? At least in async methods? (Maybe it's a stack-only type, like Span?) This wouldn't disallow any advanced usage scenarios, it would just make them more explicit. That is:

    await SomeAsync();                                      // 99% case
    Task t = SomeAsync().AsTask();                          // 0.9% case; allows deferred await, combinators
    ValueTaskAwaiter awaiter = SomeAsync().GetAwaiter();    // 0.1% case; if you call GetAwaiter,
                                                            // you better know what you're doing

    ValueTask vt = SomeAsync();                             // compiler error (in async method)

I realize that's a big change, but if it helps prevent user error/confusion, perhaps it's worth it?

Will there be an equivalent of TaskCompletionSource? Will it be resettable/reusable? What would this look like?

It would look like an implementation of IValueTaskObject<T> and IValueTaskObject (likely a class, but potentially a struct if we wanted to really live on the edge). I've currently got two different ones in my changes. I think we could/should ship a public one at some point, but I don't think that's going to be in 2.1. It's a relatively complicated thing to design in a way that's not specific to the particular usage.

Is it too late to simply remove IsCancelled from ValueTask?

It would be a source- and runtime-breaking change for anyone currently using it. So, yes.

Could we add an overload of OnComplete with a state argument, to avoid delegate allocation?

I've gone back and forth on that. Maybe. The reason I've wanted to do it is that right now AsTask for an asynchronously completing operation is two allocations, one for the task and one for an Action delegate; if we had a state argument, it could be done with just one allocation for the Task. However, if it's instead of the existing overload, it would mean that all asynchronously completing operations with await would incur two delegate invocations rather than just one delegate invocation, as we'd essentially do OnCompleted(s => ((Action)s)(), continuation, flags) rather than just OnCompleted(continuation, flags). And if it's in addition to the Action-based overload, the implementation has to be bigger to store more fields (at least the object state, if not also a separate field for the Action<object>) and potentially type checks if it uses a single object/Delegate field to store either the Action/Action<object>.

I realize that's a big change

It would certainly be a breaking change.

This wouldn't disallow any advanced usage scenarios

It would, e.g.
https://github.com/dotnet/corefx/blob/58af9adc8c6c70df4b7fd158c984508d96baca59/src/System.Net.Http/src/System/Net/Http/SocketsHttpHandler/ContentLengthReadStream.cs#L41-L62

Maybe it's a stack-only type, like Span?

C# compiler wouldn't let you use a stack-only type in an async method; so you couldn't await it

C# compiler wouldn't let you use a stack-only type in an async method; so you couldn't await it

You can use Span in an async method, you just can't store it in a local var. E.g. this works:

        static async Task TestAsync()
        {
            byte[] b = new byte[1024];
            byte[] b2 = new Span<byte>(b).Slice(0, 256).ToArray();
            // go do async stuff
        }

(at least that's my understanding)

Whoa...

Whoa...

There's even this proposal: https://github.com/dotnet/csharplang/issues/1331

It would, e.g.

Yeah, it does lose one thing, which is the ability to defer the await without converting to a Task. I can imagine ways to enable that.

That said, I sense your main objection is that you want to keep compatibility with the existing ValueTask contract. Is that right?

I do think it's worth at least briefly considering exposing this as a new type instead of an extension of ValueTask. Let's call it AsyncTask, for lack of a better term. At the very least, calling this something new forces users to understand that the semantics are not exactly like the ValueTask we've shipped previously. Moreover, there may be some implementation advantages that arise from the freedom of a new type. (Admittedly I don't have a great sense of what these would be, just some vague notions.)

I realize you've already gone to heroic lengths to get this where it is, and I'm just proposing even more work and more change :). But as you said above, this is our one chance to do this right for the foreseeable future, so it seems worth at least having the discussion.

However, if it's instead of the existing overload, it would mean that all asynchronously completing operations with await would incur two delegate invocations rather than just one delegate invocation, as we'd essentially do OnCompleted(s => ((Action)s)(), continuation, flags) rather than just OnCompleted(continuation, flags).

Ideally we'd change the compiler too. If not right away, then when possible. There's a fair amount of overhead in async completion anyway, so maybe the extra delegate invocation doesn't matter -- especially since it could go away eventually.

I do think it's worth at least briefly considering

Of course.

At the very least, calling this something new forces users to understand that the semantics are not exactly like the ValueTask we've shipped previously

ValueTask doesn't have a lot of usage yet, nor is there widespread knowledge about its semantics. So I'm not concerned about that. At the same time, I don't think we can remove exposed APIs like IsCanceled. Those are two very different things in my mind.

there may be some implementation advantages that arise from the freedom of a new type

Like? We're not going to be able to rely on additional compiler features now beyond those already available. And for all intents and purposes I see ValueTask as a new type; we're just very slightly encumbered by a few properties that we can wave our hands at because they don't really matter.

I'm just proposing even more work and more change :)

Bring it ;)

Ideally we'd change the compiler too.

That would bring other problems. For example, Task today has a single _continuations object field. And the super common case is you await a Task, once, with the Task being given an Action delegate, and there's no additional context it needs to store, so it just stores that Action directly into the _continuations field. Only if there are multiple continuations or additional context that needs to be flowed does it have to allocate anything to store that continuation. But if we switch to a model where it's given an Action<object> and a state object, now it needs to store two things, which means it's always going to allocate.

I think most of the concerns are "solved" in my mind with the AsTask support. I also think the benefits outweigh the downsides. The usage patterns that cause problems just aren't that common that it would cause an issue IMO.

ValueTask doesn't have a lot of usage yet, nor is there widespread knowledge about its semantics.

In the circles for which it's usage was designed, I'd argue it's knowledge is fairly widespread.

However, contradicting myself a bit, this may just be a case of user education about the pitfalls if the types aren't separated. After all Task itself has some API which is dangerous as well to use in the wrong circumstances, such as Result and Wait().

I think having different semantics doesn't bug me so much as having different semantics based on only source or runtime based internal usage observation.

there may be some implementation advantages that arise from the freedom of a new type

Like?

The biggest thing that comes to mind is whether there are complications that arise from being able to construct a ValueType from either Task or IValueTypeObject. If you only had to support the latter, it might make things more efficient. That's just a hypothetical since I don't know how your code is handling this.

That would bring other problems. For example, Task today has a single _continuations field.

Yeah, I'm sure it's more complicated than I'm suggesting. But if we ever want to avoid the delegate allocation, we'll need to do something. Do we? And if we don't do it now, can we do it in the future? I'm fine postponing this for later, I'm mostly concerned that we don't close a door here.

The biggest thing that comes to mind is whether there are complications that arise from being able to construct a ValueType from either Task or IValueTypeObject.

As I noted earlier https://github.com/dotnet/corefx/issues/27445#issuecomment-368317693:
"Whatever this thing is, it's going to need to wrap Tasks, as the vast majority of these async APIs are going to still return tasks under the covers. And it's going to need to be as efficient as possible with Task, which means it shouldn't go through an interface to get to it, plus the fact that there'd be no way to make it work with netstandard2.0 if Task needed to implement another interface to make that work."

Yeah, I'm sure it's more complicated than I'm suggesting. But if we ever want to avoid the delegate allocation, we'll need to do something.

I wasn't arguing against that; I already said we might want to support Action<object>+object. I was responding specifically to the comment about wanting to change the pattern the compiler targets; while using Action can result in an allocation per async method that yields, there can be much larger costs per await that yields if a multi-object pattern is employed. That's all I was commenting on.

I've made a few tweaks to the original proposal and implementation:

  • Changed "ValueTaskObject" to "ValueTaskSource"
  • Changed OnCompleted to take Action<object>+object instead of just Action

Changed OnCompleted to take Action\

This is going to force allocation on a the implementor of IValueTaskSource no? Do we really, really need it?

This is going to force allocation on a the implementor of IValueTaskSource no? Do we really, really need it?

In common code paths, with no way around it?

By making this change, I was able to get two primary wins:

  • Async methods that await any ValueTask no longer need to allocate an Action. I previously enabled this for async methods that awaited Task and ValueTask only, but I initially had to back that out for ValueTask; this change let me put it back.
  • AsTask no longer allocates a delegate.

In my IValueTaskSource implementations, on hot paths I was also able to avoid extra allocations; the object on which the delegate was being stored now also stores the object.

The problem specifically with pipelines that we'll hit is the mismatched signatures in the PipeScheduler implementation:

https://github.com/dotnet/corefx/blob/ac46f5646ea97e3d337d68ebc3ac39652d4929a5/src/System.IO.Pipelines/src/System/IO/Pipelines/ThreadPoolScheduler.NetCoreApp21.cs#L21

We need to allocate the WaitCallback internally when somebody schedules via the Action<object>, object overload.

So how about changing that line from:
```C#
System.Threading.ThreadPool.QueueUserWorkItem(_actionObjectWaitCallback, new ActionObjectAsWaitCallback(action, state), preferLocal: true);

to:
```C#
Task.Factory.StartNew(action, state, CancellationToken.None, TaskCreationOptions.DenyChildAttach, TaskScheduler.Default);

?

Task from the Factory is way bigger than the two field ActionObjectAsWaitCallback; and its now taking a slower queuing route?

WaitCallback is kinda Action<object>, anyway to may them more equivalent as far as the ThreadPool is concerned?

Task from the Factory is way bigger than the two field ActionObjectAsWaitCallback; and its now taking a slower queuing route?

There are two allocations in the original: ActionObjectAsWaitCallback and QueueUserWorkItemCallback.

Separately, @kouvel, @benaadams, maybe for 2.1, since it's a brand new API, we should change the signature of this new preferLocal overload to use Action<object> instead of WaitCallback? I realize WaitCallback is the delegate type used in the rest of the ThreadPool methods, but that's because it was from before generics and before Action<object> became the lingua franca for this sort of thing.

Or maybe we should change that new overload to use Action<T> and T... that's even better for this but other things where you'd otherwise need to box a value type argument.

Does that address your concerns, @davidfowl?

Yep! That works. The other concern may not be a concern but I'm interested to see what the implementation an implementation looks like. I guess it'll just be one extra field to store other than the delegate itself.

I guess it'll just be one extra field to store other than the delegate itself.

Right. I initially was shy about the extra field, but it's outweighed by the benefits, in particular now that I found a way to remove the Action allocation from async methods that await any Task or ValueTask. That plus avoiding a delegate allocation in AsTask, and I was swayed.

Whatever this thing is, it's going to need to wrap Tasks, as the vast majority of these async APIs are going to still return tasks under the covers. And it's going to need to be as efficient as possible with Task, which means it shouldn't go through an interface to get to it, plus the fact that there'd be no way to make it work with netstandard2.0 if Task needed to implement another interface to make that work.

I agree most APIs will still return Task; but I'm less sure that just wrapping a Task in a ValueTask is really all that common.

Most Task-returning APIs will be consumed using await, which will work fine regardless.

In a brief perusal of places in corefx where we do "new ValueTask" today, it seems like most usages (maybe 80%?) are either:

(1) Construct ValueTask from a value
(2) Construct ValueTask from Task.FromCanceled or Task.FromException. Since these are exception cases, I don't know that we care about efficiency here that much (and we could optimize to avoid wrapping Task in the future, if we do care).
(3) Construct ValueTask from TaskCompletionSource.Task. Many of these would probably be better served by using IValueTaskObject; of course that may not be reasonable to do for 2.1.

The cases where we wrap a ValueTask around a Task from an async method (i.e. not (2) or (3) above) actually seem pretty rare, and even some of these seem to be implementation details rather than strictly necessary.

That said, you're more familiar with the actual usage than I am...

I'm less sure that just wrapping a Task in a ValueTask is really all that common.

Every time we introduce a ValueTask-returning virtual method where there's already a Task-returning method, the ValueTask-returning one wraps the Task-returning one. That means every existing Stream Read/WriteAsync method we don't own will be in this camp from day 1.

And every async method that returns a ValueTask will be wrapping a Task (which isn't just a "normal" task but is actually a derived type that carries with it all of the builder / async state machine logic). That's an implementation detail, yes, but that's unlikely to change any time soon.

And there are tons of Task-returning methods today. New APIs will be introduced that will have fast-paths that return values and otherwise delegate to these Task-returning methods.

It's going to be super, super common.

Every time we introduce a ValueTask-returning virtual method where there's already a Task-returning method, the ValueTask-returning one wraps the Task-returning one. That means every existing Stream Read/WriteAsync method we don't own will be in this camp from day 1.

Yeah, that's a very good point. OK.

The only way I know of to do that would be to make ValueTask bigger, which would incur cost for all uses. I don't think that's a desirable trade-off. Exceptions are already very expensive.

@stephentoub From you pr for clr it seems like you found a way not to make it bigger by storing both task and valuetasksource in the same field. Why canā€™t you store Exception there as well? Even though thereā€™s no gain for efficiency, donā€™t you think it would be much better syntactically:
new ValueTask(ex) or ValueTask.FromException(ex) vs new ValueTask(Task.FromException(ex))

Why canā€™t you store Exception there as well? Even though thereā€™s no gain for efficiency, donā€™t you think it would be much better syntactically:
new ValueTask(ex) or ValueTask.FromException(ex) vs new ValueTask(Task.FromException(ex))

We can add syntatic sugar (e.g. ValueTask.FromException) without changing the implementation details.

Why canā€™t you store Exception there as well?

We could, and then all of the code paths need to pay for it by being able to deal with that being yet another type (this is another form of "make ValueTask bigger"). If I could make exceptions faster without negatively impacting success paths, then sure, but if there's even a minute impact to success, it's simply not something worth optimizing in the implementation.

Notes from the review today:

  • We feel like ValueTask<T> should be name but @MadsTorgersen disagrees. I'll sync with him later in person to close on this.

  • To address the behavioral differences, we might want to expose a method that:

    • Returns the underlying Task, if it's baked by a Task
    • Returns a new Task if it's baked by an IValueTaskSource<out TResult>

    public partial struct ValueTask<T>
    {
        public ValueTask<T> Preserve();
    }

  • Task separates between IsCancelled and IsFaulted. The current interface shape doesn't allow for that. This was done to reduce the number of interace calls. In order to future proof the design we could expose an enum from the interface (e.g. Status) that allows us to separate the values in the future without inflicting more interface dispatch.

    • Pipelines has a different notion of cancellation that is communicated on the TResult rather than the awaitable. They do that to avoid throwing in the case of cancellations.
  • We currently have a byte in ValueTask of which we use only 2 bits. In practice the byte will be padded to 32-bit boundary, so we could make the byte and int. Which gives us 30-bit. Should we expose as part of the IValueSource/ValueTask interface so that implementations can use it for other stuff (e.g. validation cookies). Note we cannot easily do this later due to versioning rules.

@stephentoub, I have some suggestions that would pair very nicely with this feature.

I would LOVE to see the following overloads added to enable more scenarios to take advantage of allocation free async:

  • [ ] public static bool ThreadPool.UnsafeQueueUserWorkItem(Action<object> callBack, object state)
  • [ ] public static bool ThreadPool.UnsafeQueueUserWorkItem(ActioncallBack)
  • [ ] public static bool ThreadPool.QueueUserWorkItem(Action<object> callBack, object state)
  • [ ] public static bool ThreadPool.QueueUserWorkItem(Action callBack)
  • [ ] public static void ExecutionContext.Run(ExecutionContext executionContext, Action<object> callback, object state)
  • [ ] public static void ExecutionContext.Run(ExecutionContext executionContext, Action callback)

... there might be some other obvious ones I'm missing, but these are literally killing me right now.

The lack of these overloads is killing me currently. Unless there's something obvious that I'm missing, I cannot directly convert my Action<object> to a WaitCallback or a ContextCallback, even though the signatures are identical. Moreover, in some cases, I have (or at least I allow my users to have) an Action with no state.

These overloads should be very trivial to implement, and they would be a godsend. I got inspired to improve my framework's async friendliness by enabling more allocation free paths with pass through state, but those methods are all ultimately blocks for me.

Thus, I am currently thus to allocated additional temporary delegates to wrap the Action or Action<object> instance (and my state) as either the WaitCallback or a ContextCallback, and then call the original Action or Action<object> (with the state).

Here's an example of my dilemma:

private static void Execute( ExecutionContext context, Action<object> action, object state )
{
    //I could pass either the action or the state as the ContextCallback's state, but what's the point?
    //I'm still forced to allocate a delegate or box a struct or some other allocating mechanism
    ExecutionContext.Run( context,
        _ =>
        {
            action( state );
        }, null);
}

However, in the current code for SynchronizationContext, I see this:

public virtual void Post(SendOrPostCallback d, object state)
{
  ThreadPool.QueueUserWorkItem(new WaitCallback(d.Invoke), state);
}

Will the compiler automagically not perform an allocation there and use the existing delegate instance??

If so, I could just follow that pattern:

private static void Execute( ExecutionContext context, Action<object> action, object state )
{
    //does this magically avoid an allocation?
    ExecutionContext.Run( context, new ContextCallback(action.Invoke), state);
}

(Although I don't think that would work for Action...)

If that isn't a solution, it sure would be nice to add those overloads to SynchronizationContext as well, but that might be a breaking change for people currently implementing SynchronizationContext... However, adding the API with the default implementation like this would be safe:

public virtual void Post(Action<object> d, object state)
{
  Post(new SendOrPostCallback (d.Invoke), state);
}

And then the various built in SynchronizationContext classes the framework provides could have the smarter implementation once the static overloads are in place:

public override void Post(Action<object> d, object state)
{
  ThreadPool.QueueUserWorkItem(d, state);
}

Etc., etc.... The SynchronizationContext enhancement would be nice, but not necessary. I'd happily deferred that to round 2.

I see some similar comments above. The overloads for ThreadPool and ExecutionContext are both necessary for removing allocations on a very performance critical in my code. (Getting the new overloads on SynchronizationContext would be awesome too, honestly. šŸ˜†) The ThreadPool and ExecutionContext overloads are nice at least because it might be possible to tweak the internals to avoid the allocations.

Having a CallbackHandler struct (similar to ValueTask) could be the solution (note: I haven't checked how these are implemented internally) where you've got a reference and a flag (e.g. CallbackType.ContextCallback, CallbackType.Action, CallbackType.ActionObject, CallbackType.WaitCallback, etc.) and a simple switch on the flag to execute.

I can dream, right?

public static bool ThreadPool.QueueUserWorkItem(Action<object> callBack, object state)

This was already added, just with a generic TState instead of object.

Please feel free to open issues proposing other APIs.

Good to know! That probably hasn't bubbled up to .NET framework yet (which is what I'm compiling on at the literal moment). I'll check the code on the NET Core 2.0 platform, and see if any APIs may have been added that I'm not aware of yet.

Is the master branch on CoreFx representative of what's going into 2.1? If so, I can check there too and avoid asking for anything that's already been done since 2.0.

Lastly, is it better to submit one issue per method overload that I'm asking for? The ones I noted are all on my hot path, and would have identical justifications.

That probably hasn't bubbled up to .NET framework yet

No, it's in NET Core 2.1.

Is the master branch on CoreFx representative of what's going into 2.1?

Yes

Lastly, is it better to submit one issue per method overload that I'm asking for?

I would think about what you really need, craft a proposal for the APIs, and if they're all very (on the same type, or all-or-nothing in value, etc.) put them in the same issue, otherwise different issues.

Out of curiosity, would your team take a community PR for something on that order of impact?

I wouldn't mind specing it out to get a sense of how much pain the request would be inflicting, and if it isn't much effort, I could just go ahead and finish it. (That might save someone some time regardless even if the PR isn't merged directly.)

Out of curiosity, would your team take a community PR for something on that order of impact?

Yes, once an API is approved. It needs to go through the set process:
https://github.com/dotnet/corefx/blob/master/Documentation/project-docs/api-review-process.md

using System;
using System.Runtime.CompilerServices;
using System.Threading;

class Program
{
    static void Main()
    {
        Action<object> a = x => Console.WriteLine("Action called " + x);

        // no allocation
        SendOrPostCallback s = Unsafe.As<Action<object>, SendOrPostCallback>(ref a);

        // no error (and faster)
        s("abc");
        s(123);

        // but the runtime type of s is Action'1 ...
        Console.WriteLine(s.GetType().Name);
    }
}

but these are literally killing me right now.

šŸ¤”

but these are literally killing me right now.
šŸ¤”

@jnm2, My spirits at the time, at any rate.

Although this new feature isn't out yet, @stephentoub's goal of minimizing allocations along the async pipeline inspired me to take a hard look through my own code base to minimize allocations. The code I support is cross platform to older frameworks (.NET 2.0 - latest, .NET Core, Unity, NET Standard, Xamarin, Mono, and yes even Silverlight), and I also have a safe-only version of each assembly I ship (for those running in limited security context). So I've often got to bite the bullet and achieve my goals in ways that defy convention (e.g. when unsafe code isn't allowed or when a newer API isn't available in an old framework). Regardless, in the middle of my inspired grand search I ran into several intersections where I've got to hit the ThreadPool.UnsafeQueueUserWorkItem() or ExecutionContext.Run() APIs with mismatched delegate types, and I'm forced to allocate.

Code targeting the latest platforms is separated out using partials, #if regions, etc. and I do make use of the latest conventions whenever possible. However, there are layers of "old" style callbacks, etc., or even portions of the latests frameworks that haven't gotten the new async love yet.

For instance, part of @stephentoub et al's effort is to enhance things like the Socket API to support allocation free async calls. That will be a beautiful goal, and should make a great performance enhancement. So, my socket code will have to get "forked" and I'll use the new APIs for the latest platforms once they are released, and the old APIs for the old frameworks.

@ufcpp, I like that API, however it is part of the System.Runtime.CompilerServices.Unsafe package, and I'm not too aware yet of official guidance around it. However, since it's maintained by Microsoft, I'm presuming we have a guarantee that the API surface is locked and the dependency will be forward compatible. As a framework author, I have to be very cautious about taking external dependencies because those are dependencies my consumers have to take as well.

For platforms where I can generate code at runtime, I actually have an Unsafe.As that can do the same thing using runtime IL generation. Getting OOTB support for calling these primitives with modern delegate types seems like a good idea regardless. (Although I'll still be SOL on the legacy frameworks.) I would also like to be able to do the same thing from a safe context. (I'm not sure if System.Runtime.CompilerServices.Unsafe can be used in a limited security context. I'll have to check on that.)

@jkotas how do you feel about this to avoid allocations https://github.com/dotnet/corefx/issues/27445#issuecomment-369859083? Will it crash and burn somewhere or is it safe unsafe? šŸ˜„

  • It depends on the runtime implementation details of generics. It happens to work in .NET Core today.
  • It assume that you actually want to do this to pass the delegate to some library code. It is very fragile, e.g. the library code can store the delegate as an object and then do if (d is SendOrPostCallback) { ... }. The general rule for these hacks is to not ever pass the incorrectly typed object reference to code that you do not own.

@jkotas, good points. The Unsafe.As strategy fails, for instance, if a developer tries to masquerade an Action as a ThreadStart when creating a Thread because Thread does an if (method is ThreadStart) type check. The Unsafe.As strategy also fails for events: for example, cannot add an EventHandler to an event of type EventHandler<SomeArg>. (I wish the CLI duck-typed delegates, despite the tradeoffs.)

@stephentoub, will this work be backported to .netstandard20 (and below) and included into System.Threading.Tasks.Extensions package that contains ValueTask at the moment?

@maksimkim see the answer in https://github.com/dotnet/corefx/issues/27445#issuecomment-368308590.

I am quite confused what should be used as the default return type. So, until now, default return type was the Task. Only when the operation completes synchronously we benefited from the ValueTask/ValueTask<T>. Now this types can represents asynchronously completed operations too. Since the feature is widely provided by core API, I am assuming that the default return type for the method should be ValueTask. For example, I provide some interface which return type is Task<T>. Someone implements it and read from Stream/WebSocket/.etc receiving ValueTask<T> He will be obligated to call AsTask() which allocates. Thus the whole point of ValueTask will be lost.

We still need to formalize guidance, but I expect it'll be something like this for public API surface area:

  • Task provides the most usability.
  • ValueTask provides the most options for performance optimization.
  • If you're writing an interface / virtual method that others will override, ValueTask is the right default choice.
  • If you expect the API to be used on hot paths where allocations will matter, ValueTask is a good choice.
  • Otherwise, where performance isn't critical, default to Task, as it provides better guarantees and usability.

From an implementation perspective, many of the returned ValueTask instances will still be backed by Task.

The feature has been merged.

To get this clear: You now have introduced a brand new non-generic ValueTask type (does not matter whether class or struct) which lacks that simple method:?

public static ValueTask<TResult> FromResult<TResult>(TResult result)
    => new ValueTask<TResult>(result);

I just can't believe that.

I just can't believe that.

Ok.

Should I create a ticket for that, is that comment enough, or are there reasons for not doing that?

Should I create a ticket for that

You're welcome to open an issue proposing the API. See:
https://github.com/dotnet/corefx/blob/master/Documentation/project-docs/api-review-process.md

@springy76
Why it is so important to have that method?
If one need a completed ValueTask he may use the default constructor new ValueTask(), or even default(ValueTask) for that purpose.

@AleckAgakhan

I'm speaking of the generic ValueTask<T> and there exists no generic type inference for constructors. Methods have. Why do you think Task<T> FromResult<T>(T result) exists in the non-generic Task class either at the first place? Why make the transition of existing code from Task<T> to ValueTask<T> artificial complex?

But thank you very much for the hint that I can replace return new ValueTask<ForSpecicReasons.FullyQualified.TypeName>((ForSpecicReasons.FullyQualified.TypeName)null); by just return default;

While I completely agree with this sentiment, it is easily addressable in user-code: just make your own ValueTaskHelper static class with the FromResult API that you want, and use it in the interim. That said, the APIs would definitely benefit from consistency in the framework though.

I agree with the proposal, but I have to say I find the sentiment offensive and potentially foul of the .NET Foundation code of conduct.

I find it offensive getting a thumb down for me trying to best to explain the problem with a generic constructor which additionally introduces overload ambigouty for null values because of the overload accepting a Task<T> argument.

I'm deeply sorry for the fact that English is not my native language and I'm therefore not naturally able to express everything in the US american dont-touch-dont-look-be-always-positive-and-let-shit-smell-like-flowers paradigma of "life".

You did not get a thumb down for trying your best to explain the need for the API, which as I said I agree with. I voted the comments down because you chose hostile wording each time. Nationality is not relevant. Hostility is childish and ineffective.

Whatever your views, you must abide by the https://dotnetfoundation.org/code-of-conduct to participate here.

@springy76, to answer some of your points.

This is a fairly long thread and a new type was added. Sometimes APIs are really just missed.

The proper thing to do is to open a new API proposal (following the recommended process: https://github.com/dotnet/corefx/blob/master/Documentation/project-docs/api-review-process.md) and have it go through review -- I am happy to help with this if you need it. Sometimes this is really easy (people just get why the API should be there) and sometimes it requires more discussion (you will need to convince other people that the API should be there).

Generally speaking, when you have two related types (like Task vs ValueTask) and you are wanting an API to also exist in the newer one it is on the easier side of things. "This exists in X, we should also make it exist in Y to make porting to Y easier" is a very good argument to have. That doesn't guarantee it will go in (as there could be a good reason why it wasn't included) but that is a discussion to have on the new API thread.

As a general comment. Keep in mind that communicating via text is hard, communicating via text over the internet may be harder.

It is generally good to believe that people are not trying to openly attack your ideas and are instead trying to understand your intent. Sometimes changing the wording a bit and being more explicit (generally in the information you want the person to respond with) can help.

@stephentoub
1) ValueTask.Delay() seems to be very much needed;
2) Task.WhenAny and Task.WhenAll are required to be optimised for ValueTask;
3) Object Pooling 'heap 'with manua-gc api is required;

ValueTask<T> only provides a net optimization when the value is readily available to return to avoid creating a task object, which means that the task completes synchronously, or (as @juepiezhongren notes below) when using ValueTask to wrap an implementation of IValueTaskSource for an asynchronous operation.

In that sense, I don't see much value for a ValueTask.Delay(). When would a delay ever complete synchronously? When the delay is zero? (However, a delay of zero might actually be supposed to force a delay like Thread.Sleep(0) does.)

The WhenAll and WhenAny might be useful to have though - for cases where an API returns a ValueTask, the API is declaring it will often complete synchronously or offers a cost savings by wrapping IValueTaskSource (e.g. the new sockets API changes that @stephentoub worked on). And as such, converting those to Task objects for the WhenAll or WhenAny case could well often be an unnecessary allocation.

@marksmeltzer for iValueTaskSource, ValueTask is not just for pseudo-async senarios!

@juepiezhongren, You are definitely correct about IValueTaskSource, and I edited my comment above to clarify my point. I am just commenting that the WhenAll and WhenAny APIs offer less utility in general. I would still like seeing them added for parity's sake though.

What unique utility do you see ValueTask.Delay having, since you brought it up?

The IValueTaskSource allows utilizing an existing object instance that can implement the task APIs and wrap that instance in a ValueTask. In the case of the delay we just have a value and would still need to allocate a task object (unless you're thinking of some case I'm not seeing)...

One scenario I can think of for enabling a non-allocating implementation for ValueTask.Delay() would be that it returns, essentially, a ValueTask using an additional enum value internally to declare it as representing a delay. That might allow for implementing special case logic within the task APIs to handle the delay case in a non-allocating way. That's a more substantial change and I'm not sure if it would offer any performance benefits at runtime. The idea on the IValueTaskSource being wrapped by a ValueTask was that async IO operations already allocate state transfer objects that can easily handle the demands of the task API and thus remove the additional per async call allocation for a Task. Through effective reuse of SocketEventArgs (for example), that can eliminate a large number of allocations and thus deliver tangible performance benefits. At this level, the async IO operations can be completing in mere microseconds and those allocations do otherwise cause noticeable aggregate performance drags.

In the case of a delay, however, there isn't likely to be any visible pressure from allocations because delays happen at the whole millisecond time scale. There isn't a mechanism in most architectures to reliably do any sub-millisecond delays and which means they won't accumulate noticeable performance drag from the additional Task allocations. In general, I'm not too worried about having @stephentoub and team eliminate allocations in the Task API unless those allocations are impacting CPU-bound or IO-bound workloads. Those are the workloads are actually affected by the extra allocations.

If you have some other idea in mind regarding delays that I'm not seeing, please share.

@juepiezhongren, what did you mean by "Object Pooling 'heap 'with manua-gc api is required"?

@marksmeltzer it is best that ivalueTaskSource is allocated in pooling heap where init and gc shall be manually done. Task.WhenAll and WhenAny definitely need to be tuned or new api to be done, considering that Task and ValueTask methods would be used together for sync.

@stephentoub
i believe there r a lot more work is required, considering async's 5 main senarios:
1) IO; 2) Net; 3) Timer; 4) long Computing; 5) UI;
where vTask with IValueTaskSource will definitely bring a lot of benefits;

btw, TaskCompletionSource is required to be reusable and pool-cached

@stephentoub ,Task.Delay is too often used, ValueTask.Delay with IValueTaskSource instance injected is really a strongly-needed API

@juepiezhongren, I do not understand what that would solve here. What costs do you believe would be avoided in doing so? If you believe it's important, your best bet is to code it up and show measurements of exactly how it would improve things and by how much.

@stephentoub just make less alloc(wont delay create new task instance? maybe i got some mistakes), task.delay is great already now

Task.Delay will not only create a Task, it'll also create a Timer, a TimerHolder, and a TimerQueueTimer as part of the underlying implementation. But a ValueTask.Delay would incur the latter three as well, and it would need some object to be the IValueTaskSource. Either that object would need to be allocated, in which case it might as well just be Task, or it would need to be pooled in some way, which brings with it its own costs... managing a process-wide object pool and doing it well is not easy (or necessarily cheap).

ur answer is perfect, thx.
in order to make delay totally free-alloc, some complete re-implementation is needed

Could use it for a reoccurring awaitable Task.Delay type loop; reusing the same timer?

i've heard @davidfowl has done some things for delay

There are certainly things you can layer on top, e.g. you can coalesce if you're willing to trade off accuracy, e.g. https://blogs.msdn.microsoft.com/pfxteam/2011/12/03/coalescing-cancellationtokens-from-timeouts/.

Could use it for a reoccurring awaitable Task.Delay type loop; reusing the same timer?

Sure, but that brings with it other problems, e.g. when you're done with it, who disposes of the underlying Timer? At that point you're not going to want to use ValueTask, because you're going to want an API that lets you control such things.

(One addendum... I forgot I previously made a change so that Task.Delay just goes straight to the TimerQueueTimer and skips allocating the Timer and the TimerHolder objects. Part of https://github.com/dotnet/coreclr/pull/14527.)

so, valueTask is possible!!!! Great Stephen

Sure, but that brings with it other problems, e.g. when you're done with it, who disposes of the underlying Timer?

Could wrap the source in a using e.g.

using (var timer = new PeriodicAsyncTimer(period: 1000))
{
    timer.Start();
    while (!token.IsCancellationRequested)
    {
        await timer;

       // Do stuff
   }
}

Whereas going via period on timer directly its harder to dispose as its just a callback (need access to timer via whatever is injected as state; or an external disposal)

so, valueTask is possible!!!!

@benaadams' suggestion would be that ValueTask.Delay would still allocate, you could just reuse that same ValueTask awaiting it multiple times. But, FWIW, I think that's a bad precedent to set. We do not want to suggest to developers that they can await the same ValueTask object repeatedly... in fact we explicitly say not to do that... many APIs that return ValueTask will very clearly break if that's done, and that would be the case if, for example, ValueTask.Delay were to pool. We could probably get away with a ValueTask.Delay only allocating one object rather than two, but does that really move the needle? After all, even in a "tight" loop just awaiting Task.Delay(1), on Windows that's generally only going to happen every 15ms. We should not be adding new APIs just because we can.

Could wrap the source in a using

Sure. Then it's no longer a ValueTask.

Could wrap the source in a using

Sure. Then it's no longer a ValueTask.

D'oh, I was thinking it of it using the IValueTaskSource and returning ValueTasks but there's no need for that as it can just be a regular awaitable.

@stephentoub to make me self-control timer is a good idea, for a lot of delays are together with while, so this is a perfect solution

while{
    await Task.Delay(50);
}

VTaks + IValueTaskSource is just for regular-pattern await senarios, things like while-delay really need a VTask solution, in my opinion.

things like this really need a VTask solution. In my opinion,

Why? You have a scenario where the 20 small allocations / second that might save you is critical?

Just follow your philosophy in this article, current words it's not about perf.

var myTimer=new Timer().....
while(true){
    await ValueTask.Delay(10,myTimer);
.......
}
myTimer.Dispose();

but, this is not bad

In my mind what Stephen brought to the table is about performance, but it is not simply performance for performance's sake. When doing things like sockets or other data pipelines it can be easy to be doing millions of async operations per second where there is already some state primitive that can implement the IValueTaskSource and thus prevent extra task allocation overhead. That is especially true when some of those scenarios can complete synchronously if data is immediately available.

For 20 times per second, there isn't any problem that needs to be optimized. Doing delays of 0 would still only get you a max of 100 operations per second. So, in my mind there is zero need for this.

Regards,


From: juepiezhongren notifications@github.com
Sent: Thursday, October 11, 2018 8:14:13 PM
To: dotnet/corefx
Cc: Mark Smeltzer; Mention
Subject: Re: [dotnet/corefx] Allocation-free awaitable async operations with ValueTask and ValueTask (#27445)

Just follow your philosophy in this article, current words it's not about perf.

ā€”
You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHubhttps://github.com/dotnet/corefx/issues/27445#issuecomment-429190353, or mute the threadhttps://github.com/notifications/unsubscribe-auth/ASRWY2izIS-xti2SHClmU8Xx2Zuz4fhaks5ukAkFgaJpZM4SSKTv.

@marksmeltzer DotNetCore, in my mind, is somthing that will implement everything and every dream out of Midori & Singularity, 20 times per second is not important for server app, but is for system.

What I am saying is that is empirically false: Task.Delay() as a minimum theoretical throughput of less than 20 per second no-matter-what because the underlying system hardware timers operate at a maximum of 7 millisecond intervals. Most systems lack a high resolution timer though so the actual throughput could be a low as 4 per second!

In other words the overhead on calling Task.Delay() is so high that any optimization like reducing allocations will so zero benefit in empirical testing. Allocations are not implicitly bad.

Allocations on a very hot path are the only ones worth worrying about (e.g. millions per second). Once all of the VERY hot paths are optimized (Stephen seems to be working on that), then the next step is all of the VERY warm paths, etc.

Task.Delay() would never benefit however.

Regards,


From: juepiezhongren notifications@github.com
Sent: Thursday, October 11, 2018 9:15:50 PM
To: dotnet/corefx
Cc: Mark Smeltzer; Mention
Subject: Re: [dotnet/corefx] Allocation-free awaitable async operations with ValueTask and ValueTask (#27445)

@marksmeltzerhttps://github.com/marksmeltzer DotNetCore, in my mind, is somthing that will implement everything and every dream out of Midori & Singularity, 20 times per second is not important for server app, but is for system.

ā€”
You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHubhttps://github.com/dotnet/corefx/issues/27445#issuecomment-429198905, or mute the threadhttps://github.com/notifications/unsubscribe-auth/ASRWY4hTIdKGm7Ij6rCVy4_l5Mu6whJJks5ukBd2gaJpZM4SSKTv.

I'm wondering to know is it ok that we use ValueTask<T> in ASP.NET Core Controllers?
c# public async ValueTask<IActionResult> Index() { await DoSomethingAsync(); return View(); }
@stephentoub @davidfowl

Was this page helpful?
0 / 5 - 0 ratings

Related issues

chunseoklee picture chunseoklee  Ā·  3Comments

nalywa picture nalywa  Ā·  3Comments

jzabroski picture jzabroski  Ā·  3Comments

EgorBo picture EgorBo  Ā·  3Comments

aggieben picture aggieben  Ā·  3Comments