Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

SOLR-13350: Multithreaded search #2248

Closed
wants to merge 66 commits into from
Closed

SOLR-13350: Multithreaded search #2248

wants to merge 66 commits into from

Conversation

chatman
Copy link
Contributor

@chatman chatman commented Feb 6, 2024

Abandoning previous PRs to create this new one.

https://issues.apache.org/jira/browse/SOLR-13350

@github-actions github-actions bot added the documentation Improvements or additions to documentation label Feb 6, 2024
new SolrNamedThreadFactory("searcherCollector"));
this.collectorExecutor =
ExecutorUtil.newMDCAwareCachedThreadPool(
6, new SolrNamedThreadFactory("searcherCollector"));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

1/2 I added dbd2d8a commit to tentatively make this configurable. Hope you don't mind.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @cpoerschke for your review! This is something I'm still wondering about. I'll post a comment on JIRA about the number of threads.

Comment on lines 445 to 446
cfg.getIndexSearcherExecutorThreads(), // thread count
cfg.getIndexSearcherExecutorThreads(), // queue size
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Contributor

@dsmiley dsmiley left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seems a draft with all the logging and some commented code but anyway it's nice to see this!

@@ -46,6 +46,7 @@ public void setDocSet(DocSet set) {
docListAndSet = new DocListAndSet();
}
docListAndSet.docSet = set;
// log.error("set docset {}", docListAndSet.docSet.getBits().length());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

should remove

}

/** Wraps multiple collectors for processing */
public class Collectors implements Collector {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

not sure yet if these inner classes need to be public.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

not sure yet if these inner classes need to be public.

3e66068 to reduce visibility -- implements CollectorManager<SolrMultiCollectorManager.Collectors, Object[]> says SolrMultiCollectorManager.Collectors has private access in SolrMultiCollectorManager if it's private but package visibility is okay and LeafCollectors can be private.

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/** */
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
/** */
/** A Lucene Collector collecting docs in a thread-safe manner. */

}

public DocSet getDocSet() {
log.error("Max Set Bit {}", bits.maxSetBit());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

error?


doc += base;
if (log.isErrorEnabled()) {
log.error("collect doc: {}, base: {}", doc, base, this);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

error?

}
ScoreMode scoreModeUsed =
buildAndRunCollectorChain(qr, query, collector, cmd, pf.postFilter).scoreMode();
MTCollectorQueryCheck allowMT = new MTCollectorQueryCheck();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Bunch of code being added in two places looking similar. Have you thought of attempting to factor out a common approach?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

c69006a and 1ec21d4 factored something out. Not thought much about naming in doing so and not specifically in response to this comment but just from my code reading/learning/reviewing perspective.

}

populateNextCursorMarkFromTopDocs(qr, cmd, topDocs);
// if (cmd.getSort() != null && !(cmd.getQuery() instanceof RankQuery) &&
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Forgot some logic or...?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

cb3c1cb uncomments some of it.

import org.apache.lucene.search.ScoreMode;

/**
* A {@link CollectorManager} implements which wrap a set of {@link CollectorManager} as {@link
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
* A {@link CollectorManager} implements which wrap a set of {@link CollectorManager} as {@link
* A {@link CollectorManager} implementation which wrap a set of {@link CollectorManager} as {@link

* MultiCollector} acts for {@link Collector}.
*/
public class SolrMultiCollectorManager
implements CollectorManager<SolrMultiCollectorManager.Collectors, Object[]> {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm; Object? Do we not know the type?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Lucene has a public class MultiCollectorManager implements CollectorManager<Collector, Object[]> i.e. https://github.com/apache/lucene/blob/releases/lucene/9.9.2/lucene/core/src/java/org/apache/lucene/search/MultiCollectorManager.java and I'm curious if there'd be a way to code share with that, though so far I can't see a way there.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks like you removed a symlink?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks like you removed a symlink?

Restored in 08d415f commit.

@dsmiley
Copy link
Contributor

dsmiley commented Feb 14, 2024

Observation: I don't think we need the concurrent DocSet builder (the one from Netflix). Couldn't we build segment level FixedBitSets (no safety issue) and then at the end combine into a master FixedBitSet so that we have our DocSet? It could even be made to be long-aligned, and thus the final aggregation is just copying longs with System.arraycopy (no doc iteration! or long shifting). The boundary long would be XOR.

}
log.error("new docset collector for {} max={}", numDocs, maxDoc());

return new ThreadSafeBitSetCollector(bits, maxDoc);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not sure yet if a firstCollectors[2] = ... was intended here or if not we can reduce to new Collector[2] above it seems.

