> > What I mean here is right now, when we send a batch of documents to Solr. > We still process it as concrete - unrelated documents by indexing one by > one. If indexing the fifth document causing error, that won't affect > already indexed 4 documents. Using this model we can index the batch in an > atomic way. >
It sounds like you are suggesting an alternative version of TolerantUpdateProcessorFactory (would look nothing like it though) that sends each document forward in its own thread concurrently instead of serially/synchronously? If true, I'm quite supportive of this -- Solr should just work this way IMO. It could speed up some use-cases tremendously and also cap abuse of too many client indexing requests (people who unwittingly send batches asynchronously/concurrently from a queue, possibly flooding Solr). I remember I discussed this proposal with Mikhail many years ago at a Lucene/Solr Revolution conference in Washington DC (the first one held there?), as a way to speed up all indexing, not just DIH (which was his curiosity at the time). He wasn't a fan. I'll CC him to see if it jogs his memory :-) ~ David Smiley Apache Lucene/Solr Search Developer http://www.linkedin.com/in/davidwsmiley On Thu, Oct 15, 2020 at 6:29 AM Đạt Cao Mạnh <caomanhdat...@gmail.com> wrote: > > The biggest problem I have with this is that the client doesn't know > about indexing problems without awkward callbacks later to see if something > went wrong. Even simple stuff like a schema problem (e.g. undefined > field). It's a useful *option*, any way. > > Currently we now guarantee that if solr sends you an OK response the > document WILL eventually become searchable without further action. > Maintaining that guarantee becomes impossible if we haven't verified that > the data is formatted correctly (i.e. dates are in ISO format, etc). > This may be an acceptable cost for those opting for async indexing but it > may be very hard for some folks to swallow if it became the only option > however. > > I don't mean that we're gonna replace the sync way. Plus the way sync > works can be changed to leverage the async method. For example, the sync > thread basically waits until the indexer threads finish indexing that > update. Some notice here is all the validations in update processors will > still happen in *sync* way in this model. Only indexing part to Lucene is > executed in an async way. > > > I'm a bit skeptical that would boost indexing performance. Please > understand the intent of that API is about transactionality (atomic add) > and ensuring all docs go in the same segment. Solr *does* use that API for > nested / parent-child documents, and because it has to. If that API were > to get called for normal docs, I could see the configured indexing buffer > RAM or doc limit could be exceeded substantially. Perhaps not a big deal. > You could test your performance theory on a hacked Solr without much > modifications, I think? Just buffer then send in bulk. > >> >> >> What I mean here is right now, when we send a batch of documents to Solr. > We still process it as concrete - unrelated documents by indexing one by > one. If indexing the fifth document causing error, that won't affect > already indexed 4 documents. Using this model we can index the batch in an > atomic way. > > > I think that's the real thrust of your motivations, and sounds good to > me! Also, please see https://issues.apache.org/jira/browse/SOLR-14778 for > making the optionality of the updateLog be a better supportable option in > SolrCloud. > > Thank you for bring it out, I will take a look. > > All other parts that did not get quoted are very valuable things and I > really appreciate those. > > On Wed, Oct 14, 2020 at 12:06 AM Gus Heck <gus.h...@gmail.com> wrote: > >> This is interesting, though it opens a few of cans of worms IMHO. >> >> 1. Currently we now guarantee that if solr sends you an OK response >> the document WILL eventually become searchable without further action. >> Maintaining that guarantee becomes impossible if we haven't verified that >> the data is formatted correctly (i.e. dates are in ISO format, etc). >> This may be an acceptable cost for those opting for async indexing but it >> may be very hard for some folks to swallow if it became the only option >> however. >> 2. In the case of errors we need to hold the error message >> indefinitely for later discovery by the client, this needs to not >> accumulate forever. Thus: >> 1. We have a timed cleanup, leasing or some other self limiting >> pattern... possibly by indexing the failures in a TRA with autodelete >> so >> that clients can efficiently find the status of the particular >> document(s) >> they sent, obviouysly there's at least an asyc id involved, probably >> the >> uniqueKey (where available) and timestamps for recieved, and processed >> as >> well. >> 2. We log more simply with a sequential id and let clients keep >> track of what they have seen... This can lead us down the path of >> re-inventing kafka, or making kafka a required dependency. >> 3. We provide a push oriented connection (websocket? HTTP2?) that >> clients that care about failures can listen to and store nothing. A >> less >> appetizing variant is to publish errors to a message bus. >> 3. If we have more than one thread picking up the submitted documents >> and writing them, we need a state machine that identifies in-progress >> documents to prevent multiple pickups and resets processing to new on >> startup to ensure we don't index the same document twice and don't lose >> things that were in-flight on power loss. >> 4. Backpressure/throttling. If we're losing ground continuously on >> the submissions because indexing is heavier than accepting documents, we >> may fill up the disk. Of course the index itself can do that, but need to >> think about if this makes it worse. >> >> A big plus to this however is that batches with errors could optionally >> just omit the (one or two?) errored document(s) and publish the error for >> each errored document rather than failing the whole batch, meaning that the >> indexing infrastructure submitting in batches doesn't have to leave several >> hundred docs unprocessed, or alternately do a slow doc at a time resubmit >> to weed out the offenders. >> >> Certainly the involvement of kafka sounds interesting. If one persists to >> an externally addressable location like a kafka queue one might leave the >> option for the write-on-receipt queue to be different from the >> read-to-actually-index queue and put a pipeline behind solr instead of >> infront of it... possibly atomic updates could then be given identical >> processing as initial indexing.... >> >> On Sat, Oct 10, 2020 at 12:41 AM David Smiley <dsmi...@apache.org> wrote: >> >>> >>> >>> On Thu, Oct 8, 2020 at 10:21 AM Cao Mạnh Đạt <da...@apache.org> wrote: >>> >>>> Hi guys, >>>> >>>> First of all it seems that I used the term async a lot recently :D. >>>> Recently I have been thinking a lot about changing the current indexing >>>> model of Solr from sync way like currently (user submit an update request >>>> waiting for response). What about changing it to async model, where nodes >>>> will only persist the update into tlog then return immediately much like >>>> what tlog is doing now. Then we have a dedicated executor which reads from >>>> tlog to do indexing (producer consumer model with tlog acting like the >>>> queue). >>>> >>> >>> The biggest problem I have with this is that the client doesn't know >>> about indexing problems without awkward callbacks later to see if something >>> went wrong. Even simple stuff like a schema problem (e.g. undefined >>> field). It's a useful *option*, any way. >>> >>> >>>> >>>> I do see several big benefits of this approach >>>> >>>> - We can batching updates in a single call, right now we do not use >>>> writer.add(documents) api from lucene, by batching updates this gonna >>>> boost >>>> the performance of indexing >>>> >>>> I'm a bit skeptical that would boost indexing performance. Please >>> understand the intent of that API is about transactionality (atomic add) >>> and ensuring all docs go in the same segment. Solr *does* use that API for >>> nested / parent-child documents, and because it has to. If that API were >>> to get called for normal docs, I could see the configured indexing buffer >>> RAM or doc limit could be exceeded substantially. Perhaps not a big deal. >>> You could test your performance theory on a hacked Solr without much >>> modifications, I think? Just buffer then send in bulk. >>> >>>> >>>> - One common problems with Solr now is we have lot of threads doing >>>> indexing so that can ends up with many small segments. Using this model >>>> we >>>> can have bigger segments so less merge cost >>>> >>>> This is app/use-case dependent of course. If you observe the segment >>> count to be high, I think it's more likely due to a sub-optimal commit >>> strategy. Many users should put more thought into this. If update >>> requests to Solr have a small number of docs and it explicitly gives a >>> commit (commitWithin on the other hand is fine), then this is a recipe for >>> small segments and is generally expensive as well (commits are kinda >>> expensive). Many apps would do well to either pass commitWithin or rely on >>> a configured commitWithin, accomplishing the same instead of commit. For >>> apps that can't do that (e.g. need to immediately read-your-write or for >>> other reasons where I work), then such an app can't use async any way. >>> >>> An idea I've had to help throughput for this case would be for commits >>> that are about to happen concurrently with other indexing to voluntarily >>> wait a short period of time (500ms?) in an attempt to coalesce the commit >>> needs of both concurrent indexing requests. Very opt-in, probably a >>> solrconfig value, and probably a wait time in the 100s of milliseconds >>> range. An ideal implementation would be conservative to avoid this waiting >>> if there is no concurrent indexing request that did not start after the >>> current request or that which doesn't require a commit as well. >>> >>> If your goal is fewer segments, then definitely check out the recent >>> improvements to Lucene to do some lightweight merge-on-commit. The >>> Solr-side hook is SOLR-14582 >>> <https://issues.apache.org/jira/browse/SOLR-14582> and it requires a >>> custom MergePolicy. I contributed such a MergePolicy policy here: >>> https://github.com/apache/lucene-solr/pull/1222 although it needs to be >>> updated in light of Lucene changes that occured since then. We've been >>> using that MergePolicy at Salesforce for a couple years and it has cut our >>> segment count in half! Of course if you can generate fewer segments in the >>> first place, that's preferable and is more to your point. >>> >>>> >>>> - Another huge reason here is after switching to this model, we can >>>> remove tlog and use a distributed queue like Kafka, Pulsar. Since the >>>> purpose of leader in SolrCloud now is ordering updates, the distributed >>>> queue is already ordering updates for us, so no need to have a dedicated >>>> leader. That is just the beginning of things that we can do after using >>>> a >>>> distributed queue. >>>> >>>> I think that's the real thrust of your motivations, and sounds good to >>> me! Also, please see https://issues.apache.org/jira/browse/SOLR-14778 >>> for making the optionality of the updateLog be a better supportable option >>> in SolrCloud. >>> >>> >>>> What do your guys think about this? Just want to hear from your guys >>>> before going deep into this rabbit hole. >>>> >>>> Thanks! >>>> >>>> >> >> -- >> http://www.needhamsoftware.com (work) >> http://www.the111shift.com (play) >> > > > -- > *Best regards,* > *Cao Mạnh Đạt* > *E-mail: caomanhdat...@gmail.com <caomanhdat...@gmail.com>* >