Closed lorban closed 2 weeks ago
@lorban I've pushed a fixed for the read side of servlets in ee10. Thoughts?
@lorban @sbordet we could extend CompletableFuture to an InvocableCompletableFuture that either:
andThen
which pass in functional references. If any such method is called, then the invocation type will be blocking, otherwise it is non-blockingandThen
and looks at the invocation type of the functional references passed, which it combines as the invocation type. andThen
and redirects them to andThenAsync
. The invocation type can then always be non-blocking.Note the returned completablefuture might also need to be wrapped (maybe not for the andThenAsync redirection).
@lorban @sbordet would this help:
class NonBlockingCompletableFuture<V> extends CompletableFuture<V> implements Invocable
{
private final Executor _executor;
public NonBlockingCompletableFuture(Executor executor)
{
_executor = executor;
}
@Override
public InvocationType getInvocationType()
{
return InvocationType.NON_BLOCKING;
}
@Override
public CompletableFuture<Void> acceptEither(CompletionStage<? extends V> other, Consumer<? super V> action)
{
return super.acceptEitherAsync(other, action, _executor);
}
@Override
public <U> CompletableFuture<U> applyToEither(CompletionStage<? extends V> other, Function<? super V, U> fn)
{
return super.applyToEitherAsync(other, fn, _executor);
}
@Override
public CompletableFuture<V> exceptionallyCompose(Function<Throwable, ? extends CompletionStage<V>> fn)
{
return super.exceptionallyComposeAsync(fn, _executor);
}
@Override
public <U> CompletableFuture<U> handle(BiFunction<? super V, Throwable, ? extends U> fn)
{
return super.handleAsync(fn, _executor);
}
@Override
public CompletableFuture<Void> runAfterBoth(CompletionStage<?> other, Runnable action)
{
return super.runAfterBothAsync(other, action, _executor);
}
@Override
public CompletableFuture<Void> runAfterEither(CompletionStage<?> other, Runnable action)
{
return super.runAfterEitherAsync(other, action, _executor);
}
@Override
public CompletableFuture<Void> thenAccept(Consumer<? super V> action)
{
return super.thenAcceptAsync(action, _executor);
}
@Override
public <U> CompletableFuture<Void> thenAcceptBoth(CompletionStage<? extends U> other, BiConsumer<? super V, ? super U> action)
{
return super.thenAcceptBothAsync(other, action, _executor);
}
@Override
public <U> CompletableFuture<U> thenApply(Function<? super V, ? extends U> fn)
{
return super.thenApplyAsync(fn, _executor);
}
@Override
public <U, V1> CompletableFuture<V1> thenCombine(CompletionStage<? extends U> other, BiFunction<? super V, ? super U, ? extends V1> fn)
{
return super.thenCombineAsync(other, fn, _executor);
}
@Override
public <U> CompletableFuture<U> thenCompose(Function<? super V, ? extends CompletionStage<U>> fn)
{
return super.thenComposeAsync(fn, _executor);
}
@Override
public CompletableFuture<Void> thenRun(Runnable action)
{
return super.thenRunAsync(action, _executor);
}
@Override
public CompletableFuture<V> whenComplete(BiConsumer<? super V, ? super Throwable> action)
{
return super.whenCompleteAsync(action, _executor);
}
}
I honestly would not go there.
Changing the semantic of thenRun()
into a thenRunAsync()
is breaking the least surprise.
I think we can just say in the javadocs to never block on the CF returned, or deprecate the API.
I honestly would not go there. Changing the semantic of
thenRun()
into athenRunAsync()
is breaking the least surprise.I think we can just say in the javadocs to never block on the CF returned, or deprecate the API.
We need more than javadocs, as we need the Runnable or Callback created from the CF to be Invocable. So how about:
/**
* An extension of {@link java.util.concurrent.CompletableFuture} that is an {@link Invocable}.
* The {@link InvocationType} is initially the type used in construction (default NON_BLOCKING).
* If a non async method is called, then the invocation type of any passed function is used.
* @param <V>
*/
class InvocableCompletableFuture<V> extends java.util.concurrent.CompletableFuture<V> implements Invocable
{
private final AtomicReference<InvocationType> _invocationType = new AtomicReference<>();
public InvocableCompletableFuture()
{
this(null);
}
public InvocableCompletableFuture(InvocationType invocationType)
{
_invocationType.set(Objects.requireNonNullElse(invocationType, InvocationType.NON_BLOCKING));
}
@Override
public InvocationType getInvocationType()
{
return _invocationType.get();
}
@Override
public java.util.concurrent.CompletableFuture<Void> acceptEither(CompletionStage<? extends V> other, Consumer<? super V> action)
{
_invocationType.set(Invocable.combine(Invocable.getInvocationType(other), Invocable.getInvocationType(action)));
return super.acceptEither(other, action);
}
@Override
public <U> java.util.concurrent.CompletableFuture<U> applyToEither(CompletionStage<? extends V> other, Function<? super V, U> fn)
{
_invocationType.set(Invocable.combine(Invocable.getInvocationType(other), Invocable.getInvocationType(fn)));
return super.applyToEither(other, fn);
}
@Override
public <U> java.util.concurrent.CompletableFuture<U> handle(BiFunction<? super V, Throwable, ? extends U> fn)
{
_invocationType.set(Invocable.getInvocationType(fn));
return super.handle(fn);
}
@Override
public java.util.concurrent.CompletableFuture<Void> runAfterBoth(CompletionStage<?> other, Runnable action)
{
_invocationType.set(Invocable.combine(Invocable.getInvocationType(other), Invocable.getInvocationType(action)));
return super.runAfterBoth(other, action);
}
@Override
public java.util.concurrent.CompletableFuture<Void> runAfterEither(CompletionStage<?> other, Runnable action)
{
_invocationType.set(Invocable.combine(Invocable.getInvocationType(other), Invocable.getInvocationType(action)));
return super.runAfterEither(other, action);
}
@Override
public java.util.concurrent.CompletableFuture<Void> thenAccept(Consumer<? super V> action)
{
_invocationType.set(Invocable.getInvocationType(action));
return super.thenAccept(action);
}
@Override
public <U> java.util.concurrent.CompletableFuture<Void> thenAcceptBoth(CompletionStage<? extends U> other, BiConsumer<? super V, ? super U> action)
{
_invocationType.set(Invocable.combine(Invocable.getInvocationType(other), Invocable.getInvocationType(action)));
return super.thenAcceptBoth(other, action);
}
@Override
public <U> java.util.concurrent.CompletableFuture<U> thenApply(Function<? super V, ? extends U> fn)
{
_invocationType.set(Invocable.getInvocationType(fn));
return super.thenApply(fn);
}
@Override
public <U, V1> java.util.concurrent.CompletableFuture<V1> thenCombine(CompletionStage<? extends U> other, BiFunction<? super V, ? super U, ? extends V1> fn)
{
_invocationType.set(Invocable.combine(Invocable.getInvocationType(other), Invocable.getInvocationType(fn)));
return super.thenCombine(other, fn);
}
@Override
public <U> java.util.concurrent.CompletableFuture<U> thenCompose(Function<? super V, ? extends CompletionStage<U>> fn)
{
_invocationType.set(Invocable.getInvocationType(fn));
return super.thenCompose(fn);
}
@Override
public java.util.concurrent.CompletableFuture<Void> thenRun(Runnable action)
{
_invocationType.set(Invocable.getInvocationType(action));
return super.thenRun(action);
}
@Override
public java.util.concurrent.CompletableFuture<V> whenComplete(BiConsumer<? super V, ? super Throwable> action)
{
_invocationType.set(Invocable.getInvocationType(action));
return super.whenComplete(action);
}
}
That would then be used with:
public abstract class ContentSourceCompletableFuture<X> extends Invocable.InvocableCompletableFuture<X> implements Runnable
{
private final Content.Source _content;
public ContentSourceCompletableFuture(Content.Source content)
{
_content = content;
}
/**
* <p>Initiates the parsing of the {@link Content.Source}.</p>
* <p>For every valid chunk that is read, {@link #parse(Content.Chunk)}
* is called, until a result is produced that is used to
* complete this {@link CompletableFuture}.</p>
* <p>Internally, this method is called multiple times to progress
* the parsing in response to {@link Content.Source#demand(Runnable)}
* calls.</p>
* <p>Exceptions thrown during parsing result in this
* {@link CompletableFuture} to be completed exceptionally.</p>
*/
public void parse()
{
while (true)
{
Content.Chunk chunk = _content.read();
if (chunk == null)
{
_content.demand(this);
return;
}
if (Content.Chunk.isFailure(chunk))
{
if (chunk.isLast())
{
completeExceptionally(chunk.getFailure());
}
else
{
if (onTransientFailure(chunk.getFailure()))
continue;
_content.fail(chunk.getFailure());
completeExceptionally(chunk.getFailure());
}
return;
}
try
{
X x = parse(chunk);
if (x != null)
{
complete(x);
return;
}
}
catch (Throwable failure)
{
completeExceptionally(failure);
return;
}
finally
{
chunk.release();
}
if (chunk.isLast())
{
completeExceptionally(new EOFException());
return;
}
}
}
@Override
public void run()
{
parse();
}
/**
* <p>Called by {@link #parse()} to parse a {@link org.eclipse.jetty.io.Content.Chunk}.</p>
*
* @param chunk The chunk containing content to parse. The chunk will never be {@code null} nor a
* {@link org.eclipse.jetty.io.Content.Chunk#isFailure(Content.Chunk) failure chunk}.
* If the chunk is stored away to be used later beyond the scope of this call,
* then implementations must call {@link Content.Chunk#retain()} and
* {@link Content.Chunk#release()} as appropriate.
* @return The parsed {@code X} result instance or {@code null} if parsing is not yet complete
* @throws Throwable If there is an error parsing
*/
protected abstract X parse(Content.Chunk chunk) throws Throwable;
/**
* <p>Callback method that informs the parsing about how to handle transient failures.</p>
*
* @param cause A transient failure obtained by reading a {@link Content.Chunk#isLast() non-last}
* {@link org.eclipse.jetty.io.Content.Chunk#isFailure(Content.Chunk) failure chunk}
* @return {@code true} if the transient failure can be ignored, {@code false} otherwise
*/
protected boolean onTransientFailure(Throwable cause)
{
return false;
}
}
@sbordet @lorban I've used the approach above to extend Invocable.InvocableCompletableFuture for ContentSourceCompletableFuture. I've tested this with a new form based starvation test, that fails with the current future and passes with this fix.
I honestly would not go there. Changing the semantic of
thenRun()
into athenRunAsync()
is breaking the least surprise.
I think there is no problem changing thenRun()
into thenRunAsync()
, as the user of a CF has no control of what thread calls them in the first place, so the differences will be transparent to them. I don't think anybody would be surprised.
But, the current approach doesn't need this.
IMHO the CompletableFuture
API is orthogonal to the main problem, so we should not distract ourselves too much with this side problem.
That being said, I think the main problem I have with that API is that it makes it so easy to write broken code that I'm tempted to say it encourages. This simple example brings back the original problem in a non-obvious way:
String s = Content.Source.asStringAsync(source, charset).thenRun(() -> { /* whatever */ }).get();
The above is my main motivation to say we should deprecate all our CompletableFuture
API.
Back to the main problem, I remember a few months ago I said that we made a mistake when we designed the Content.Source.demand()
call to take a Runnable
: it should have taken a Invocable.Task
instead to make it apparent that InvocationType
is meaningful. I wonder if we could do that change in 12.1 without breaking backward compatibility, as it's source-compatible but not binary compatible?
@lorban unfortunately it's not a compatible change: we have several occurrences of allocating a Runnable
anonymous, and then calling demand(this)
inside that Runnable
.
I think there is no problem changing thenRun() into thenRunAsync(), as the user of a CF has no control of what thread calls them in the first place, so the differences will be transparent to them. I don't think anybody would be surprised.
They will be, as thenRunAsync()
will occupy a common pool thread, and after N of them they won't be available, so tasks won't be run.
In the other case I would block some other thread, but not the precious ones in the common pool.
Furthermore, it would not solve the problem of a double composition, e.g. async().thenApply(...).thenRun(<JDBC>)
.
I really would not go into trying to put another smartness to fix this, but rather just document clearly the API or deprecate it.
I've just noticed some other complicating factor we must be careful about: a lot of places where an Invocable
is passed to demand()
, the InvocationType
is pre-computed at the time demand()
is called, not when the demand callback is to be executed.
This means if we ever decide to manipulate the InvocationType
dynamically, we have to be extra careful that all demand()
calls take that into account, which isn't the case now.
@gregw, also, Promise.Completable
and Callback.Completable
are already Invocable
, so perhaps we just need a way to specify the InvocationType
, or just assume it is non-blocking.
See alternative at #12406.
@lorban
a lot of places where an
Invocable
is passed todemand()
, theInvocationType
is pre-computed at the timedemand()
is called, not when the demand callback is to be executed.
This is somewhat of an issue, but it is something we have elsewhere in the code (see dyanamic handler invocation type).
Ultimately it is a race we can never win, if a passed callback is non-blocking, but can asynchronously become blocking, then we can always lose that race. Basically if code that self described as asynchronous suddenly decides it is not, changes its self description and blocks, then we can do nothing.
It is something worth javadocing, but I do not think it is fixable.
@sbordet
I really would not go into trying to put another smartness to fix this, but rather just document clearly the API or deprecate it.
"This fix" is not attempting this smartness we are discussing here.... but I think it is worthwhile discussing it so that we fully understand the problem and possible ways out. Let's not cede too much authority to the application by saying it would be surprised if it was called back by one thread or another. The thread we use to call back a CF is entirely our choice, either before we call succeeded, or within the implementation of any CF that we create.
I think there is no problem changing thenRun() into thenRunAsync(), as the user of a CF has no control of what thread calls them in the first place, so the differences will be transparent to them. I don't think anybody would be surprised. They will be, as
thenRunAsync()
will occupy a common pool thread, and after N of them they won't be available, so tasks won't be run. In the other case I would block some other thread, but not the precious ones in the common pool.
Firstly, this uses a passed executor, so there is no common pool thread involved, only our executor.
But I do not think this changes behaviour. If our thread pool is exhausted, currently our CF is seen as blocking and just never called, so deadlock happens always. If we did a change based on converting non-async methods to async, then in the blocking case, we still hit the exhausted thread pool and are deadlocked.
Furthermore, it would not solve the problem of a double composition, e.g.
async().thenApply(...).thenRun(<JDBC>)
.
Yeah, that would require further wrapping... hence I don't like this approach.
So I'm not proposing we do this, and this PR is not doing it. But I do believe that if we want to control the thread that does the callback, it is within our authority both as the caller and as the implementor of the CF, so there would be no surprise.
The above is my main motivation to say we should deprecate all our
CompletableFuture
API.
I think we should fix all our CF usages (it is not proving to be too difficult) and then separately consider if it is indeed the best API to use for many of our utility classes. As you say, it is orthogonal.
There are several ways we can "fix" our existing CF usage:
1) Change our CFs to automatically determine their invocation type by checking what callbacks are installed on them. This appears to work, but there is some risk of CFs dynamically changing their type after calling demand.
2) Change our CFs to have a fixed NON_BLOCKING invocation type of non-blocking and just javadoc that the andThen
style APIs should not be used. This would "work", but would be fragile.
3) Change our CFs to have a fixed NON_BLOCKING invocation type and throw UnsupportedOperation exception if any andThen
style APIs were used. This is kind of deprecating full CF support and a step towards coming up with our own API (which we could do as well).
4) Change our CFs to have a fixed NON_BLOCKING invocation type and throw IllegalArgumentException if any andThen
style APIs are used with BLOCKING functions etc.
My contributions to this PR implement 1). They are not too complex, appear to be working, but are vulnerable to dynamic changes of invocation type. 2) is essentially the same as the alternate PR, as that it just changes to type and hopes for the best.
I don't mind 3) or 4) as they solve the dynamic type issue. 3) is a big neutering of the CF API and really shouts that we should not use it and do something else. 4) is less so, as it gives a way all the API can be used.
Ultimately, we have always known that the invocation type of our callbacks is important and previously we have always dealt with it. For some reason, we forgot that for a little while and introduced a few utility classes that didn't do the work required to ensure a good invocation type. I think least surprise is to not radically change our API/contract and to just go back to caring about invocation type in all our utility classes and better javadoc of the necessity to do so for others that implement the API directly (we could even give warning for non invocable demand callbacks?).
There is still a role for Blocking callbacks to demand. It is just that they cannot be used if their only job is to wakeup an otherwise blocked thread, as that is deadlock. If the demand callback is genuinely blocking (eg on a database), then it will just have to wait until a thread is available before it is called and then it can block on the database. It just cannot wait if its job is to free up a thread that would go back to the pool to call itself in the first place.
For our CF based utility classes, we have two distinct styles of usage:
1) in our adaption to blocking APIs (e.g. in ServletApiRequest) we often do:
Fields formFields = FormFields.from(request).get();
2) in truly async (e.g. DelayedHandler) we do:
CompletableFuture<Fields> futureFormFields = FormFields.from(getRequest(), _charset);
// if we are done already, then we are still in the scope of the original process call and can
// process directly, otherwise we must execute a call to process as we are within a serialized
// demand callback.
boolean done = futureFormFields.isDone();
futureFormFields.whenComplete(done ? this::process : this::executeProcess);
We typically do not mix these styles, other than the DelayedHandler
is designed to go in front of the blocking ServletHandler
, so the whenComplete
style usage may proceed the get()
style usage. However, this is not really a problem as in this case, the get()
will never block, as it is already done/complete by the time it is given an opportunity.
However, I don't think my fix here is good for the 2) usage. The parsing, and hence demanding, is commenced within the from
method, which is before the call to whenComplete
. So we cannot use calling that method to trigger switching the demand
callback to be blocking. We need to know the invocation type during the from call so we could just pass in:
CompletableFuture<Fields> futureFormFields = FormFields.from(getRequest(), _charset, InvocationType.BLOCKING);
// if we are done already, then we are still in the scope of the original process call and can
// process directly, otherwise we must execute a call to process as we are within a serialized
// demand callback.
boolean done = futureFormFields.isDone();
futureFormFields.whenComplete(done ? this::process : this::executeProcess);
In this case the CF returned should have the get()
methods extended to do something like:
public T get()
{
if (getInvocationType() != InvocationType.NON_BLOCKING && !isDone())
throw new IllegalStateException("Cannot call get unless NON_BLOCKING or complete");
return super.get();
}
But it would be a bit strange as we'd then need to do in ServletAPIRequest
:
Fields formFields = FormFields.from(request, InvocationType.NON_BLOCKING).get();
which I think is correct, but reads strangely. It is also strange that the passed invocation type is only used if this is the first call to from
As I think the current "fix" is wrong. I'll change it today to do this.
The CF fix works but is counter intuitive, fragile and mind blowing when combining CFs as we do going from core Parts to servlet Parts.
So I've pushed some changes that deprecate all the CF APIs in Fields and MultiPart and replace with explicit onXxx
and getXxx
methods. Internally, these use the "fixed" CF for now, but we can replace with something different once we can actually remove the CFs
This is still a WIP, but pushed so you can see it.
@gregw I have not looked yet, but regarding this comment:
The thread we use to call back a CF is entirely our choice, either before we call succeeded, or within the implementation of any CF that we create.
That is true, but what I would like to see is actually the following:
Promise
, Callback
, Blocker
, etc. that specify the InvocationType
(likely NON_BLOCKING) until the operation is finished.CompletableFuture
using ThreadPool.executeImmediately()
, or similar.In this way, the semantic of CompletableFuture
is preserved, and we do not need any special subclass of CompletableFuture
with overrides that would need to be reviewed when the JDK adds new methods to it.
Then, if the application blocks on the CompletableFuture
, it's the application choice, and it won't block a selector thread.
I'll review as soon as I am back.
@sbordet I think we are agreed that ultimately we want to remove CF for our implementation and API. However, currently it is in our API (although deprecated by this PR), so we do need to fix it. Hence I think we should have the extended CF (also deprecated) as an interim fix until we can remove the CFs entirely.
So the key things we need to do in this PR is: 1) fix/work around the immediate problem in our usage of utility classes (mostly done) 2) Come up with a new API that will allow others to use these classes without using CFs (done - but needs work see below).
As we are close to release time, can you focus on reviewing 2), as new API is forever.
I like the general style of the new API in this PR: getParts(request)
for blocking and onParts(request, consumer, consumer)
for async. But the arguments of the onParts
methods need some consideration.
Currently they are like:
void onParts(Request request, BiConsumer<Parts, Throwable> immediate, InvocableBiConsumer<Parts, Throwable> future);
default void onParts(Request request, InvocableBiConsumer<Parts, Throwable> future)
{ onParts(request, future, future); }
Thoughts about this are:
BiConsumer
and the InvocableBiConsumer
makes it clear which one needs to have an InvocationType. Better than javadoc.BiConsumer
is a single method interface, so lambdas can still be used, although less so with the InvocableBiConsumer
An alternative would be:
void onParts(Request request, Promise<Parts> immediate, Promise<Parts> future);
default void onParts(Request request, Promise<Parts> future)
{ onParts(request, future, future); }
Thoughts:
@lorban thoughts on the API comment above?
I like Promise<Parts>
because:
Blocker.Promise
so it's trivial to implement the blocking API with the async onePromise
is already part of our async APIbut I do not like that it is not Invocable
, but I think we can make it extend that interface without problem.
I don't mind that this API is not as fluent as CompletableFuture
: you can't use lambdas and you cannot chain actions, this isn't so bad as this is already the case with Callback
and we don't mind it.
I don't like InvocableBiConsumer
as it really is equivalent to Promise
and would IMHO just add confusions about when we would use one over the other. But maybe BiConsumer<Parts,Throwable>
could make sense to clearly show in the API what needs an invocation type. Or maybe use Promise<Part>
and introduce Promise.Invocable<Part>
?
Oh, and there are other places where we expose CompletableFuture
in our API, like in the Content.Source
static helpers. Those should be fixed and deprecated too.
@sbordet @lorban I've converrted the FormFields class to use Promise
on the new API and it works fine. Can you review that as a priority and if OK I'll convert the MultiPart classes as well.
@sbordet @lorban I'm liking the new getXxx()
and onXxx(Promise)
APIs.
So my mind is turning more towards the implementation. Currently we are still using a CF<Xxx>
internally, as it is a useful object to add as a request attribute, so that we can make multiple calls to getXxx()
and onXxx(Promise)
idempotent. E.g. a DelayedHandler or EagerHandler can first call the onXxx(Promise)
to read the content asynchronously before dispatching to a blocking handler/servlet that uses getXxx()
that finds the CF<Xxx>
as an attribute and does a get()
from it, which will not block. But if the async handler is not in front, that same get()
will block whilst the content is read in blocking mode.
But I'm wondering if we need to be so idempotent. Will we ever get a call to getXxx()
and onXxx(Promise)
whilst a previous one is pending? I don't think so as the first blocks and the second only dispatches once the content is read. So the attribute could just be the Xxx
put there at completion, rather than a CF<Xxx>
put there at commencement. The danger is if we get concurrent calls to getXxx()
and onXxx(Promise)
, but I just don't see how that could/should happen.
Note that the parsers themselves are still written in terms of CF
, so there is more work to do to remove all CF usage. But I've deprecated some of those APIs (But FormFields is problematic and will need a breaking change in 12.1 ).
I'm happy if this PR fixes the bug ugly, but introduces a new good API... and then clean up the implementation over the next few releases (probably only in 12.1).
For SerializedInvoker
an experiment I made is the following:
In Link.run()
we remember the InvocationType
of the running task.
If, after running a task, it is detected that another task should be run, we check the InvocationType
of the new task against the previous task.
Only in the case prev=NON_BLOCKING and next=BLOCKING, we wrap the next task, execute it and return.
The wrapper will resume Link.run()
to look for a next-next task.
This preserves the task serialization, although not anymore same thread-ness (although guarantees single thread-ness).
@gregw the SerializedInvoker
issue we discussed is as follows:
HttpChannelState._onContentAvailable
, and the invocation type checked against it directly.HttpChannelState.onContentAvailable()
is called, that wraps the application demand callback in a SerializedInvoker.Link
, whose InvocationType
is always BLOCKING
.Runnable
returned from HttpChannelState.onContentAvailable()
is then:
AdaptiveExecutionStrategy
, which will consume it accordingly to its invocation type, i.e. dispatch it because it is BLOCKING
.Example: request arrives, but no content yet; application registers a non-blocking demand callback; then:
Link
, but then it is run immediately despite being BLOCKING
(HttpConnection.DemandContentCallback
).Link
, and then given back to the AdaptiveExecutionStrategy
that it runs it as BLOCKING
which is going to be inefficient (HttpStreamOverHTTP2.onDataAvailable()
).@gregw the
SerializedInvoker
issue we discussed is as follows:
@sbordet So do you think we should write a simplified SerializedInvoker that only prevents infinite recursion but does not support queuing infinite jobs (only 0 or 1), so that it can correctly report an invocation type?
TODO:
Promise
implement Invocable
.Promise.Invocable
Invocable.InvocableCompletableFuture
.Invocable.ReadyTask
in more places where Invocable.Task
is used.TODO:
- Make
Promise
implementInvocable
.
I don't think we should. Not all Promises are Invocable, and this makes it clear in the signature when the caller has to think about InvocationType or not.
- Remove
Promise.Invocable
Keep it.
- Remove
Invocable.InvocableCompletableFuture
.
I'm 50:50 on this one. It has helped me a couple of times when I got the wrong invocation type during development.
- Use
Invocable.ReadyTask
in more places whereInvocable.Task
is used.
I find the name ReadyTask
meaningless and it does very little. Directly implementing Invocable.Task
is more readable. Also, we end up creating Runnable lambdas just to pass into the constructor of ReadyTask, so might as well just create a nested class anyway.
- Remove
Invocable.InvocableCompletableFuture
.I'm 50:50 on this one. It has helped me a couple of times when I got the wrong invocation type during development.
I've got a compromise. I've removed InvocableCompletableFuture
, but moved its methods to ContentSourceCompletableFuture
, which was the only use anyway. We have not yet deprecated CSCF as we do not have an alternative, so the extra protection in using it is worthwhile I think.
@sbordet Another compromise.... How about we introduce Invocable.AbstractTask
that implements the getInvocationType, but keeps run abstract so we don't have to create a runnable lambda just to pass into the ReadyTask constructor?
testReadStarvation
are broken in ee9, ee10 and core.They uncovered a problem with the core API not always correctly using
InvocationType
s and usingCompletableFuture
as a return type for async calls.