Comment on lines 2075 to 2083
for (Collector collector : firstCollectors) {
if (collector != null) {
if (scoreMode == null) {
scoreMode = collector.scoreMode();
} else if (scoreMode != collector.scoreMode()) {
scoreMode = ScoreMode.COMPLETE;
}
}
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this needs to happen after the super.search call since at this point here the collectors will still all be null? 2501f7b to tentatively move it.

For new clusters, the solr.xml contains `indexSearcherExecutorThreads` as 4, and hence 4 threads will be used for search.
However, for older setups that upgrade to this version, the `indexSearcherExecutorThreads` defaults to 1 when unspecified in the solr.xml.
@chatman
Copy link
Contributor Author

chatman commented May 6, 2024

I think that's an anti-pattern or broken and isn't what I meant in JIRA. We could use a SynchronousQueue (with fairness) if we want to block for a thread -- probably what we should do. FYI that queue is the default for Executors.newCachedThreadPool(). The "caller runs" behavior I meant could be done via an ExecutorService delegate that catches RejectedException and simply runs the Runnable.

@dsmiley instead of using a rejected tasks execution handler, I went with @noblepaul 's suggestion of having a reasonably large queue for the threadpool (number of threads * 1000). Beyond this, if tasks are submitted, it is okay to reject them. We can revisit these limits later as well.

@chatman
Copy link
Contributor Author

chatman commented May 6, 2024

Thanks @cpoerschke and @dsmiley. I've merged to main, where this can bake for some days before merging to branch_9x. If there are any major outstanding issues, or any changes needed to documentation or default behaviour, we can take it up in another PR.

@dsmiley
Copy link
Contributor

dsmiley commented May 6, 2024

Merging to main was unexpected to me because of the healthy code review happening here didn't conclude. Next time please announce an intention to do so in a couple days. I would have given this another look! For example I thought the DocSet merging aspect was still tentative... I was awaiting another comment from Christine. And I was looking forward to checking out the thread pool aspect again.

@dsmiley
Copy link
Contributor

dsmiley commented May 6, 2024

This pull request is closed, but the jira/solr-13350 branch has unmerged commits.

Not sure what didn't make it

@chatman
Copy link
Contributor Author

chatman commented May 7, 2024

Merging to main was unexpected to me because of the healthy code review happening here didn't conclude.

I didn't want this to languish for a long time, and there were large PRs in the waiting that could affect this PR. Example: #2382

I would have given this another look!

...

And I was looking forward to checking out the thread pool aspect again.

Please feel free to look at it, would be happy to address all loose ends, if any.

@cpoerschke
Copy link
Contributor

Merging to main was unexpected to me because of the healthy code review happening here didn't conclude. ...

I was surprised by this merge too and my initial thought was that it might have been an accident and well that happens sometimes and we can just revert the commit and resume on a new PR then.

@gus-asf
Copy link
Contributor

gus-asf commented May 16, 2024

The discussion on this is long, so maybe I've missed it, but the actual merged code has introduced the possibility (though I suspect it might never happen) of a non-numeric Max Score...

    public float getMaxScore(int totalHits) {
      if (totalHits > 0) {
        for (Object res : result) {
          if (res instanceof MaxScoreResult) {
            return ((MaxScoreResult) res).maxScore;
          }
        }
        return Float.NaN;    <<<<<<<<<<<<<<<<<<<<<<
      } else {
        return 0.0f;
      }
    }

Did I miss discussion of this?

@@ -335,7 +335,7 @@ public SolrIndexSearcher(
boolean reserveDirectory,
DirectoryFactory directoryFactory)
throws IOException {
super(wrapReader(core, r));
super(wrapReader(core, r), core.getCoreContainer().getCollectorExecutor());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This change here is not controlled by the request level multiThreaded=true|false flag. #2570 explores how node-level opt-out could be supported.

}
if (needDocSet) {
int maxDoc = searcher.getRawReader().maxDoc();
log.error("raw read max={}", searcher.getRawReader().maxDoc());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Am unclear on why we log an error here. Also since the overall bit set is now computed as union of per-segment bit sets, perhaps there is no need to 'know' the max doc and it could be computed from the per-segment bit sets?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

logging: surely it was temporary during dev to ensure a log message could be seen easily. It's easier than logging at trace and reconfiguring logging. This must be fixed.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

oops, looks like this wasn't fixed yet: https://issues.apache.org/jira/browse/SOLR-17454

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

#2720 to remedy.

Comment on lines -188 to +190
"50"));
"50",
"multiThreaded",
"false"));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

#2590 proposes to explicitly not (yet) support multi-threading when query limits are used, and with that then the test change here could be reverted.

Comment on lines -79 to +89
params("q", "id:*", "sort", "id asc", "facet", "true", "facet.field", "val_i"));
params(
"q",
"id:*",
"sort",
"id asc",
"facet",
"true",
"facet.field",
"val_i",
"multiThreaded",
"false"));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

#2590 proposes to explicitly not (yet) support multi-threading when query limits are used, and with that then the test change here could be reverted.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
cat:search configs documentation Improvements or additions to documentation start-scripts tests
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants