Mailing List Archive

SolrClient and making requests asynchronously
Hello devs,

I'd like to have an API design discussion here on the dev list concerning
adding asynchronous request semantics to SolrClient. I think the
SolrClient APIs require great peer-review consideration and so I'm having
this discussion here instead of a JIRA issue.

Background: Sometimes, a SolrJ user (including Solr itself) may wish to
issue a request but not block/wait for the response because it can do other
useful work before eventually needing the results. For example, maybe the
user has multiple requests to send to Solr that can be done in parallel and
_then_ process the results. This is in fact what Solr needs internally for
during distributed-search (see HttpShardHandler) and during
distributed-indexing (see ConcurrentUpdateHttp2SolrClient).

Why?: Greater efficiency. This can be accomplished easily in a generic
fashion with threads (e.g.
CompletableFuture.supplyAsync(Supplier,Executor)) with use of the
SolrClient concurrently. However, this can result in lots of threads and
context switching. I really like Cao Dat's explanation in the description
here: https://issues.apache.org/jira/browse/SOLR-14354 Thanks to HTTP/2,
there is a new approach in which the underlying HTTP client can handle the
asynchronous nature more efficiently with minimal threads. Complete
plumbing of this implies that SolrClient (and/or its derivatives) need a
similar async mechanism. Disclaimer: I haven't measured any of this but
am piggybacking off of Cao's's insights on SOLR-14354

Where?: An async API _could_ go only on some SolrClient classes that
natively support it, avoiding changing SolrClient itself. Maybe this is
what should occur first before "graduating" the method to SolrClient where
we devise a default approach, although I would prefer just putting it on
SolrClient. The default approach might be configured to throw
UnsupportedOperationException, or perhaps might simply use an Executor to
get it done in an obvious way (assuming we can get ahold of an Executor
somewhere?). If you're writing a delegating SolrClient (which I've done in
the past), you might want to take-care to delegate this method.
Another aspect of "Where" is whether SolrRequest should additionally have
an API alternative to "process" which is currently synchronous and calls
out to SolrClient.request(this).

What?: What should this API look like? This is my primary interest right
now and the meat of the discussion.

SolrClient's primary signature that actually does the work (synchronously)
is this:

public abstract NamedList<Object>
request(@SuppressWarnings({"rawtypes"})final SolrRequest request,
String collection)
throws SolrServerException, IOException;

I don't like the "collection" there as it's redundant with either the
configured SolrClient default or a setting in SolrRequest, but I digress
from the important matter at hand.

In SOLR-14354, recently committed by Cao Dat destined for Solr 8.7, there
is a new (undocumented) method on Http2SolrClient (*not* SolrClient):

