Re: [PR] LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor of IndexSearcher#search(Query, CollectorManager) - TopFieldCollectorManager & TopScoreDocCollectorManager [lucene]
zacharymorn commented on PR #240: URL: https://github.com/apache/lucene/pull/240#issuecomment-1811972923 > Hi @mikemccand @jpountz @javanna @gsmiller , I have updated this PR to pick up the latest from `main`, as well as revert some changes to save them for follow-up PRs that address other collectors. This PR is now focused on the following: > > * Refactor out TopScoreDocCollectorManager, TopFieldCollectorManager > * Refactor some tests to use the above collector manager > > Could you please take a look, and let me know if you have any feedback? Hi @jpountz @javanna @gsmiller, just want to have a quick follow-up to see if you may have any further feedback on this PR? I plan to merge it in the next few days if there's no major concern. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[I] Simplifying TextAreaPrintStream in Luke [lucene]
picimako opened a new issue, #12809: URL: https://github.com/apache/lucene/issues/12809 ### Description Hi, I've been looking into how [`org.apache.lucene.luke.app.desktop.util.TextAreaPrintStream`](https://github.com/apache/lucene/blob/main/lucene/luke/src/java/org/apache/lucene/luke/app/desktop/util/TextAreaPrintStream.java) is used in Luke, and I think its usage and its implementation may be simplified. - It is used in `OptimizeIndexDialogFactory.ListenerFunctions` and `CheckIndexDialogFactory.ListenerFunctions`, both in a try-catch. Since `TextAreaPrintStream` is closable, my first suggestion would be to use try-with-resources instead at those locations. - Since there is always a new instance of `TextAreaPrintStream` instance created at the aforementioned locations, and they are flushed afterwards, it seems unnecessary to store the `ByteArrayOutputStream baos` field and call .reset() at all, when the `TextAreaPrintStream` instances are not reused. Let me know what you think. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[PR] Simplify advancing on postings/impacts enums [lucene]
jpountz opened a new pull request, #12810: URL: https://github.com/apache/lucene/pull/12810 Currently `advance(int target)` needs to perform two checks: - is there a need to use skip lists? - is there a need for decoding a new block? Ideally we would track the last doc ID in a block and use it to check these two things with a single comparison. However neither `accum` nor `nextSkipDoc` are suitable for this because they both start at 0, so `advance(0)` on an postings enum that is not initialized would think that it doesn't need to decode a new block while it does. Instead of introducing a 3rd variable, this PR changes postings and skip lists to start at -1 so that checking `target > accum` is enough to know whether skip lists may need to be advanced and a new block needs to be decoded. Specifically, check out changes to `BlockImpactsEnum#advance` which is now simpler. This feels like a good change anyway, as now a value that would be present in every single document would be able to take advantage of the `allEqual` optimization on the first block, which currently fails because the first doc delta is 0, while all other deltas are 1. This also enabled a few other simplifications, see e.g. `BlockDocsEnum` and `EverythingEnum`, which no longer need to track the `nextSkipDoc` and can do everything they need using `accum`. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Allow FST builder to use different writer (#12543) [lucene]
dungba88 commented on code in PR #12624: URL: https://github.com/apache/lucene/pull/12624#discussion_r1393547261 ## lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java: ## @@ -21,12 +21,13 @@ import java.util.List; import org.apache.lucene.store.DataInput; import org.apache.lucene.store.DataOutput; +import org.apache.lucene.util.Accountable; import org.apache.lucene.util.RamUsageEstimator; -// TODO: merge with PagedBytes, except PagedBytes doesn't -// let you read while writing which FST needs +// TODO: merge with or use PagedBytes/ByteBuffersDataOutput, as we no longer need to read while +// writing -class BytesStore extends DataOutput implements FSTReader { +class BytesStore extends DataOutput implements Accountable { Review Comment: > Hmm where do we do this TBH I haven't gone through all of them, but one instance is when we write the arc in backward, another is when we first skip the header and after all is written we go back and write the header, presence bits and label. Anyhow I changed to a byte[] to simplify its implementation. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Simplify advancing on postings/impacts enums [lucene]
jpountz commented on PR #12810: URL: https://github.com/apache/lucene/pull/12810#issuecomment-1812097551 This change seems to be neutral on wikibigall. No speedup, but not slowdown either. ``` TaskQPS baseline StdDevQPS my_modified_version StdDevPct diff p-value HighTermTitleSort 160.92 (4.2%) 158.41 (4.6%) -1.6% ( -9% -7%) 0.261 CountTerm12508.81 (3.8%)12320.72 (5.0%) -1.5% ( -9% -7%) 0.285 HighTerm 364.21 (8.1%) 359.26 (6.2%) -1.4% ( -14% - 14%) 0.553 MedTerm 652.09 (6.4%) 643.53 (4.2%) -1.3% ( -11% -9%) 0.442 LowTerm 872.30 (5.2%) 862.94 (3.4%) -1.1% ( -9% -7%) 0.439 PKLookup 269.53 (2.2%) 267.46 (2.6%) -0.8% ( -5% -4%) 0.311 OrHighLow 626.96 (1.6%) 623.38 (1.5%) -0.6% ( -3% -2%) 0.250 Respell 45.26 (1.4%) 45.01 (1.9%) -0.6% ( -3% -2%) 0.276 AndHighHigh 63.84 (2.8%) 63.49 (2.5%) -0.5% ( -5% -4%) 0.518 Fuzzy2 64.01 (1.1%) 63.76 (1.4%) -0.4% ( -2% -2%) 0.325 Wildcard 100.52 (2.6%) 100.14 (2.8%) -0.4% ( -5% -5%) 0.660 Fuzzy1 77.30 (1.3%) 77.05 (1.4%) -0.3% ( -2% -2%) 0.440 AndHighMed 195.90 (3.3%) 195.47 (3.0%) -0.2% ( -6% -6%) 0.824 Prefix3 164.89 (3.4%) 164.68 (2.5%) -0.1% ( -5% -5%) 0.893 CountAndHighHigh 44.24 (1.6%) 44.19 (2.0%) -0.1% ( -3% -3%) 0.848 HighSpanNear3.45 (2.7%)3.45 (2.6%) -0.1% ( -5% -5%) 0.943 LowSpanNear 10.70 (3.3%) 10.70 (3.4%) -0.0% ( -6% -6%) 0.992 MedSpanNear 10.07 (2.5%) 10.07 (2.5%)0.0% ( -4% -5%) 0.994 CountAndHighMed 130.25 (1.9%) 130.31 (2.1%)0.1% ( -3% -4%) 0.935 IntNRQ 105.34 (9.7%) 105.49 (10.3%)0.1% ( -18% - 22%) 0.965 HighTermMonthSort 4583.26 (1.6%) 4598.28 (1.8%)0.3% ( -3% -3%) 0.545 OrHighHigh 59.55 (2.2%) 59.86 (2.2%)0.5% ( -3% -5%) 0.453 CountPhrase4.16 (1.6%)4.18 (1.9%)0.6% ( -2% -4%) 0.250 AndHighLow 854.37 (3.0%) 859.99 (2.6%)0.7% ( -4% -6%) 0.455 HighSloppyPhrase3.20 (3.7%)3.22 (3.6%)0.7% ( -6% -8%) 0.567 OrHighMed 169.55 (2.8%) 170.85 (2.8%)0.8% ( -4% -6%) 0.385 HighTermDayOfYearSort 454.09 (2.5%) 457.71 (2.3%)0.8% ( -3% -5%) 0.298 MedSloppyPhrase9.30 (2.6%)9.40 (2.7%)1.0% ( -4% -6%) 0.215 LowSloppyPhrase4.57 (3.7%)4.62 (3.8%)1.2% ( -6% -9%) 0.331 HighPhrase 24.71 (7.5%) 25.23 (3.4%)2.1% ( -8% - 14%) 0.255 LowPhrase 16.71 (7.1%) 17.06 (3.0%)2.1% ( -7% - 13%) 0.223 MedPhrase 39.91 (7.6%) 40.76 (3.4%)2.1% ( -8% - 14%) 0.256 CountOrHighMed 86.68 (15.1%) 89.47 (16.9%)3.2% ( -25% - 41%) 0.526 CountOrHighHigh 55.80 (15.5%) 57.64 (17.3%)3.3% ( -25% - 42%) 0.525 ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Simplify advancing on postings/impacts enums [lucene]
jpountz commented on PR #12810: URL: https://github.com/apache/lucene/pull/12810#issuecomment-1812099057 For reference, starting postings and skip lists at -1 changes file formats, so I'm keen to getting this change in 9.9 since we had to change the file format anyway because of the move from PFOR to FOR for doc blocks. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Utilize exact kNN search when gathering k > numVectors in a segment [lucene]
jpountz commented on code in PR #12806: URL: https://github.com/apache/lucene/pull/12806#discussion_r1394090752 ## lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java: ## @@ -110,6 +110,12 @@ private TopDocs getLeafResults(LeafReaderContext ctx, Weight filterWeight) throw int maxDoc = ctx.reader().maxDoc(); if (filterWeight == null) { + int cost = liveDocs == null ? maxDoc : liveDocs.length(); Review Comment: liveDocs.length() is always equal to `maxDoc` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [I] HnwsGraph creates disconnected components [lucene]
benwtrent commented on issue #12627: URL: https://github.com/apache/lucene/issues/12627#issuecomment-1812408858 @nitirajrathore @msokolov I had an idea around this, and it will cost an extra 4bytes per node on each layer its a member (maybe we only need this on the bottom layer...) What if we added an "incoming connection" count for every node? Then when disconnecting non-diverse neighbors, we enforce that we can never disconnect the last connection to a node. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Utilize exact kNN search when gathering k > numVectors in a segment [lucene]
benwtrent commented on PR #12806: URL: https://github.com/apache/lucene/pull/12806#issuecomment-1812421692 > The idea makes sense to me, what is less clear to me is whether this logic belongs to the Query or to the vector reader: should searchNearestNeighbors implicitly do a linear scan when k is greater than size()? I like that idea, we should do that. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Minor change to IndexOrDocValuesQuery#toString [lucene]
mikemccand commented on PR #12791: URL: https://github.com/apache/lucene/pull/12791#issuecomment-1812469386 Nightly sparse (NYC taxis) benchy was a bit unhappy with this change because it (weirdly) relies on `Query.toString` (I tried to fix the benchy [here](https://github.com/mikemccand/luceneutil/commit/b9b4bcbc6d4b04a78be38ee13ca888777304539c)), but what was interesting is how heinous the `dvQuery` part of the query becomes: ``` IndexOrDocValuesQuery(indexQuery=pickup_latitude:[40.75 TO 40.9], dvQuery=pickup_latitude:[4630931869959979008 TO 4630952980583232307]) ``` I assume this is because this is a `DoubleDocValuesField` which encodes the double using `NumericUtils.doubleToSortableLong`. I don't think there's much we can do (Lucene doesn't have the knowledge that this `long` doc values field is "really" a double), and it's another datapoint showing that apps should not assume if you run any `Query.toString` through your favorite `QueryParser` that you'll get your query back: the process it not guaranteed to be fully invertible, but sometimes it is. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [I] HnwsGraph creates disconnected components [lucene]
msokolov commented on issue #12627: URL: https://github.com/apache/lucene/issues/12627#issuecomment-1812559330 Is the problem primarily to do with single isolated nodes or do we also see disconnected subgraphs containing multiple nodes? I think this idea would prevent the isolated nodes, but not fix the other case. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [I] HnwsGraph creates disconnected components [lucene]
benwtrent commented on issue #12627: URL: https://github.com/apache/lucene/issues/12627#issuecomment-1812605864 @msokolov good point. It seems to me we would only fully disconnect a sub-graph only if its very clustered. Is there a way to detect this in the diversity selection? One other thing I just found that confuses me: https://github.com/apache/lucene/pull/12235 This slightly changed the diversity connection in a way to to attempt to improve performance. The key issue is here: https://github.com/apache/lucene/blob/main/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphBuilder.java#L386-L407 If we have "checked the indices", we always just remove the furthest one, which doesn't seem correct to me. We should check for diversity starting at the furthest one, not always remove it? @nitirajrathore for your connection checker, are you testing on a Lucene > 9.7.0? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Utilize exact kNN search when gathering k > numVectors in a segment [lucene]
jpountz commented on code in PR #12806: URL: https://github.com/apache/lucene/pull/12806#discussion_r1394256922 ## lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsReader.java: ## @@ -238,11 +238,23 @@ public void search(String field, float[] target, KnnCollector knnCollector, Bits return; } RandomVectorScorer scorer = flatVectorsReader.getRandomVectorScorer(field, target); -HnswGraphSearcher.search( -scorer, -new OrdinalTranslatedKnnCollector(knnCollector, scorer::ordToDoc), -getGraph(fieldEntry), -scorer.getAcceptOrds(acceptDocs)); +if (knnCollector.k() < scorer.maxOrd()) { + HnswGraphSearcher.search( + scorer, + new OrdinalTranslatedKnnCollector(knnCollector, scorer::ordToDoc), + getGraph(fieldEntry), + scorer.getAcceptOrds(acceptDocs)); + return; Review Comment: nit: from a style perspective, I'd rather not return here and have an `else` block instead -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [I] HnwsGraph creates disconnected components [lucene]
msokolov commented on issue #12627: URL: https://github.com/apache/lucene/issues/12627#issuecomment-1812647124 My memory of the way this diversity criterion has evolved is kind of hazy, but I believe in the very first implementation we would not impose any diversity check until the neighbor array was full? It seems as if that would have tended to preserve more links, but then we (I think?) decided that wasn't the approach advocated by the academic papers we were emulating, and implemented the diversity check to every added neighbor. Now we have this checked/unchecked distinction. I seem to remember that the checked ones have had a diversity check applied -- so we know that as a group, they are diverse (none of them is closer to any of the others than it is to the target node). So this would seem to support the idea that we can add unchecked nodes and then only check when the list gets full?? So, what are we doing?! Sorry, I've lost track. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Simplify advancing on postings/impacts enums [lucene]
msokolov commented on PR #12810: URL: https://github.com/apache/lucene/pull/12810#issuecomment-1812678173 this sounds reasonable to me, and the code does seem simpler, but I'm not able to give a thorough review. +1 to rationalize / simplify even if it doesn't show significant peformance improvement -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [I] HnwsGraph creates disconnected components [lucene]
benwtrent commented on issue #12627: URL: https://github.com/apache/lucene/issues/12627#issuecomment-1812901631 @nitirajrathore could you add something to [KnnGraphTester](https://github.com/mikemccand/luceneutil/blob/master/src/main/KnnGraphTester.java) that is a test for connectedness? If you can't get to it, I can try. It would be good to have it along side all our other performance testing as another metric to record and optimize for. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Add support for similarity-based vector searches [lucene]
kaivalnp commented on PR #12679: URL: https://github.com/apache/lucene/pull/12679#issuecomment-1812941899 > You still need to score the vectors to realize that they are in the iteration set or not Right, I meant that we need not score all *other* vectors to determine if the vector itself is a "hit" or not (we just need its similarity score to be above the `resultSimilarity`) - as opposed to KNN where it's not a simple "filter" like you mentioned > we do all this work in approximateSearch (because we need to score the values) only to throw it away I've tried to re-use some of this work to [directly reject](https://github.com/apache/lucene/blob/cad565439be512ac6e95a698007b1fc971173f00/lucene/core/src/java/org/apache/lucene/search/AbstractVectorSimilarityQuery.java#L119-L121) vectors that are above the `traversalSimilarity` but below the `resultSimilarity` (the ones that were [already scored from HNSW search](https://github.com/apache/lucene/blob/cad565439be512ac6e95a698007b1fc971173f00/lucene/core/src/java/org/apache/lucene/search/VectorSimilarityCollector.java#L66-L68)), without re-computing their scores I wonder if we can extend this further: [`visited`](https://github.com/apache/lucene/blob/main/lucene/core/src/java/org/apache/lucene/util/hnsw/HnswGraphSearcher.java#L236) marks all the nodes for which we have computed scores from HNSW search. However, anything that is "visited but not collected" will not make it to the final results. We can do this by passing the `visited` variable back to the `KnnCollector` by adding a new method like `setVisited(Bits)`? This is also usable in the current KNN-based search, wherever we fall back from `approximateSearch` to `exactSearch`. If the `KnnCollector` had information about whatever we have already scored in graph searches (but is not present in the results) -- we can prevent computing its similarity scores again from `exactSearch`, because we already know they are not present in the `topK` Right now we [score all vectors](https://github.com/apache/lucene/blob/main/lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java#L177-L187) present in the `filter`, even if many of them are already scored and rejected in graph search [Here](https://github.com/apache/lucene/commit/2d6c0bfd4134b04c60be3864567211c824e7bc3c) are some very rough changes to support this -- what do you think @benwtrent? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Add support for similarity-based vector searches [lucene]
kaivalnp commented on PR #12679: URL: https://github.com/apache/lucene/pull/12679#issuecomment-1812956627 > could you test on cohere with Max-inner product? Thanks, the gist was really helpful and gave some files including normalized and un-normalized vectors. I assume that since you mentioned `MAXIMUM_INNER_PRODUCT`, you wanted the un-normalized vectors I saw \~476k vectors of 768 dimensions there and indexed the first 400k in a *single segment*, while querying the next 10k, using the following command: ```sh ./gradlew :lucene:core:similarity-benchmark --args=" --vecPath=/home/kaivalnp/working/similarity-benchmark/cohere-768.vec --indexPath=/home/kaivalnp/working/similarity-benchmark/cohere-indexes --dim=768 --function=MAXIMUM_INNER_PRODUCT --numDocs=40 --numQueries=1 --topKs=5000,2500,1000,500,100 --topK-thresholds=300,305,310,315,320 --traversalSimilarities=295,300,305,310,315 --resultSimilarities=300,305,310,315,320" ``` ### KNN search | maxConn | beamWidth | topK | threshold | count | numVisited | latency | recall | | --- | - | | - | --- | -- | --- | -- | | 16 | 100 | 5000 | 300.00| 1123.19 | 40056.44 | 98.96 | 0.89 | | 16 | 100 | 2500 | 305.00| 480.82 | 23258.29 | 54.91 | 0.83 | | 16 | 100 | 1000 | 310.00| 191.52 | 11249.93 | 26.12 | 0.73 | | 16 | 100 | 500 | 315.00| 83.21 | 6487.60| 14.87 | 0.69 | | 16 | 100 | 100 | 320.00| 23.80 | 1832.45| 4.00| 0.43 | | 16 | 200 | 5000 | 300.00| 1126.33 | 44928.96 | 107.69 | 0.89 | | 16 | 200 | 2500 | 305.00| 482.17 | 26242.83 | 61.47 | 0.83 | | 16 | 200 | 1000 | 310.00| 192.13 | 12751.78 | 29.42 | 0.73 | | 16 | 200 | 500 | 315.00| 83.49 | 7360.26| 16.67 | 0.70 | | 16 | 200 | 100 | 320.00| 23.89 | 2056.14| 4.51| 0.44 | | 32 | 100 | 5000 | 300.00| 1128.81 | 51636.98 | 122.67 | 0.89 | | 32 | 100 | 2500 | 305.00| 483.29 | 30892.01 | 72.01 | 0.84 | | 32 | 100 | 1000 | 310.00| 192.65 | 15424.38 | 35.12 | 0.73 | | 32 | 100 | 500 | 315.00| 83.72 | 9060.78| 20.28 | 0.70 | | 32 | 100 | 100 | 320.00| 24.00 | 2606.37| 5.70| 0.44 | | 32 | 200 | 5000 | 300.00| 1130.18 | 61350.93 | 145.76 | 0.89 | | 32 | 200 | 2500 | 305.00| 483.95 | 37178.70 | 86.05 | 0.84 | | 32 | 200 | 1000 | 310.00| 192.99 | 18778.34 | 42.14 | 0.73 | | 32 | 200 | 500 | 315.00| 83.90 | 11083.97 | 24.54 | 0.70 | | 32 | 200 | 100 | 320.00| 24.08 | 3172.91| 6.83| 0.44 | | 64 | 100 | 5000 | 300.00| 1129.81 | 58389.13 | 138.14 | 0.89 | | 64 | 100 | 2500 | 305.00| 483.77 | 35567.55 | 81.62 | 0.84 | | 64 | 100 | 1000 | 310.00| 192.87 | 18093.55 | 40.34 | 0.73 | | 64 | 100 | 500 | 315.00| 83.84 | 10734.50 | 23.76 | 0.70 | | 64 | 100 | 100 | 320.00| 24.06 | 3122.13| 6.77| 0.44 | | 64 | 200 | 5000 | 300.00| 1130.78 | 72620.92 | 169.86 | 0.89 | | 64 | 200 | 2500 | 305.00| 484.24 | 45052.36 | 101.93 | 0.84 | | 64 | 200 | 1000 | 310.00| 193.16 | 23283.96 | 51.61 | 0.73 | | 64 | 200 | 500 | 315.00| 83.99 | 13908.95 | 30.44 | 0.70 | | 64 | 200 | 100 | 320.00| 24.13 | 4035.89| 8.61| 0.44 | ### Similarity-based search | maxConn | beamWidth | traversalSimilarity | resultSimilarity | count | numVisited | latency | recall | | --- | - | --- | | --- | -- | --- | -- | | 16 | 100 | 295.00 | 300.00 | 1209.53 | 18270.70 | 44.38 | 0.95 | | 16 | 100 | 300.00 | 305.00 | 538.00 | 8833.17| 21.02 | 0.93 | | 16 | 100 | 305.00 | 310.00 | 239.11 | 4249.13| 9.97| 0.91 | | 16 | 100 | 310.00 | 315.00 | 105.02 | 2050.95| 4.87| 0.87 | | 16 | 100 | 315.00 | 320.00 | 45.71 | 1028.26| 2.35| 0.83 | | 16 | 200 | 295.00 | 300.00 | 1217.74 | 20335.62 | 49.38 | 0.96 | | 16 | 200 | 300.00 | 305.00 | 542.19 | 9851.65| 23.54 | 0.94 | | 16 | 200 | 305.00 | 310.00 | 240.68 | 4726.50| 11.04
Re: [PR] Utilize exact kNN search when gathering k > numVectors in a segment [lucene]
benwtrent merged PR #12806: URL: https://github.com/apache/lucene/pull/12806 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Improve vector search speed by using FixedBitSet [lucene]
jpountz commented on PR #12789: URL: https://github.com/apache/lucene/pull/12789#issuecomment-1813030726 ++ This feels similar to `IndexOrDocValuesQuery`: we probably can't guess the absolute best threshold, but we can probably figure out something that is right more often than wrong. Hopefully we can keep it simple and not include `maxConn` and other parameters in the equation. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [I] USearch integration and potential Vector Search performance improvements [lucene]
chadbrewbaker commented on issue #12502: URL: https://github.com/apache/lucene/issues/12502#issuecomment-1813112623 > Yes: > > * no external libraries for Lucene Core > * no native code Put it in an "examples" directory to show how to extend Lucene with JNI. If you have a $1m spend on Lucene you will figure out JNI issues. As accelerators pop up you will also likely want MOJO native drivers. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[PR] Simple rename of unreleased quantization parameter [lucene]
benwtrent opened a new pull request, #12811: URL: https://github.com/apache/lucene/pull/12811 the `quantile` parameter is actually a `confidence_interval` this is a simple rename of this parameter for the hnsw scalar quantized format. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Simple rename of unreleased quantization parameter [lucene]
benwtrent merged PR #12811: URL: https://github.com/apache/lucene/pull/12811 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[PR] Introduce workflow for stale PRs [lucene]
stefanvodita opened a new pull request, #12813: URL: https://github.com/apache/lucene/pull/12813 PRs get stale and we miss out on good contributions. This workflow will mark PRs that are becoming stale. Addresses #12796 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [I] Port PR management bot from Apache Beam [lucene]
stefanvodita commented on issue #12796: URL: https://github.com/apache/lucene/issues/12796#issuecomment-1813265420 +1 to starting super simple. I tried to hack a workflow for marking stale PRs (#12813). Fortunately, GitHub provides good [support](https://github.com/actions/stale) for this type of task. It also has a [labeler](https://github.com/actions/labeler) which we can try later. The stale workflow is tested on my fork, but it would be good to have a GitHub Actions expert review. Examples I followed: 1. [starter-wokflows stale](https://github.com/actions/starter-workflows/blob/main/automation/stale.yml) 2. [beam stale](https://github.com/apache/beam/blob/master/.github/workflows/pr-bot-prs-needing-attention.yml) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Fix segmentInfos replace doesn't set userData [lucene]
Shibi-bala commented on PR #12626: URL: https://github.com/apache/lucene/pull/12626#issuecomment-1813343695 @uschindler Ah I needed to re-sync my forked repo 😅 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Allow FST builder to use different writer (#12543) [lucene]
dungba88 commented on code in PR #12624: URL: https://github.com/apache/lucene/pull/12624#discussion_r1395069004 ## lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java: ## @@ -337,11 +349,23 @@ public long size() { return getPosition(); } + /** Similar to {@link #truncate(long)} with newLen=0 but keep the first block to reduce GC. */ + public void reset() { Review Comment: Yeah it seems the main complexity is due to the fact that we need to override the previously written bytes, sometimes when we want to remove the labels, sometimes when we need to update the header after writing everything. So I guess making it forward would not eliminate those backward writing operation. But as the implementation is much simpler now (with the growable byte array), would we still need to pursuit this? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Allow FST builder to use different writer (#12543) [lucene]
dungba88 commented on code in PR #12624: URL: https://github.com/apache/lucene/pull/12624#discussion_r1395069004 ## lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java: ## @@ -337,11 +349,23 @@ public long size() { return getPosition(); } + /** Similar to {@link #truncate(long)} with newLen=0 but keep the first block to reduce GC. */ + public void reset() { Review Comment: Yeah it seems the main complexity is due to the fact that we need to override the previously written bytes, sometimes when we want to remove the labels, sometimes when we need to update the header after writing everything. So I guess making it forward would not eliminate those backward writing operation. I simplified the writing operation a bit in the direct addressing mode. Seems like we can convert it to append-only mode. But the binary search one seems to be difficult, as when we fix the arc we need to do in backward (I assume to avoid overriding?). Anyhow, the remaining backward operations are: - writeBytes: As mentioned, this is used in the binary search mode. If we first copy the whole buffer to a new one, we can start appending back to back. But this isn't as efficient? - reverse: This is only done at the end before writing to the DataOutput and the NodeHash.ByteBlockPool -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Allow FST builder to use different writer (#12543) [lucene]
dungba88 commented on code in PR #12624: URL: https://github.com/apache/lucene/pull/12624#discussion_r1393547261 ## lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java: ## @@ -21,12 +21,13 @@ import java.util.List; import org.apache.lucene.store.DataInput; import org.apache.lucene.store.DataOutput; +import org.apache.lucene.util.Accountable; import org.apache.lucene.util.RamUsageEstimator; -// TODO: merge with PagedBytes, except PagedBytes doesn't -// let you read while writing which FST needs +// TODO: merge with or use PagedBytes/ByteBuffersDataOutput, as we no longer need to read while +// writing -class BytesStore extends DataOutput implements FSTReader { +class BytesStore extends DataOutput implements Accountable { Review Comment: > Hmm where do we do this TBH I haven't gone through all of them, but one instance is when we write the arc in backward, another is when we first skip the header and after all is written we go back and write the header, presence bits and label. Anyhow I changed to a byte[] to simplify its implementation. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Allow FST builder to use different writer (#12543) [lucene]
dungba88 commented on code in PR #12624: URL: https://github.com/apache/lucene/pull/12624#discussion_r1395069004 ## lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java: ## @@ -337,11 +349,23 @@ public long size() { return getPosition(); } + /** Similar to {@link #truncate(long)} with newLen=0 but keep the first block to reduce GC. */ + public void reset() { Review Comment: Yeah it seems the main complexity is due to the fact that we need to override the previously written bytes, sometimes when we want to remove the labels, sometimes when we need to update the header after writing everything. So I guess making it forward would not eliminate those backward writing operation. I simplified the writing operation a bit in the direct addressing mode. Seems like we can convert it to append-only mode. But the binary search one seems to be difficult, as when we fix the arc we need to do in backward (I assume to avoid overriding?). Anyhow, the remaining backward operations are: - writeBytes: As mentioned, this is used in the binary search mode. If we first copy the whole buffer to a new one, we can start appending back to back. But this isn't as efficient as the current in-place copy? - reverse: This is only done at the end before writing to the DataOutput and the NodeHash.ByteBlockPool -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Fix segmentInfos replace doesn't set userData [lucene]
MarcusSorealheis commented on PR #12626: URL: https://github.com/apache/lucene/pull/12626#issuecomment-1813833061 Looks good now. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Allow FST builder to use different writer (#12543) [lucene]
dungba88 commented on PR #12624: URL: https://github.com/apache/lucene/pull/12624#issuecomment-1813856323 Seems like this PR is getting long, so I spawned 2 PR out of it: - https://github.com/apache/lucene/pull/12814: Simplify `BytesStore` operations (which was changed to GrowableByteArrayDataOutput in this PR) - https://github.com/apache/lucene/pull/12802: Remove the size() method in FSTReader -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Allow FST builder to use different writer (#12543) [lucene]
dungba88 commented on code in PR #12624: URL: https://github.com/apache/lucene/pull/12624#discussion_r1393462969 ## lucene/core/src/java/org/apache/lucene/util/fst/OnHeapFSTStore.java: ## @@ -64,22 +66,13 @@ public FSTStore init(DataInput in, long numBytes) throws IOException { return this; } - @Override Review Comment: This change is in a different PR: https://github.com/apache/lucene/pull/12802. Ideally we would want to merge that one first. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Allow FST builder to use different writer (#12543) [lucene]
dungba88 commented on code in PR #12624: URL: https://github.com/apache/lucene/pull/12624#discussion_r1395069004 ## lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java: ## @@ -337,11 +349,23 @@ public long size() { return getPosition(); } + /** Similar to {@link #truncate(long)} with newLen=0 but keep the first block to reduce GC. */ + public void reset() { Review Comment: Yeah it seems the main complexity is due to the fact that we need to override the previously written bytes, sometimes when we want to remove the labels, sometimes when we need to update the header after writing everything. So I guess making it forward would not eliminate those backward writing operation. I simplified the writing operation a bit in the direct addressing mode. Seems like we can convert it to append-only mode. But the binary search one seems to be difficult, as when we fix the arc we need to do in backward (I assume to avoid overriding?). Anyhow, the remaining backward operations are: - writeBytes: As mentioned, this is used in the binary search mode. If we first copy the whole buffer to a new one, we can start appending back to back. But this isn't as efficient as the current in-place copy? The whole method seems to be expanding each arc from variable arc length to fixed arc lengths. - reverse: This is only done at the end before writing to the DataOutput and the NodeHash.ByteBlockPool -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org