[jira] [Commented] (LUCENE-10559) Add preFilter/postFilter options to KnnGraphTester
[ https://issues.apache.org/jira/browse/LUCENE-10559?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17539350#comment-17539350 ] Kaival Parikh commented on LUCENE-10559: We can maintain a *FixedBitSet* to record passing docs, and randomly *set* bits based on a *selectivity* argument, so over a large set of docs we would have a close proportion of passing docs We can use this bitset both while finding brute force NN in *computeNN* and HNSW kNN in {*}testSearch{*}. To incorporate pre-filtering, we can take an additional argument *prefilter* which wraps our bitset into a query, and passes it to *doKnnVectorQuery* to give pre-filtered results For post-filtering, we can request *topK/selectivity* number of results, to get close to *topK* after filtering out non passing docs I ran some tests with these changes (topK = 100, numDocs = 1M (wikimedia dataset), numIter = 1K): ||selectivity||effective topK||post-filter recall||post-filter time||pre-filter recall||pre-filter time|| |0.8|125|0.974|1.67|0.987|1.71| |0.6|166|0.965|2.06|0.989|2.09| |0.4|250|0.967|2.85|0.991|2.83| |0.2|500|0.941|5.06|0.996|4.85| |0.1|1000|0.972|8.81|0.997|7.99| |0.01|1|0.953|59.37|1.000|9.76| Note: I have directly updated the bitset in *BitSetCollector* to work around the arbitrary time it takes to *collect* results by iterating over the entire bitset (as it would be different for different types of queries, and {_}has to be considered separately{_}) The changes are available [here|https://github.com/kaivalnp/lucene/commit/c985fad694dfff4e9341649228b3a08d64c98a7f] in a separate branch. Looking for suggestions/comments on this approach > Add preFilter/postFilter options to KnnGraphTester > -- > > Key: LUCENE-10559 > URL: https://issues.apache.org/jira/browse/LUCENE-10559 > Project: Lucene - Core > Issue Type: Improvement >Reporter: Michael Sokolov >Priority: Major > > We want to be able to test the efficacy of pre-filtering in KnnVectorQuery: > if you (say) want the top K nearest neighbors subject to a constraint Q, are > you better off over-selecting (say 2K) top hits and *then* filtering > (post-filtering), or incorporating the filtering into the query > (pre-filtering). How does it depend on the selectivity of the filter? > I think we can get a reasonable testbed by generating a uniform random filter > with some selectivity (that is consistent and repeatable). Possibly we'd also > want to try filters that are correlated with index order, but it seems they'd > be unlikely to be correlated with vector values in a way that the graph > structure would notice, so random is a pretty good starting point for this. -- This message was sent by Atlassian Jira (v8.20.7#820007) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Resolved] (LUCENE-9634) Highlighting of degenerate spans on fields *with offsets* doesn't work properly
[ https://issues.apache.org/jira/browse/LUCENE-9634?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dawid Weiss resolved LUCENE-9634. - Resolution: Fixed > Highlighting of degenerate spans on fields *with offsets* doesn't work > properly > --- > > Key: LUCENE-9634 > URL: https://issues.apache.org/jira/browse/LUCENE-9634 > Project: Lucene - Core > Issue Type: Sub-task >Reporter: Dawid Weiss >Assignee: Dawid Weiss >Priority: Minor > Time Spent: 40m > Remaining Estimate: 0h > > Match highlighter works fine with degenerate interval positions when > {{OffsetsFromPositions}} strategy is used to compute offsets but will show > incorrect offset ranges if offsets are read from directly from the > {{MatchIterator}} ({{OffsetsFromMatchIterator}}). -- This message was sent by Atlassian Jira (v8.20.7#820007) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Commented] (LUCENE-10574) Remove O(n^2) from TieredMergePolicy or change defaults to one that doesn't do this
[ https://issues.apache.org/jira/browse/LUCENE-10574?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17539354#comment-17539354 ] ASF subversion and git services commented on LUCENE-10574: -- Commit bf07d98f13043860d5dffae094eacd6bb113fdde in lucene's branch refs/heads/main from Adrien Grand [ https://gitbox.apache.org/repos/asf?p=lucene.git;h=bf07d98f130 ] LUCENE-10574: Fix test failure. LogDocMergePolicy would previously always force-merge an index that has 10 segments of size 1 to 10, due to the min doc count. This is not the case anymore, but the test was assuming that such an index would get merged, so I fixed the test's expectations. Also changed the merge policy to keep working when RAM buffers are flushed in such a way that segments do not appear in decreasing size order using the same logic as LogMergePolicy. > Remove O(n^2) from TieredMergePolicy or change defaults to one that doesn't > do this > --- > > Key: LUCENE-10574 > URL: https://issues.apache.org/jira/browse/LUCENE-10574 > Project: Lucene - Core > Issue Type: Bug >Reporter: Robert Muir >Priority: Major > Fix For: 9.3 > > Time Spent: 1h 40m > Remaining Estimate: 0h > > Remove {{floorSegmentBytes}} parameter, or change lucene's default to a merge > policy that doesn't merge in an O(n^2) way. > I have the feeling it might have to be the latter, as folks seem really wed > to this crazy O(n^2) behavior. -- This message was sent by Atlassian Jira (v8.20.7#820007) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Commented] (LUCENE-10574) Remove O(n^2) from TieredMergePolicy or change defaults to one that doesn't do this
[ https://issues.apache.org/jira/browse/LUCENE-10574?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17539356#comment-17539356 ] ASF subversion and git services commented on LUCENE-10574: -- Commit 4155f4bb65d899466cb66b99bd12bfe949c487c0 in lucene's branch refs/heads/branch_9x from Adrien Grand [ https://gitbox.apache.org/repos/asf?p=lucene.git;h=4155f4bb65d ] LUCENE-10574: Fix test failure. LogDocMergePolicy would previously always force-merge an index that has 10 segments of size 1 to 10, due to the min doc count. This is not the case anymore, but the test was assuming that such an index would get merged, so I fixed the test's expectations. Also changed the merge policy to keep working when RAM buffers are flushed in such a way that segments do not appear in decreasing size order using the same logic as LogMergePolicy. > Remove O(n^2) from TieredMergePolicy or change defaults to one that doesn't > do this > --- > > Key: LUCENE-10574 > URL: https://issues.apache.org/jira/browse/LUCENE-10574 > Project: Lucene - Core > Issue Type: Bug >Reporter: Robert Muir >Priority: Major > Fix For: 9.3 > > Time Spent: 1h 40m > Remaining Estimate: 0h > > Remove {{floorSegmentBytes}} parameter, or change lucene's default to a merge > policy that doesn't merge in an O(n^2) way. > I have the feeling it might have to be the latter, as folks seem really wed > to this crazy O(n^2) behavior. -- This message was sent by Atlassian Jira (v8.20.7#820007) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Commented] (LUCENE-10574) Remove O(n^2) from TieredMergePolicy or change defaults to one that doesn't do this
[ https://issues.apache.org/jira/browse/LUCENE-10574?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17539365#comment-17539365 ] ASF subversion and git services commented on LUCENE-10574: -- Commit 2be2b02e21fc5784523152a30ded7ba82c657d03 in lucene's branch refs/heads/branch_9x from Adrien Grand [ https://gitbox.apache.org/repos/asf?p=lucene.git;h=2be2b02e21f ] LUCENE-10574: More test failures. - MergeOnFlushMergePolicy doesn't try to avoid O(n^2) merges, so I'm disabling the test on it for now. - TestUpgradeIndexMergePolicy would sometimes wrap with a non-standard merge policy like the alcoholic merge policy, I forced it to wrap a TieredMergePolicy. > Remove O(n^2) from TieredMergePolicy or change defaults to one that doesn't > do this > --- > > Key: LUCENE-10574 > URL: https://issues.apache.org/jira/browse/LUCENE-10574 > Project: Lucene - Core > Issue Type: Bug >Reporter: Robert Muir >Priority: Major > Fix For: 9.3 > > Time Spent: 1h 40m > Remaining Estimate: 0h > > Remove {{floorSegmentBytes}} parameter, or change lucene's default to a merge > policy that doesn't merge in an O(n^2) way. > I have the feeling it might have to be the latter, as folks seem really wed > to this crazy O(n^2) behavior. -- This message was sent by Atlassian Jira (v8.20.7#820007) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Commented] (LUCENE-10574) Remove O(n^2) from TieredMergePolicy or change defaults to one that doesn't do this
[ https://issues.apache.org/jira/browse/LUCENE-10574?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17539364#comment-17539364 ] ASF subversion and git services commented on LUCENE-10574: -- Commit 3960c16296777ff14e1741921d59abf2e9cbf037 in lucene's branch refs/heads/main from Adrien Grand [ https://gitbox.apache.org/repos/asf?p=lucene.git;h=3960c162967 ] LUCENE-10574: More test failures. - MergeOnFlushMergePolicy doesn't try to avoid O(n^2) merges, so I'm disabling the test on it for now. - TestUpgradeIndexMergePolicy would sometimes wrap with a non-standard merge policy like the alcoholic merge policy, I forced it to wrap a TieredMergePolicy. > Remove O(n^2) from TieredMergePolicy or change defaults to one that doesn't > do this > --- > > Key: LUCENE-10574 > URL: https://issues.apache.org/jira/browse/LUCENE-10574 > Project: Lucene - Core > Issue Type: Bug >Reporter: Robert Muir >Priority: Major > Fix For: 9.3 > > Time Spent: 1h 40m > Remaining Estimate: 0h > > Remove {{floorSegmentBytes}} parameter, or change lucene's default to a merge > policy that doesn't merge in an O(n^2) way. > I have the feeling it might have to be the latter, as folks seem really wed > to this crazy O(n^2) behavior. -- This message was sent by Atlassian Jira (v8.20.7#820007) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] jpountz opened a new pull request, #905: LUCENE-10574: Keep allowing unbalanced merges if they would reclaim lots of deletes.
jpountz opened a new pull request, #905: URL: https://github.com/apache/lucene/pull/905 `TestTieredMergePolicy` caught this special case: if a segment has lots of deletes, we should still allow unbalanced merges. -- 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
[GitHub] [lucene] jpountz commented on pull request #905: LUCENE-10574: Keep allowing unbalanced merges if they would reclaim lots of deletes.
jpountz commented on PR #905: URL: https://github.com/apache/lucene/pull/905#issuecomment-1131447327 @mikemccand I think you called out this case on the PR review and we assumed it'd be ok because sizes are pro-rated by the ratio of live docs, but it looks like we need more. (It's good that `BaseMergePolicyTestCase` detects such issues.) -- 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
[jira] [Commented] (LUCENE-10312) Add PersianStemmer
[ https://issues.apache.org/jira/browse/LUCENE-10312?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17539468#comment-17539468 ] Alan Woodward commented on LUCENE-10312: No problem [~tomoko] - I think we can add a simple boolean parameter to PersianAnalyzer to configure stemming. It would be a shame to just drop this out of 9.2 entirely as it's a nice addition to our analysis module. I'll raise a PR to add configurability. > Add PersianStemmer > -- > > Key: LUCENE-10312 > URL: https://issues.apache.org/jira/browse/LUCENE-10312 > Project: Lucene - Core > Issue Type: Wish > Components: modules/analysis >Affects Versions: 9.0 >Reporter: Ramin Alirezaee >Priority: Major > Fix For: 10.0 (main), 9.2 > > Attachments: image.png > > Time Spent: 7h 40m > Remaining Estimate: 0h > -- This message was sent by Atlassian Jira (v8.20.7#820007) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] mocobeta commented on a diff in pull request #906: LUCENE-10312: Make stemming configurable on PersianAnalyzer
mocobeta commented on code in PR #906: URL: https://github.com/apache/lucene/pull/906#discussion_r876984088 ## lucene/analysis/common/src/java/org/apache/lucene/analysis/fa/PersianAnalyzer.java: ## @@ -88,30 +88,43 @@ private static class DefaultSetHolder { } private final CharArraySet stemExclusionSet; + private final boolean useStemming; /** Builds an analyzer with the default stop words: {@link #DEFAULT_STOPWORD_FILE}. */ public PersianAnalyzer() { this(DefaultSetHolder.DEFAULT_STOP_SET); } /** - * Builds an analyzer with the given stop words + * Builds an analyzer with the default stop words: {@link #DEFAULT_STOPWORD_FILE} + * + * @param useStemming whether or not to enable stemming + */ + public PersianAnalyzer(boolean useStemming) { +this(DefaultSetHolder.DEFAULT_STOP_SET, useStemming, CharArraySet.EMPTY_SET); + } + + /** + * Builds an analyzer with the given stop words and no stemming * * @param stopwords a stopword set */ public PersianAnalyzer(CharArraySet stopwords) { -this(stopwords, CharArraySet.EMPTY_SET); +this(stopwords, false, CharArraySet.EMPTY_SET); } /** - * Builds an analyzer with the given stop word. If a none-empty stem exclusion set is provided - * this analyzer will add a {@link SetKeywordMarkerFilter} before {@link PersianStemFilter}. + * Builds an analyzer with the given stop word. If a non-empty stem exclusion set is provided this + * analyzer will add a {@link SetKeywordMarkerFilter} before {@link PersianStemFilter}. * * @param stopwords a stopword set + * @param useStemming whether or not to enable stemming * @param stemExclusionSet a set of terms not to be stemmed */ - public PersianAnalyzer(CharArraySet stopwords, CharArraySet stemExclusionSet) { + public PersianAnalyzer( + CharArraySet stopwords, boolean useStemming, CharArraySet stemExclusionSet) { Review Comment: How about this three-args constructor private and keep the two-args constructor `public PersianAnalyzer(CharArraySet stopwords, CharArraySet stemExclusionSet)` so that we make the API changes minimum on the next major release? -- 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
[GitHub] [lucene] rmuir commented on pull request #906: LUCENE-10312: Make stemming configurable on PersianAnalyzer
rmuir commented on PR #906: URL: https://github.com/apache/lucene/pull/906#issuecomment-1131618031 I don't think we should make this configurable. Please let's not do this to the analyzers. Just add the stemmer in 10.x, done. -- 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
[GitHub] [lucene] mocobeta commented on a diff in pull request #906: LUCENE-10312: Make stemming configurable on PersianAnalyzer
mocobeta commented on code in PR #906: URL: https://github.com/apache/lucene/pull/906#discussion_r876990252 ## lucene/analysis/common/src/java/org/apache/lucene/analysis/fa/PersianAnalyzer.java: ## @@ -88,30 +88,43 @@ private static class DefaultSetHolder { } private final CharArraySet stemExclusionSet; + private final boolean useStemming; /** Builds an analyzer with the default stop words: {@link #DEFAULT_STOPWORD_FILE}. */ public PersianAnalyzer() { this(DefaultSetHolder.DEFAULT_STOP_SET); } /** - * Builds an analyzer with the given stop words + * Builds an analyzer with the default stop words: {@link #DEFAULT_STOPWORD_FILE} + * + * @param useStemming whether or not to enable stemming + */ + public PersianAnalyzer(boolean useStemming) { +this(DefaultSetHolder.DEFAULT_STOP_SET, useStemming, CharArraySet.EMPTY_SET); + } + + /** + * Builds an analyzer with the given stop words and no stemming * * @param stopwords a stopword set */ public PersianAnalyzer(CharArraySet stopwords) { -this(stopwords, CharArraySet.EMPTY_SET); +this(stopwords, false, CharArraySet.EMPTY_SET); } /** - * Builds an analyzer with the given stop word. If a none-empty stem exclusion set is provided - * this analyzer will add a {@link SetKeywordMarkerFilter} before {@link PersianStemFilter}. + * Builds an analyzer with the given stop word. If a non-empty stem exclusion set is provided this + * analyzer will add a {@link SetKeywordMarkerFilter} before {@link PersianStemFilter}. * * @param stopwords a stopword set + * @param useStemming whether or not to enable stemming * @param stemExclusionSet a set of terms not to be stemmed */ - public PersianAnalyzer(CharArraySet stopwords, CharArraySet stemExclusionSet) { + public PersianAnalyzer( + CharArraySet stopwords, boolean useStemming, CharArraySet stemExclusionSet) { Review Comment: I'd suppose when `stemExclusionSet` is set `useStemming` flag is always set to true, so I think the three-args constructor can be internal-use only. -- 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
[GitHub] [lucene] romseygeek commented on pull request #906: LUCENE-10312: Make stemming configurable on PersianAnalyzer
romseygeek commented on PR #906: URL: https://github.com/apache/lucene/pull/906#issuecomment-1131623067 > Just add the stemmer in 10.x, done. That also works for me, the TokenFilter will be available in 9.2 but you have to create an analyzer yourself to use it, and we delegate BWC handling to factories. I'll update this PR to just add a MIGRATE entry explaining how the analyzer has changed, and open a separate one against 9x to remove the stem filter from PersianAnalyzer. -- 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
[jira] [Commented] (LUCENE-10312) Add PersianStemmer
[ https://issues.apache.org/jira/browse/LUCENE-10312?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17539517#comment-17539517 ] Robert Muir commented on LUCENE-10312: -- There need not be backwards compatibility issues: Just add the stemmer to PersianAnalyzer in 10.x only. That's what major releases are for. You can add to MIGRATE.txt, if you really want, how to build a CustomAnalyzer that looks like the old one, for people that are stuck in the past. Please, do not add crazy configuration options like this. > Add PersianStemmer > -- > > Key: LUCENE-10312 > URL: https://issues.apache.org/jira/browse/LUCENE-10312 > Project: Lucene - Core > Issue Type: Wish > Components: modules/analysis >Affects Versions: 9.0 >Reporter: Ramin Alirezaee >Priority: Major > Fix For: 10.0 (main), 9.2 > > Attachments: image.png > > Time Spent: 8h 20m > Remaining Estimate: 0h > -- This message was sent by Atlassian Jira (v8.20.7#820007) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] mocobeta commented on pull request #906: LUCENE-10312: Make stemming configurable on PersianAnalyzer
mocobeta commented on PR #906: URL: https://github.com/apache/lucene/pull/906#issuecomment-1131627669 > open a separate one against 9x to remove the stem filter from PersianAnalyzer. I wonder if my first PR #904 would work? -- 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
[GitHub] [lucene] romseygeek commented on pull request #906: LUCENE-10312: Make stemming configurable on PersianAnalyzer
romseygeek commented on PR #906: URL: https://github.com/apache/lucene/pull/906#issuecomment-1131629875 > I wonder if my first PR https://github.com/apache/lucene/pull/904 would work? It absolutely will, I had misread it as reverting everything rather than just the changes to the Analyzer. I'll go approve that one, thanks @mocobeta! -- 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
[GitHub] [lucene] mocobeta commented on pull request #906: LUCENE-10312: Make stemming configurable on PersianAnalyzer
mocobeta commented on PR #906: URL: https://github.com/apache/lucene/pull/906#issuecomment-1131630610 > I'll update this PR to just add a MIGRATE entry explaining how the analyzer has changed It's my omission in #540. I'll add MIGRATE entry as a follow-up of #540 if you don't mind. -- 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
[GitHub] [lucene] rmuir commented on a diff in pull request #904: LUCENE-10312: Revert changes in PersianAnalyzer
rmuir commented on code in PR #904: URL: https://github.com/apache/lucene/pull/904#discussion_r877000127 ## lucene/analysis/common/src/java/org/apache/lucene/analysis/fa/PersianAnalyzer.java: ## @@ -136,11 +121,7 @@ protected TokenStreamComponents createComponents(String fieldName) { * the order here is important: the stopword list is normalized with the * above! */ -result = new StopFilter(result, stopwords); -if (!stemExclusionSet.isEmpty()) { - result = new SetKeywordMarkerFilter(result, stemExclusionSet); -} -return new TokenStreamComponents(source, new PersianStemFilter(result)); +return new TokenStreamComponents(source, new StopFilter(result, stopwords)); Review Comment: I guess no javadocs changes were needed for this method because they never got updated in the first place :) Maybe something to fix in 10.x. -- 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
[GitHub] [lucene] romseygeek commented on pull request #906: LUCENE-10312: Make stemming configurable on PersianAnalyzer
romseygeek commented on PR #906: URL: https://github.com/apache/lucene/pull/906#issuecomment-1131631989 Closing in favour of #904 -- 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
[GitHub] [lucene] romseygeek closed pull request #906: LUCENE-10312: Make stemming configurable on PersianAnalyzer
romseygeek closed pull request #906: LUCENE-10312: Make stemming configurable on PersianAnalyzer URL: https://github.com/apache/lucene/pull/906 -- 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
[GitHub] [lucene] mocobeta commented on a diff in pull request #904: LUCENE-10312: Revert changes in PersianAnalyzer
mocobeta commented on code in PR #904: URL: https://github.com/apache/lucene/pull/904#discussion_r877002862 ## lucene/analysis/common/src/java/org/apache/lucene/analysis/fa/PersianAnalyzer.java: ## @@ -136,11 +121,7 @@ protected TokenStreamComponents createComponents(String fieldName) { * the order here is important: the stopword list is normalized with the * above! */ -result = new StopFilter(result, stopwords); -if (!stemExclusionSet.isEmpty()) { - result = new SetKeywordMarkerFilter(result, stemExclusionSet); -} -return new TokenStreamComponents(source, new PersianStemFilter(result)); +return new TokenStreamComponents(source, new StopFilter(result, stopwords)); Review Comment: OK, I'll update the javadocs in main later. -- 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
[GitHub] [lucene] mocobeta commented on pull request #904: LUCENE-10312: Revert changes in PersianAnalyzer
mocobeta commented on PR #904: URL: https://github.com/apache/lucene/pull/904#issuecomment-1131633700 Thank you both! -- 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
[GitHub] [lucene] mocobeta merged pull request #904: LUCENE-10312: Revert changes in PersianAnalyzer
mocobeta merged PR #904: URL: https://github.com/apache/lucene/pull/904 -- 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
[jira] [Commented] (LUCENE-10312) Add PersianStemmer
[ https://issues.apache.org/jira/browse/LUCENE-10312?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17539526#comment-17539526 ] ASF subversion and git services commented on LUCENE-10312: -- Commit 212fd3a74829268800e39623df1adcfbaf93f8e8 in lucene's branch refs/heads/branch_9x from Tomoko Uchida [ https://gitbox.apache.org/repos/asf?p=lucene.git;h=212fd3a7482 ] LUCENE-10312: Revert changes in PersianAnalyzer from 9x (#904) > Add PersianStemmer > -- > > Key: LUCENE-10312 > URL: https://issues.apache.org/jira/browse/LUCENE-10312 > Project: Lucene - Core > Issue Type: Wish > Components: modules/analysis >Affects Versions: 9.0 >Reporter: Ramin Alirezaee >Priority: Major > Fix For: 10.0 (main), 9.2 > > Attachments: image.png > > Time Spent: 9h 50m > Remaining Estimate: 0h > -- This message was sent by Atlassian Jira (v8.20.7#820007) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] mocobeta commented on pull request #904: LUCENE-10312: Revert changes in PersianAnalyzer
mocobeta commented on PR #904: URL: https://github.com/apache/lucene/pull/904#issuecomment-1131635115 I'll cherry-pick this in branch_9_2 soon. -- 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
[jira] [Commented] (LUCENE-10312) Add PersianStemmer
[ https://issues.apache.org/jira/browse/LUCENE-10312?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17539528#comment-17539528 ] ASF subversion and git services commented on LUCENE-10312: -- Commit ba8c3a806ada3d7b3c34d408e449a92376a8481b in lucene's branch refs/heads/branch_9_2 from Tomoko Uchida [ https://gitbox.apache.org/repos/asf?p=lucene.git;h=ba8c3a806ad ] LUCENE-10312: Revert changes in PersianAnalyzer from 9x (#904) > Add PersianStemmer > -- > > Key: LUCENE-10312 > URL: https://issues.apache.org/jira/browse/LUCENE-10312 > Project: Lucene - Core > Issue Type: Wish > Components: modules/analysis >Affects Versions: 9.0 >Reporter: Ramin Alirezaee >Priority: Major > Fix For: 10.0 (main), 9.2 > > Attachments: image.png > > Time Spent: 10h > Remaining Estimate: 0h > -- This message was sent by Atlassian Jira (v8.20.7#820007) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Commented] (LUCENE-10527) Use bigger maxConn for last layer in HNSW
[ https://issues.apache.org/jira/browse/LUCENE-10527?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17539529#comment-17539529 ] Mayya Sharipova commented on LUCENE-10527: -- Thanks Julie, I was also expecting this degradation. > Use bigger maxConn for last layer in HNSW > - > > Key: LUCENE-10527 > URL: https://issues.apache.org/jira/browse/LUCENE-10527 > Project: Lucene - Core > Issue Type: Task >Reporter: Julie Tibshirani >Assignee: Mayya Sharipova >Priority: Minor > Attachments: Screen Shot 2022-05-18 at 4.26.14 PM.png, Screen Shot > 2022-05-18 at 4.26.24 PM.png, Screen Shot 2022-05-18 at 4.27.37 PM.png, > image-2022-04-20-14-53-58-484.png > > Time Spent: 4h 40m > Remaining Estimate: 0h > > Recently I was rereading the HNSW paper > ([https://arxiv.org/pdf/1603.09320.pdf)] and noticed that they suggest using > a different maxConn for the upper layers vs. the bottom one (which contains > the full neighborhood graph). Specifically, they suggest using maxConn=M for > upper layers and maxConn=2*M for the bottom. This differs from what we do, > which is to use maxConn=M for all layers. > I tried updating our logic using a hacky patch, and noticed an improvement in > latency for higher recall values (which is consistent with the paper's > observation): > *Results on glove-100-angular* > Parameters: M=32, efConstruction=100 > !image-2022-04-20-14-53-58-484.png|width=400,height=367! > As we'd expect, indexing becomes a bit slower: > {code:java} > Baseline: Indexed 1183514 documents in 733s > Candidate: Indexed 1183514 documents in 948s{code} > When we benchmarked Lucene HNSW against hnswlib in LUCENE-9937, we noticed a > big difference in recall for the same settings of M and efConstruction. (Even > adding graph layers in LUCENE-10054 didn't really affect recall.) With this > change, the recall is now very similar: > *Results on glove-100-angular* > Parameters: M=32, efConstruction=100 > {code:java} > kApproach Recall > QPS > 10 luceneknn dim=100 {'M': 32, 'efConstruction': 100}0.563 >4410.499 > 50 luceneknn dim=100 {'M': 32, 'efConstruction': 100}0.798 >1956.280 > 100 luceneknn dim=100 {'M': 32, 'efConstruction': 100}0.862 >1209.734 > 500 luceneknn dim=100 {'M': 32, 'efConstruction': 100}0.958 > 341.428 > 800 luceneknn dim=100 {'M': 32, 'efConstruction': 100}0.974 > 230.396 > 1000 luceneknn dim=100 {'M': 32, 'efConstruction': 100}0.980 > 188.757 > 10 hnswlib ({'M': 32, 'efConstruction': 100})0.552 > 16745.433 > 50 hnswlib ({'M': 32, 'efConstruction': 100})0.794 >5738.468 > 100 hnswlib ({'M': 32, 'efConstruction': 100})0.860 >3336.386 > 500 hnswlib ({'M': 32, 'efConstruction': 100})0.956 > 832.982 > 800 hnswlib ({'M': 32, 'efConstruction': 100})0.973 > 541.097 > 1000 hnswlib ({'M': 32, 'efConstruction': 100})0.979 > 442.163 > {code} > I think it'd be nice update to maxConn so that we faithfully implement the > paper's algorithm. This is probably least surprising for users, and I don't > see a strong reason to take a different approach from the paper? Let me know > what you think! -- This message was sent by Atlassian Jira (v8.20.7#820007) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Resolved] (LUCENE-10527) Use bigger maxConn for last layer in HNSW
[ https://issues.apache.org/jira/browse/LUCENE-10527?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mayya Sharipova resolved LUCENE-10527. -- Fix Version/s: 9.2 Resolution: Fixed > Use bigger maxConn for last layer in HNSW > - > > Key: LUCENE-10527 > URL: https://issues.apache.org/jira/browse/LUCENE-10527 > Project: Lucene - Core > Issue Type: Task >Reporter: Julie Tibshirani >Assignee: Mayya Sharipova >Priority: Minor > Fix For: 9.2 > > Attachments: Screen Shot 2022-05-18 at 4.26.14 PM.png, Screen Shot > 2022-05-18 at 4.26.24 PM.png, Screen Shot 2022-05-18 at 4.27.37 PM.png, > image-2022-04-20-14-53-58-484.png > > Time Spent: 4h 40m > Remaining Estimate: 0h > > Recently I was rereading the HNSW paper > ([https://arxiv.org/pdf/1603.09320.pdf)] and noticed that they suggest using > a different maxConn for the upper layers vs. the bottom one (which contains > the full neighborhood graph). Specifically, they suggest using maxConn=M for > upper layers and maxConn=2*M for the bottom. This differs from what we do, > which is to use maxConn=M for all layers. > I tried updating our logic using a hacky patch, and noticed an improvement in > latency for higher recall values (which is consistent with the paper's > observation): > *Results on glove-100-angular* > Parameters: M=32, efConstruction=100 > !image-2022-04-20-14-53-58-484.png|width=400,height=367! > As we'd expect, indexing becomes a bit slower: > {code:java} > Baseline: Indexed 1183514 documents in 733s > Candidate: Indexed 1183514 documents in 948s{code} > When we benchmarked Lucene HNSW against hnswlib in LUCENE-9937, we noticed a > big difference in recall for the same settings of M and efConstruction. (Even > adding graph layers in LUCENE-10054 didn't really affect recall.) With this > change, the recall is now very similar: > *Results on glove-100-angular* > Parameters: M=32, efConstruction=100 > {code:java} > kApproach Recall > QPS > 10 luceneknn dim=100 {'M': 32, 'efConstruction': 100}0.563 >4410.499 > 50 luceneknn dim=100 {'M': 32, 'efConstruction': 100}0.798 >1956.280 > 100 luceneknn dim=100 {'M': 32, 'efConstruction': 100}0.862 >1209.734 > 500 luceneknn dim=100 {'M': 32, 'efConstruction': 100}0.958 > 341.428 > 800 luceneknn dim=100 {'M': 32, 'efConstruction': 100}0.974 > 230.396 > 1000 luceneknn dim=100 {'M': 32, 'efConstruction': 100}0.980 > 188.757 > 10 hnswlib ({'M': 32, 'efConstruction': 100})0.552 > 16745.433 > 50 hnswlib ({'M': 32, 'efConstruction': 100})0.794 >5738.468 > 100 hnswlib ({'M': 32, 'efConstruction': 100})0.860 >3336.386 > 500 hnswlib ({'M': 32, 'efConstruction': 100})0.956 > 832.982 > 800 hnswlib ({'M': 32, 'efConstruction': 100})0.973 > 541.097 > 1000 hnswlib ({'M': 32, 'efConstruction': 100})0.979 > 442.163 > {code} > I think it'd be nice update to maxConn so that we faithfully implement the > paper's algorithm. This is probably least surprising for users, and I don't > see a strong reason to take a different approach from the paper? Let me know > what you think! -- This message was sent by Atlassian Jira (v8.20.7#820007) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Closed] (LUCENE-10527) Use bigger maxConn for last layer in HNSW
[ https://issues.apache.org/jira/browse/LUCENE-10527?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mayya Sharipova closed LUCENE-10527. > Use bigger maxConn for last layer in HNSW > - > > Key: LUCENE-10527 > URL: https://issues.apache.org/jira/browse/LUCENE-10527 > Project: Lucene - Core > Issue Type: Task >Reporter: Julie Tibshirani >Assignee: Mayya Sharipova >Priority: Minor > Fix For: 9.2 > > Attachments: Screen Shot 2022-05-18 at 4.26.14 PM.png, Screen Shot > 2022-05-18 at 4.26.24 PM.png, Screen Shot 2022-05-18 at 4.27.37 PM.png, > image-2022-04-20-14-53-58-484.png > > Time Spent: 4h 40m > Remaining Estimate: 0h > > Recently I was rereading the HNSW paper > ([https://arxiv.org/pdf/1603.09320.pdf)] and noticed that they suggest using > a different maxConn for the upper layers vs. the bottom one (which contains > the full neighborhood graph). Specifically, they suggest using maxConn=M for > upper layers and maxConn=2*M for the bottom. This differs from what we do, > which is to use maxConn=M for all layers. > I tried updating our logic using a hacky patch, and noticed an improvement in > latency for higher recall values (which is consistent with the paper's > observation): > *Results on glove-100-angular* > Parameters: M=32, efConstruction=100 > !image-2022-04-20-14-53-58-484.png|width=400,height=367! > As we'd expect, indexing becomes a bit slower: > {code:java} > Baseline: Indexed 1183514 documents in 733s > Candidate: Indexed 1183514 documents in 948s{code} > When we benchmarked Lucene HNSW against hnswlib in LUCENE-9937, we noticed a > big difference in recall for the same settings of M and efConstruction. (Even > adding graph layers in LUCENE-10054 didn't really affect recall.) With this > change, the recall is now very similar: > *Results on glove-100-angular* > Parameters: M=32, efConstruction=100 > {code:java} > kApproach Recall > QPS > 10 luceneknn dim=100 {'M': 32, 'efConstruction': 100}0.563 >4410.499 > 50 luceneknn dim=100 {'M': 32, 'efConstruction': 100}0.798 >1956.280 > 100 luceneknn dim=100 {'M': 32, 'efConstruction': 100}0.862 >1209.734 > 500 luceneknn dim=100 {'M': 32, 'efConstruction': 100}0.958 > 341.428 > 800 luceneknn dim=100 {'M': 32, 'efConstruction': 100}0.974 > 230.396 > 1000 luceneknn dim=100 {'M': 32, 'efConstruction': 100}0.980 > 188.757 > 10 hnswlib ({'M': 32, 'efConstruction': 100})0.552 > 16745.433 > 50 hnswlib ({'M': 32, 'efConstruction': 100})0.794 >5738.468 > 100 hnswlib ({'M': 32, 'efConstruction': 100})0.860 >3336.386 > 500 hnswlib ({'M': 32, 'efConstruction': 100})0.956 > 832.982 > 800 hnswlib ({'M': 32, 'efConstruction': 100})0.973 > 541.097 > 1000 hnswlib ({'M': 32, 'efConstruction': 100})0.979 > 442.163 > {code} > I think it'd be nice update to maxConn so that we faithfully implement the > paper's algorithm. This is probably least surprising for users, and I don't > see a strong reason to take a different approach from the paper? Let me know > what you think! -- This message was sent by Atlassian Jira (v8.20.7#820007) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Resolved] (LUCENE-10312) Add PersianStemmer
[ https://issues.apache.org/jira/browse/LUCENE-10312?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tomoko Uchida resolved LUCENE-10312. Resolution: Fixed We addressed the backward compatibility issue. I'm closing this to avoid possible confusion in the 9.2 release operation but will make some necessary follow-ups (MIGRATE entry, javadocs, minor refactoring on tests) on main. > Add PersianStemmer > -- > > Key: LUCENE-10312 > URL: https://issues.apache.org/jira/browse/LUCENE-10312 > Project: Lucene - Core > Issue Type: Wish > Components: modules/analysis >Affects Versions: 9.0 >Reporter: Ramin Alirezaee >Priority: Major > Fix For: 10.0 (main), 9.2 > > Attachments: image.png > > Time Spent: 10h > Remaining Estimate: 0h > -- This message was sent by Atlassian Jira (v8.20.7#820007) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Commented] (LUCENE-3373) waitForMerges deadlocks if background merge fails
[ https://issues.apache.org/jira/browse/LUCENE-3373?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17539536#comment-17539536 ] Thomas Hoffmann commented on LUCENE-3373: - I think I encountered the same behaviour with version 8.11.1: "https-openssl-nio-443-exec-30" #166 daemon prio=5 os_prio=0 cpu=78703.13ms elapsed=81248.18s tid=0x2860af10 nid=0x237c in Object.wait() [0x413fc000] java.lang.Thread.State: TIMED_WAITING (on object monitor) at java.lang.Object.wait(java.base@17.0.2/Native Method) - waiting on at org.apache.lucene.index.IndexWriter.doWait(IndexWriter.java:4983) - locked <0x0006ef1fc020> (a org.apache.lucene.index.IndexWriter) at org.apache.lucene.index.IndexWriter.waitForMerges(IndexWriter.java:2697) - locked <0x0006ef1fc020> (a org.apache.lucene.index.IndexWriter) at org.apache.lucene.index.IndexWriter.shutdown(IndexWriter.java:1236) at org.apache.lucene.index.IndexWriter.close(IndexWriter.java:1278) at com.xxx.module.search.SearchService.updateSearchIndex(SearchService.java:1723) - locked <0x0006d5c00208> (a org.apache.lucene.store.MMapDirectory) at com.xxx.module.businessrelations.ticket.TicketChangedListener.postUpdate(TicketChangedListener.java:142) The call never returned and caused the application to hook. > waitForMerges deadlocks if background merge fails > - > > Key: LUCENE-3373 > URL: https://issues.apache.org/jira/browse/LUCENE-3373 > Project: Lucene - Core > Issue Type: Bug > Components: core/index >Affects Versions: 3.0.3 >Reporter: Tim Smith >Priority: Major > > waitForMerges can deadlock if a merge fails for ConcurrentMergeScheduler > this is because the merge thread will die, but pending merges are still > available > normally, the merge thread will pick up the next merge once it finishes the > previous merge, but in the event of a merge exception, the pending work is > not resumed, but waitForMerges won't complete until all pending work is > complete > i worked around this by overriding doMerge() like so: > {code} > protected final void doMerge(MergePolicy.OneMerge merge) throws IOException > { > try { > super.doMerge(merge); > } catch (Throwable exc) { > // Just logging the exception and not rethrowing > // insert logging code here > } > } > {code} > Here's the rough steps i used to reproduce this issue: > override doMerge like so > {code} > protected final void doMerge(MergePolicy.OneMerge merge) throws IOException > { > try {Thread.sleep(500L);} catch (InterruptedException e) { } > super.doMerge(merge); > throw new IOException("fail"); > } > {code} > then, if you do the following: > loop 50 times: > addDocument // any doc > commit > waitForMerges // This will deadlock sometimes > SOLR-2017 may be related to this (stack trace for deadlock looked related) -- This message was sent by Atlassian Jira (v8.20.7#820007) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Comment Edited] (LUCENE-3373) waitForMerges deadlocks if background merge fails
[ https://issues.apache.org/jira/browse/LUCENE-3373?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17539536#comment-17539536 ] Thomas Hoffmann edited comment on LUCENE-3373 at 5/19/22 1:07 PM: -- I think I encountered the same behaviour with version 8.11.1: "https-openssl-nio-443-exec-30" #166 daemon prio=5 os_prio=0 cpu=78703.13ms elapsed=81248.18s tid=0x2860af10 nid=0x237c in Object.wait() [0x413fc000] java.lang.Thread.State: TIMED_WAITING (on object monitor) at java.lang.Object.wait(java.base@17.0.2/Native Method) - waiting on at org.apache.lucene.index.IndexWriter.doWait(IndexWriter.java:4983) - locked <0x0006ef1fc020> (a org.apache.lucene.index.IndexWriter) at org.apache.lucene.index.IndexWriter.waitForMerges(IndexWriter.java:2697) - locked <0x0006ef1fc020> (a org.apache.lucene.index.IndexWriter) at org.apache.lucene.index.IndexWriter.shutdown(IndexWriter.java:1236) at org.apache.lucene.index.IndexWriter.close(IndexWriter.java:1278) at com.xxx.module.search.SearchService.updateSearchIndex(SearchService.java:1723) - locked <0x0006d5c00208> (a org.apache.lucene.store.MMapDirectory) at com.xxx.module.businessrelations.ticket.TicketChangedListener.postUpdate(TicketChangedListener.java:142) The call never returned and caused the application to hook. OS: Windows 2019 was (Author: tom_s4t): I think I encountered the same behaviour with version 8.11.1: "https-openssl-nio-443-exec-30" #166 daemon prio=5 os_prio=0 cpu=78703.13ms elapsed=81248.18s tid=0x2860af10 nid=0x237c in Object.wait() [0x413fc000] java.lang.Thread.State: TIMED_WAITING (on object monitor) at java.lang.Object.wait(java.base@17.0.2/Native Method) - waiting on at org.apache.lucene.index.IndexWriter.doWait(IndexWriter.java:4983) - locked <0x0006ef1fc020> (a org.apache.lucene.index.IndexWriter) at org.apache.lucene.index.IndexWriter.waitForMerges(IndexWriter.java:2697) - locked <0x0006ef1fc020> (a org.apache.lucene.index.IndexWriter) at org.apache.lucene.index.IndexWriter.shutdown(IndexWriter.java:1236) at org.apache.lucene.index.IndexWriter.close(IndexWriter.java:1278) at com.xxx.module.search.SearchService.updateSearchIndex(SearchService.java:1723) - locked <0x0006d5c00208> (a org.apache.lucene.store.MMapDirectory) at com.xxx.module.businessrelations.ticket.TicketChangedListener.postUpdate(TicketChangedListener.java:142) The call never returned and caused the application to hook. > waitForMerges deadlocks if background merge fails > - > > Key: LUCENE-3373 > URL: https://issues.apache.org/jira/browse/LUCENE-3373 > Project: Lucene - Core > Issue Type: Bug > Components: core/index >Affects Versions: 3.0.3 >Reporter: Tim Smith >Priority: Major > > waitForMerges can deadlock if a merge fails for ConcurrentMergeScheduler > this is because the merge thread will die, but pending merges are still > available > normally, the merge thread will pick up the next merge once it finishes the > previous merge, but in the event of a merge exception, the pending work is > not resumed, but waitForMerges won't complete until all pending work is > complete > i worked around this by overriding doMerge() like so: > {code} > protected final void doMerge(MergePolicy.OneMerge merge) throws IOException > { > try { > super.doMerge(merge); > } catch (Throwable exc) { > // Just logging the exception and not rethrowing > // insert logging code here > } > } > {code} > Here's the rough steps i used to reproduce this issue: > override doMerge like so > {code} > protected final void doMerge(MergePolicy.OneMerge merge) throws IOException > { > try {Thread.sleep(500L);} catch (InterruptedException e) { } > super.doMerge(merge); > throw new IOException("fail"); > } > {code} > then, if you do the following: > loop 50 times: > addDocument // any doc > commit > waitForMerges // This will deadlock sometimes > SOLR-2017 may be related to this (stack trace for deadlock looked related) -- This message was sent by Atlassian Jira (v8.20.7#820007) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Resolved] (LUCENE-10553) WANDScorer's handling of 0 and +Infty is backwards
[ https://issues.apache.org/jira/browse/LUCENE-10553?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Adrien Grand resolved LUCENE-10553. --- Fix Version/s: 9.2 Resolution: Fixed > WANDScorer's handling of 0 and +Infty is backwards > -- > > Key: LUCENE-10553 > URL: https://issues.apache.org/jira/browse/LUCENE-10553 > Project: Lucene - Core > Issue Type: Improvement >Reporter: Adrien Grand >Priority: Minor > Fix For: 9.2 > > Time Spent: 1h > Remaining Estimate: 0h > > WANDScorer has special logic to deal with corner cases when the sum of the > maximum scores of sub queries is either 0 or +Infty, but both code and tests > have backwards logic regarding this special case, doing +1 instead of -1 and > vice-versa. > This leads to a failed assertion in the case when the sum of the scores of > the sub queries overflows, which typically happens if one of the clauses has > a default implementation that returns MAX_VALUE if it cannot reason about max > scores. -- This message was sent by Atlassian Jira (v8.20.7#820007) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] ctlgcustodio commented on pull request #874: LUCENE-10471 Increse max dims for vectors to 2048
ctlgcustodio commented on PR #874: URL: https://github.com/apache/lucene/pull/874#issuecomment-1131711152 > My concerns are on the JIRA issue, I don't want them to be forgotten. https://issues.apache.org/jira/browse/LUCENE-10471 > > I don't know how we can say "we will not recommend further increase". What happens when the latest trendy dataset comes out with 4096 dimensions? > > I want to understand, why so many dimensions are really needed for search purposes. What is the concrete benefit in terms of quality, because we know what the performance hit is going to be. I understand that in general the more features you have in a vector of embeddings, the more details the model returns from the classification. So you have a more refined result. However, while it does not support greater than 1024, if possible use a weighted average and evaluate your result. In my case I used Fixed Average and it worked fine for Elmo model, as mentioned here in 3 Alternative Weighting Schemes https://arxiv.org/pdf/1904.02954.pdf [Other option](https://github.com/lior-k/fast-elasticsearch-vector-scoring) If I'm not mistaken this git is capable of supporting vectors larger than 1024. -- 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
[GitHub] [lucene] msokolov commented on a diff in pull request #899: Lucene 10577
msokolov commented on code in PR #899: URL: https://github.com/apache/lucene/pull/899#discussion_r877085508 ## lucene/core/src/java/org/apache/lucene/codecs/lucene92/ExpandingRandomAccessVectorValues.java: ## @@ -0,0 +1,57 @@ +package org.apache.lucene.codecs.lucene92; + +import org.apache.lucene.index.RandomAccessVectorValues; +import org.apache.lucene.index.RandomAccessVectorValuesProducer; +import org.apache.lucene.util.BytesRef; + +import java.io.IOException; + +public class ExpandingRandomAccessVectorValues implements RandomAccessVectorValuesProducer { + + private final RandomAccessVectorValuesProducer delegate; + private final float scale; + + /** + * Wraps an existing vector values producer. Floating point vector values will be produced by scaling + * byte-quantized values read from the values produced by the input. + */ + protected ExpandingRandomAccessVectorValues(RandomAccessVectorValuesProducer in, float scale) { +this.delegate = in; +assert scale != 0; +this.scale = scale; + } + + @Override + public RandomAccessVectorValues randomAccess() throws IOException { +RandomAccessVectorValues delegateValues = delegate.randomAccess(); +float[] value = new float[delegateValues.dimension()];; + +return new RandomAccessVectorValues() { + + @Override + public int size() { +return delegateValues.size(); + } + + @Override + public int dimension() { +return delegateValues.dimension(); + } + + @Override + public float[] vectorValue(int targetOrd) throws IOException { +BytesRef binaryValue = delegateValues.binaryValue(targetOrd); +byte[] bytes = binaryValue.bytes; +for (int i = 0, j = binaryValue.offset; i < value.length; i++, j++) { + value[i] = bytes[j] * scale; Review Comment: Well, there are definitely byte-oriented vectors. I don't think we should try to use some kind of 8-bit floating point (what would that do, have an exponent and a mantissa?) rather if we can scale to use the entire 8 bits as significant bits (no exponent) then we maximize the precision and can use the native instruction set, which ByteVector does seem to be accessing (based on some preliminary JMH I ran it seems quite a bit faster than 32-bit FloatVector). For now I see two directions: (1) figure out how to scale vectors and store as signed bytes at full precision (this issue), and (2) work up performant Vector API implementations of that scaling and the dot product in anticipation of the vector API hatching along the lines of the patch you already did, @rcmuir. Maybe we can push that to a branch that builds with `--add-modules jdk.incubator.vector` for posterity and ease of testing. I don't think we need to block (1) for (2) since it seems clear there will be a path to vectorization. In the meantime we can implement a byte-at-a-time dot product to replace the float-at-a-time dot product? -- 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
[GitHub] [lucene] rmuir commented on a diff in pull request #899: Lucene 10577
rmuir commented on code in PR #899: URL: https://github.com/apache/lucene/pull/899#discussion_r877101477 ## lucene/core/src/java/org/apache/lucene/codecs/lucene92/ExpandingRandomAccessVectorValues.java: ## @@ -0,0 +1,57 @@ +package org.apache.lucene.codecs.lucene92; + +import org.apache.lucene.index.RandomAccessVectorValues; +import org.apache.lucene.index.RandomAccessVectorValuesProducer; +import org.apache.lucene.util.BytesRef; + +import java.io.IOException; + +public class ExpandingRandomAccessVectorValues implements RandomAccessVectorValuesProducer { + + private final RandomAccessVectorValuesProducer delegate; + private final float scale; + + /** + * Wraps an existing vector values producer. Floating point vector values will be produced by scaling + * byte-quantized values read from the values produced by the input. + */ + protected ExpandingRandomAccessVectorValues(RandomAccessVectorValuesProducer in, float scale) { +this.delegate = in; +assert scale != 0; +this.scale = scale; + } + + @Override + public RandomAccessVectorValues randomAccess() throws IOException { +RandomAccessVectorValues delegateValues = delegate.randomAccess(); +float[] value = new float[delegateValues.dimension()];; + +return new RandomAccessVectorValues() { + + @Override + public int size() { +return delegateValues.size(); + } + + @Override + public int dimension() { +return delegateValues.dimension(); + } + + @Override + public float[] vectorValue(int targetOrd) throws IOException { +BytesRef binaryValue = delegateValues.binaryValue(targetOrd); +byte[] bytes = binaryValue.bytes; +for (int i = 0, j = binaryValue.offset; i < value.length; i++, j++) { + value[i] = bytes[j] * scale; Review Comment: I'm not really seeing the path to vectorization here though. At least not based on what i see here. Like i said, I really do think it should block the issue because if we add support to the index format it is difficult/impossible to get rid of. -- 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
[GitHub] [lucene] rmuir commented on a diff in pull request #899: Lucene 10577
rmuir commented on code in PR #899: URL: https://github.com/apache/lucene/pull/899#discussion_r877128319 ## lucene/core/src/java/org/apache/lucene/codecs/lucene92/ExpandingRandomAccessVectorValues.java: ## @@ -0,0 +1,57 @@ +package org.apache.lucene.codecs.lucene92; + +import org.apache.lucene.index.RandomAccessVectorValues; +import org.apache.lucene.index.RandomAccessVectorValuesProducer; +import org.apache.lucene.util.BytesRef; + +import java.io.IOException; + +public class ExpandingRandomAccessVectorValues implements RandomAccessVectorValuesProducer { + + private final RandomAccessVectorValuesProducer delegate; + private final float scale; + + /** + * Wraps an existing vector values producer. Floating point vector values will be produced by scaling + * byte-quantized values read from the values produced by the input. + */ + protected ExpandingRandomAccessVectorValues(RandomAccessVectorValuesProducer in, float scale) { +this.delegate = in; +assert scale != 0; +this.scale = scale; + } + + @Override + public RandomAccessVectorValues randomAccess() throws IOException { +RandomAccessVectorValues delegateValues = delegate.randomAccess(); +float[] value = new float[delegateValues.dimension()];; + +return new RandomAccessVectorValues() { + + @Override + public int size() { +return delegateValues.size(); + } + + @Override + public int dimension() { +return delegateValues.dimension(); + } + + @Override + public float[] vectorValue(int targetOrd) throws IOException { +BytesRef binaryValue = delegateValues.binaryValue(targetOrd); +byte[] bytes = binaryValue.bytes; +for (int i = 0, j = binaryValue.offset; i < value.length; i++, j++) { + value[i] = bytes[j] * scale; Review Comment: I think i'd be able to help more, if the current PR was actually doing said byte-based formula and not converting to floats. The way the code is structured now (doing floating point based math), I can't really help, it won't be efficient. -- 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
[jira] [Commented] (LUCENE-10559) Add preFilter/postFilter options to KnnGraphTester
[ https://issues.apache.org/jira/browse/LUCENE-10559?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17539583#comment-17539583 ] Michael Sokolov commented on LUCENE-10559: -- I think it makes sense to use a fixed bit set so that we can test HNSW performance with filtering independently from the cost of the filter Query. I think your test seems to be demonstrating that for similar latencies (~cost) we can achieve significantly higher recall with pre-filtering? I wonder if we could also demonstrate the converse -- what effective topK is required when post-filtering to drive recall to be the same as pre-filtering? Also, these recall numbers seem curiously high, higher than we usually see. Could you publish the graph construction and HNSW search time parameters you used? I'm also wondering whether perhaps you tested with vectors from the training set? > Add preFilter/postFilter options to KnnGraphTester > -- > > Key: LUCENE-10559 > URL: https://issues.apache.org/jira/browse/LUCENE-10559 > Project: Lucene - Core > Issue Type: Improvement >Reporter: Michael Sokolov >Priority: Major > > We want to be able to test the efficacy of pre-filtering in KnnVectorQuery: > if you (say) want the top K nearest neighbors subject to a constraint Q, are > you better off over-selecting (say 2K) top hits and *then* filtering > (post-filtering), or incorporating the filtering into the query > (pre-filtering). How does it depend on the selectivity of the filter? > I think we can get a reasonable testbed by generating a uniform random filter > with some selectivity (that is consistent and repeatable). Possibly we'd also > want to try filters that are correlated with index order, but it seems they'd > be unlikely to be correlated with vector values in a way that the graph > structure would notice, so random is a pretty good starting point for this. -- This message was sent by Atlassian Jira (v8.20.7#820007) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] rmuir commented on pull request #901: remove commented-out/obselete AwaitsFix
rmuir commented on PR #901: URL: https://github.com/apache/lucene/pull/901#issuecomment-1131804325 Thanks @dweiss for taking one off the list! I looked into the MoreLikeThis one, https://issues.apache.org/jira/browse/LUCENE-7161 This test has been `@AwaitsFix` for over 6 years now, this is my concern with this annotation :( I beasted the test (probably "incorrectly/illegally") and made it pop relatively easily, but then resulting seeds don't reproduce. -- 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
[jira] [Assigned] (LUCENE-10504) KnnGraphTester should use KnnVectorQuery
[ https://issues.apache.org/jira/browse/LUCENE-10504?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Sokolov reassigned LUCENE-10504: Assignee: Michael Sokolov > KnnGraphTester should use KnnVectorQuery > > > Key: LUCENE-10504 > URL: https://issues.apache.org/jira/browse/LUCENE-10504 > Project: Lucene - Core > Issue Type: Improvement >Reporter: Michael Sokolov >Assignee: Michael Sokolov >Priority: Major > Time Spent: 1h 40m > Remaining Estimate: 0h > > to get a more realistic picture, and to track developments in the query > implementation, the tester should use that rather than implementing its own > per-segment search and merging logic. -- This message was sent by Atlassian Jira (v8.20.7#820007) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Resolved] (LUCENE-10504) KnnGraphTester should use KnnVectorQuery
[ https://issues.apache.org/jira/browse/LUCENE-10504?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Sokolov resolved LUCENE-10504. -- Fix Version/s: 10.0 (main) Resolution: Fixed > KnnGraphTester should use KnnVectorQuery > > > Key: LUCENE-10504 > URL: https://issues.apache.org/jira/browse/LUCENE-10504 > Project: Lucene - Core > Issue Type: Improvement >Reporter: Michael Sokolov >Assignee: Michael Sokolov >Priority: Major > Fix For: 10.0 (main) > > Time Spent: 1h 40m > Remaining Estimate: 0h > > to get a more realistic picture, and to track developments in the query > implementation, the tester should use that rather than implementing its own > per-segment search and merging logic. -- This message was sent by Atlassian Jira (v8.20.7#820007) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Commented] (LUCENE-7161) TestMoreLikeThis.testMultiFieldShouldReturnPerFieldBooleanQuery assertion error
[ https://issues.apache.org/jira/browse/LUCENE-7161?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17539601#comment-17539601 ] Robert Muir commented on LUCENE-7161: - I beasted it and can reproduce it like this: {noformat} $ git checkout 8e777a132039fe2e6b70a600f5f02753a2d80e10 $ ./gradlew -p lucene/queries test --tests TestMoreLikeThis.testMultiFieldShouldReturnPerFieldBooleanQuery -Dtests.seed=815E65E6B88AAE43 {noformat} The test prints this when it fails: {noformat} org.apache.lucene.queries.mlt.TestMoreLikeThis > testMultiFieldShouldReturnPerFieldBooleanQuery FAILED java.lang.AssertionError at __randomizedtesting.SeedInfo.seed([815E65E6B88AAE43:E5ED3D20056428B8]:0) at junit@4.13.1/org.junit.Assert.fail(Assert.java:87) at junit@4.13.1/org.junit.Assert.assertTrue(Assert.java:42) at junit@4.13.1/org.junit.Assert.assertTrue(Assert.java:53) at org.apache.lucene.queries.mlt.TestMoreLikeThis.testMultiFieldShouldReturnPerFieldBooleanQuery(TestMoreLikeThis.java:534) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77) at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.base/java.lang.reflect.Method.invoke(Method.java:568) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.RandomizedRunner.invoke(RandomizedRunner.java:1754) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.RandomizedRunner$8.evaluate(RandomizedRunner.java:942) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.RandomizedRunner$9.evaluate(RandomizedRunner.java:978) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.RandomizedRunner$10.evaluate(RandomizedRunner.java:992) at org.apache.lucene.test_framework@10.0.0-SNAPSHOT/org.apache.lucene.tests.util.TestRuleSetupTeardownChained$1.evaluate(TestRuleSetupTeardownChained.java:44) at org.apache.lucene.test_framework@10.0.0-SNAPSHOT/org.apache.lucene.tests.util.AbstractBeforeAfterRule$1.evaluate(AbstractBeforeAfterRule.java:43) at org.apache.lucene.test_framework@10.0.0-SNAPSHOT/org.apache.lucene.tests.util.TestRuleThreadAndTestName$1.evaluate(TestRuleThreadAndTestName.java:45) at org.apache.lucene.test_framework@10.0.0-SNAPSHOT/org.apache.lucene.tests.util.TestRuleIgnoreAfterMaxFailures$1.evaluate(TestRuleIgnoreAfterMaxFailures.java:60) at org.apache.lucene.test_framework@10.0.0-SNAPSHOT/org.apache.lucene.tests.util.TestRuleMarkFailure$1.evaluate(TestRuleMarkFailure.java:44) at junit@4.13.1/org.junit.rules.RunRules.evaluate(RunRules.java:20) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.rules.StatementAdapter.evaluate(StatementAdapter.java:36) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.ThreadLeakControl$StatementRunner.run(ThreadLeakControl.java:370) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.ThreadLeakControl.forkTimeoutingTask(ThreadLeakControl.java:819) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.ThreadLeakControl$3.evaluate(ThreadLeakControl.java:470) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.RandomizedRunner.runSingleTest(RandomizedRunner.java:951) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.RandomizedRunner$5.evaluate(RandomizedRunner.java:836) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.RandomizedRunner$6.evaluate(RandomizedRunner.java:887) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.RandomizedRunner$7.evaluate(RandomizedRunner.java:898) at org.apache.lucene.test_framework@10.0.0-SNAPSHOT/org.apache.lucene.tests.util.AbstractBeforeAfterRule$1.evaluate(AbstractBeforeAfterRule.java:43) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.rules.StatementAdapter.evaluate(StatementAdapter.java:36) at org.apache.lucene.test_framework@10.0.0-SNAPSHOT/org.apache.lucene.tests.util.TestRuleStoreClassName$1.evaluate(TestRuleStoreClassName.java:38) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.rules.NoShadowingOrOverridesOnMethodsRule$1.evaluate(NoShadowingOrOverridesOnMethodsRule.java:40) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.rules.NoShadowingOrOverridesOnMethodsRule$1.evaluate(NoShadowingOrOverridesOnMethodsRule.java:40) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.rules.StatementAdapter.evaluate(StatementAdapter.java:36) at randomizedtesting.runner@2.7.6/com.carro
[GitHub] [lucene] msokolov commented on a diff in pull request #899: Lucene 10577
msokolov commented on code in PR #899: URL: https://github.com/apache/lucene/pull/899#discussion_r877179592 ## lucene/core/src/java/org/apache/lucene/codecs/lucene92/ExpandingRandomAccessVectorValues.java: ## @@ -0,0 +1,57 @@ +package org.apache.lucene.codecs.lucene92; + +import org.apache.lucene.index.RandomAccessVectorValues; +import org.apache.lucene.index.RandomAccessVectorValuesProducer; +import org.apache.lucene.util.BytesRef; + +import java.io.IOException; + +public class ExpandingRandomAccessVectorValues implements RandomAccessVectorValuesProducer { + + private final RandomAccessVectorValuesProducer delegate; + private final float scale; + + /** + * Wraps an existing vector values producer. Floating point vector values will be produced by scaling + * byte-quantized values read from the values produced by the input. + */ + protected ExpandingRandomAccessVectorValues(RandomAccessVectorValuesProducer in, float scale) { +this.delegate = in; +assert scale != 0; +this.scale = scale; + } + + @Override + public RandomAccessVectorValues randomAccess() throws IOException { +RandomAccessVectorValues delegateValues = delegate.randomAccess(); +float[] value = new float[delegateValues.dimension()];; + +return new RandomAccessVectorValues() { + + @Override + public int size() { +return delegateValues.size(); + } + + @Override + public int dimension() { +return delegateValues.dimension(); + } + + @Override + public float[] vectorValue(int targetOrd) throws IOException { +BytesRef binaryValue = delegateValues.binaryValue(targetOrd); +byte[] bytes = binaryValue.bytes; +for (int i = 0, j = binaryValue.offset; i < value.length; i++, j++) { + value[i] = bytes[j] * scale; Review Comment: Yeah - I hear you. I will try to work out more concretely what that would look like. My initial preoccupation has been with (1) a reasonable API and (2) ensuring that 8 bits is enough to get good recall. -- 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
[GitHub] [lucene] rmuir commented on a diff in pull request #899: Lucene 10577
rmuir commented on code in PR #899: URL: https://github.com/apache/lucene/pull/899#discussion_r877182870 ## lucene/core/src/java/org/apache/lucene/codecs/lucene92/ExpandingRandomAccessVectorValues.java: ## @@ -0,0 +1,57 @@ +package org.apache.lucene.codecs.lucene92; + +import org.apache.lucene.index.RandomAccessVectorValues; +import org.apache.lucene.index.RandomAccessVectorValuesProducer; +import org.apache.lucene.util.BytesRef; + +import java.io.IOException; + +public class ExpandingRandomAccessVectorValues implements RandomAccessVectorValuesProducer { + + private final RandomAccessVectorValuesProducer delegate; + private final float scale; + + /** + * Wraps an existing vector values producer. Floating point vector values will be produced by scaling + * byte-quantized values read from the values produced by the input. + */ + protected ExpandingRandomAccessVectorValues(RandomAccessVectorValuesProducer in, float scale) { +this.delegate = in; +assert scale != 0; +this.scale = scale; + } + + @Override + public RandomAccessVectorValues randomAccess() throws IOException { +RandomAccessVectorValues delegateValues = delegate.randomAccess(); +float[] value = new float[delegateValues.dimension()];; + +return new RandomAccessVectorValues() { + + @Override + public int size() { +return delegateValues.size(); + } + + @Override + public int dimension() { +return delegateValues.dimension(); + } + + @Override + public float[] vectorValue(int targetOrd) throws IOException { +BytesRef binaryValue = delegateValues.binaryValue(targetOrd); +byte[] bytes = binaryValue.bytes; +for (int i = 0, j = binaryValue.offset; i < value.length; i++, j++) { + value[i] = bytes[j] * scale; Review Comment: from my perspective, if you are able to do calculations with integer-based math, on 8-bit byte or 16-bit short, then we should be able to get things going. But i don't know what the formula/method signature would look like to do any testing :) -- 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
[jira] [Assigned] (LUCENE-7161) TestMoreLikeThis.testMultiFieldShouldReturnPerFieldBooleanQuery assertion error
[ https://issues.apache.org/jira/browse/LUCENE-7161?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tommaso Teofili reassigned LUCENE-7161: --- Assignee: (was: Tommaso Teofili) > TestMoreLikeThis.testMultiFieldShouldReturnPerFieldBooleanQuery assertion > error > --- > > Key: LUCENE-7161 > URL: https://issues.apache.org/jira/browse/LUCENE-7161 > Project: Lucene - Core > Issue Type: Bug >Reporter: Michael McCandless >Priority: Major > Fix For: 6.7, 7.0 > > > I just hit this unrelated but reproducible on master > #cc75be53f9b3b86ec59cb93896c4fd5a9a5926b2 while tweaking earth's radius: > {noformat} >[junit4] Suite: org.apache.lucene.queries.mlt.TestMoreLikeThis >[junit4] 2> NOTE: reproduce with: ant test -Dtestcase=TestMoreLikeThis > -Dtests.method=testMultiFieldShouldReturnPerFieldBooleanQuery > -Dtests.seed=794526110651C8E6 -Dtests.locale=es-HN > -Dtests.timezone=Brazil/West -Dtests.asserts=true > -Dtests.file.encoding=US-ASCII >[junit4] FAILURE 0.25s | > TestMoreLikeThis.testMultiFieldShouldReturnPerFieldBooleanQuery <<< >[junit4]> Throwable #1: java.lang.AssertionError >[junit4]> at > __randomizedtesting.SeedInfo.seed([794526110651C8E6:1DF67ED7BBBF4E1D]:0) >[junit4]> at > org.apache.lucene.queries.mlt.TestMoreLikeThis.testMultiFieldShouldReturnPerFieldBooleanQuery(TestMoreLikeThis.java:320) >[junit4]> at java.lang.Thread.run(Thread.java:745) >[junit4] 2> NOTE: test params are: codec=CheapBastard, > sim=ClassicSimilarity, locale=es-HN, timezone=Brazil/West >[junit4] 2> NOTE: Linux 3.13.0-71-generic amd64/Oracle Corporation > 1.8.0_60 (64-bit)/cpus=8,threads=1,free=409748864,total=504889344 >[junit4] 2> NOTE: All tests run in this JVM: [TestMoreLikeThis] >[junit4] Completed [1/1 (1!)] in 0.45s, 1 test, 1 failure <<< FAILURES! >[junit4] >[junit4] >[junit4] Tests with failures [seed: 794526110651C8E6]: >[junit4] - > org.apache.lucene.queries.mlt.TestMoreLikeThis.testMultiFieldShouldReturnPerFieldBooleanQuery > {noformat} > Likely related to LUCENE-6954? -- This message was sent by Atlassian Jira (v8.20.7#820007) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Comment Edited] (LUCENE-7161) TestMoreLikeThis.testMultiFieldShouldReturnPerFieldBooleanQuery assertion error
[ https://issues.apache.org/jira/browse/LUCENE-7161?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17539601#comment-17539601 ] Robert Muir edited comment on LUCENE-7161 at 5/19/22 3:03 PM: -- I beasted it and can reproduce it like this: {noformat} $ git checkout 8e777a132039fe2e6b70a600f5f02753a2d80e10 << edit MoreLikeThis.java and remove AwaitsFix >> $ ./gradlew -p lucene/queries test --tests TestMoreLikeThis.testMultiFieldShouldReturnPerFieldBooleanQuery -Dtests.seed=815E65E6B88AAE43 {noformat} The test prints this when it fails: {noformat} org.apache.lucene.queries.mlt.TestMoreLikeThis > testMultiFieldShouldReturnPerFieldBooleanQuery FAILED java.lang.AssertionError at __randomizedtesting.SeedInfo.seed([815E65E6B88AAE43:E5ED3D20056428B8]:0) at junit@4.13.1/org.junit.Assert.fail(Assert.java:87) at junit@4.13.1/org.junit.Assert.assertTrue(Assert.java:42) at junit@4.13.1/org.junit.Assert.assertTrue(Assert.java:53) at org.apache.lucene.queries.mlt.TestMoreLikeThis.testMultiFieldShouldReturnPerFieldBooleanQuery(TestMoreLikeThis.java:534) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77) at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.base/java.lang.reflect.Method.invoke(Method.java:568) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.RandomizedRunner.invoke(RandomizedRunner.java:1754) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.RandomizedRunner$8.evaluate(RandomizedRunner.java:942) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.RandomizedRunner$9.evaluate(RandomizedRunner.java:978) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.RandomizedRunner$10.evaluate(RandomizedRunner.java:992) at org.apache.lucene.test_framework@10.0.0-SNAPSHOT/org.apache.lucene.tests.util.TestRuleSetupTeardownChained$1.evaluate(TestRuleSetupTeardownChained.java:44) at org.apache.lucene.test_framework@10.0.0-SNAPSHOT/org.apache.lucene.tests.util.AbstractBeforeAfterRule$1.evaluate(AbstractBeforeAfterRule.java:43) at org.apache.lucene.test_framework@10.0.0-SNAPSHOT/org.apache.lucene.tests.util.TestRuleThreadAndTestName$1.evaluate(TestRuleThreadAndTestName.java:45) at org.apache.lucene.test_framework@10.0.0-SNAPSHOT/org.apache.lucene.tests.util.TestRuleIgnoreAfterMaxFailures$1.evaluate(TestRuleIgnoreAfterMaxFailures.java:60) at org.apache.lucene.test_framework@10.0.0-SNAPSHOT/org.apache.lucene.tests.util.TestRuleMarkFailure$1.evaluate(TestRuleMarkFailure.java:44) at junit@4.13.1/org.junit.rules.RunRules.evaluate(RunRules.java:20) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.rules.StatementAdapter.evaluate(StatementAdapter.java:36) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.ThreadLeakControl$StatementRunner.run(ThreadLeakControl.java:370) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.ThreadLeakControl.forkTimeoutingTask(ThreadLeakControl.java:819) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.ThreadLeakControl$3.evaluate(ThreadLeakControl.java:470) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.RandomizedRunner.runSingleTest(RandomizedRunner.java:951) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.RandomizedRunner$5.evaluate(RandomizedRunner.java:836) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.RandomizedRunner$6.evaluate(RandomizedRunner.java:887) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.RandomizedRunner$7.evaluate(RandomizedRunner.java:898) at org.apache.lucene.test_framework@10.0.0-SNAPSHOT/org.apache.lucene.tests.util.AbstractBeforeAfterRule$1.evaluate(AbstractBeforeAfterRule.java:43) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.rules.StatementAdapter.evaluate(StatementAdapter.java:36) at org.apache.lucene.test_framework@10.0.0-SNAPSHOT/org.apache.lucene.tests.util.TestRuleStoreClassName$1.evaluate(TestRuleStoreClassName.java:38) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.rules.NoShadowingOrOverridesOnMethodsRule$1.evaluate(NoShadowingOrOverridesOnMethodsRule.java:40) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.rules.NoShadowingOrOverridesOnMethodsRule$1.evaluate(NoShadowingOrOverridesOnMethodsRule.java:40) at randomizedtesting.runner@2.7.6/com.carrotsearch.randomizedtesting.rules.State
[GitHub] [lucene] dweiss commented on pull request #901: remove commented-out/obselete AwaitsFix
dweiss commented on PR #901: URL: https://github.com/apache/lucene/pull/901#issuecomment-1131910048 > This test has been @AwaitsFix for over 6 years now, this is my concern with this annotation :( Well, we dug it up, eventually. What other means are there for marking tests that fail for some reason (that we don't know how to fix)? If you remove the test, it'll be gone forever. The replication tests (forks) I think I can handle - just not today. -- 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
[GitHub] [lucene] gsmiller commented on a diff in pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities
gsmiller commented on code in PR #841: URL: https://github.com/apache/lucene/pull/841#discussion_r877264170 ## lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java: ## @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.lucene.facet.hyperrectangle; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import org.apache.lucene.document.LongPoint; +import org.apache.lucene.facet.FacetResult; +import org.apache.lucene.facet.Facets; +import org.apache.lucene.facet.FacetsCollector; +import org.apache.lucene.facet.LabelAndValue; +import org.apache.lucene.index.BinaryDocValues; +import org.apache.lucene.index.DocValues; +import org.apache.lucene.search.DocIdSetIterator; + +/** Get counts given a list of HyperRectangles (which must be of the same type) */ +public class HyperRectangleFacetCounts extends Facets { + /** Hypper rectangles passed to constructor. */ + protected final HyperRectangle[] hyperRectangles; + + /** Counts, initialized in subclass. */ + protected final int[] counts; + + /** Our field name. */ + protected final String field; + + /** Number of dimensions for field */ + protected final int dims; + + /** Total number of hits. */ + protected int totCount; + + /** + * Create HyperRectangleFacetCounts using this + * + * @param field Field name + * @param hits Hits to facet on + * @param hyperRectangles List of hyper rectangle facets + * @throws IOException If there is a problem reading the field + */ + public HyperRectangleFacetCounts( + String field, FacetsCollector hits, HyperRectangle... hyperRectangles) throws IOException { +assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty"; +assert areHyperRectangleDimsConsistent(hyperRectangles) +: "All hyper rectangles must be the same dimensionality"; +this.field = field; +this.hyperRectangles = hyperRectangles; +this.dims = hyperRectangles[0].dims; +this.counts = new int[hyperRectangles.length]; +count(field, hits.getMatchingDocs()); + } + + private boolean areHyperRectangleDimsConsistent(HyperRectangle[] hyperRectangles) { +int dims = hyperRectangles[0].dims; +return Arrays.stream(hyperRectangles).allMatch(hyperRectangle -> hyperRectangle.dims == dims); + } + + /** Counts from the provided field. */ + private void count(String field, List matchingDocs) + throws IOException { + +for (int i = 0; i < matchingDocs.size(); i++) { Review Comment: minor: I'd suggest `for (FacetsCollector.MatchingDocs hits : matchingDocs)` as a slightly more idiomatic loop style since you don't actually care about the index. ## lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java: ## @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.lucene.facet.hyperrectangle; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import org.apache.lucene.document.LongPoint; +import org.apache.lucene.facet.FacetResult; +import org.apache.lucene.facet.Facets; +import org.apache.lucene.facet.FacetsCollector; +import org.apache.lucene.facet.LabelAndValue; +import org.apache.lucene.index.BinaryDocValues; +import org.apache.lucene.index.DocValues; +import org.apache.luc
[GitHub] [lucene] rmuir commented on pull request #901: remove commented-out/obselete AwaitsFix
rmuir commented on PR #901: URL: https://github.com/apache/lucene/pull/901#issuecomment-1131960030 > Well, we dug it up, eventually. What other means are there for marking tests that fail for some reason (that we don't know how to fix)? If you remove the test, it'll be gone forever. I don't have a better solution. I like that at least the annotation forces an issue to be opened. > The replication tests (forks) I think I can handle - just not today. Thanks! I wonder if forking is truly needed? I think it just wants to simulate "crashing" or bringing down nodes. -- 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
[GitHub] [lucene-solr] kiranchitturi opened a new pull request, #2659: SOLR-16199: Fix query syntax for LIKE queries with wildcard
kiranchitturi opened a new pull request, #2659: URL: https://github.com/apache/lucene-solr/pull/2659 Backport of https://github.com/apache/solr/pull/870 -- 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
[jira] [Created] (LUCENE-10580) Should we add a "slow range query" to xxxPoint classes?
Greg Miller created LUCENE-10580: Summary: Should we add a "slow range query" to xxxPoint classes? Key: LUCENE-10580 URL: https://issues.apache.org/jira/browse/LUCENE-10580 Project: Lucene - Core Issue Type: Improvement Components: core/index Reporter: Greg Miller Users that index 2D point data have the option of running a range query with, 1) the points index (via {{LongPoint#newRangeQuery}}), or 2) a doc values field (via {{SortedNumericDocValuesField#newSlowRangeQuery}}). But if users are indexing points data in higher dimensions, there's no equivalent "slow" query that I'm aware of (relying on doc values). It's useful to have both and be able to wrap them in {{IndexOrDocValuesQuery}}. I wonder if we should model a "point" doc value type (could just extend from {{BinaryDocValuesField}}) that supports creating "slow" range queries? -- This message was sent by Atlassian Jira (v8.20.7#820007) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] shahrs87 commented on pull request #897: LUCENE-10266 Move nearest-neighbor search on points to core
shahrs87 commented on PR #897: URL: https://github.com/apache/lucene/pull/897#issuecomment-1132091362 @jpountz @iverase @mikemccand Can you guys please help me review this patch ? Tagging Ignacio and Michael since you showed interest in the corresponding jira. Thank you. -- 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
[GitHub] [lucene-solr] thelabdude closed pull request #2657: SOLR-16199: Fix query syntax for LIKE queries with wildcard
thelabdude closed pull request #2657: SOLR-16199: Fix query syntax for LIKE queries with wildcard URL: https://github.com/apache/lucene-solr/pull/2657 -- 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
[GitHub] [lucene-solr] thelabdude commented on pull request #2657: SOLR-16199: Fix query syntax for LIKE queries with wildcard
thelabdude commented on PR #2657: URL: https://github.com/apache/lucene-solr/pull/2657#issuecomment-1132137271 Thanks @mkhludnev ... we ended up moving over to another PR for this one but appreciate the review. -- 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
[jira] [Commented] (LUCENE-10481) FacetsCollector does not need scores when not keeping them
[ https://issues.apache.org/jira/browse/LUCENE-10481?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17539786#comment-17539786 ] Adrien Grand commented on LUCENE-10481: --- I pushed an annotation to luceneutil, this triggered a good speedup for facets on term and boolean queries, e.g. http://people.apache.org/~mikemccand/lucenebench/TermDateFacets.html. > FacetsCollector does not need scores when not keeping them > -- > > Key: LUCENE-10481 > URL: https://issues.apache.org/jira/browse/LUCENE-10481 > Project: Lucene - Core > Issue Type: Improvement > Components: modules/facet >Reporter: Mike Drob >Assignee: Mike Drob >Priority: Major > Fix For: 8.11.2, 9.2 > > Time Spent: 50m > Remaining Estimate: 0h > > FacetsCollector currently always specifies ScoreMode.COMPLETE, we could get > better performance by not requesting scores when we don't need them. -- This message was sent by Atlassian Jira (v8.20.7#820007) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] shahrs87 opened a new pull request, #907: LUCENE-10357 Ghost fields and postings/points
shahrs87 opened a new pull request, #907: URL: https://github.com/apache/lucene/pull/907 # Description Please provide a short description of the changes you're making with this pull request. # Solution Please provide a short description of the approach taken to implement your solution. # Tests Please describe the tests you've developed or run to confirm this patch implements the feature or solves the problem. # Checklist Please review the following and check all that apply: - [ ] I have reviewed the guidelines for [How to Contribute](https://github.com/apache/lucene/blob/main/CONTRIBUTING.md) and my code conforms to the standards described there to the best of my ability. - [ ] I have given Lucene maintainers [access](https://help.github.com/en/articles/allowing-changes-to-a-pull-request-branch-created-from-a-fork) to contribute to my PR branch. (optional but recommended) - [ ] I have developed this patch against the `main` branch. - [ ] I have run `./gradlew check`. - [ ] I have added tests for my changes. -- 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
[GitHub] [lucene] shahrs87 commented on pull request #907: LUCENE-10357 Ghost fields and postings/points
shahrs87 commented on PR #907: URL: https://github.com/apache/lucene/pull/907#issuecomment-1132206775 Hi @jpountz, I have tried to put up a patch from the suggestions you have made in [LUCENE-10357](https://issues.apache.org/jira/browse/LUCENE-10357). Can you please review and provide feedback that I am going in the right direction ? Thank you. -- 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
[GitHub] [lucene-solr] thelabdude merged pull request #2659: SOLR-16199: Fix query syntax for LIKE queries with wildcard
thelabdude merged PR #2659: URL: https://github.com/apache/lucene-solr/pull/2659 -- 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
[GitHub] [lucene] gsmiller commented on a diff in pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities
gsmiller commented on code in PR #841: URL: https://github.com/apache/lucene/pull/841#discussion_r877627338 ## lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/DoubleHyperRectangle.java: ## @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.lucene.facet.hyperrectangle; + +import java.util.Arrays; +import org.apache.lucene.util.NumericUtils; + +/** Stores a hyper rectangle as an array of DoubleRangePairs */ +public class DoubleHyperRectangle extends HyperRectangle { Review Comment: As sort of a blanket comment for all the new classes you created, I might suggest we add `@lucene.experimental`. This is a pretty big, new bit of work, and I could see us wanting to tweak and change this a bit here-and-there. ## lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/DoubleHyperRectangle.java: ## @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.lucene.facet.hyperrectangle; + +import java.util.Arrays; +import org.apache.lucene.util.NumericUtils; + +/** Stores a hyper rectangle as an array of DoubleRangePairs */ +public class DoubleHyperRectangle extends HyperRectangle { + + /** Creates DoubleHyperRectangle */ + public DoubleHyperRectangle(String label, DoubleRangePair... pairs) { +super(label, convertToLongRangePairs(pairs)); + } + + private static LongRangePair[] convertToLongRangePairs(DoubleRangePair... pairs) { +if (pairs == null || pairs.length == 0) { + throw new IllegalArgumentException("Pairs cannot be null or empty"); +} +return Arrays.stream(pairs).map(DoubleRangePair::toLongRangePair).toArray(LongRangePair[]::new); + } + + /** Defines a single range in a DoubleHyperRectangle */ + public static class DoubleRangePair { +/** Inclusive min */ +public final double min; + +/** Inclusive max */ +public final double max; + +/** + * Creates a DoubleRangePair, very similar to the constructor of {@link + * org.apache.lucene.facet.range.DoubleRange} + * + * @param minIn Min value of pair + * @param minInclusive If minIn is inclusive + * @param maxIn Max value of pair + * @param maxInclusive If maxIn is inclusive + */ +public DoubleRangePair(double minIn, boolean minInclusive, double maxIn, boolean maxInclusive) { + if (Double.isNaN(minIn) || Double.isNaN(maxIn)) { +throw new IllegalArgumentException( +"min and max cannot be NaN: min=" + minIn + ", max=" + maxIn); + } + + if (!minInclusive) { +minIn = Math.nextUp(minIn); + } + + if (!maxInclusive) { +maxIn = Math.nextDown(maxIn); + } + + if (minIn > maxIn) { +throw new IllegalArgumentException( +"Minimum cannot be greater than maximum, max=" + maxIn + ", min=" + minIn); + } + + this.min = minIn; + this.max = maxIn; +} + +/** + * Converts this to a LongRangePair with sortable long equivalents + * + * @return A LongRangePair equivalent of this object + */ +public LongRangePair toLongRangePair() { Review Comment: Does this need to be public? I think it's only used internally in `DoubleHyperRectangle` right? Should we reduce visibility (unless we expect users need this functionality directly?). ## lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java: ###
[GitHub] [lucene] mocobeta opened a new pull request, #908: LUCENE-10312: small follow-ups
mocobeta opened a new pull request, #908: URL: https://github.com/apache/lucene/pull/908 - Add MIGRATE entry to preserve 9.x PersianAnalyzer behavior - Fix PersianAnalyzer javadocs - Forward port of minor test refactoring in #904 -- 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
[jira] [Commented] (LUCENE-10516) reduce unnecessary loop matches in BKDReader
[ https://issues.apache.org/jira/browse/LUCENE-10516?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17539903#comment-17539903 ] kkewwei commented on LUCENE-10516: -- For the spareDocValues, we use compression to store data: sameCount, detailValue, In the BKDReader, we compare the same batch docIds in the loop, the iterator seems useless. {code:java} // read cardinality and point private void visitSparseRawDocValues(int[] commonPrefixLengths, byte[] scratchPackedValue, IndexInput in, BKDReaderDocIDSetIterator scratchIterator, int count, IntersectVisitor visitor) throws IOException { int i; for (i = 0; i < count;) { // read the same values count int length = in.readVInt(); // read the detail values for(int dim = 0; dim < numDataDims; dim++) { int prefix = commonPrefixLengths[dim]; in.readBytes(scratchPackedValue, dim*bytesPerDim + prefix, bytesPerDim - prefix); } scratchIterator.reset(i, length); // iterate compare every same values. visitor.visit(scratchIterator, scratchPackedValue); i += length; } if (i != count) { throw new CorruptIndexException("Sub blocks do not add up to the expected count: " + count + " != " + i, in); } } {code} > reduce unnecessary loop matches in BKDReader > > > Key: LUCENE-10516 > URL: https://issues.apache.org/jira/browse/LUCENE-10516 > Project: Lucene - Core > Issue Type: Improvement > Components: core/other >Affects Versions: 8.6.2 >Reporter: kkewwei >Priority: Major > > In *BKDReader.visitSparseRawDocValues()*, we will read a batch of docIds > which have the same point value:*scratchPackedValue*, then call > *visitor.visit(scratchIterator, scratchPackedValue)* to find which docIDs > match the range. > {code:java} > default void visit(DocIdSetIterator iterator, byte[] packedValue) throws > IOException { > int docID; > while ((docID = iterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) { > visit(docID, packedValue); > } > } > {code} > We know that the packedValue are same for the batch of docIds, if the first > doc match the range, the batch of other docIds will also match the range, so > the loop seems useless. > We should call the method as follow: > {code:java} > public void visit(DocIdSetIterator iterator, byte[] packedValue) > throws IOException { > if (matches(packedValue)) { > int docID; > while ((docID = iterator.nextDoc()) != > DocIdSetIterator.NO_MORE_DOCS) { > visit(docID); > } > } > } > {code} > https://github.com/apache/lucene/blob/2e941fcfed6cad3d9c8667ff5324cd04858ba547/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java#L196 > If we should override the *visit(DocIdSetIterator iterator, byte[] > packedValue)* in *ExitableDirectoryReader$ExitableIntersectVisitor* to avoid > calling the default implement: > {code:java} > @Override > public void visit(DocIdSetIterator iterator, byte[] packedValue) > throws IOException { > queryCancellation.checkCancelled(); > in.visit(iterator, packedValue); > } > {code} -- This message was sent by Atlassian Jira (v8.20.7#820007) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org