public Cancellable asyncRequest(@SuppressWarnings({"rawtypes"})
SolrRequest solrRequest, String collection,
AsyncListener<NamedList<Object>> asyncListener) {

So firstly, it's only on Http2SolrClient, which means if you're using
CloudHttp2SolrClient (which does not subclass it, counterintuitively to
users) then too bad -- SOLR-14675 filed by my colleague.
Secondly, I really dislike this API signature (sorry Cao). It introduces
two custom and obscure Solr interfaces, Cancellable and AsyncListener. I
claim that we can and should use one abstraction provided by the JDK --
CompletableFuture<NamedList<Object>>. The user can cancel() it if they
like, they can use it like a Future which is maybe what they prefer, or
they can attach a listener / handler via whenComplete(lambda) that even has
access to an exception if there was one. Based on some toying around with
CompletableFuture lately, I suspect what would be most straight-forward is
to have an asyncRequest method that *returns* a CompletableFuture, and
which merely takes the SolrRequest parameter and nothing else.
Alternatively the client could supply a CompletableFuture parameter that
Solr will call complete() on etc. but that seems a bit less natural to the
notion of a method that returns it's results, albeit with a wrapper.
Proposal:

public CompletableFuture<NamedList<Object>>
requestAsync(SolrRequest<?> request);

BTW I'm trying to avoid implementation details here. My objective is to
devise a user-friendly API, and my hope is that the eventual implementation
is reasonable.

I hope I haven't bored you all to tears and that you find this public
discussion useful.

~ David Smiley
Apache Lucene/Solr Search Developer
http://www.linkedin.com/in/davidwsmiley
Re: SolrClient and making requests asynchronously [ In reply to ]
Thanks David for bringing this up.

I want us to stop using concrete classes. We should exclusively use
interfaces in our public APIs.

We should stop using NamedList and start using this interface

https://github.com/apache/lucene-solr/blob/b91b461632b19c9488926a9302126a2158df3298/solr/solrj/src/java/org/apache/solr/common/util/SimpleMap.java

NamedList will implement this interface too

The SolrRequest is an abstract class and it has a huge surface area.
We must create a simple small interface for this as will




On Sat, Aug 1, 2020 at 6:25 AM David Smiley <dsmiley@apache.org> wrote:
>
> Hello devs,
>
> I'd like to have an API design discussion here on the dev list concerning adding asynchronous request semantics to SolrClient. I think the SolrClient APIs require great peer-review consideration and so I'm having this discussion here instead of a JIRA issue.
>
> Background: Sometimes, a SolrJ user (including Solr itself) may wish to issue a request but not block/wait for the response because it can do other useful work before eventually needing the results. For example, maybe the user has multiple requests to send to Solr that can be done in parallel and _then_ process the results. This is in fact what Solr needs internally for during distributed-search (see HttpShardHandler) and during distributed-indexing (see ConcurrentUpdateHttp2SolrClient).
>
> Why?: Greater efficiency. This can be accomplished easily in a generic fashion with threads (e.g. CompletableFuture.supplyAsync(Supplier,Executor)) with use of the SolrClient concurrently. However, this can result in lots of threads and context switching. I really like Cao Dat's explanation in the description here: https://issues.apache.org/jira/browse/SOLR-14354 Thanks to HTTP/2, there is a new approach in which the underlying HTTP client can handle the asynchronous nature more efficiently with minimal threads. Complete plumbing of this implies that SolrClient (and/or its derivatives) need a similar async mechanism. Disclaimer: I haven't measured any of this but am piggybacking off of Cao's's insights on SOLR-14354
>
> Where?: An async API _could_ go only on some SolrClient classes that natively support it, avoiding changing SolrClient itself. Maybe this is what should occur first before "graduating" the method to SolrClient where we devise a default approach, although I would prefer just putting it on SolrClient. The default approach might be configured to throw UnsupportedOperationException, or perhaps might simply use an Executor to get it done in an obvious way (assuming we can get ahold of an Executor somewhere?). If you're writing a delegating SolrClient (which I've done in the past), you might want to take-care to delegate this method.
> Another aspect of "Where" is whether SolrRequest should additionally have an API alternative to "process" which is currently synchronous and calls out to SolrClient.request(this).
>
> What?: What should this API look like? This is my primary interest right now and the meat of the discussion.
>
> SolrClient's primary signature that actually does the work (synchronously) is this:
>
> public abstract NamedList<Object> request(@SuppressWarnings({"rawtypes"})final SolrRequest request, String collection)
> throws SolrServerException, IOException;
>
> I don't like the "collection" there as it's redundant with either the configured SolrClient default or a setting in SolrRequest, but I digress from the important matter at hand.
>
> In SOLR-14354, recently committed by Cao Dat destined for Solr 8.7, there is a new (undocumented) method on Http2SolrClient (*not* SolrClient):
>
> public Cancellable asyncRequest(@SuppressWarnings({"rawtypes"}) SolrRequest solrRequest, String collection, AsyncListener<NamedList<Object>> asyncListener) {
>
> So firstly, it's only on Http2SolrClient, which means if you're using CloudHttp2SolrClient (which does not subclass it, counterintuitively to users) then too bad -- SOLR-14675 filed by my colleague.
> Secondly, I really dislike this API signature (sorry Cao). It introduces two custom and obscure Solr interfaces, Cancellable and AsyncListener. I claim that we can and should use one abstraction provided by the JDK -- CompletableFuture<NamedList<Object>>. The user can cancel() it if they like, they can use it like a Future which is maybe what they prefer, or they can attach a listener / handler via whenComplete(lambda) that even has access to an exception if there was one. Based on some toying around with CompletableFuture lately, I suspect what would be most straight-forward is to have an asyncRequest method that *returns* a CompletableFuture, and which merely takes the SolrRequest parameter and nothing else. Alternatively the client could supply a CompletableFuture parameter that Solr will call complete() on etc. but that seems a bit less natural to the notion of a method that returns it's results, albeit with a wrapper. Proposal:
>
> public CompletableFuture<NamedList<Object>> requestAsync(SolrRequest<?> request);
>
> BTW I'm trying to avoid implementation details here. My objective is to devise a user-friendly API, and my hope is that the eventual implementation is reasonable.
>
> I hope I haven't bored you all to tears and that you find this public discussion useful.
>
> ~ David Smiley
> Apache Lucene/Solr Search Developer
> http://www.linkedin.com/in/davidwsmiley



--
-----------------------------------------------------
Noble Paul

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org
Re: SolrClient and making requests asynchronously [ In reply to ]
I would also wish to deprecate all the public APIs which uses concrete
classes and move over to interfaces.

The objectives are.
* We should be able to use POJOs that help with strong typing
* We should be able to easily switch between JSON/javabin/XML or even
well known protocols like protobuf

On Sat, Aug 1, 2020 at 12:02 PM Noble Paul <noble.paul@gmail.com> wrote:
>
> Thanks David for bringing this up.
>
> I want us to stop using concrete classes. We should exclusively use
> interfaces in our public APIs.
>
> We should stop using NamedList and start using this interface
>
> https://github.com/apache/lucene-solr/blob/b91b461632b19c9488926a9302126a2158df3298/solr/solrj/src/java/org/apache/solr/common/util/SimpleMap.java
>
> NamedList will implement this interface too
>
> The SolrRequest is an abstract class and it has a huge surface area.
> We must create a simple small interface for this as will
>
>
>
>
> On Sat, Aug 1, 2020 at 6:25 AM David Smiley <dsmiley@apache.org> wrote:
> >
> > Hello devs,
> >
> > I'd like to have an API design discussion here on the dev list concerning adding asynchronous request semantics to SolrClient. I think the SolrClient APIs require great peer-review consideration and so I'm having this discussion here instead of a JIRA issue.
> >
> > Background: Sometimes, a SolrJ user (including Solr itself) may wish to issue a request but not block/wait for the response because it can do other useful work before eventually needing the results. For example, maybe the user has multiple requests to send to Solr that can be done in parallel and _then_ process the results. This is in fact what Solr needs internally for during distributed-search (see HttpShardHandler) and during distributed-indexing (see ConcurrentUpdateHttp2SolrClient).
> >
> > Why?: Greater efficiency. This can be accomplished easily in a generic fashion with threads (e.g. CompletableFuture.supplyAsync(Supplier,Executor)) with use of the SolrClient concurrently. However, this can result in lots of threads and context switching. I really like Cao Dat's explanation in the description here: https://issues.apache.org/jira/browse/SOLR-14354 Thanks to HTTP/2, there is a new approach in which the underlying HTTP client can handle the asynchronous nature more efficiently with minimal threads. Complete plumbing of this implies that SolrClient (and/or its derivatives) need a similar async mechanism. Disclaimer: I haven't measured any of this but am piggybacking off of Cao's's insights on SOLR-14354
> >
> > Where?: An async API _could_ go only on some SolrClient classes that natively support it, avoiding changing SolrClient itself. Maybe this is what should occur first before "graduating" the method to SolrClient where we devise a default approach, although I would prefer just putting it on SolrClient. The default approach might be configured to throw UnsupportedOperationException, or perhaps might simply use an Executor to get it done in an obvious way (assuming we can get ahold of an Executor somewhere?). If you're writing a delegating SolrClient (which I've done in the past), you might want to take-care to delegate this method.
> > Another aspect of "Where" is whether SolrRequest should additionally have an API alternative to "process" which is currently synchronous and calls out to SolrClient.request(this).
> >
> > What?: What should this API look like? This is my primary interest right now and the meat of the discussion.
> >
> > SolrClient's primary signature that actually does the work (synchronously) is this:
> >
> > public abstract NamedList<Object> request(@SuppressWarnings({"rawtypes"})final SolrRequest request, String collection)
> > throws SolrServerException, IOException;
> >
> > I don't like the "collection" there as it's redundant with either the configured SolrClient default or a setting in SolrRequest, but I digress from the important matter at hand.
> >
> > In SOLR-14354, recently committed by Cao Dat destined for Solr 8.7, there is a new (undocumented) method on Http2SolrClient (*not* SolrClient):
> >
> > public Cancellable asyncRequest(@SuppressWarnings({"rawtypes"}) SolrRequest solrRequest, String collection, AsyncListener<NamedList<Object>> asyncListener) {
> >
> > So firstly, it's only on Http2SolrClient, which means if you're using CloudHttp2SolrClient (which does not subclass it, counterintuitively to users) then too bad -- SOLR-14675 filed by my colleague.
> > Secondly, I really dislike this API signature (sorry Cao). It introduces two custom and obscure Solr interfaces, Cancellable and AsyncListener. I claim that we can and should use one abstraction provided by the JDK -- CompletableFuture<NamedList<Object>>. The user can cancel() it if they like, they can use it like a Future which is maybe what they prefer, or they can attach a listener / handler via whenComplete(lambda) that even has access to an exception if there was one. Based on some toying around with CompletableFuture lately, I suspect what would be most straight-forward is to have an asyncRequest method that *returns* a CompletableFuture, and which merely takes the SolrRequest parameter and nothing else. Alternatively the client could supply a CompletableFuture parameter that Solr will call complete() on etc. but that seems a bit less natural to the notion of a method that returns it's results, albeit with a wrapper. Proposal:
> >
> > public CompletableFuture<NamedList<Object>> requestAsync(SolrRequest<?> request);
> >
> > BTW I'm trying to avoid implementation details here. My objective is to devise a user-friendly API, and my hope is that the eventual implementation is reasonable.
> >
> > I hope I haven't bored you all to tears and that you find this public discussion useful.
> >
> > ~ David Smiley
> > Apache Lucene/Solr Search Developer
> > http://www.linkedin.com/in/davidwsmiley
>
>
>
> --
> -----------------------------------------------------
> Noble Paul



--
-----------------------------------------------------
Noble Paul

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org
Re: SolrClient and making requests asynchronously [ In reply to ]
Noble: Those sound like some big and bold ideas that are deserving of a
separate conversation? My proposal here is based on the APIs we have today.

~ David


On Fri, Jul 31, 2020 at 10:06 PM Noble Paul <noble.paul@gmail.com> wrote:

> I would also wish to deprecate all the public APIs which uses concrete
> classes and move over to interfaces.
>
> The objectives are.
> * We should be able to use POJOs that help with strong typing
> * We should be able to easily switch between JSON/javabin/XML or even
> well known protocols like protobuf
>
> On Sat, Aug 1, 2020 at 12:02 PM Noble Paul <noble.paul@gmail.com> wrote:
> >
> > Thanks David for bringing this up.
> >
> > I want us to stop using concrete classes. We should exclusively use
> > interfaces in our public APIs.
> >
> > We should stop using NamedList and start using this interface
> >
> >
> https://github.com/apache/lucene-solr/blob/b91b461632b19c9488926a9302126a2158df3298/solr/solrj/src/java/org/apache/solr/common/util/SimpleMap.java
> >
> > NamedList will implement this interface too
> >
> > The SolrRequest is an abstract class and it has a huge surface area.
> > We must create a simple small interface for this as will
> >
> >
> >
> >
> > On Sat, Aug 1, 2020 at 6:25 AM David Smiley <dsmiley@apache.org> wrote:
> > >
> > > Hello devs,
> > >
> > > I'd like to have an API design discussion here on the dev list
> concerning adding asynchronous request semantics to SolrClient. I think
> the SolrClient APIs require great peer-review consideration and so I'm
> having this discussion here instead of a JIRA issue.
> > >
> > > Background: Sometimes, a SolrJ user (including Solr itself) may wish
> to issue a request but not block/wait for the response because it can do
> other useful work before eventually needing the results. For example,
> maybe the user has multiple requests to send to Solr that can be done in
> parallel and _then_ process the results. This is in fact what Solr needs
> internally for during distributed-search (see HttpShardHandler) and during
> distributed-indexing (see ConcurrentUpdateHttp2SolrClient).
> > >
> > > Why?: Greater efficiency. This can be accomplished easily in a
> generic fashion with threads (e.g.
> CompletableFuture.supplyAsync(Supplier,Executor)) with use of the
> SolrClient concurrently. However, this can result in lots of threads and
> context switching. I really like Cao Dat's explanation in the description
> here: https://issues.apache.org/jira/browse/SOLR-14354 Thanks to HTTP/2,
> there is a new approach in which the underlying HTTP client can handle the
> asynchronous nature more efficiently with minimal threads. Complete
> plumbing of this implies that SolrClient (and/or its derivatives) need a
> similar async mechanism. Disclaimer: I haven't measured any of this but
> am piggybacking off of Cao's's insights on SOLR-14354
> > >
> > > Where?: An async API _could_ go only on some SolrClient classes that
> natively support it, avoiding changing SolrClient itself. Maybe this is
> what should occur first before "graduating" the method to SolrClient where
> we devise a default approach, although I would prefer just putting it on
> SolrClient. The default approach might be configured to throw
> UnsupportedOperationException, or perhaps might simply use an Executor to
> get it done in an obvious way (assuming we can get ahold of an Executor
> somewhere?). If you're writing a delegating SolrClient (which I've done in
> the past), you might want to take-care to delegate this method.
> > > Another aspect of "Where" is whether SolrRequest should additionally
> have an API alternative to "process" which is currently synchronous and
> calls out to SolrClient.request(this).
> > >
> > > What?: What should this API look like? This is my primary interest
> right now and the meat of the discussion.
> > >
> > > SolrClient's primary signature that actually does the work
> (synchronously) is this:
> > >
> > > public abstract NamedList<Object>
> request(@SuppressWarnings({"rawtypes"})final SolrRequest request, String
> collection)
> > > throws SolrServerException, IOException;
> > >
> > > I don't like the "collection" there as it's redundant with either the
> configured SolrClient default or a setting in SolrRequest, but I digress
> from the important matter at hand.
> > >
> > > In SOLR-14354, recently committed by Cao Dat destined for Solr 8.7,
> there is a new (undocumented) method on Http2SolrClient (*not* SolrClient):
> > >
> > > public Cancellable asyncRequest(@SuppressWarnings({"rawtypes"})
> SolrRequest solrRequest, String collection,
> AsyncListener<NamedList<Object>> asyncListener) {
> > >
> > > So firstly, it's only on Http2SolrClient, which means if you're using
> CloudHttp2SolrClient (which does not subclass it, counterintuitively to
> users) then too bad -- SOLR-14675 filed by my colleague.
> > > Secondly, I really dislike this API signature (sorry Cao). It
> introduces two custom and obscure Solr interfaces, Cancellable and
> AsyncListener. I claim that we can and should use one abstraction provided
> by the JDK -- CompletableFuture<NamedList<Object>>. The user can cancel()
> it if they like, they can use it like a Future which is maybe what they
> prefer, or they can attach a listener / handler via whenComplete(lambda)
> that even has access to an exception if there was one. Based on some
> toying around with CompletableFuture lately, I suspect what would be most
> straight-forward is to have an asyncRequest method that *returns* a
> CompletableFuture, and which merely takes the SolrRequest parameter and
> nothing else. Alternatively the client could supply a CompletableFuture
> parameter that Solr will call complete() on etc. but that seems a bit less
> natural to the notion of a method that returns it's results, albeit with a
> wrapper. Proposal:
> > >
> > > public CompletableFuture<NamedList<Object>>
> requestAsync(SolrRequest<?> request);
> > >
> > > BTW I'm trying to avoid implementation details here. My objective is
> to devise a user-friendly API, and my hope is that the eventual
> implementation is reasonable.
> > >
> > > I hope I haven't bored you all to tears and that you find this public
> discussion useful.
> > >
> > > ~ David Smiley
> > > Apache Lucene/Solr Search Developer
> > > http://www.linkedin.com/in/davidwsmiley
> >
> >
> >
> > --
> > -----------------------------------------------------
> > Noble Paul
>
>
>
> --
> -----------------------------------------------------
> Noble Paul
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
> For additional commands, e-mail: dev-help@lucene.apache.org
>
>
Re: SolrClient and making requests asynchronously [ In reply to ]
The new API seems good to me. it does be able to handle all cases : future,
listener and error handling.

On Sat, Aug 1, 2020 at 10:15 AM David Smiley <david.w.smiley@gmail.com>
wrote:

> Noble: Those sound like some big and bold ideas that are deserving of a
> separate conversation? My proposal here is based on the APIs we have today.
>
> ~ David
>
>
> On Fri, Jul 31, 2020 at 10:06 PM Noble Paul <noble.paul@gmail.com> wrote:
>
>> I would also wish to deprecate all the public APIs which uses concrete
>> classes and move over to interfaces.
>>
>> The objectives are.
>> * We should be able to use POJOs that help with strong typing
>> * We should be able to easily switch between JSON/javabin/XML or even
>> well known protocols like protobuf
>>
>> On Sat, Aug 1, 2020 at 12:02 PM Noble Paul <noble.paul@gmail.com> wrote:
>> >
>> > Thanks David for bringing this up.
>> >
>> > I want us to stop using concrete classes. We should exclusively use
>> > interfaces in our public APIs.
>> >
>> > We should stop using NamedList and start using this interface
>> >
>> >
>> https://github.com/apache/lucene-solr/blob/b91b461632b19c9488926a9302126a2158df3298/solr/solrj/src/java/org/apache/solr/common/util/SimpleMap.java
>> >
>> > NamedList will implement this interface too
>> >
>> > The SolrRequest is an abstract class and it has a huge surface area.
>> > We must create a simple small interface for this as will
>> >
>> >
>> >
>> >
>> > On Sat, Aug 1, 2020 at 6:25 AM David Smiley <dsmiley@apache.org> wrote:
>> > >
>> > > Hello devs,
>> > >
>> > > I'd like to have an API design discussion here on the dev list
>> concerning adding asynchronous request semantics to SolrClient. I think
>> the SolrClient APIs require great peer-review consideration and so I'm
>> having this discussion here instead of a JIRA issue.
>> > >
>> > > Background: Sometimes, a SolrJ user (including Solr itself) may wish
>> to issue a request but not block/wait for the response because it can do
>> other useful work before eventually needing the results. For example,
>> maybe the user has multiple requests to send to Solr that can be done in
>> parallel and _then_ process the results. This is in fact what Solr needs
>> internally for during distributed-search (see HttpShardHandler) and during
>> distributed-indexing (see ConcurrentUpdateHttp2SolrClient).
>> > >
>> > > Why?: Greater efficiency. This can be accomplished easily in a
>> generic fashion with threads (e.g.
>> CompletableFuture.supplyAsync(Supplier,Executor)) with use of the
>> SolrClient concurrently. However, this can result in lots of threads and
>> context switching. I really like Cao Dat's explanation in the description
>> here: https://issues.apache.org/jira/browse/SOLR-14354 Thanks to
>> HTTP/2, there is a new approach in which the underlying HTTP client can
>> handle the asynchronous nature more efficiently with minimal threads.
>> Complete plumbing of this implies that SolrClient (and/or its derivatives)
>> need a similar async mechanism. Disclaimer: I haven't measured any of
>> this but am piggybacking off of Cao's's insights on SOLR-14354
>> > >
>> > > Where?: An async API _could_ go only on some SolrClient classes that
>> natively support it, avoiding changing SolrClient itself. Maybe this is
>> what should occur first before "graduating" the method to SolrClient where
>> we devise a default approach, although I would prefer just putting it on
>> SolrClient. The default approach might be configured to throw
>> UnsupportedOperationException, or perhaps might simply use an Executor to
>> get it done in an obvious way (assuming we can get ahold of an Executor
>> somewhere?). If you're writing a delegating SolrClient (which I've done in
>> the past), you might want to take-care to delegate this method.
>> > > Another aspect of "Where" is whether SolrRequest should additionally
>> have an API alternative to "process" which is currently synchronous and
>> calls out to SolrClient.request(this).
>> > >
>> > > What?: What should this API look like? This is my primary interest
>> right now and the meat of the discussion.
>> > >
>> > > SolrClient's primary signature that actually does the work
>> (synchronously) is this:
>> > >
>> > > public abstract NamedList<Object>
>> request(@SuppressWarnings({"rawtypes"})final SolrRequest request, String
>> collection)
>> > > throws SolrServerException, IOException;
>> > >
>> > > I don't like the "collection" there as it's redundant with either the
>> configured SolrClient default or a setting in SolrRequest, but I digress
>> from the important matter at hand.
>> > >
>> > > In SOLR-14354, recently committed by Cao Dat destined for Solr 8.7,
>> there is a new (undocumented) method on Http2SolrClient (*not* SolrClient):
>> > >
>> > > public Cancellable asyncRequest(@SuppressWarnings({"rawtypes"})
>> SolrRequest solrRequest, String collection,
>> AsyncListener<NamedList<Object>> asyncListener) {
>> > >
>> > > So firstly, it's only on Http2SolrClient, which means if you're using
>> CloudHttp2SolrClient (which does not subclass it, counterintuitively to
>> users) then too bad -- SOLR-14675 filed by my colleague.
>> > > Secondly, I really dislike this API signature (sorry Cao). It
>> introduces two custom and obscure Solr interfaces, Cancellable and
>> AsyncListener. I claim that we can and should use one abstraction provided
>> by the JDK -- CompletableFuture<NamedList<Object>>. The user can cancel()
>> it if they like, they can use it like a Future which is maybe what they
>> prefer, or they can attach a listener / handler via whenComplete(lambda)
>> that even has access to an exception if there was one. Based on some
>> toying around with CompletableFuture lately, I suspect what would be most
>> straight-forward is to have an asyncRequest method that *returns* a
>> CompletableFuture, and which merely takes the SolrRequest parameter and
>> nothing else. Alternatively the client could supply a CompletableFuture
>> parameter that Solr will call complete() on etc. but that seems a bit less
>> natural to the notion of a method that returns it's results, albeit with a
>> wrapper. Proposal:
>> > >
>> > > public CompletableFuture<NamedList<Object>>
>> requestAsync(SolrRequest<?> request);
>> > >
>> > > BTW I'm trying to avoid implementation details here. My objective is
>> to devise a user-friendly API, and my hope is that the eventual
>> implementation is reasonable.
>> > >
>> > > I hope I haven't bored you all to tears and that you find this public
>> discussion useful.
>> > >
>> > > ~ David Smiley
>> > > Apache Lucene/Solr Search Developer
>> > > http://www.linkedin.com/in/davidwsmiley
>> >
>> >
>> >
>> > --
>> > -----------------------------------------------------
>> > Noble Paul
>>
>>
>>
>> --
>> -----------------------------------------------------
>> Noble Paul
>>
>> ---------------------------------------------------------------------
>> To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
>> For additional commands, e-mail: dev-help@lucene.apache.org
>>
>>

--
*Best regards,*
*Cao M?nh ??t*
*E-mail: caomanhdat317@gmail.com <caomanhdat317@gmail.com>*
Re: SolrClient and making requests asynchronously [ In reply to ]
I am +1 to this approach. Some thoughts inline.

How would query timeout be respected in this approach?

>

The default approach might be configured to throw
> UnsupportedOperationException, or perhaps might simply use an Executor to
> get it done in an obvious way (assuming we can get ahold of an Executor
> somewhere?).
>

Would that mean that we use an Executor to execute a single thread?


>>
CompletableFuture, and which merely takes the SolrRequest parameter and
nothing else. Alternatively the client could supply a CompletableFuture
parameter that Solr will call complete() on etc. but that seems a bit less
natural to the notion of a method that returns it's results, albeit with a
wrapper.

I would think that we allow users to specify their callback. One of the
advantages of AsyncListener is that it a custom implementation can allow
users to handle the behaviour of timeout and other events. We should retain
that behaviour.
--
Regards,

Atri
Apache Concerted
Re: SolrClient and making requests asynchronously [ In reply to ]
+1 I and a few of my former Lucidworks colleagues know the pain of the
synchronous client very well.

Thanks for the first step toward an improved design Dat and David Smiley!

Marcus
On Sun, Aug 2, 2020 at 07:47 Atri Sharma <atri@apache.org> wrote:

> I am +1 to this approach. Some thoughts inline.
>
> How would query timeout be respected in this approach?
>
>>
>
> The default approach might be configured to throw
>> UnsupportedOperationException, or perhaps might simply use an Executor to
>> get it done in an obvious way (assuming we can get ahold of an Executor
>> somewhere?).
>>
>
> Would that mean that we use an Executor to execute a single thread?
>
>
> >>
> CompletableFuture, and which merely takes the SolrRequest parameter and
> nothing else. Alternatively the client could supply a CompletableFuture
> parameter that Solr will call complete() on etc. but that seems a bit less
> natural to the notion of a method that returns it's results, albeit with a
> wrapper.
>
> I would think that we allow users to specify their callback. One of the
> advantages of AsyncListener is that it a custom implementation can allow
> users to handle the behaviour of timeout and other events. We should retain
> that behaviour.
> --
> Regards,
>
> Atri
> Apache Concerted
>
--
Marcus Eagan
Re: SolrClient and making requests asynchronously [ In reply to ]
On Sun, Aug 2, 2020 at 10:47 AM Atri Sharma <atri@apache.org> wrote:

> I am +1 to this approach. Some thoughts inline.
>
> How would query timeout be respected in this approach?
>

The backend would need to "complete exceptionally" with a timeout
exception we think is appropriate. It'd call
completableFuture.completeExceptionally(exception).


>
> The default approach might be configured to throw
>> UnsupportedOperationException, or perhaps might simply use an Executor to
>> get it done in an obvious way (assuming we can get ahold of an Executor
>> somewhere?).
>>
>
> Would that mean that we use an Executor to execute a single thread?
>

Either I don't know what you mean, or I suspect that you think I'm
referring to the synchronous (not asynchronous) code path), which I am
not. I am referring to what a proposed SolrClient.requestAsync should do
if the SolrClient impl doesn't have an underlying asynchronous mechanism.
I think I'm leaning towards using a configured executor, and if you don't
supply one then default to JDK ForkJoinPool.commonPool(). I am not
proposing changing SolrClient.request() (a synchronous method) to pass off
work to another thread (in an executor) unless it's already doing that for
some implementations (e.g. ConcurrentUpdateHttp2SolrClient).


> >>
> CompletableFuture, and which merely takes the SolrRequest parameter and
> nothing else. Alternatively the client could supply a CompletableFuture
> parameter that Solr will call complete() on etc. but that seems a bit less
> natural to the notion of a method that returns it's results, albeit with a
> wrapper.
>
> I would think that we allow users to specify their callback. One of the
> advantages of AsyncListener is that it a custom implementation can allow
> users to handle the behaviour of timeout and other events. We should retain
> that behaviour.
>

CompletableFuture is a swiss army knife. If a client wants to provide an
async callback, it can do that via the CompletableFuture *returned* by the
proposed API via completableFuture.whenComplete on the result. Code
example:

public static void main(String[] args) {
final CompletableFuture<String> future = requestAsync();
future.whenComplete((result, throwable) -> {
System.out.println("Handler executed; got: " + result + "
throwable: " + throwable);
});
}

public static CompletableFuture<String> requestAsync() {
final CompletableFuture<String> future = new CompletableFuture<>();
new Thread(() -> {
System.out.println("Starting async work");
try {
Thread.sleep(1000);
} catch (InterruptedException e) {
e.printStackTrace();
}
System.out.println("Ending async work");
future.complete("hello world");
}).start();
return future;
}

The code will print the following (after some other stuff) before finally
exiting:
"Handler executed; got: hello world throwable: null"

The caller/client is also able to cancel the operation, and the backend can
have its own handler to react just-in-time without polling.

~ David

--
> Regards,
>
> Atri
> Apache Concerted
>
Re: SolrClient and making requests asynchronously [ In reply to ]
> public CompletableFuture<NamedList<Object>>
requestAsync(SolrRequest<?> request);

NamedList<Object> aside, this looks like a great async API to me. But I
would still like some control over the thread pool/executor that's being
used.

Maybe that doesn't have to be part of the requestAysnc method signature,
maybe it could be part of the SolrClient constructor/builder, but then
maybe I'll want a different one for reads and writes? So maybe an
overloaded version of requestAsync with an extra paramater for a thread
pool/executor?

- Bram

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org
Re: SolrClient and making requests asynchronously [ In reply to ]
Maybe the method could be overloaded with an executor... there's a balance
between expert control and simplicity. Shrug.

~ David Smiley
Apache Lucene/Solr Search Developer
http://www.linkedin.com/in/davidwsmiley


On Thu, Aug 6, 2020 at 10:56 AM Bram Van Dam <bram.vandam@intix.eu> wrote:

> > public CompletableFuture<NamedList<Object>>
> requestAsync(SolrRequest<?> request);
>
> NamedList<Object> aside, this looks like a great async API to me. But I
> would still like some control over the thread pool/executor that's being
> used.
>
> Maybe that doesn't have to be part of the requestAysnc method signature,
> maybe it could be part of the SolrClient constructor/builder, but then
> maybe I'll want a different one for reads and writes? So maybe an
> overloaded version of requestAsync with an extra paramater for a thread
> pool/executor?
>
> - Bram
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
> For additional commands, e-mail: dev-help@lucene.apache.org
>
>