[jira] [Commented] (LUCENE-9983) Stop sorting determinize powersets unnecessarily
[ https://issues.apache.org/jira/browse/LUCENE-9983?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354856#comment-17354856 ] Dawid Weiss commented on LUCENE-9983: - bq. from 6 min to 16 sec on my local machine Ouch. That's what I call a nice improvement... I guess you know where to focus the attention now then. > Stop sorting determinize powersets unnecessarily > > > Key: LUCENE-9983 > URL: https://issues.apache.org/jira/browse/LUCENE-9983 > Project: Lucene - Core > Issue Type: Improvement >Reporter: Michael McCandless >Priority: Major > Time Spent: 2h 10m > Remaining Estimate: 0h > > Spinoff from LUCENE-9981. > Today, our {{Operations.determinize}} implementation builds powersets of all > subsets of NFA states that "belong" in the same determinized state, using > [this algorithm|https://en.wikipedia.org/wiki/Powerset_construction]. > To hold each powerset, we use a malleable {{SortedIntSet}} and periodically > freeze it to a {{FrozenIntSet}}, also sorted. We pay a high price to keep > these growing maps of int key, int value sorted by key, e.g. upgrading to a > {{TreeMap}} once the map is large enough (> 30 entries). > But I think sorting is entirely unnecessary here! Really all we need is the > ability to add/delete keys from the map, and hashCode / equals (by key only – > ignoring value!), and to freeze the map (a small optimization that we could > skip initially). We only use these maps to lookup in the (growing) > determinized automaton whether this powerset has already been seen. > Maybe we could simply poach the {{IntIntScatterMap}} implementation from > [HPPC|https://github.com/carrotsearch/hppc]? And then change its > {{hashCode}}/{{equals }}to only use keys (not values). > This change should be a big speedup for the kinds of (admittedly adversarial) > regexps we saw on LUCENE-9981. > -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] dweiss commented on a change in pull request #163: LUCENE-9983: Stop sorting determinize powersets unnecessarily
dweiss commented on a change in pull request #163: URL: https://github.com/apache/lucene/pull/163#discussion_r642820266 ## File path: lucene/core/src/java/org/apache/lucene/util/automaton/StateSet.java ## @@ -0,0 +1,107 @@ +/* + * 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.util.automaton; + +import com.carrotsearch.hppc.BitMixer; +import com.carrotsearch.hppc.IntIntHashMap; +import com.carrotsearch.hppc.cursors.IntCursor; +import java.util.Arrays; + +/** A thin wrapper of {@link com.carrotsearch.hppc.IntIntHashMap} */ +final class StateSet extends IntSet { + + private final IntIntHashMap inner; + private int hashCode; + private boolean changed; + private int[] arrayCache = new int[0]; + + StateSet(int capacity) { +inner = new IntIntHashMap(capacity); + } + + // Adds this state to the set + void incr(int num) { +if (inner.addTo(num, 1) == 1) { + changed = true; +} + } + + // Removes this state from the set, if count decrs to 0 + void decr(int num) { +assert inner.containsKey(num); +int keyIndex = inner.indexOf(num); +int count = inner.indexGet(keyIndex) - 1; +if (count == 0) { + inner.remove(num); + changed = true; +} else { + inner.indexReplace(keyIndex, count); +} + } + + void computeHash() { +if (changed == false) { + return; +} +hashCode = inner.size(); +for (IntCursor cursor : inner.keys()) { + hashCode += BitMixer.mix(cursor.value); Review comment: +1. An alternative: 37 * hashCode + cursor.value with a final mix step. -- 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. 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] dweiss commented on a change in pull request #163: LUCENE-9983: Stop sorting determinize powersets unnecessarily
dweiss commented on a change in pull request #163: URL: https://github.com/apache/lucene/pull/163#discussion_r642819034 ## File path: lucene/core/build.gradle ## @@ -20,6 +20,8 @@ apply plugin: 'java-library' description = 'Lucene core library' dependencies { + implementation 'com.carrotsearch:hppc' Review comment: Feel free to cut and trim to your will. This is exactly why it's licensed the way it is. @bruno-roustant came up with some clever new hashing improvements recently - these are not published as a public revision but you can get them from the repository and compile it locally. See this for details: https://issues.carrot2.org/browse/HPPC-176 -- 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. 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] dweiss commented on a change in pull request #163: LUCENE-9983: Stop sorting determinize powersets unnecessarily
dweiss commented on a change in pull request #163: URL: https://github.com/apache/lucene/pull/163#discussion_r642817638 ## File path: lucene/core/src/java/org/apache/lucene/util/automaton/StateSet.java ## @@ -0,0 +1,107 @@ +/* + * 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.util.automaton; + +import com.carrotsearch.hppc.BitMixer; +import com.carrotsearch.hppc.IntIntHashMap; +import com.carrotsearch.hppc.cursors.IntCursor; +import java.util.Arrays; + +/** A thin wrapper of {@link com.carrotsearch.hppc.IntIntHashMap} */ +final class StateSet extends IntSet { + + private final IntIntHashMap inner; + private int hashCode; Review comment: Not just moving to a long but also using a better hash function. The typical accumulative default (prime * current + elementValue) is fine but if the number of hashed elements is small (or if their values are small) this leads to poor distributions. Throw a murmur mix function in between (or at the end at least) and things typically look much better. -- 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. 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] dweiss commented on a change in pull request #163: LUCENE-9983: Stop sorting determinize powersets unnecessarily
dweiss commented on a change in pull request #163: URL: https://github.com/apache/lucene/pull/163#discussion_r642816694 ## File path: lucene/core/src/java/org/apache/lucene/util/automaton/StateSet.java ## @@ -0,0 +1,107 @@ +/* + * 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.util.automaton; + +import com.carrotsearch.hppc.BitMixer; +import com.carrotsearch.hppc.IntIntHashMap; +import com.carrotsearch.hppc.cursors.IntCursor; +import java.util.Arrays; + +/** A thin wrapper of {@link com.carrotsearch.hppc.IntIntHashMap} */ +final class StateSet extends IntSet { + + private final IntIntHashMap inner; + private int hashCode; + private boolean changed; + private int[] arrayCache = new int[0]; + + StateSet(int capacity) { +inner = new IntIntHashMap(capacity); + } + + // Adds this state to the set + void incr(int num) { +if (inner.addTo(num, 1) == 1) { + changed = true; +} + } + + // Removes this state from the set, if count decrs to 0 + void decr(int num) { +assert inner.containsKey(num); +int keyIndex = inner.indexOf(num); +int count = inner.indexGet(keyIndex) - 1; +if (count == 0) { + inner.remove(num); + changed = true; +} else { + inner.indexReplace(keyIndex, count); +} + } + + void computeHash() { +if (changed == false) { + return; +} +hashCode = inner.size(); +for (IntCursor cursor : inner.keys()) { + hashCode += BitMixer.mix(cursor.value); +} + } + + /** + * Create a snapshot of this int set associated with a given state. The snapshot will not retain + * any frequency information about the elements of this set, only existence. + * + * It is the caller's responsibility to ensure that the hashCode and data are up to date via + * the {@link #computeHash()} method before calling this method. + * + * @param state the state to associate with the frozen set. + * @return A new FrozenIntSet with the same values as this set. + */ + FrozenIntSet freeze(int state) { +if (changed == false) { + assert arrayCache != null; + return new FrozenIntSet(arrayCache, hashCode, state); +} +return new FrozenIntSet(getArray(), hashCode, state); + } + + @Override + int[] getArray() { +if (changed == false) { + assert arrayCache != null; + return arrayCache; +} +changed = false; +arrayCache = inner.keys().toArray(); +// we need to sort this array since "equals" method depend on this +Arrays.sort(arrayCache); Review comment: Yeah... I think a good empirical breakdown of what actually is happening may be much better than theoretical complexity analysis. This doesn't mean it's not worth experimenting with int-int hash maps but I have a gut feeling you can get pretty darn close with less memory used using existing sorted arrays. -- 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. 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] zacharymorn commented on pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index check across segments
zacharymorn commented on pull request #128: URL: https://github.com/apache/lucene/pull/128#issuecomment-851783171 I've created a spinoff issue to track the default setting change https://issues.apache.org/jira/browse/LUCENE-9984 -- 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. 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-9984) Make CheckIndex doChecksumsOnly / -fast as default
Zach Chen created LUCENE-9984: - Summary: Make CheckIndex doChecksumsOnly / -fast as default Key: LUCENE-9984 URL: https://issues.apache.org/jira/browse/LUCENE-9984 Project: Lucene - Core Issue Type: Improvement Components: core/index Affects Versions: 9.0 Reporter: Zach Chen Assignee: Zach Chen This issue is a spin-off from discussion in https://github.com/apache/lucene/pull/128 Currently _CheckIndex_ defaults to checking both checksum as well as content inside each segment files for correctness, and requires _-fast_ flag to be explicitly passed in to do checksum only. However, this default setting was there due to lack of checksum feature historically, and is slow for most end-users nowadays as they probably only care about their indices being intact (from random bit flipping for example). This issue is to change the default settings for CheckIndex so that they are more appropriate for end-users. One proposal from @rmuir is the following: # Make {{-fast}} the new default. # The previous {{-slow}} could be moved to {{-slower}} # The current behavior (checksum + segment file content - slow check) could be activated by {{-slow}}. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Comment Edited] (LUCENE-9950) Support both single- and multi-value string fields in facet counting (non-taxonomy based approaches)
[ https://issues.apache.org/jira/browse/LUCENE-9950?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354735#comment-17354735 ] Alexander L edited comment on LUCENE-9950 at 6/1/21, 2:20 AM: -- Thank you for adding the new facet implementation, [~gsmiller]! ??It seems like the only advantage it might offer over a taxonomy-based approach is not requiring the side-car index?? A couple of SSDVFF advantages we found is the ability to perform fast index merge operation, since it is a regular index and does not require [global ordinals translation logic|https://github.com/apache/lucene/blob/main/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyMergeUtils.java] (regular index merge with HardlinkCopyDirectoryWrapper takes 3 minutes in our tests, while main+taxonomy pairs merge is about 85 minutes for ~ 200Gb index size). Also, SSDVFF indexing performance is better and unlike the Taxonomy approach, scales with added threads. These advantages tipped the scales in favor of SSDVFF in our case, although Taxonomy provides a bit better query performance and allows hierarchical faceting. _— I didn't go so far as to mark {{SortedSetDocValueFacetCounts}} as {{deprecated}} at this point, thinking there may still be some use-cases for "packing" multiple "dimensions" into one field (like the taxonomy-based approach) while not requiring a taxonomy index._ I wonder what use cases do you have in mind for that? Also, any chance that you have done some performance comparison with SortedSetDocValuesFacetField implementation? I remember reading somewhere that facet dimensions stored in a single field can provide better performance (e.g due to CPU reference locality), but not sure how big the difference can be. was (Author: sqshq): Thank you for adding the new facet implementation, [~gsmiller]! ??It seems like the only advantage it might offer over a taxonomy-based approach is not requiring the side-car index?? A couple of SSDVFF advantages we found is the ability to perform fast index merge operation, since it is a regular index and does not require [global ordinals translation logic|https://github.com/apache/lucene/blob/main/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyMergeUtils.java] (regular index merge with HardlinkCopyDirectoryWrapper takes 3 minutes in our tests, while main+taxonomy pairs merge is about 85 minutes for ~ 200Gb index size). Also, SSDVFF indexing performance is better and unlike the Taxonomy approach, scales with added threads. These advantages tipped the scales in favor of SSDVFF in our case, although Taxonomy provides a bit better query performance and allows hierarchical faceting. _— I didn't go so far as to mark {{SortedSetDocValueFacetCounts}} as {{deprecated}} at this point, thinking there may still be some use-cases for "packing" multiple "dimensions" into one field (like the taxonomy-based approach) while not requiring a taxonomy index._ I wonder what use cases do you have in mind for that, or maybe you have some performance comparison with SortedSetDocValuesFacetField implementation available? I remember reading somewhere that facet dimensions stored in a single field can provide better performance (e.g due to CPU reference locality), but not sure how big the difference can be. > Support both single- and multi-value string fields in facet counting > (non-taxonomy based approaches) > > > Key: LUCENE-9950 > URL: https://issues.apache.org/jira/browse/LUCENE-9950 > Project: Lucene - Core > Issue Type: Improvement > Components: modules/facet >Affects Versions: main (9.0) >Reporter: Greg Miller >Priority: Minor > Fix For: main (9.0), 8.9 > > Time Spent: 3h > Remaining Estimate: 0h > > Users wanting to facet count string-based fields using a non-taxonomy-based > approach can use {{SortedSetDocValueFacetCounts}}, which accumulates facet > counts based on a {{SortedSetDocValues}} field. This requires the stored doc > values to be multi-valued (i.e., {{SORTED_SET}}), and doesn't work on > single-valued fields (i.e., SORTED). In contrast, if a user wants to facet > count on a stored numeric field, they can use {{LongValueFacetCounts}}, which > supports both single- and multi-valued fields (and in LUCENE-9948, we now > auto-detect instead of asking the user to specify). > Let's update {{SortedSetDocValueFacetCounts}} to also support, and > automatically detect single- and multi-value fields. Note that this is a > spin-off issue from LUCENE-9946, where [~rcmuir] points out that this can > essentially be a one-line change, but we may want to do some class renaming > at the same time. Also note that we should do thi
[jira] [Comment Edited] (LUCENE-9950) Support both single- and multi-value string fields in facet counting (non-taxonomy based approaches)
[ https://issues.apache.org/jira/browse/LUCENE-9950?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354735#comment-17354735 ] Alexander L edited comment on LUCENE-9950 at 6/1/21, 1:59 AM: -- Thank you for adding the new facet implementation, [~gsmiller]! ??It seems like the only advantage it might offer over a taxonomy-based approach is not requiring the side-car index?? A couple of SSDVFF advantages we found is the ability to perform fast index merge operation, since it is a regular index and does not require [global ordinals translation logic|https://github.com/apache/lucene/blob/main/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyMergeUtils.java] (regular index merge with HardlinkCopyDirectoryWrapper takes 3 minutes in our tests, while main+taxonomy pairs merge is about 85 minutes for ~ 200Gb index size). Also, SSDVFF indexing performance is better and unlike the Taxonomy approach, scales with added threads. These advantages tipped the scales in favor of SSDVFF in our case, although Taxonomy provides a bit better query performance and allows hierarchical faceting. _— I didn't go so far as to mark {{SortedSetDocValueFacetCounts}} as {{deprecated}} at this point, thinking there may still be some use-cases for "packing" multiple "dimensions" into one field (like the taxonomy-based approach) while not requiring a taxonomy index._ I wonder what use cases do you have in mind for that, or maybe you have some performance comparison with SortedSetDocValuesFacetField implementation available? I remember reading somewhere that facet dimensions stored in a single field can provide better performance (e.g due to CPU reference locality), but not sure how big the difference can be. was (Author: sqshq): Thank you for adding the new facet implementation, [~gsmiller]! ??It seems like the only advantage it might offer over a taxonomy-based approach is not requiring the side-car index?? A couple of SSDVFF advantages we found is the ability to perform fast index merge operation, since it is a regular index and does not require [global ordinals translation logic|https://github.com/apache/lucene/blob/main/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyMergeUtils.java] (regular index merge with HardlinkCopyDirectoryWrapper takes 3 minutes in our tests, while main+taxonomy pairs merge is about 85 minutes for ~ 200Gb index size). Also, SSDVFF indexing performance is better and unlike the Taxonomy approach, scales with added threads. These advantages tipped the scales in favor of SSDVFF in our case, although Taxonomy provides a bit better query performance and allows hierarchical faceting. ?? I didn't go so far as to mark {{SortedSetDocValueFacetCounts}} as {{deprecated}} at this point, thinking there may still be some use-cases for "packing" multiple "dimensions" into one field (like the taxonomy-based approach) while not requiring a taxonomy index.?? I wonder what use cases do you have in mind for that, or maybe you have some performance comparison with SortedSetDocValuesFacetField implementation available? I remember reading somewhere that facet dimensions stored in a single field can provide better performance (e.g due to CPU reference locality), but not sure how big the difference can be. > Support both single- and multi-value string fields in facet counting > (non-taxonomy based approaches) > > > Key: LUCENE-9950 > URL: https://issues.apache.org/jira/browse/LUCENE-9950 > Project: Lucene - Core > Issue Type: Improvement > Components: modules/facet >Affects Versions: main (9.0) >Reporter: Greg Miller >Priority: Minor > Fix For: main (9.0), 8.9 > > Time Spent: 3h > Remaining Estimate: 0h > > Users wanting to facet count string-based fields using a non-taxonomy-based > approach can use {{SortedSetDocValueFacetCounts}}, which accumulates facet > counts based on a {{SortedSetDocValues}} field. This requires the stored doc > values to be multi-valued (i.e., {{SORTED_SET}}), and doesn't work on > single-valued fields (i.e., SORTED). In contrast, if a user wants to facet > count on a stored numeric field, they can use {{LongValueFacetCounts}}, which > supports both single- and multi-valued fields (and in LUCENE-9948, we now > auto-detect instead of asking the user to specify). > Let's update {{SortedSetDocValueFacetCounts}} to also support, and > automatically detect single- and multi-value fields. Note that this is a > spin-off issue from LUCENE-9946, where [~rcmuir] points out that this can > essentially be a one-line change, but we may want to do some class renaming > at the same time. Also note that we should do this in >
[jira] [Comment Edited] (LUCENE-9950) Support both single- and multi-value string fields in facet counting (non-taxonomy based approaches)
[ https://issues.apache.org/jira/browse/LUCENE-9950?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354735#comment-17354735 ] Alexander L edited comment on LUCENE-9950 at 6/1/21, 1:58 AM: -- Thank you for adding the new facet implementation, [~gsmiller]! ??It seems like the only advantage it might offer over a taxonomy-based approach is not requiring the side-car index?? A couple of SSDVFF advantages we found is the ability to perform fast index merge operation, since it is a regular index and does not require [global ordinals translation logic|https://github.com/apache/lucene/blob/main/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyMergeUtils.java] (regular index merge with HardlinkCopyDirectoryWrapper takes 3 minutes in our tests, while main+taxonomy pairs merge is about 85 minutes for ~ 200Gb index size). Also, SSDVFF indexing performance is better and unlike the Taxonomy approach, scales with added threads. These advantages tipped the scales in favor of SSDVFF in our case, although Taxonomy provides a bit better query performance and allows hierarchical faceting. ?? I didn't go so far as to mark {{SortedSetDocValueFacetCounts}} as {{deprecated}} at this point, thinking there may still be some use-cases for "packing" multiple "dimensions" into one field (like the taxonomy-based approach) while not requiring a taxonomy index.?? I wonder what use cases do you have in mind for that, or maybe you have some performance comparison with SortedSetDocValuesFacetField implementation available? I remember reading somewhere that facet dimensions stored in a single field can provide better performance (e.g due to CPU reference locality), but not sure how big the difference can be. was (Author: sqshq): Thank you for adding the new facet implementation, [~gsmiller]! ??It seems like the only advantage it might offer over a taxonomy-based approach is not requiring the side-car index?? A couple of SSDVFF advantages we found is the ability to perform fast index merge operation, since it is a regular index and does not require [global ordinals translation logic|https://github.com/apache/lucene/blob/main/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyMergeUtils.java] (regular index merge with HardlinkCopyDirectoryWrapper takes 3 minutes in our tests, while main+taxonomy pairs merge is about 85 minutes for ~ 200Gb index size). Also, SSDVFF indexing performance is better and unlike the Taxonomy approach, scales with added threads. These advantages tipped the scales in favor of SSDVFF in our case, although Taxonomy provides a bit better query performance and allows hierarchical faceting. ?? I didn't go so far as to mark {{SortedSetDocValueFacetCounts}} as {{deprecated}} at this point, thinking there may still be some use-cases for "packing" multiple "dimensions" into one field (like the taxonomy-based approach) while not requiring a taxonomy index.?? I wonder what use cases do you have in mind for that, or maybe you have some performance comparison with SortedSetDocValuesFacetField implementation available? I remember reading somewhere that facet dimensions stored in a single field can provide better performance (e.g due to CPU reference locality), but not sure how big the difference can be. > Support both single- and multi-value string fields in facet counting > (non-taxonomy based approaches) > > > Key: LUCENE-9950 > URL: https://issues.apache.org/jira/browse/LUCENE-9950 > Project: Lucene - Core > Issue Type: Improvement > Components: modules/facet >Affects Versions: main (9.0) >Reporter: Greg Miller >Priority: Minor > Fix For: main (9.0), 8.9 > > Time Spent: 3h > Remaining Estimate: 0h > > Users wanting to facet count string-based fields using a non-taxonomy-based > approach can use {{SortedSetDocValueFacetCounts}}, which accumulates facet > counts based on a {{SortedSetDocValues}} field. This requires the stored doc > values to be multi-valued (i.e., {{SORTED_SET}}), and doesn't work on > single-valued fields (i.e., SORTED). In contrast, if a user wants to facet > count on a stored numeric field, they can use {{LongValueFacetCounts}}, which > supports both single- and multi-valued fields (and in LUCENE-9948, we now > auto-detect instead of asking the user to specify). > Let's update {{SortedSetDocValueFacetCounts}} to also support, and > automatically detect single- and multi-value fields. Note that this is a > spin-off issue from LUCENE-9946, where [~rcmuir] points out that this can > essentially be a one-line change, but we may want to do some class renaming > at the same time. Also note that we should do this in
[jira] [Comment Edited] (LUCENE-9950) Support both single- and multi-value string fields in facet counting (non-taxonomy based approaches)
[ https://issues.apache.org/jira/browse/LUCENE-9950?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354735#comment-17354735 ] Alexander L edited comment on LUCENE-9950 at 6/1/21, 1:57 AM: -- Thank you for adding the new facet implementation, [~gsmiller]! ??It seems like the only advantage it might offer over a taxonomy-based approach is not requiring the side-car index?? A couple of SSDVFF advantages we found is the ability to perform fast index merge operation, since it is a regular index and does not require [global ordinals translation logic|https://github.com/apache/lucene/blob/main/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyMergeUtils.java] (regular index merge with HardlinkCopyDirectoryWrapper takes 3 minutes in our tests, while main+taxonomy pairs merge is about 85 minutes for ~ 200Gb index size). Also, SSDVFF indexing performance is better and unlike the Taxonomy approach, scales with added threads. These advantages tipped the scales in favor of SSDVFF in our case, although Taxonomy provides a bit better query performance and allows hierarchical faceting. ?? I didn't go so far as to mark {{SortedSetDocValueFacetCounts}} as {{deprecated}} at this point, thinking there may still be some use-cases for "packing" multiple "dimensions" into one field (like the taxonomy-based approach) while not requiring a taxonomy index.?? I wonder what use cases do you have in mind for that, or maybe you have some performance comparison with SortedSetDocValuesFacetField implementation available? I remember reading somewhere that facet dimensions stored in a single field can provide better performance (e.g due to CPU reference locality), but not sure how big the difference can be. was (Author: sqshq): Thank you for adding the new facet implementation, [~gsmiller]! ??It seems like the only advantage it might offer over a taxonomy-based approach is not requiring the side-car index?? A couple of SSDVFF advantages we found is the ability to perform fast index merge operation, since it is a regular index and does not require [global ordinals translation logic|https://github.com/apache/lucene/blob/main/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyMergeUtils.java] (regular index merge with HardlinkCopyDirectoryWrapper takes 3 minutes in our tests, while main+taxonomy pairs merge is about 85 minutes for ~ 200Gb index size). Also, SSDVFF indexing performance is better and unlike the Taxonomy approach, scales with added threads. These advantages tipped the scales in favor of SSDVFF in our case, although Taxonomy provides a bit better query performance and allows hierarchical faceting. ?? there may still be some use-cases for "packing" multiple "dimensions" into one field?? I wonder what use cases do you have in mind for that, or maybe you have some performance comparison with SortedSetDocValuesFacetField implementation available? I remember reading somewhere that facet dimensions stored in a single field can provide better performance (e.g due to CPU reference locality), but not sure how big the difference can be. > Support both single- and multi-value string fields in facet counting > (non-taxonomy based approaches) > > > Key: LUCENE-9950 > URL: https://issues.apache.org/jira/browse/LUCENE-9950 > Project: Lucene - Core > Issue Type: Improvement > Components: modules/facet >Affects Versions: main (9.0) >Reporter: Greg Miller >Priority: Minor > Fix For: main (9.0), 8.9 > > Time Spent: 3h > Remaining Estimate: 0h > > Users wanting to facet count string-based fields using a non-taxonomy-based > approach can use {{SortedSetDocValueFacetCounts}}, which accumulates facet > counts based on a {{SortedSetDocValues}} field. This requires the stored doc > values to be multi-valued (i.e., {{SORTED_SET}}), and doesn't work on > single-valued fields (i.e., SORTED). In contrast, if a user wants to facet > count on a stored numeric field, they can use {{LongValueFacetCounts}}, which > supports both single- and multi-valued fields (and in LUCENE-9948, we now > auto-detect instead of asking the user to specify). > Let's update {{SortedSetDocValueFacetCounts}} to also support, and > automatically detect single- and multi-value fields. Note that this is a > spin-off issue from LUCENE-9946, where [~rcmuir] points out that this can > essentially be a one-line change, but we may want to do some class renaming > at the same time. Also note that we should do this in > {{ConcurrentSortedSetDocValuesFacetCounts}} while we're at it. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (LUCENE-9950) Support both single- and multi-value string fields in facet counting (non-taxonomy based approaches)
[ https://issues.apache.org/jira/browse/LUCENE-9950?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354735#comment-17354735 ] Alexander L commented on LUCENE-9950: - Thank you for adding the new facet implementation, [~gsmiller]! ??It seems like the only advantage it might offer over a taxonomy-based approach is not requiring the side-car index?? A couple of SSDVFF advantages we found is the ability to perform fast index merge operation, since it is a regular index and does not require [global ordinals translation logic|https://github.com/apache/lucene/blob/main/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyMergeUtils.java] (regular index merge with HardlinkCopyDirectoryWrapper takes 3 minutes in our tests, while main+taxonomy pairs merge is about 85 minutes for ~ 200Gb index size). Also, SSDVFF indexing performance is better and unlike the Taxonomy approach, scales with added threads. These advantages tipped the scales in favor of SSDVFF in our case, although Taxonomy provides a bit better query performance and allows hierarchical faceting. ?? there may still be some use-cases for "packing" multiple "dimensions" into one field?? I wonder what use cases do you have in mind for that, or maybe you have some performance comparison with SortedSetDocValuesFacetField implementation available? I remember reading somewhere that facet dimensions stored in a single field can provide better performance (e.g due to CPU reference locality), but not sure how big the difference can be. > Support both single- and multi-value string fields in facet counting > (non-taxonomy based approaches) > > > Key: LUCENE-9950 > URL: https://issues.apache.org/jira/browse/LUCENE-9950 > Project: Lucene - Core > Issue Type: Improvement > Components: modules/facet >Affects Versions: main (9.0) >Reporter: Greg Miller >Priority: Minor > Fix For: main (9.0), 8.9 > > Time Spent: 3h > Remaining Estimate: 0h > > Users wanting to facet count string-based fields using a non-taxonomy-based > approach can use {{SortedSetDocValueFacetCounts}}, which accumulates facet > counts based on a {{SortedSetDocValues}} field. This requires the stored doc > values to be multi-valued (i.e., {{SORTED_SET}}), and doesn't work on > single-valued fields (i.e., SORTED). In contrast, if a user wants to facet > count on a stored numeric field, they can use {{LongValueFacetCounts}}, which > supports both single- and multi-valued fields (and in LUCENE-9948, we now > auto-detect instead of asking the user to specify). > Let's update {{SortedSetDocValueFacetCounts}} to also support, and > automatically detect single- and multi-value fields. Note that this is a > spin-off issue from LUCENE-9946, where [~rcmuir] points out that this can > essentially be a one-line change, but we may want to do some class renaming > at the same time. Also note that we should do this in > {{ConcurrentSortedSetDocValuesFacetCounts}} while we're at it. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] zacharymorn commented on pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index check across segments
zacharymorn commented on pull request #128: URL: https://github.com/apache/lucene/pull/128#issuecomment-851739934 > > To get the best speedup, even at -slow, we should do concurrency both ways, and then sort those tasks by decreasing expected cost. > > This fine-grained concurrency (one task for "part X segment") would then mean we can get the postings check of the big segments all going, early, concurrently across all segments, and likely the biggest segments postings check would be the long pole, and {{CheckIndex}} would finish in (about) that much time. > > Versus "thread per segment" concurrency, where the long pole will be one thread checking the biggest segment. > > The really fun part of concurrency across _and_ within segments will be figure out the rough heuristic of how to assign estimated cost to each task :) > > But this all can come later! "thread per segment" is already a big step forward! Yup this indeed will be both fun and challenging to implement! I think maybe it's also worth looking into breaking posting check into parallelize-able parts, as they alone can often take up 80% - 90% of the entire segment check time? -- 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. 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] zacharymorn commented on a change in pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index check across segments
zacharymorn commented on a change in pull request #128: URL: https://github.com/apache/lucene/pull/128#discussion_r642722424 ## File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java ## @@ -843,6 +812,258 @@ public Status checkIndex(List onlySegments) throws IOException { return result; } + private void updateMaxSegmentName(Status result, SegmentCommitInfo info) { +long segmentName = Long.parseLong(info.info.name.substring(1), Character.MAX_RADIX); +if (segmentName > result.maxSegmentName) { + result.maxSegmentName = segmentName; +} + } + + private void processSegmentInfoStatusResult( + Status result, SegmentCommitInfo info, Status.SegmentInfoStatus segmentInfoStatus) { +result.segmentInfos.add(segmentInfoStatus); +if (segmentInfoStatus.error != null) { + result.totLoseDocCount += segmentInfoStatus.toLoseDocCount; + result.numBadSegments++; +} else { + // Keeper + result.newSegments.add(info.clone()); +} + } + + private CompletableFuture runAsyncSegmentCheck( + Callable asyncCallable, ExecutorService executorService) { +return CompletableFuture.supplyAsync(callableToSupplier(asyncCallable), executorService); + } + + private Supplier callableToSupplier(Callable callable) { +return () -> { + try { +return callable.call(); + } catch (RuntimeException | Error e) { +throw e; + } catch (Throwable e) { +throw new CompletionException(e); + } +}; + } + + private Status.SegmentInfoStatus testSegment( + SegmentInfos sis, SegmentCommitInfo info, PrintStream infoStream) throws IOException { +Status.SegmentInfoStatus segInfoStat = new Status.SegmentInfoStatus(); +segInfoStat.name = info.info.name; +segInfoStat.maxDoc = info.info.maxDoc(); + +final Version version = info.info.getVersion(); +if (info.info.maxDoc() <= 0) { + throw new CheckIndexException(" illegal number of documents: maxDoc=" + info.info.maxDoc()); +} + +int toLoseDocCount = info.info.maxDoc(); Review comment: I took a closer look at this code. I think the reason of inconsistency here is that, between `toLoseDocCount = info.info.maxDoc()` and `toLoseDocCount = reader.numDocs()`, there are a few places where exception can be thrown (e.g. `reader = new SegmentReader` may throw IOException), but we still want to catch the exception and add the value into result via `segInfoStat.toLoseDocCount = toLoseDocCount` (using maxDoc count as the upper bound). So I guess this inconsistency is intended? -- 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. 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] zhaih commented on a change in pull request #163: LUCENE-9983: Stop sorting determinize powersets unnecessarily
zhaih commented on a change in pull request #163: URL: https://github.com/apache/lucene/pull/163#discussion_r642721094 ## File path: lucene/core/src/java/org/apache/lucene/util/automaton/StateSet.java ## @@ -0,0 +1,107 @@ +/* + * 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.util.automaton; + +import com.carrotsearch.hppc.BitMixer; +import com.carrotsearch.hppc.IntIntHashMap; +import com.carrotsearch.hppc.cursors.IntCursor; +import java.util.Arrays; + +/** A thin wrapper of {@link com.carrotsearch.hppc.IntIntHashMap} */ +final class StateSet extends IntSet { + + private final IntIntHashMap inner; + private int hashCode; + private boolean changed; + private int[] arrayCache = new int[0]; + + StateSet(int capacity) { +inner = new IntIntHashMap(capacity); + } + + // Adds this state to the set + void incr(int num) { +if (inner.addTo(num, 1) == 1) { + changed = true; +} + } + + // Removes this state from the set, if count decrs to 0 + void decr(int num) { +assert inner.containsKey(num); +int keyIndex = inner.indexOf(num); +int count = inner.indexGet(keyIndex) - 1; +if (count == 0) { + inner.remove(num); + changed = true; +} else { + inner.indexReplace(keyIndex, count); +} + } + + void computeHash() { +if (changed == false) { + return; +} +hashCode = inner.size(); +for (IntCursor cursor : inner.keys()) { + hashCode += BitMixer.mix(cursor.value); +} + } + + /** + * Create a snapshot of this int set associated with a given state. The snapshot will not retain + * any frequency information about the elements of this set, only existence. + * + * It is the caller's responsibility to ensure that the hashCode and data are up to date via + * the {@link #computeHash()} method before calling this method. + * + * @param state the state to associate with the frozen set. + * @return A new FrozenIntSet with the same values as this set. + */ + FrozenIntSet freeze(int state) { +if (changed == false) { + assert arrayCache != null; + return new FrozenIntSet(arrayCache, hashCode, state); +} +return new FrozenIntSet(getArray(), hashCode, state); + } + + @Override + int[] getArray() { +if (changed == false) { + assert arrayCache != null; + return arrayCache; +} +changed = false; +arrayCache = inner.keys().toArray(); +// we need to sort this array since "equals" method depend on this +Arrays.sort(arrayCache); Review comment: Oh another thought is, we were using raw array for states less than 30, which have `add/delete` cost of `O(n)`, and I would expect most of the powerset will be constructed using these 30 states (since `C(30,4) > 1` already), meaning we might be actually paying `O(n)` price per `add/delete` most times -- 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. 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] zacharymorn commented on a change in pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index check across segments
zacharymorn commented on a change in pull request #128: URL: https://github.com/apache/lucene/pull/128#discussion_r642719647 ## File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java ## @@ -605,209 +680,103 @@ public Status checkIndex(List onlySegments) throws IOException { result.newSegments.clear(); result.maxSegmentName = -1; -for (int i = 0; i < numSegments; i++) { - final SegmentCommitInfo info = sis.info(i); - long segmentName = Long.parseLong(info.info.name.substring(1), Character.MAX_RADIX); - if (segmentName > result.maxSegmentName) { -result.maxSegmentName = segmentName; - } - if (onlySegments != null && !onlySegments.contains(info.info.name)) { -continue; - } - Status.SegmentInfoStatus segInfoStat = new Status.SegmentInfoStatus(); - result.segmentInfos.add(segInfoStat); - msg( - infoStream, - " " - + (1 + i) - + " of " - + numSegments - + ": name=" - + info.info.name - + " maxDoc=" - + info.info.maxDoc()); - segInfoStat.name = info.info.name; - segInfoStat.maxDoc = info.info.maxDoc(); - - final Version version = info.info.getVersion(); - if (info.info.maxDoc() <= 0) { -throw new RuntimeException("illegal number of documents: maxDoc=" + info.info.maxDoc()); - } - - int toLoseDocCount = info.info.maxDoc(); - - SegmentReader reader = null; - - try { -msg(infoStream, "version=" + (version == null ? "3.0" : version)); -msg(infoStream, "id=" + StringHelper.idToString(info.info.getId())); -final Codec codec = info.info.getCodec(); -msg(infoStream, "codec=" + codec); -segInfoStat.codec = codec; -msg(infoStream, "compound=" + info.info.getUseCompoundFile()); -segInfoStat.compound = info.info.getUseCompoundFile(); -msg(infoStream, "numFiles=" + info.files().size()); -Sort indexSort = info.info.getIndexSort(); -if (indexSort != null) { - msg(infoStream, "sort=" + indexSort); -} -segInfoStat.numFiles = info.files().size(); -segInfoStat.sizeMB = info.sizeInBytes() / (1024. * 1024.); -msg(infoStream, "size (MB)=" + nf.format(segInfoStat.sizeMB)); -Map diagnostics = info.info.getDiagnostics(); -segInfoStat.diagnostics = diagnostics; -if (diagnostics.size() > 0) { - msg(infoStream, "diagnostics = " + diagnostics); +// checks segments sequentially +if (executorService == null) { + for (int i = 0; i < numSegments; i++) { +final SegmentCommitInfo info = sis.info(i); +updateMaxSegmentName(result, info); +if (onlySegments != null && !onlySegments.contains(info.info.name)) { + continue; } -if (!info.hasDeletions()) { - msg(infoStream, "no deletions"); - segInfoStat.hasDeletions = false; -} else { - msg(infoStream, "has deletions [delGen=" + info.getDelGen() + "]"); - segInfoStat.hasDeletions = true; - segInfoStat.deletionsGen = info.getDelGen(); -} - -long startOpenReaderNS = System.nanoTime(); -if (infoStream != null) infoStream.print("test: open reader."); -reader = new SegmentReader(info, sis.getIndexCreatedVersionMajor(), IOContext.DEFAULT); msg( infoStream, -String.format( -Locale.ROOT, "OK [took %.3f sec]", nsToSec(System.nanoTime() - startOpenReaderNS))); +(1 + i) ++ " of " ++ numSegments ++ ": name=" ++ info.info.name ++ " maxDoc=" ++ info.info.maxDoc()); +Status.SegmentInfoStatus segmentInfoStatus = testSegment(sis, info, infoStream); + +processSegmentInfoStatusResult(result, info, segmentInfoStatus); + } +} else { + ByteArrayOutputStream[] outputs = new ByteArrayOutputStream[numSegments]; + @SuppressWarnings({"unchecked", "rawtypes"}) + CompletableFuture[] futures = new CompletableFuture[numSegments]; + + // checks segments concurrently + for (int i = 0; i < numSegments; i++) { +final SegmentCommitInfo info = sis.info(i); +updateMaxSegmentName(result, info); +if (onlySegments != null && !onlySegments.contains(info.info.name)) { + continue; +} -segInfoStat.openReaderPassed = true; +SegmentInfos finalSis = sis; -long startIntegrityNS = System.nanoTime(); -if (infoStream != null) infoStream.print("test: check integrity."); -reader.checkIntegrity(); +ByteArrayOutputStream output = new ByteArrayOutputStream(); +PrintStream stream; +if (i > 0) { + // buffer
[GitHub] [lucene] zhaih commented on a change in pull request #163: LUCENE-9983: Stop sorting determinize powersets unnecessarily
zhaih commented on a change in pull request #163: URL: https://github.com/apache/lucene/pull/163#discussion_r642719501 ## File path: lucene/core/build.gradle ## @@ -20,6 +20,8 @@ apply plugin: 'java-library' description = 'Lucene core library' dependencies { + implementation 'com.carrotsearch:hppc' Review comment: Oh I see, we might need to modify the source so that we won't bring in a whole interface set used in HPPC. -- 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. 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] zhaih commented on a change in pull request #163: LUCENE-9983: Stop sorting determinize powersets unnecessarily
zhaih commented on a change in pull request #163: URL: https://github.com/apache/lucene/pull/163#discussion_r642718657 ## File path: lucene/core/src/java/org/apache/lucene/util/automaton/StateSet.java ## @@ -0,0 +1,107 @@ +/* + * 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.util.automaton; + +import com.carrotsearch.hppc.BitMixer; +import com.carrotsearch.hppc.IntIntHashMap; +import com.carrotsearch.hppc.cursors.IntCursor; +import java.util.Arrays; + +/** A thin wrapper of {@link com.carrotsearch.hppc.IntIntHashMap} */ +final class StateSet extends IntSet { + + private final IntIntHashMap inner; + private int hashCode; Review comment: Ah yeah that's a good idea, I forgot to integrate it here, will do. -- 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. 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] zhaih commented on a change in pull request #163: LUCENE-9983: Stop sorting determinize powersets unnecessarily
zhaih commented on a change in pull request #163: URL: https://github.com/apache/lucene/pull/163#discussion_r642718504 ## File path: lucene/core/src/java/org/apache/lucene/util/automaton/StateSet.java ## @@ -0,0 +1,107 @@ +/* + * 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.util.automaton; + +import com.carrotsearch.hppc.BitMixer; +import com.carrotsearch.hppc.IntIntHashMap; +import com.carrotsearch.hppc.cursors.IntCursor; +import java.util.Arrays; + +/** A thin wrapper of {@link com.carrotsearch.hppc.IntIntHashMap} */ +final class StateSet extends IntSet { + + private final IntIntHashMap inner; + private int hashCode; + private boolean changed; + private int[] arrayCache = new int[0]; + + StateSet(int capacity) { +inner = new IntIntHashMap(capacity); + } + + // Adds this state to the set + void incr(int num) { +if (inner.addTo(num, 1) == 1) { + changed = true; +} + } + + // Removes this state from the set, if count decrs to 0 + void decr(int num) { +assert inner.containsKey(num); +int keyIndex = inner.indexOf(num); +int count = inner.indexGet(keyIndex) - 1; +if (count == 0) { + inner.remove(num); + changed = true; +} else { + inner.indexReplace(keyIndex, count); +} + } + + void computeHash() { +if (changed == false) { + return; +} +hashCode = inner.size(); +for (IntCursor cursor : inner.keys()) { + hashCode += BitMixer.mix(cursor.value); +} + } + + /** + * Create a snapshot of this int set associated with a given state. The snapshot will not retain + * any frequency information about the elements of this set, only existence. + * + * It is the caller's responsibility to ensure that the hashCode and data are up to date via + * the {@link #computeHash()} method before calling this method. + * + * @param state the state to associate with the frozen set. + * @return A new FrozenIntSet with the same values as this set. + */ + FrozenIntSet freeze(int state) { +if (changed == false) { + assert arrayCache != null; + return new FrozenIntSet(arrayCache, hashCode, state); +} +return new FrozenIntSet(getArray(), hashCode, state); + } + + @Override + int[] getArray() { +if (changed == false) { + assert arrayCache != null; + return arrayCache; +} +changed = false; +arrayCache = inner.keys().toArray(); +// we need to sort this array since "equals" method depend on this +Arrays.sort(arrayCache); Review comment: I have posted some stats in the JIRA issue, it turns out we are doing way way much more add/delete than freeze. Since `(#add + #delete)/#freeze > 1` so a rough analysis is each `add` or `delete` use `O(log(n))` and `freeze` use `O(n * log(n))`, we see the cost of `#add + #delete` is still larger, though I'm not sure why it is this much faster. -- 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. 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-9949) Flaky test in TestCachePurging.testBackgroundPurges
[ https://issues.apache.org/jira/browse/LUCENE-9949?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354721#comment-17354721 ] Gautam Worah commented on LUCENE-9949: -- Hmm beasting does not help either: {{./gradlew cleanTest test --tests TestCachePurging.testBackgroundPurges -Ptests.seed=4DDD0C97DB8E50CC:5D664A5C8BA9CD8C -Dtests.slow=true -Dtests.badapples=true -Dtests.locale=shi-Tfng -Ptests.iters=200}} runs successfully for all iterations > Flaky test in TestCachePurging.testBackgroundPurges > --- > > Key: LUCENE-9949 > URL: https://issues.apache.org/jira/browse/LUCENE-9949 > Project: Lucene - Core > Issue Type: Bug > Components: core/other >Affects Versions: main (9.0) > Environment: Ubuntu 18.04.5 LTS > Java 11 >Reporter: Gautam Worah >Priority: Minor > > While executing the `./gradlew check` command on an unrelated change, my > tests crashed on this test case with the following error log: > > > org.apache.lucene.monitor.TestCachePurging > testBackgroundPurges FAILED > java.lang.AssertionError: expected:<-1> but was:<21196529334563> > at __randomizedtesting.SeedInfo.seed([4DDD0C97DB8E50CC:5D664A5C8BA9CD8C]:0) > at org.junit.Assert.fail(Assert.java:89) > at org.junit.Assert.failNotEquals(Assert.java:835) > at org.junit.Assert.assertEquals(Assert.java:647) > at org.junit.Assert.assertEquals(Assert.java:633) > at > org.apache.lucene.monitor.TestCachePurging.testBackgroundPurges(TestCachePurging.java:142) > at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native > Method) > at > java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) > at > java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.base/java.lang.reflect.Method.invoke(Method.java:567) > at > com.carrotsearch.randomizedtesting.RandomizedRunner.invoke(RandomizedRunner.java:1754) > at > com.carrotsearch.randomizedtesting.RandomizedRunner$8.evaluate(RandomizedRunner.java:942) > at > com.carrotsearch.randomizedtesting.RandomizedRunner$9.evaluate(RandomizedRunner.java:978) > at > com.carrotsearch.randomizedtesting.RandomizedRunner$10.evaluate(RandomizedRunner.java:992) > at > org.apache.lucene.util.TestRuleSetupTeardownChained$1.evaluate(TestRuleSetupTeardownChained.java:44) > at > org.apache.lucene.util.AbstractBeforeAfterRule$1.evaluate(AbstractBeforeAfterRule.java:43) > at > org.apache.lucene.util.TestRuleThreadAndTestName$1.evaluate(TestRuleThreadAndTestName.java:45) > at > org.apache.lucene.util.TestRuleIgnoreAfterMaxFailures$1.evaluate(TestRuleIgnoreAfterMaxFailures.java:60) > at > org.apache.lucene.util.TestRuleMarkFailure$1.evaluate(TestRuleMarkFailure.java:44) > at org.junit.rules.RunRules.evaluate(RunRules.java:20) > at > com.carrotsearch.randomizedtesting.rules.StatementAdapter.evaluate(StatementAdapter.java:36) > at > com.carrotsearch.randomizedtesting.ThreadLeakControl$StatementRunner.run(ThreadLeakControl.java:370) > at > com.carrotsearch.randomizedtesting.ThreadLeakControl.forkTimeoutingTask(ThreadLeakControl.java:819) > at > com.carrotsearch.randomizedtesting.ThreadLeakControl$3.evaluate(ThreadLeakControl.java:470) > at > com.carrotsearch.randomizedtesting.RandomizedRunner.runSingleTest(RandomizedRunner.java:951) > at > com.carrotsearch.randomizedtesting.RandomizedRunner$5.evaluate(RandomizedRunner.java:836) > at > com.carrotsearch.randomizedtesting.RandomizedRunner$6.evaluate(RandomizedRunner.java:887) > at > com.carrotsearch.randomizedtesting.RandomizedRunner$7.evaluate(RandomizedRunner.java:898) > at > org.apache.lucene.util.AbstractBeforeAfterRule$1.evaluate(AbstractBeforeAfterRule.java:43) > at > com.carrotsearch.randomizedtesting.rules.StatementAdapter.evaluate(StatementAdapter.java:36) > at > org.apache.lucene.util.TestRuleStoreClassName$1.evaluate(TestRuleStoreClassName.java:38) > at > com.carrotsearch.randomizedtesting.rules.NoShadowingOrOverridesOnMethodsRule$1.evaluate(NoShadowingOrOverridesOnMethodsRule.java:40) > at > com.carrotsearch.randomizedtesting.rules.NoShadowingOrOverridesOnMethodsRule$1.evaluate(NoShadowingOrOverridesOnMethodsRule.java:40) > at > com.carrotsearch.randomizedtesting.rules.StatementAdapter.evaluate(StatementAdapter.java:36) > at > com.carrotsearch.randomizedtesting.rules.StatementAdapter.evaluate(StatementAdapter.java:36) > at > org.apache.lucene.util.TestRuleAssertionsRequired$1.evaluate(TestRuleAssertionsRequired.java:53) > at > org.apache.lucene.util.AbstractBeforeAfterRule$1.evaluate(AbstractBeforeAfterRule.java:43) > at > org.apache.lucene.util.TestRuleMarkFailure$1.evaluate(TestRuleMarkFailure.java:44) > at > org.apache.lucene.util.TestRuleIgnoreAfterMaxFailu
[GitHub] [lucene] gautamworah96 commented on a change in pull request #138: LUCENE-9956: Make getBaseQuery, getDrillDownQueries API from DrillDownQuery public
gautamworah96 commented on a change in pull request #138: URL: https://github.com/apache/lucene/pull/138#discussion_r642711762 ## File path: lucene/facet/src/java/org/apache/lucene/facet/DrillDownQuery.java ## @@ -170,16 +185,36 @@ private BooleanQuery getBooleanQuery() { return bq.build(); } - Query getBaseQuery() { + /** + * Returns the internal baseQuery of the DrillDownQuery + * + * @return The baseQuery used on initialization of DrillDownQuery + */ + public Query getBaseQuery() { return baseQuery; } - Query[] getDrillDownQueries() { -Query[] dimQueries = new Query[this.dimQueries.size()]; -for (int i = 0; i < dimQueries.length; ++i) { - dimQueries[i] = this.dimQueries.get(i).build(); + /** + * Returns the dimension queries added either via {@link #add(String, Query)} or {@link + * #add(String, String...)} + * + * @return The array of dimQueries + */ + public Query[] getDrillDownQueries() { +if (dirtyDimQueryIndex.isEmpty()) { + // returns previously built dimQueries + Query[] builtDimQueriesCopy = new Query[builtDimQueries.size()]; + return builtDimQueries.toArray(builtDimQueriesCopy); +} +for (int i = 0; i < this.dimQueries.size(); ++i) { + if (dirtyDimQueryIndex.contains(i)) { +builtDimQueries.set(i, this.dimQueries.get(i).build()); +dirtyDimQueryIndex.remove(i); + } } -return dimQueries; +assert dirtyDimQueryIndex.isEmpty(); +// by this time builtDimQueries has all the built queries and dirtyDimQueryIndex is empty +return getDrillDownQueries(); Review comment: Added ## File path: lucene/facet/src/java/org/apache/lucene/facet/DrillDownQuery.java ## @@ -170,16 +185,36 @@ private BooleanQuery getBooleanQuery() { return bq.build(); } - Query getBaseQuery() { + /** + * Returns the internal baseQuery of the DrillDownQuery + * + * @return The baseQuery used on initialization of DrillDownQuery + */ + public Query getBaseQuery() { return baseQuery; } - Query[] getDrillDownQueries() { -Query[] dimQueries = new Query[this.dimQueries.size()]; -for (int i = 0; i < dimQueries.length; ++i) { - dimQueries[i] = this.dimQueries.get(i).build(); + /** + * Returns the dimension queries added either via {@link #add(String, Query)} or {@link + * #add(String, String...)} + * + * @return The array of dimQueries + */ + public Query[] getDrillDownQueries() { Review comment: Added -- 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. 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] mikemccand commented on a change in pull request #163: LUCENE-9983: Stop sorting determinize powersets unnecessarily
mikemccand commented on a change in pull request #163: URL: https://github.com/apache/lucene/pull/163#discussion_r642695443 ## File path: lucene/core/build.gradle ## @@ -20,6 +20,8 @@ apply plugin: 'java-library' description = 'Lucene core library' dependencies { + implementation 'com.carrotsearch:hppc' Review comment: Can you add a `// nocommit` here? We can't take (external) dependencies in Lucene's `core` ... but hopefully we can fork the one HPPC class we need here (`IntIntHashMap`?). ## File path: lucene/core/src/java/org/apache/lucene/util/automaton/StateSet.java ## @@ -0,0 +1,107 @@ +/* + * 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.util.automaton; + +import com.carrotsearch.hppc.BitMixer; +import com.carrotsearch.hppc.IntIntHashMap; +import com.carrotsearch.hppc.cursors.IntCursor; +import java.util.Arrays; + +/** A thin wrapper of {@link com.carrotsearch.hppc.IntIntHashMap} */ +final class StateSet extends IntSet { + + private final IntIntHashMap inner; + private int hashCode; Review comment: @dweiss suggested upgrading the hash to `long` to further reduce chance of false collisions. ## File path: lucene/core/src/java/org/apache/lucene/util/automaton/StateSet.java ## @@ -0,0 +1,107 @@ +/* + * 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.util.automaton; + +import com.carrotsearch.hppc.BitMixer; +import com.carrotsearch.hppc.IntIntHashMap; +import com.carrotsearch.hppc.cursors.IntCursor; +import java.util.Arrays; + +/** A thin wrapper of {@link com.carrotsearch.hppc.IntIntHashMap} */ +final class StateSet extends IntSet { + + private final IntIntHashMap inner; + private int hashCode; + private boolean changed; + private int[] arrayCache = new int[0]; + + StateSet(int capacity) { +inner = new IntIntHashMap(capacity); + } + + // Adds this state to the set + void incr(int num) { +if (inner.addTo(num, 1) == 1) { + changed = true; +} + } + + // Removes this state from the set, if count decrs to 0 + void decr(int num) { +assert inner.containsKey(num); +int keyIndex = inner.indexOf(num); +int count = inner.indexGet(keyIndex) - 1; +if (count == 0) { + inner.remove(num); + changed = true; +} else { + inner.indexReplace(keyIndex, count); +} + } + + void computeHash() { +if (changed == false) { + return; +} +hashCode = inner.size(); +for (IntCursor cursor : inner.keys()) { + hashCode += BitMixer.mix(cursor.value); +} + } + + /** + * Create a snapshot of this int set associated with a given state. The snapshot will not retain + * any frequency information about the elements of this set, only existence. + * + * It is the caller's responsibility to ensure that the hashCode and data are up to date via + * the {@link #computeHash()} method before calling this method. + * + * @param state the state to associate with the frozen set. + * @return A new FrozenIntSet with the same values as this set. + */ + FrozenIntSet freeze(int state) { +if (changed == false) { + assert arrayCache != null; + return new FrozenIntSet(arrayCache, hashCode, state); +} +return new FrozenIntSet(getArray(), hashCode, state); + } + + @Override + int[] getArray() { +if (changed == false) { + assert arrayCache != null; + return arrayCache; +} +
[jira] [Commented] (LUCENE-9983) Stop sorting determinize powersets unnecessarily
[ https://issues.apache.org/jira/browse/LUCENE-9983?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354660#comment-17354660 ] Haoyu Zhai commented on LUCENE-9983: I've added a simple static counter just for the adversarial test, and here's the stats: * {{incr}} called: 106073079 * entry added to set: 100076079 * {{decr}} called: 106069079 * entry removed from set: 100072079 * {{computeHash}} called: 40057 * {{freeze}} called: 14056 So seems to me my guess above holds, we're doing way more put/remove entry operations than others > Stop sorting determinize powersets unnecessarily > > > Key: LUCENE-9983 > URL: https://issues.apache.org/jira/browse/LUCENE-9983 > Project: Lucene - Core > Issue Type: Improvement >Reporter: Michael McCandless >Priority: Major > Time Spent: 40m > Remaining Estimate: 0h > > Spinoff from LUCENE-9981. > Today, our {{Operations.determinize}} implementation builds powersets of all > subsets of NFA states that "belong" in the same determinized state, using > [this algorithm|https://en.wikipedia.org/wiki/Powerset_construction]. > To hold each powerset, we use a malleable {{SortedIntSet}} and periodically > freeze it to a {{FrozenIntSet}}, also sorted. We pay a high price to keep > these growing maps of int key, int value sorted by key, e.g. upgrading to a > {{TreeMap}} once the map is large enough (> 30 entries). > But I think sorting is entirely unnecessary here! Really all we need is the > ability to add/delete keys from the map, and hashCode / equals (by key only – > ignoring value!), and to freeze the map (a small optimization that we could > skip initially). We only use these maps to lookup in the (growing) > determinized automaton whether this powerset has already been seen. > Maybe we could simply poach the {{IntIntScatterMap}} implementation from > [HPPC|https://github.com/carrotsearch/hppc]? And then change its > {{hashCode}}/{{equals }}to only use keys (not values). > This change should be a big speedup for the kinds of (admittedly adversarial) > regexps we saw on LUCENE-9981. > -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Comment Edited] (LUCENE-9983) Stop sorting determinize powersets unnecessarily
[ https://issues.apache.org/jira/browse/LUCENE-9983?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354659#comment-17354659 ] Haoyu Zhai edited comment on LUCENE-9983 at 5/31/21, 9:07 PM: -- Thanks [~mikemccand] and [~dweiss]. I've opened a PR based on {{IntIntHashMap}}: [https://github.com/apache/lucene/pull/163] I've applied the test attached in LUCENE-9981 to verify this PR helps. Seems it successfully reduce the time it need before throwing the exception from 6 min to 16 sec on my local machine (they both stoped at the same point as well). I still kept the state array to be sorted when get it, so we'll be slower when actually getting array but way faster on putting/removing keys. I'm not quite sure why the speed up is this much, but my guess is we're doing way more operations and spending way more times on increasing/decreasing state count and putting/removing states from the set than introducing new states? was (Author: zhai7631): Thanks [~mikemccand] and [~dweiss]. I've opened a PR based on {{IntIntHashMap}}: [https://github.com/apache/lucene/pull/162] I've applied the test attached in LUCENE-9981 to verify this PR helps. Seems it successfully reduce the time it need before throwing the exception from 6 min to 16 sec on my local machine (they both stoped at the same point as well). I still kept the state array to be sorted when get it, so we'll be slower when actually getting array but way faster on putting/removing keys. I'm not quite sure why the speed up is this much, but my guess is we're doing way more operations and spending way more times on increasing/decreasing state count and putting/removing states from the set than introducing new states? > Stop sorting determinize powersets unnecessarily > > > Key: LUCENE-9983 > URL: https://issues.apache.org/jira/browse/LUCENE-9983 > Project: Lucene - Core > Issue Type: Improvement >Reporter: Michael McCandless >Priority: Major > Time Spent: 40m > Remaining Estimate: 0h > > Spinoff from LUCENE-9981. > Today, our {{Operations.determinize}} implementation builds powersets of all > subsets of NFA states that "belong" in the same determinized state, using > [this algorithm|https://en.wikipedia.org/wiki/Powerset_construction]. > To hold each powerset, we use a malleable {{SortedIntSet}} and periodically > freeze it to a {{FrozenIntSet}}, also sorted. We pay a high price to keep > these growing maps of int key, int value sorted by key, e.g. upgrading to a > {{TreeMap}} once the map is large enough (> 30 entries). > But I think sorting is entirely unnecessary here! Really all we need is the > ability to add/delete keys from the map, and hashCode / equals (by key only – > ignoring value!), and to freeze the map (a small optimization that we could > skip initially). We only use these maps to lookup in the (growing) > determinized automaton whether this powerset has already been seen. > Maybe we could simply poach the {{IntIntScatterMap}} implementation from > [HPPC|https://github.com/carrotsearch/hppc]? And then change its > {{hashCode}}/{{equals }}to only use keys (not values). > This change should be a big speedup for the kinds of (admittedly adversarial) > regexps we saw on LUCENE-9981. > -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Commented] (LUCENE-9983) Stop sorting determinize powersets unnecessarily
[ https://issues.apache.org/jira/browse/LUCENE-9983?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354659#comment-17354659 ] Haoyu Zhai commented on LUCENE-9983: Thanks [~mikemccand] and [~dweiss]. I've opened a PR based on {{IntIntHashMap}}: [https://github.com/apache/lucene/pull/162] I've applied the test attached in LUCENE-9981 to verify this PR helps. Seems it successfully reduce the time it need before throwing the exception from 6 min to 16 sec on my local machine (they both stoped at the same point as well). I still kept the state array to be sorted when get it, so we'll be slower when actually getting array but way faster on putting/removing keys. I'm not quite sure why the speed up is this much, but my guess is we're doing way more operations and spending way more times on increasing/decreasing state count and putting/removing states from the set than introducing new states? > Stop sorting determinize powersets unnecessarily > > > Key: LUCENE-9983 > URL: https://issues.apache.org/jira/browse/LUCENE-9983 > Project: Lucene - Core > Issue Type: Improvement >Reporter: Michael McCandless >Priority: Major > Time Spent: 40m > Remaining Estimate: 0h > > Spinoff from LUCENE-9981. > Today, our {{Operations.determinize}} implementation builds powersets of all > subsets of NFA states that "belong" in the same determinized state, using > [this algorithm|https://en.wikipedia.org/wiki/Powerset_construction]. > To hold each powerset, we use a malleable {{SortedIntSet}} and periodically > freeze it to a {{FrozenIntSet}}, also sorted. We pay a high price to keep > these growing maps of int key, int value sorted by key, e.g. upgrading to a > {{TreeMap}} once the map is large enough (> 30 entries). > But I think sorting is entirely unnecessary here! Really all we need is the > ability to add/delete keys from the map, and hashCode / equals (by key only – > ignoring value!), and to freeze the map (a small optimization that we could > skip initially). We only use these maps to lookup in the (growing) > determinized automaton whether this powerset has already been seen. > Maybe we could simply poach the {{IntIntScatterMap}} implementation from > [HPPC|https://github.com/carrotsearch/hppc]? And then change its > {{hashCode}}/{{equals }}to only use keys (not values). > This change should be a big speedup for the kinds of (admittedly adversarial) > regexps we saw on LUCENE-9981. > -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] zhaih opened a new pull request #163: LUCENE-9983: Stop sorting determinize powersets unnecessarily
zhaih opened a new pull request #163: URL: https://github.com/apache/lucene/pull/163 # Description In `determinize` function (which truns NFA to DFA), we use `SortedIntSet` to store powerset and it uses a sorted data structure which could be very costly in some cases. And we actually don't need to keep the set always sorted. # Solution Replace usage of `SortedIntSet` by `StateSet`, a wrapper of `IntIntHashMap`. Note I haven't removed `SortedIntSet` but probably we should since it is only used in `determinize`? # Tests * Reused old tests that used to test `SortedIntSet` * I also applied the patch attached in [LUCENE-9981](https://issues.apache.org/jira/projects/LUCENE/issues/LUCENE-9981). Before the commit it runs 6min before throw an exception and after this commit it is 16 sec. # Checklist Please review the following and check all that apply: - [x] I have reviewed the guidelines for [How to Contribute](https://wiki.apache.org/lucene/HowToContribute) and my code conforms to the standards described there to the best of my ability. - [x] I have created a Jira issue and added the issue ID to my pull request title. - [x] 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) - [x] I have developed this patch against the `main` branch. - [x] I have run `./gradlew check`. - [x] 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. 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-9981) CompiledAutomaton.getCommonSuffix can be extraordinarily slow, even with default maxDeterminizedStates limit
[ https://issues.apache.org/jira/browse/LUCENE-9981?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354650#comment-17354650 ] Dawid Weiss commented on LUCENE-9981: - Thanks for the explanation, Mike. Makes sense. > CompiledAutomaton.getCommonSuffix can be extraordinarily slow, even with > default maxDeterminizedStates limit > > > Key: LUCENE-9981 > URL: https://issues.apache.org/jira/browse/LUCENE-9981 > Project: Lucene - Core > Issue Type: Task >Reporter: Robert Muir >Priority: Major > Attachments: LUCENE-9981.patch, LUCENE-9981.patch, LUCENE-9981.patch, > LUCENE-9981.patch, LUCENE-9981.patch, LUCENE-9981_nfaprefix.patch, > LUCENE-9981_test.patch, three-repeats-reverse-det.png, three-repeats.png > > > We have a {{maxDeterminizedStates = 1}} limit designed to keep > regexp-type queries from blowing up. > But we have an adversary that will run for 268s on my laptop before hitting > exception, first reported here: > https://github.com/opensearch-project/OpenSearch/issues/687 > When I run the test and jstack the threads, this what I see: > {noformat} > "TEST-TestOpensearch687.testInteresting-seed#[4B9C20A027A9850C]" #15 prio=5 > os_prio=0 cpu=56960.04ms elapsed=57.49s tid=0x7fff7006ca80 nid=0x231c8 > runnable [0x7fff8b7f] >java.lang.Thread.State: RUNNABLE > at > org.apache.lucene.util.automaton.SortedIntSet.decr(SortedIntSet.java:106) > at > org.apache.lucene.util.automaton.Operations.determinize(Operations.java:769) > at > org.apache.lucene.util.automaton.Operations.getCommonSuffixBytesRef(Operations.java:1155) > at > org.apache.lucene.util.automaton.CompiledAutomaton.(CompiledAutomaton.java:247) > at > org.apache.lucene.search.AutomatonQuery.(AutomatonQuery.java:104) > at > org.apache.lucene.search.AutomatonQuery.(AutomatonQuery.java:82) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:138) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:114) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:72) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:62) > at > org.apache.lucene.TestOpensearch687.testInteresting(TestOpensearch687.java:42) > {noformat} > This is really sad, as {{getCommonSuffixBytesRef()}} is only supposed to be > an "up-front" optimization to make the actual subsequent terms-intensive part > of the query faster. But it makes the whole query run for nearly 5 minutes > before it does anything. > So I definitely think we should improve {{getCommonSuffixBytesRef}} to be > more "best-effort". For example, it can reduce the lower bound to {{1000}} > and catch the exception like such: > {code} > try { >// this is slow, and just an opto anyway, so don't burn cycles on it for > some crazy worst-case. >// if we don't set this common suffix, the query will just run a bit > slower, that's all. >int limit = Math.min(1000, maxDeterminizedStates); >BytesRef suffix = Operations.getCommonSuffixBytesRef(binary, limit); >... (setting commonSuffixRef) > } catch (TooComplexTooDeterminizeException notWorthIt) { > commonSuffixRef = null; > } > {code} > Another, maybe simpler option, is to just check that input state/transitions > accounts don't exceed some low limit N. > Basically this opto is geared at stuff like leading wildcard query of "*foo". > By computing that the common suffix is "foo" we can spend less CPU in the > terms dictionary because we can first do a memcmp before having to run data > thru any finite state machine. It's really a microopt and we shouldn't be > spending whole seconds of cpu on it, ever. > But I still don't quite understand how the current limits are giving the > behavior today, maybe there is a bigger issue and I don't want to shove > something under the rug. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Commented] (LUCENE-9983) Stop sorting determinize powersets unnecessarily
[ https://issues.apache.org/jira/browse/LUCENE-9983?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354649#comment-17354649 ] Dawid Weiss commented on LUCENE-9983: - I generally agree but I'd like to confirm that it's actually the sorting that costly here before trying to optimize. The algorithmic complexity is one thing but if these sets are short (and they will be, right?) then it's a small constant. What you gain is comparisons are fast later on on collisions. An alternative representation (sets) will require more memory and comparing sets will be slower than comparing sorted arrays. It's all a tradeoff. What I'd do first is try to figure out whether the sorting is indeed the key problem here. If it is, switch to sets and see if it makes a difference. Then, as a final optimization, change the hash code of those sets to a long and distribute the hash better to limit the number of collisions - this will limit the number of the required hash set comparisons to a minimum. > Stop sorting determinize powersets unnecessarily > > > Key: LUCENE-9983 > URL: https://issues.apache.org/jira/browse/LUCENE-9983 > Project: Lucene - Core > Issue Type: Improvement >Reporter: Michael McCandless >Priority: Major > Time Spent: 0.5h > Remaining Estimate: 0h > > Spinoff from LUCENE-9981. > Today, our {{Operations.determinize}} implementation builds powersets of all > subsets of NFA states that "belong" in the same determinized state, using > [this algorithm|https://en.wikipedia.org/wiki/Powerset_construction]. > To hold each powerset, we use a malleable {{SortedIntSet}} and periodically > freeze it to a {{FrozenIntSet}}, also sorted. We pay a high price to keep > these growing maps of int key, int value sorted by key, e.g. upgrading to a > {{TreeMap}} once the map is large enough (> 30 entries). > But I think sorting is entirely unnecessary here! Really all we need is the > ability to add/delete keys from the map, and hashCode / equals (by key only – > ignoring value!), and to freeze the map (a small optimization that we could > skip initially). We only use these maps to lookup in the (growing) > determinized automaton whether this powerset has already been seen. > Maybe we could simply poach the {{IntIntScatterMap}} implementation from > [HPPC|https://github.com/carrotsearch/hppc]? And then change its > {{hashCode}}/{{equals }}to only use keys (not values). > This change should be a big speedup for the kinds of (admittedly adversarial) > regexps we saw on LUCENE-9981. > -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Resolved] (LUCENE-9982) ../../gradlew help should be just as helpful as ./gradlew help?
[ https://issues.apache.org/jira/browse/LUCENE-9982?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael McCandless resolved LUCENE-9982. Resolution: Not A Problem > ../../gradlew help should be just as helpful as ./gradlew help? > --- > > Key: LUCENE-9982 > URL: https://issues.apache.org/jira/browse/LUCENE-9982 > Project: Lucene - Core > Issue Type: Bug >Reporter: Michael McCandless >Priority: Major > > If you are in the root of a Lucene git clone, and run {{./gradlew help}}, you > get useful help. > But if you first {{cd lucene/core}} and then run {{../../gradlew help}}, it > is less useful, and seems to be {{gradle}}'s generic help or something? > Is {{gradlew help}} known to be different depending on your cwd? > A good workaround (thanks [~rcmuir]) is to just look at the files under the > {{help}} directory in the root of the Lucene git clone. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Commented] (LUCENE-9982) ../../gradlew help should be just as helpful as ./gradlew help?
[ https://issues.apache.org/jira/browse/LUCENE-9982?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354648#comment-17354648 ] Michael McCandless commented on LUCENE-9982: OK thanks you for the detailed explanations, and links to specific parts of the {{help}} content, [~dweiss]! I think there is nothing more to do here ... resolving. > ../../gradlew help should be just as helpful as ./gradlew help? > --- > > Key: LUCENE-9982 > URL: https://issues.apache.org/jira/browse/LUCENE-9982 > Project: Lucene - Core > Issue Type: Bug >Reporter: Michael McCandless >Priority: Major > > If you are in the root of a Lucene git clone, and run {{./gradlew help}}, you > get useful help. > But if you first {{cd lucene/core}} and then run {{../../gradlew help}}, it > is less useful, and seems to be {{gradle}}'s generic help or something? > Is {{gradlew help}} known to be different depending on your cwd? > A good workaround (thanks [~rcmuir]) is to just look at the files under the > {{help}} directory in the root of the Lucene git clone. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Commented] (LUCENE-9982) ../../gradlew help should be just as helpful as ./gradlew help?
[ https://issues.apache.org/jira/browse/LUCENE-9982?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354647#comment-17354647 ] Dawid Weiss commented on LUCENE-9982: - bq. But my one hesitation here is that the top-level tasks do work when I am in cwd=lucene/core, i.e. I can run ../../gradlew test --tests TestRegexpRandom2 and it indeed only runs that one test, as expected. No need to hesitate here. When you run an unqualified task name (such as 'test') it searches (and schedules) that task in all of the modules under your current context (folder you're in). So your example works because that test is in the lucene/core package. You can use fully qualified task path instead of the context. See here, for example: https://github.com/apache/lucene/blob/main/help/tests.txt#L26-L27 bq. Given that ../../gradlew indeed seems to accept the same tasks as when I am in the root directory, shouldn't its help advertise that too? It's not the same task. It's two tasks with the same name (belonging to different modules). If you run with {{--console=plain}} you can see fully qualified task names. When you run an unqualified task name, it means literally "all tasks named 'foobar' in the current module and its submodules". bq. Aha! That : does the trick – I will try remember this (weirdly cryptic) way of asking gradlew for help It's a fully qualified task name, not cryptic at all. You could also say :lucene:core:help, for example. For the help task, it doesn't matter. For other tasks, it may matter - for example the list of tasks for a given module will be typically different, depending on what the module actually is (and what plugins it consumes). > ../../gradlew help should be just as helpful as ./gradlew help? > --- > > Key: LUCENE-9982 > URL: https://issues.apache.org/jira/browse/LUCENE-9982 > Project: Lucene - Core > Issue Type: Bug >Reporter: Michael McCandless >Priority: Major > > If you are in the root of a Lucene git clone, and run {{./gradlew help}}, you > get useful help. > But if you first {{cd lucene/core}} and then run {{../../gradlew help}}, it > is less useful, and seems to be {{gradle}}'s generic help or something? > Is {{gradlew help}} known to be different depending on your cwd? > A good workaround (thanks [~rcmuir]) is to just look at the files under the > {{help}} directory in the root of the Lucene git clone. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] zacharymorn commented on a change in pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index check across segments
zacharymorn commented on a change in pull request #128: URL: https://github.com/apache/lucene/pull/128#discussion_r642646093 ## File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java ## @@ -605,209 +680,103 @@ public Status checkIndex(List onlySegments) throws IOException { result.newSegments.clear(); result.maxSegmentName = -1; -for (int i = 0; i < numSegments; i++) { - final SegmentCommitInfo info = sis.info(i); - long segmentName = Long.parseLong(info.info.name.substring(1), Character.MAX_RADIX); - if (segmentName > result.maxSegmentName) { -result.maxSegmentName = segmentName; - } - if (onlySegments != null && !onlySegments.contains(info.info.name)) { -continue; - } - Status.SegmentInfoStatus segInfoStat = new Status.SegmentInfoStatus(); - result.segmentInfos.add(segInfoStat); - msg( - infoStream, - " " - + (1 + i) - + " of " - + numSegments - + ": name=" - + info.info.name - + " maxDoc=" - + info.info.maxDoc()); - segInfoStat.name = info.info.name; - segInfoStat.maxDoc = info.info.maxDoc(); - - final Version version = info.info.getVersion(); - if (info.info.maxDoc() <= 0) { -throw new RuntimeException("illegal number of documents: maxDoc=" + info.info.maxDoc()); - } - - int toLoseDocCount = info.info.maxDoc(); - - SegmentReader reader = null; - - try { -msg(infoStream, "version=" + (version == null ? "3.0" : version)); -msg(infoStream, "id=" + StringHelper.idToString(info.info.getId())); -final Codec codec = info.info.getCodec(); -msg(infoStream, "codec=" + codec); -segInfoStat.codec = codec; -msg(infoStream, "compound=" + info.info.getUseCompoundFile()); -segInfoStat.compound = info.info.getUseCompoundFile(); -msg(infoStream, "numFiles=" + info.files().size()); -Sort indexSort = info.info.getIndexSort(); -if (indexSort != null) { - msg(infoStream, "sort=" + indexSort); -} -segInfoStat.numFiles = info.files().size(); -segInfoStat.sizeMB = info.sizeInBytes() / (1024. * 1024.); -msg(infoStream, "size (MB)=" + nf.format(segInfoStat.sizeMB)); -Map diagnostics = info.info.getDiagnostics(); -segInfoStat.diagnostics = diagnostics; -if (diagnostics.size() > 0) { - msg(infoStream, "diagnostics = " + diagnostics); +// checks segments sequentially +if (executorService == null) { + for (int i = 0; i < numSegments; i++) { +final SegmentCommitInfo info = sis.info(i); +updateMaxSegmentName(result, info); +if (onlySegments != null && !onlySegments.contains(info.info.name)) { + continue; } -if (!info.hasDeletions()) { - msg(infoStream, "no deletions"); - segInfoStat.hasDeletions = false; -} else { - msg(infoStream, "has deletions [delGen=" + info.getDelGen() + "]"); - segInfoStat.hasDeletions = true; - segInfoStat.deletionsGen = info.getDelGen(); -} - -long startOpenReaderNS = System.nanoTime(); -if (infoStream != null) infoStream.print("test: open reader."); -reader = new SegmentReader(info, sis.getIndexCreatedVersionMajor(), IOContext.DEFAULT); msg( infoStream, -String.format( -Locale.ROOT, "OK [took %.3f sec]", nsToSec(System.nanoTime() - startOpenReaderNS))); +(1 + i) ++ " of " ++ numSegments ++ ": name=" ++ info.info.name ++ " maxDoc=" ++ info.info.maxDoc()); +Status.SegmentInfoStatus segmentInfoStatus = testSegment(sis, info, infoStream); + +processSegmentInfoStatusResult(result, info, segmentInfoStatus); + } +} else { + ByteArrayOutputStream[] outputs = new ByteArrayOutputStream[numSegments]; + @SuppressWarnings({"unchecked", "rawtypes"}) + CompletableFuture[] futures = new CompletableFuture[numSegments]; + + // checks segments concurrently + for (int i = 0; i < numSegments; i++) { +final SegmentCommitInfo info = sis.info(i); +updateMaxSegmentName(result, info); +if (onlySegments != null && !onlySegments.contains(info.info.name)) { + continue; +} -segInfoStat.openReaderPassed = true; +SegmentInfos finalSis = sis; -long startIntegrityNS = System.nanoTime(); -if (infoStream != null) infoStream.print("test: check integrity."); -reader.checkIntegrity(); +ByteArrayOutputStream output = new ByteArrayOutputStream(); +PrintStream stream; +if (i > 0) { + // buffer
[GitHub] [lucene] zhaih commented on pull request #162: LUCENE-9983: Stop sorting determinize powersets unnecessarily
zhaih commented on pull request #162: URL: https://github.com/apache/lucene/pull/162#issuecomment-851642477 Close this one temporarily for debugging -- 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. 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] zhaih closed pull request #162: LUCENE-9983: Stop sorting determinize powersets unnecessarily
zhaih closed pull request #162: URL: https://github.com/apache/lucene/pull/162 -- 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. 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] zacharymorn commented on a change in pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index check across segments
zacharymorn commented on a change in pull request #128: URL: https://github.com/apache/lucene/pull/128#discussion_r642641460 ## File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java ## @@ -450,6 +479,14 @@ public void setChecksumsOnly(boolean v) { private boolean checksumsOnly; + /** Set threadCount used for parallelizing index integrity checking. */ + public void setThreadCount(int tc) { +threadCount = tc; + } + + // capped threadCount at 4 + private int threadCount = Math.min(Runtime.getRuntime().availableProcessors(), 4); Review comment: Ah I thought we would like to cap it to 4 (or 8) per earlier comment. But I think in latest changes I actually removed away the usage of this default, as when the user does not pass in `-threadCount`, the execution will use 0 for `threadCount` and fallback to sequential check. Should we give user this flexibility to not use concurrent check? -- 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. 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] zacharymorn commented on a change in pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index check across segments
zacharymorn commented on a change in pull request #128: URL: https://github.com/apache/lucene/pull/128#discussion_r642639399 ## File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java ## @@ -605,209 +680,103 @@ public Status checkIndex(List onlySegments) throws IOException { result.newSegments.clear(); result.maxSegmentName = -1; -for (int i = 0; i < numSegments; i++) { - final SegmentCommitInfo info = sis.info(i); - long segmentName = Long.parseLong(info.info.name.substring(1), Character.MAX_RADIX); - if (segmentName > result.maxSegmentName) { -result.maxSegmentName = segmentName; - } - if (onlySegments != null && !onlySegments.contains(info.info.name)) { -continue; - } - Status.SegmentInfoStatus segInfoStat = new Status.SegmentInfoStatus(); - result.segmentInfos.add(segInfoStat); - msg( - infoStream, - " " - + (1 + i) - + " of " - + numSegments - + ": name=" - + info.info.name - + " maxDoc=" - + info.info.maxDoc()); - segInfoStat.name = info.info.name; - segInfoStat.maxDoc = info.info.maxDoc(); - - final Version version = info.info.getVersion(); - if (info.info.maxDoc() <= 0) { -throw new RuntimeException("illegal number of documents: maxDoc=" + info.info.maxDoc()); - } - - int toLoseDocCount = info.info.maxDoc(); - - SegmentReader reader = null; - - try { -msg(infoStream, "version=" + (version == null ? "3.0" : version)); -msg(infoStream, "id=" + StringHelper.idToString(info.info.getId())); -final Codec codec = info.info.getCodec(); -msg(infoStream, "codec=" + codec); -segInfoStat.codec = codec; -msg(infoStream, "compound=" + info.info.getUseCompoundFile()); -segInfoStat.compound = info.info.getUseCompoundFile(); -msg(infoStream, "numFiles=" + info.files().size()); -Sort indexSort = info.info.getIndexSort(); -if (indexSort != null) { - msg(infoStream, "sort=" + indexSort); -} -segInfoStat.numFiles = info.files().size(); -segInfoStat.sizeMB = info.sizeInBytes() / (1024. * 1024.); -msg(infoStream, "size (MB)=" + nf.format(segInfoStat.sizeMB)); -Map diagnostics = info.info.getDiagnostics(); -segInfoStat.diagnostics = diagnostics; -if (diagnostics.size() > 0) { - msg(infoStream, "diagnostics = " + diagnostics); +// checks segments sequentially +if (executorService == null) { Review comment: This is actually done to avoid deadlock during test (I put a comment in `MockDirectoryWrapper#close` on passing a flag to have null `executorService`, but it might not be immediately obvious here). Essentially, the deadlock can be formed in test as such even with a single threaded executor: 1. At the end of the tests that use directory, `MockDirectoryWrapper#close` was called, which would hold directory's monitor as the method is `synchronized` 2. `MockDirectoryWrapper#close` would call `TestUtil#checkIndex` and passed in itself for directory reference as 1st argument 3. With concurrent execution across segments in `TestUtil#checkIndex`, another thread checking segment would be making call back to methods from directory, such as `MockDirectoryWrapper#fileLength`, which again require directory's monitor access as they are also `synchronized` 4. Deadlock occurred as another thread is waiting for directory's monitor, which is held by the main thread waiting for the other thread to complete ## File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java ## @@ -843,6 +812,258 @@ public Status checkIndex(List onlySegments) throws IOException { return result; } + private void updateMaxSegmentName(Status result, SegmentCommitInfo info) { +long segmentName = Long.parseLong(info.info.name.substring(1), Character.MAX_RADIX); +if (segmentName > result.maxSegmentName) { + result.maxSegmentName = segmentName; +} + } + + private void processSegmentInfoStatusResult( + Status result, SegmentCommitInfo info, Status.SegmentInfoStatus segmentInfoStatus) { +result.segmentInfos.add(segmentInfoStatus); +if (segmentInfoStatus.error != null) { + result.totLoseDocCount += segmentInfoStatus.toLoseDocCount; + result.numBadSegments++; +} else { + // Keeper + result.newSegments.add(info.clone()); +} + } + + private CompletableFuture runAsyncSegmentCheck( + Callable asyncCallable, ExecutorService executorService) { +return CompletableFuture.supplyAsync(callableToSupplier(asyncCallable), executorService); + } + + private Supplier callableToSupplier(Callable callable) { +return () -> { + try { +return callable.call(); +
[GitHub] [lucene] zhaih opened a new pull request #162: LUCENE-9983: Stop sorting determinize powersets unnecessarily
zhaih opened a new pull request #162: URL: https://github.com/apache/lucene/pull/162 # Description In `determinize` function (which truns NFA to DFA), we use `SortedIntSet` to store powerset and it uses a sorted data structure which could be very costly in some cases. And we actually don't need the sorting functionality. # Solution Replace usage of `SortedIntSet` by `StateSet`, a wrapper of `IntIntHashMap`. Note I haven't removed `SortedIntSet` but probably we should since it is only used in `determinize`? # Tests * Reused old tests that used to test `SortedIntSet` * I also applied the patch attached in [LUCENE-9981](https://issues.apache.org/jira/projects/LUCENE/issues/LUCENE-9981). Before the commit it runs 6min before throw an exception and after patch it is 1+ sec. (Seems from the error message this has something to do with the order we iterate the key, but I believe it still provides some speedups) # Checklist Please review the following and check all that apply: - [x] I have reviewed the guidelines for [How to Contribute](https://wiki.apache.org/lucene/HowToContribute) and my code conforms to the standards described there to the best of my ability. - [x] I have created a Jira issue and added the issue ID to my pull request title. - [x] 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) - [x] I have developed this patch against the `main` branch. - [x] I have run `./gradlew check`. - [x] 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. 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] [Updated] (LUCENE-9981) CompiledAutomaton.getCommonSuffix can be extraordinarily slow, even with default maxDeterminizedStates limit
[ https://issues.apache.org/jira/browse/LUCENE-9981?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael McCandless updated LUCENE-9981: --- Attachment: LUCENE-9981.patch Status: Open (was: Open) Another iteration! I think this is ready. {{gradle check}} passes. I fixed the typo [~dweiss] spotted (thanks!), added a dedicated test case for the new determinize effort check using this fun regexp, improved javadocs to link to the {{DEFAULT_DETERMINIZE_WORK_LIMIT}} and removed an unrelated change to {{ConcatenateGraphFilter}} (not sure how that snuck in). I also beasted {{TestRegexpRandom2}} and Lucene's core tests for a while, no failures, except for {{gradle}} timing out on file locks because I launched too many (128) {{gradle}} subprocesses: {noformat} BUILD FAILED in 2m 17s FAILURE: Build failed with an exception. * What went wrong: Timeout waiting to lock buildSrc build lock. It is currently in use by another Gradle instance. Owner PID: 1555872 Our PID: 1557315 Owner Operation: Our operation: Lock file: /l/regexp/buildSrc/.gradle/noVersion/buildSrc.lock {noformat} I think this is ready, to bake on {{main}} for a while before backporting! > CompiledAutomaton.getCommonSuffix can be extraordinarily slow, even with > default maxDeterminizedStates limit > > > Key: LUCENE-9981 > URL: https://issues.apache.org/jira/browse/LUCENE-9981 > Project: Lucene - Core > Issue Type: Task >Reporter: Robert Muir >Priority: Major > Attachments: LUCENE-9981.patch, LUCENE-9981.patch, LUCENE-9981.patch, > LUCENE-9981.patch, LUCENE-9981.patch, LUCENE-9981_nfaprefix.patch, > LUCENE-9981_test.patch, three-repeats-reverse-det.png, three-repeats.png > > > We have a {{maxDeterminizedStates = 1}} limit designed to keep > regexp-type queries from blowing up. > But we have an adversary that will run for 268s on my laptop before hitting > exception, first reported here: > https://github.com/opensearch-project/OpenSearch/issues/687 > When I run the test and jstack the threads, this what I see: > {noformat} > "TEST-TestOpensearch687.testInteresting-seed#[4B9C20A027A9850C]" #15 prio=5 > os_prio=0 cpu=56960.04ms elapsed=57.49s tid=0x7fff7006ca80 nid=0x231c8 > runnable [0x7fff8b7f] >java.lang.Thread.State: RUNNABLE > at > org.apache.lucene.util.automaton.SortedIntSet.decr(SortedIntSet.java:106) > at > org.apache.lucene.util.automaton.Operations.determinize(Operations.java:769) > at > org.apache.lucene.util.automaton.Operations.getCommonSuffixBytesRef(Operations.java:1155) > at > org.apache.lucene.util.automaton.CompiledAutomaton.(CompiledAutomaton.java:247) > at > org.apache.lucene.search.AutomatonQuery.(AutomatonQuery.java:104) > at > org.apache.lucene.search.AutomatonQuery.(AutomatonQuery.java:82) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:138) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:114) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:72) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:62) > at > org.apache.lucene.TestOpensearch687.testInteresting(TestOpensearch687.java:42) > {noformat} > This is really sad, as {{getCommonSuffixBytesRef()}} is only supposed to be > an "up-front" optimization to make the actual subsequent terms-intensive part > of the query faster. But it makes the whole query run for nearly 5 minutes > before it does anything. > So I definitely think we should improve {{getCommonSuffixBytesRef}} to be > more "best-effort". For example, it can reduce the lower bound to {{1000}} > and catch the exception like such: > {code} > try { >// this is slow, and just an opto anyway, so don't burn cycles on it for > some crazy worst-case. >// if we don't set this common suffix, the query will just run a bit > slower, that's all. >int limit = Math.min(1000, maxDeterminizedStates); >BytesRef suffix = Operations.getCommonSuffixBytesRef(binary, limit); >... (setting commonSuffixRef) > } catch (TooComplexTooDeterminizeException notWorthIt) { > commonSuffixRef = null; > } > {code} > Another, maybe simpler option, is to just check that input state/transitions > accounts don't exceed some low limit N. > Basically this opto is geared at stuff like leading wildcard query of "*foo". > By computing that the common suffix is "foo" we can spend less CPU in the > terms dictionary because we can first do a memcmp before having to run data > thru any finite state machine. It's really a microopt and we shouldn't be > spending whole seconds of cpu on it, ever. > But I still don't quite understand how the current limits are giving the > behavior today, maybe there is a bigger
[jira] [Commented] (LUCENE-9981) CompiledAutomaton.getCommonSuffix can be extraordinarily slow, even with default maxDeterminizedStates limit
[ https://issues.apache.org/jira/browse/LUCENE-9981?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354547#comment-17354547 ] Michael McCandless commented on LUCENE-9981: Or, maybe on backport, we leave the old constant in place, {{@Deprecated}} with an explanation to migrate to the new constant. > CompiledAutomaton.getCommonSuffix can be extraordinarily slow, even with > default maxDeterminizedStates limit > > > Key: LUCENE-9981 > URL: https://issues.apache.org/jira/browse/LUCENE-9981 > Project: Lucene - Core > Issue Type: Task >Reporter: Robert Muir >Priority: Major > Attachments: LUCENE-9981.patch, LUCENE-9981.patch, LUCENE-9981.patch, > LUCENE-9981.patch, LUCENE-9981_nfaprefix.patch, LUCENE-9981_test.patch, > three-repeats-reverse-det.png, three-repeats.png > > > We have a {{maxDeterminizedStates = 1}} limit designed to keep > regexp-type queries from blowing up. > But we have an adversary that will run for 268s on my laptop before hitting > exception, first reported here: > https://github.com/opensearch-project/OpenSearch/issues/687 > When I run the test and jstack the threads, this what I see: > {noformat} > "TEST-TestOpensearch687.testInteresting-seed#[4B9C20A027A9850C]" #15 prio=5 > os_prio=0 cpu=56960.04ms elapsed=57.49s tid=0x7fff7006ca80 nid=0x231c8 > runnable [0x7fff8b7f] >java.lang.Thread.State: RUNNABLE > at > org.apache.lucene.util.automaton.SortedIntSet.decr(SortedIntSet.java:106) > at > org.apache.lucene.util.automaton.Operations.determinize(Operations.java:769) > at > org.apache.lucene.util.automaton.Operations.getCommonSuffixBytesRef(Operations.java:1155) > at > org.apache.lucene.util.automaton.CompiledAutomaton.(CompiledAutomaton.java:247) > at > org.apache.lucene.search.AutomatonQuery.(AutomatonQuery.java:104) > at > org.apache.lucene.search.AutomatonQuery.(AutomatonQuery.java:82) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:138) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:114) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:72) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:62) > at > org.apache.lucene.TestOpensearch687.testInteresting(TestOpensearch687.java:42) > {noformat} > This is really sad, as {{getCommonSuffixBytesRef()}} is only supposed to be > an "up-front" optimization to make the actual subsequent terms-intensive part > of the query faster. But it makes the whole query run for nearly 5 minutes > before it does anything. > So I definitely think we should improve {{getCommonSuffixBytesRef}} to be > more "best-effort". For example, it can reduce the lower bound to {{1000}} > and catch the exception like such: > {code} > try { >// this is slow, and just an opto anyway, so don't burn cycles on it for > some crazy worst-case. >// if we don't set this common suffix, the query will just run a bit > slower, that's all. >int limit = Math.min(1000, maxDeterminizedStates); >BytesRef suffix = Operations.getCommonSuffixBytesRef(binary, limit); >... (setting commonSuffixRef) > } catch (TooComplexTooDeterminizeException notWorthIt) { > commonSuffixRef = null; > } > {code} > Another, maybe simpler option, is to just check that input state/transitions > accounts don't exceed some low limit N. > Basically this opto is geared at stuff like leading wildcard query of "*foo". > By computing that the common suffix is "foo" we can spend less CPU in the > terms dictionary because we can first do a memcmp before having to run data > thru any finite state machine. It's really a microopt and we shouldn't be > spending whole seconds of cpu on it, ever. > But I still don't quite understand how the current limits are giving the > behavior today, maybe there is a bigger issue and I don't want to shove > something under the rug. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Commented] (LUCENE-9981) CompiledAutomaton.getCommonSuffix can be extraordinarily slow, even with default maxDeterminizedStates limit
[ https://issues.apache.org/jira/browse/LUCENE-9981?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354546#comment-17354546 ] Michael McCandless commented on LUCENE-9981: We plan to backport this to 8.x, after it has baked in {{main}} for a while, but that will include a small API break due to renaming {{Operations.DEFAULT_MAX_DETERMINIZED_STATES}} to {{Operations.DEFAULT_DETERMINIZE_WORK_LIMIT}}. I think this is acceptable, but it will likely require small code changes for applications upgrading. > CompiledAutomaton.getCommonSuffix can be extraordinarily slow, even with > default maxDeterminizedStates limit > > > Key: LUCENE-9981 > URL: https://issues.apache.org/jira/browse/LUCENE-9981 > Project: Lucene - Core > Issue Type: Task >Reporter: Robert Muir >Priority: Major > Attachments: LUCENE-9981.patch, LUCENE-9981.patch, LUCENE-9981.patch, > LUCENE-9981.patch, LUCENE-9981_nfaprefix.patch, LUCENE-9981_test.patch, > three-repeats-reverse-det.png, three-repeats.png > > > We have a {{maxDeterminizedStates = 1}} limit designed to keep > regexp-type queries from blowing up. > But we have an adversary that will run for 268s on my laptop before hitting > exception, first reported here: > https://github.com/opensearch-project/OpenSearch/issues/687 > When I run the test and jstack the threads, this what I see: > {noformat} > "TEST-TestOpensearch687.testInteresting-seed#[4B9C20A027A9850C]" #15 prio=5 > os_prio=0 cpu=56960.04ms elapsed=57.49s tid=0x7fff7006ca80 nid=0x231c8 > runnable [0x7fff8b7f] >java.lang.Thread.State: RUNNABLE > at > org.apache.lucene.util.automaton.SortedIntSet.decr(SortedIntSet.java:106) > at > org.apache.lucene.util.automaton.Operations.determinize(Operations.java:769) > at > org.apache.lucene.util.automaton.Operations.getCommonSuffixBytesRef(Operations.java:1155) > at > org.apache.lucene.util.automaton.CompiledAutomaton.(CompiledAutomaton.java:247) > at > org.apache.lucene.search.AutomatonQuery.(AutomatonQuery.java:104) > at > org.apache.lucene.search.AutomatonQuery.(AutomatonQuery.java:82) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:138) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:114) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:72) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:62) > at > org.apache.lucene.TestOpensearch687.testInteresting(TestOpensearch687.java:42) > {noformat} > This is really sad, as {{getCommonSuffixBytesRef()}} is only supposed to be > an "up-front" optimization to make the actual subsequent terms-intensive part > of the query faster. But it makes the whole query run for nearly 5 minutes > before it does anything. > So I definitely think we should improve {{getCommonSuffixBytesRef}} to be > more "best-effort". For example, it can reduce the lower bound to {{1000}} > and catch the exception like such: > {code} > try { >// this is slow, and just an opto anyway, so don't burn cycles on it for > some crazy worst-case. >// if we don't set this common suffix, the query will just run a bit > slower, that's all. >int limit = Math.min(1000, maxDeterminizedStates); >BytesRef suffix = Operations.getCommonSuffixBytesRef(binary, limit); >... (setting commonSuffixRef) > } catch (TooComplexTooDeterminizeException notWorthIt) { > commonSuffixRef = null; > } > {code} > Another, maybe simpler option, is to just check that input state/transitions > accounts don't exceed some low limit N. > Basically this opto is geared at stuff like leading wildcard query of "*foo". > By computing that the common suffix is "foo" we can spend less CPU in the > terms dictionary because we can first do a memcmp before having to run data > thru any finite state machine. It's really a microopt and we shouldn't be > spending whole seconds of cpu on it, ever. > But I still don't quite understand how the current limits are giving the > behavior today, maybe there is a bigger issue and I don't want to shove > something under the rug. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Commented] (LUCENE-9981) CompiledAutomaton.getCommonSuffix can be extraordinarily slow, even with default maxDeterminizedStates limit
[ https://issues.apache.org/jira/browse/LUCENE-9981?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354542#comment-17354542 ] Michael McCandless commented on LUCENE-9981: {quote}Caught one typo ("Automataon") in CHANGES.txt {quote} Woops, I'll fix. {quote}One thing that bugs me is the int argument's "defaults". Currently it just says: {quote}Set higher to allow more complex queries and lower to prevent memory exhaustion. {quote}{quote} We do provide a {{public static final int DEFAULT_DETERMINIZE_WORK_LIMIT = 1}}{{}} though it's not obvious :) I'll update the javadocs to try to advertise that better. There is some danger here that the {{10}} multiplier is "too much". I picked this number as a rough conversion from maximum states in the final det machine, to total sum'd effort (which is just the sum of all powerset sizes of all DFA states iterated through), i.e. "on average" each det state consists of 10 NFA states. If we make this too high, we risk ReDoS sort of attacks since we work too hard before giving up. But if it's too low, then possibly legitimate regexps are incorrectly blocked. I think 10 is OK? > CompiledAutomaton.getCommonSuffix can be extraordinarily slow, even with > default maxDeterminizedStates limit > > > Key: LUCENE-9981 > URL: https://issues.apache.org/jira/browse/LUCENE-9981 > Project: Lucene - Core > Issue Type: Task >Reporter: Robert Muir >Priority: Major > Attachments: LUCENE-9981.patch, LUCENE-9981.patch, LUCENE-9981.patch, > LUCENE-9981.patch, LUCENE-9981_nfaprefix.patch, LUCENE-9981_test.patch, > three-repeats-reverse-det.png, three-repeats.png > > > We have a {{maxDeterminizedStates = 1}} limit designed to keep > regexp-type queries from blowing up. > But we have an adversary that will run for 268s on my laptop before hitting > exception, first reported here: > https://github.com/opensearch-project/OpenSearch/issues/687 > When I run the test and jstack the threads, this what I see: > {noformat} > "TEST-TestOpensearch687.testInteresting-seed#[4B9C20A027A9850C]" #15 prio=5 > os_prio=0 cpu=56960.04ms elapsed=57.49s tid=0x7fff7006ca80 nid=0x231c8 > runnable [0x7fff8b7f] >java.lang.Thread.State: RUNNABLE > at > org.apache.lucene.util.automaton.SortedIntSet.decr(SortedIntSet.java:106) > at > org.apache.lucene.util.automaton.Operations.determinize(Operations.java:769) > at > org.apache.lucene.util.automaton.Operations.getCommonSuffixBytesRef(Operations.java:1155) > at > org.apache.lucene.util.automaton.CompiledAutomaton.(CompiledAutomaton.java:247) > at > org.apache.lucene.search.AutomatonQuery.(AutomatonQuery.java:104) > at > org.apache.lucene.search.AutomatonQuery.(AutomatonQuery.java:82) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:138) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:114) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:72) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:62) > at > org.apache.lucene.TestOpensearch687.testInteresting(TestOpensearch687.java:42) > {noformat} > This is really sad, as {{getCommonSuffixBytesRef()}} is only supposed to be > an "up-front" optimization to make the actual subsequent terms-intensive part > of the query faster. But it makes the whole query run for nearly 5 minutes > before it does anything. > So I definitely think we should improve {{getCommonSuffixBytesRef}} to be > more "best-effort". For example, it can reduce the lower bound to {{1000}} > and catch the exception like such: > {code} > try { >// this is slow, and just an opto anyway, so don't burn cycles on it for > some crazy worst-case. >// if we don't set this common suffix, the query will just run a bit > slower, that's all. >int limit = Math.min(1000, maxDeterminizedStates); >BytesRef suffix = Operations.getCommonSuffixBytesRef(binary, limit); >... (setting commonSuffixRef) > } catch (TooComplexTooDeterminizeException notWorthIt) { > commonSuffixRef = null; > } > {code} > Another, maybe simpler option, is to just check that input state/transitions > accounts don't exceed some low limit N. > Basically this opto is geared at stuff like leading wildcard query of "*foo". > By computing that the common suffix is "foo" we can spend less CPU in the > terms dictionary because we can first do a memcmp before having to run data > thru any finite state machine. It's really a microopt and we shouldn't be > spending whole seconds of cpu on it, ever. > But I still don't quite understand how the current limits are giving the > behavior today, maybe there is a bigger issue and I don't want to shov
[jira] [Resolved] (LUCENE-9980) Do not expose deleted commits in IndexDeletionPolicy#onCommit
[ https://issues.apache.org/jira/browse/LUCENE-9980?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Nhat Nguyen resolved LUCENE-9980. - Fix Version/s: 9.0 8.9 Resolution: Fixed > Do not expose deleted commits in IndexDeletionPolicy#onCommit > - > > Key: LUCENE-9980 > URL: https://issues.apache.org/jira/browse/LUCENE-9980 > Project: Lucene - Core > Issue Type: Bug > Components: core/index >Affects Versions: 8.8.1 >Reporter: Nhat Nguyen >Priority: Major > Fix For: 8.9, 9.0 > > Time Spent: 40m > Remaining Estimate: 0h > > If we fail to delete files that belong to a commit point, then we will expose > that deleted commit in the next calls of IndexDeletionPolicy#onCommit(). I > think we should never expose those deleted commit points as some of their > files might have been deleted already. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Commented] (LUCENE-9980) Do not expose deleted commits in IndexDeletionPolicy#onCommit
[ https://issues.apache.org/jira/browse/LUCENE-9980?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354512#comment-17354512 ] ASF subversion and git services commented on LUCENE-9980: - Commit ddc238e5df870dc643f862738b4ca059b6b7c8c0 in lucene-solr's branch refs/heads/branch_8x from Nhat Nguyen [ https://gitbox.apache.org/repos/asf?p=lucene-solr.git;h=ddc238e ] LUCENE-9980: Do not expose deleted commits (#158) If we fail to delete files that belong to a commit point, then we will expose that deleted commit in the next calls of IndexDeletionPolicy#onCommit. I think we should never expose those deleted commit points as some of their files might have been deleted already. > Do not expose deleted commits in IndexDeletionPolicy#onCommit > - > > Key: LUCENE-9980 > URL: https://issues.apache.org/jira/browse/LUCENE-9980 > Project: Lucene - Core > Issue Type: Bug > Components: core/index >Affects Versions: 8.8.1 >Reporter: Nhat Nguyen >Priority: Major > Time Spent: 40m > Remaining Estimate: 0h > > If we fail to delete files that belong to a commit point, then we will expose > that deleted commit in the next calls of IndexDeletionPolicy#onCommit(). I > think we should never expose those deleted commit points as some of their > files might have been deleted already. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] mikemccand commented on a change in pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index check across segments
mikemccand commented on a change in pull request #128: URL: https://github.com/apache/lucene/pull/128#discussion_r642550067 ## File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java ## @@ -605,209 +680,103 @@ public Status checkIndex(List onlySegments) throws IOException { result.newSegments.clear(); result.maxSegmentName = -1; -for (int i = 0; i < numSegments; i++) { - final SegmentCommitInfo info = sis.info(i); - long segmentName = Long.parseLong(info.info.name.substring(1), Character.MAX_RADIX); - if (segmentName > result.maxSegmentName) { -result.maxSegmentName = segmentName; - } - if (onlySegments != null && !onlySegments.contains(info.info.name)) { -continue; - } - Status.SegmentInfoStatus segInfoStat = new Status.SegmentInfoStatus(); - result.segmentInfos.add(segInfoStat); - msg( - infoStream, - " " - + (1 + i) - + " of " - + numSegments - + ": name=" - + info.info.name - + " maxDoc=" - + info.info.maxDoc()); - segInfoStat.name = info.info.name; - segInfoStat.maxDoc = info.info.maxDoc(); - - final Version version = info.info.getVersion(); - if (info.info.maxDoc() <= 0) { -throw new RuntimeException("illegal number of documents: maxDoc=" + info.info.maxDoc()); - } - - int toLoseDocCount = info.info.maxDoc(); - - SegmentReader reader = null; - - try { -msg(infoStream, "version=" + (version == null ? "3.0" : version)); -msg(infoStream, "id=" + StringHelper.idToString(info.info.getId())); -final Codec codec = info.info.getCodec(); -msg(infoStream, "codec=" + codec); -segInfoStat.codec = codec; -msg(infoStream, "compound=" + info.info.getUseCompoundFile()); -segInfoStat.compound = info.info.getUseCompoundFile(); -msg(infoStream, "numFiles=" + info.files().size()); -Sort indexSort = info.info.getIndexSort(); -if (indexSort != null) { - msg(infoStream, "sort=" + indexSort); -} -segInfoStat.numFiles = info.files().size(); -segInfoStat.sizeMB = info.sizeInBytes() / (1024. * 1024.); -msg(infoStream, "size (MB)=" + nf.format(segInfoStat.sizeMB)); -Map diagnostics = info.info.getDiagnostics(); -segInfoStat.diagnostics = diagnostics; -if (diagnostics.size() > 0) { - msg(infoStream, "diagnostics = " + diagnostics); +// checks segments sequentially +if (executorService == null) { Review comment: Maybe we should make a single threaded executor so we don't have to bifurcate the code? ## File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java ## @@ -843,6 +812,258 @@ public Status checkIndex(List onlySegments) throws IOException { return result; } + private void updateMaxSegmentName(Status result, SegmentCommitInfo info) { +long segmentName = Long.parseLong(info.info.name.substring(1), Character.MAX_RADIX); +if (segmentName > result.maxSegmentName) { + result.maxSegmentName = segmentName; +} + } + + private void processSegmentInfoStatusResult( + Status result, SegmentCommitInfo info, Status.SegmentInfoStatus segmentInfoStatus) { +result.segmentInfos.add(segmentInfoStatus); +if (segmentInfoStatus.error != null) { + result.totLoseDocCount += segmentInfoStatus.toLoseDocCount; + result.numBadSegments++; +} else { + // Keeper + result.newSegments.add(info.clone()); +} + } + + private CompletableFuture runAsyncSegmentCheck( + Callable asyncCallable, ExecutorService executorService) { +return CompletableFuture.supplyAsync(callableToSupplier(asyncCallable), executorService); + } + + private Supplier callableToSupplier(Callable callable) { +return () -> { + try { +return callable.call(); + } catch (RuntimeException | Error e) { +throw e; + } catch (Throwable e) { +throw new CompletionException(e); + } +}; + } + + private Status.SegmentInfoStatus testSegment( + SegmentInfos sis, SegmentCommitInfo info, PrintStream infoStream) throws IOException { +Status.SegmentInfoStatus segInfoStat = new Status.SegmentInfoStatus(); +segInfoStat.name = info.info.name; +segInfoStat.maxDoc = info.info.maxDoc(); + +final Version version = info.info.getVersion(); +if (info.info.maxDoc() <= 0) { + throw new CheckIndexException(" illegal number of documents: maxDoc=" + info.info.maxDoc()); +} + +int toLoseDocCount = info.info.maxDoc(); Review comment: Hmm, sometimes this seems to be `maxDoc` (here) and other times `numDocs` (accounting for deleted documents properly) -- let's try to be consistent with what it was before (I think `numDocs`)? ## Fi
[GitHub] [lucene] mikemccand commented on pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index check across segments
mikemccand commented on pull request #128: URL: https://github.com/apache/lucene/pull/128#issuecomment-851551676 Also, note that the "first segment" might not necessarily always be the largest! The way our default merge policy works, that tends to be the case, but users can customize the merge policy, and we recently added a feature to let `IndexWriter` customize how it sorts the segments. We should maybe sort the segments by descending `docCount` and work on them (concurrently) in that order? -- 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. 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] dnhatn merged pull request #158: LUCENE-9980: Do not expose deleted commit points in IndexDeletionPolicy
dnhatn merged pull request #158: URL: https://github.com/apache/lucene/pull/158 -- 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. 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-9980) Do not expose deleted commits in IndexDeletionPolicy#onCommit
[ https://issues.apache.org/jira/browse/LUCENE-9980?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354501#comment-17354501 ] ASF subversion and git services commented on LUCENE-9980: - Commit c46bcf75cc6592cc492fabd42e1ec41dbe96304d in lucene's branch refs/heads/main from Nhat Nguyen [ https://gitbox.apache.org/repos/asf?p=lucene.git;h=c46bcf7 ] LUCENE-9980: Do not expose deleted commits (#158) If we fail to delete files that belong to a commit point, then we will expose that deleted commit in the next calls of IndexDeletionPolicy#onCommit. I think we should never expose those deleted commit points as some of their files might have been deleted already. > Do not expose deleted commits in IndexDeletionPolicy#onCommit > - > > Key: LUCENE-9980 > URL: https://issues.apache.org/jira/browse/LUCENE-9980 > Project: Lucene - Core > Issue Type: Bug > Components: core/index >Affects Versions: 8.8.1 >Reporter: Nhat Nguyen >Priority: Major > Time Spent: 0.5h > Remaining Estimate: 0h > > If we fail to delete files that belong to a commit point, then we will expose > that deleted commit in the next calls of IndexDeletionPolicy#onCommit(). I > think we should never expose those deleted commit points as some of their > files might have been deleted already. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] dnhatn commented on pull request #158: LUCENE-9980: Do not expose deleted commit points in IndexDeletionPolicy
dnhatn commented on pull request #158: URL: https://github.com/apache/lucene/pull/158#issuecomment-851545808 Thanks Mike for reviewing :). > Was there an exciting test failure somewhere the lead to this discovery? Yes, there's a test failure in Elasticsearch: https://github.com/elastic/elasticsearch/issues/39591#issuecomment-655920692. -- 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. 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] mikemccand commented on pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index check across segments
mikemccand commented on pull request #128: URL: https://github.com/apache/lucene/pull/128#issuecomment-851538612 > To get the best speedup, even at -slow, we should do concurrency both ways, and then sort those tasks by decreasing expected cost. This fine-grained concurrency (one task for "part X segment") would then mean we can get the postings check of the big segments all going, early, concurrently across all segments, and likely the biggest segments postings check would be the long pole, and {{CheckIndex}} would finish in (about) that much time. Versus "thread per segment" concurrency, where the long pole will be one thread checking the biggest segment. The really fun part of concurrency across *and* within segments will be figure out the rough heuristic of how to assign estimated cost to each task :) But this all can come later! "thread per segment" is already a big step forward! -- 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. 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] mikemccand commented on pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index check across segments
mikemccand commented on pull request #128: URL: https://github.com/apache/lucene/pull/128#issuecomment-851532048 > Oh one more thing. As the log output was buffered during parallel execution and printed later in sequential fashion to maintain order, to help out those who might be eager to see the output, for the first segment (which consumes the most of time during check) I have used the "global" infoStream to print log as they are available - this gives the "weird" printing behavior that the first segment check prints slowly while it progress, and once the first segment finishes then all the subsequent segment outputs got printed at once. Not sure if this behavior is ok as it may be perceived as buggy by the user? Hmm, could we instead just output the full segment's output as each finishes? So we would presumably see small segments finish first, then all checks for that small segment are printed? -- 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. 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] mikemccand commented on pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index check across segments
mikemccand commented on pull request #128: URL: https://github.com/apache/lucene/pull/128#issuecomment-851528281 > One issue though is that this bit flipping is causing checksum integrity check failures before the concurrent segment part checks kick in, so it may not test the changes here? I think we may actually need to write a semantically buggy segment file with good checksum verification to see the error still gets detected and propagated correctly? You're right -- it is able to do that. It's random, so if it flips a "really important" bit, such that the `segments_N` file cannot be loaded, then `CheckIndex` will fail, quickly, without checking the segments. If you watch its output and re-run it until you get a single segment corruption event, then it should showcase the gains in this PR. -- 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. 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] mikemccand commented on pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index check across segments
mikemccand commented on pull request #128: URL: https://github.com/apache/lucene/pull/128#issuecomment-851527406 Thank you for all the awesome iterations here @zacharymorn! To get the best speedup, even at `-slow`, we should do concurrency both ways, and then sort those tasks by decreasing expected cost. This way the work queue would first output all postings checks (across all segments), one per thread, followed by doc values, etc. We could even get a bit crazy, e.g. checking postings for a tiny segment is surely expected to be faster than checking doc values for a massive segment. But we can add such complexity later -- the PR now ("thread per segment") is surely a great step forward too :) And +1 to spinoff a separate issue to change `CheckIndex` to default to `-fast` -- this is really long overdue since we added end-to-end checksums to Lucene! -- 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. 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] [Updated] (LUCENE-9971) Inconsistent SSDVFF and Taxonomy facet behavior in case of unseen dimension
[ https://issues.apache.org/jira/browse/LUCENE-9971?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Miller updated LUCENE-9971: Fix Version/s: main (9.0) Resolution: Fixed Status: Resolved (was: Patch Available) > Inconsistent SSDVFF and Taxonomy facet behavior in case of unseen dimension > --- > > Key: LUCENE-9971 > URL: https://issues.apache.org/jira/browse/LUCENE-9971 > Project: Lucene - Core > Issue Type: Improvement >Reporter: Alexander L >Priority: Minor > Fix For: main (9.0) > > Time Spent: 2h 20m > Remaining Estimate: 0h > > Unlike other org.apache.lucene.facet.Facet interface implementations, > SortedSetDocValuesFacetCounts and ConcurrentSortedSetDocValuesFacetCounts > [throw an exception in case of unseen dimension in the getTopChildren() > method > |https://github.com/apache/lucene/blob/main/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesFacetCounts.java#L99] > > It is inconsistent with [Facet interface > documentation|https://github.com/apache/lucene/blob/main/lucene/facet/src/java/org/apache/lucene/facet/Facets.java#L33] > and not pleasant to deal with: in our case requested dimensions can be > manually specified by the user, so we should always catch that and check the > exception message to be sure of the exact cause. > During the PR review, it was also decided to make similar changes for > Taxonomy implementation. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Commented] (LUCENE-9971) Inconsistent SSDVFF and Taxonomy facet behavior in case of unseen dimension
[ https://issues.apache.org/jira/browse/LUCENE-9971?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354460#comment-17354460 ] Greg Miller commented on LUCENE-9971: - Thanks [~sqshq]! > Inconsistent SSDVFF and Taxonomy facet behavior in case of unseen dimension > --- > > Key: LUCENE-9971 > URL: https://issues.apache.org/jira/browse/LUCENE-9971 > Project: Lucene - Core > Issue Type: Improvement >Reporter: Alexander L >Priority: Minor > Time Spent: 2h 20m > Remaining Estimate: 0h > > Unlike other org.apache.lucene.facet.Facet interface implementations, > SortedSetDocValuesFacetCounts and ConcurrentSortedSetDocValuesFacetCounts > [throw an exception in case of unseen dimension in the getTopChildren() > method > |https://github.com/apache/lucene/blob/main/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesFacetCounts.java#L99] > > It is inconsistent with [Facet interface > documentation|https://github.com/apache/lucene/blob/main/lucene/facet/src/java/org/apache/lucene/facet/Facets.java#L33] > and not pleasant to deal with: in our case requested dimensions can be > manually specified by the user, so we should always catch that and check the > exception message to be sure of the exact cause. > During the PR review, it was also decided to make similar changes for > Taxonomy implementation. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Commented] (LUCENE-9971) Inconsistent SSDVFF and Taxonomy facet behavior in case of unseen dimension
[ https://issues.apache.org/jira/browse/LUCENE-9971?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354459#comment-17354459 ] ASF subversion and git services commented on LUCENE-9971: - Commit d76dd6454ed5ebe010a7db5f2a8153f90c59e71c in lucene's branch refs/heads/main from Greg Miller [ https://gitbox.apache.org/repos/asf?p=lucene.git;h=d76dd64 ] Add CHANGES.txt entry for LUCENE-9971 (#161) > Inconsistent SSDVFF and Taxonomy facet behavior in case of unseen dimension > --- > > Key: LUCENE-9971 > URL: https://issues.apache.org/jira/browse/LUCENE-9971 > Project: Lucene - Core > Issue Type: Improvement >Reporter: Alexander L >Priority: Minor > Time Spent: 2h 10m > Remaining Estimate: 0h > > Unlike other org.apache.lucene.facet.Facet interface implementations, > SortedSetDocValuesFacetCounts and ConcurrentSortedSetDocValuesFacetCounts > [throw an exception in case of unseen dimension in the getTopChildren() > method > |https://github.com/apache/lucene/blob/main/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesFacetCounts.java#L99] > > It is inconsistent with [Facet interface > documentation|https://github.com/apache/lucene/blob/main/lucene/facet/src/java/org/apache/lucene/facet/Facets.java#L33] > and not pleasant to deal with: in our case requested dimensions can be > manually specified by the user, so we should always catch that and check the > exception message to be sure of the exact cause. > During the PR review, it was also decided to make similar changes for > Taxonomy implementation. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] gsmiller merged pull request #161: Add CHANGES.txt entry for LUCENE-9971
gsmiller merged pull request #161: URL: https://github.com/apache/lucene/pull/161 -- 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. 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-9983) Stop sorting determinize powersets unnecessarily
[ https://issues.apache.org/jira/browse/LUCENE-9983?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354456#comment-17354456 ] Michael McCandless commented on LUCENE-9983: Hi [~zhai7631], thank you for jumping on this :) [Disclaimer: [~zhai7631] and I both work at Amazon, on customer facing product search, now [100% migrated to Apache Lucene|https://www.youtube.com/watch?v=EkkzSLstSAE]!] For the record, this code is attempting to implement the [Powerset Construction for NFA -> DFA|https://en.wikipedia.org/wiki/Powerset_construction]. I think it's really three different data structures we need. *1* (currently SortedIntSet) Map where the key is a state, and the value is its "reference count". We increment/decrement by state here, removing the entry from the map when its ref count drops to 0. This thing is sort of a factory to make *2* below: we periodically call {{freeze}} on this map to make a copy of just its keys, a set of ints, producing an instance of *2* below: *2* (currently FrozenIntSet) ** Set to hold a single powerset – this can maybe just be an simple int[] or maybe packed form or something *3* (currently HashMap Map,int> – the keys in this set are the Set from *2* and the values are int state numbers for the eventual (output) determinized automaton. For *3* we need hash/equals on *2* and that is why we sort *1*'s keys today. But that sorting is costly, and I think a big part of the cost for regepxs like the one in LUCENE-9981 and likely even for non-adversarial regexps too. If we stop the sorting, and use something like HPPC's {{IntHashSet}} (except, since this is Lucene's core, we cannot take a hard dependency, so we would need to poach/fork/specialize that sources) I think we can have O(N) hashCode() and equals(), same big-oh cost as those operations have today? And reduce the O(N * log(N)) we are paying for *1* today. > Stop sorting determinize powersets unnecessarily > > > Key: LUCENE-9983 > URL: https://issues.apache.org/jira/browse/LUCENE-9983 > Project: Lucene - Core > Issue Type: Improvement >Reporter: Michael McCandless >Priority: Major > > Spinoff from LUCENE-9981. > Today, our {{Operations.determinize}} implementation builds powersets of all > subsets of NFA states that "belong" in the same determinized state, using > [this algorithm|https://en.wikipedia.org/wiki/Powerset_construction]. > To hold each powerset, we use a malleable {{SortedIntSet}} and periodically > freeze it to a {{FrozenIntSet}}, also sorted. We pay a high price to keep > these growing maps of int key, int value sorted by key, e.g. upgrading to a > {{TreeMap}} once the map is large enough (> 30 entries). > But I think sorting is entirely unnecessary here! Really all we need is the > ability to add/delete keys from the map, and hashCode / equals (by key only – > ignoring value!), and to freeze the map (a small optimization that we could > skip initially). We only use these maps to lookup in the (growing) > determinized automaton whether this powerset has already been seen. > Maybe we could simply poach the {{IntIntScatterMap}} implementation from > [HPPC|https://github.com/carrotsearch/hppc]? And then change its > {{hashCode}}/{{equals }}to only use keys (not values). > This change should be a big speedup for the kinds of (admittedly adversarial) > regexps we saw on LUCENE-9981. > -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Resolved] (LUCENE-9946) Support multi-value fields in range facet counting
[ https://issues.apache.org/jira/browse/LUCENE-9946?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Miller resolved LUCENE-9946. - Fix Version/s: main (9.0) Resolution: Fixed > Support multi-value fields in range facet counting > -- > > Key: LUCENE-9946 > URL: https://issues.apache.org/jira/browse/LUCENE-9946 > Project: Lucene - Core > Issue Type: Improvement > Components: modules/facet >Affects Versions: main (9.0) >Reporter: Greg Miller >Priority: Minor > Fix For: main (9.0) > > Time Spent: 5h 10m > Remaining Estimate: 0h > > The {{RangeFacetCounts}} implementations ({{LongRangeFacetCounts}} and > {{DoubleRangeFacetCount}}) only work on single-valued fields today. In > contrast, the more recently added {{LongValueFacetCounts}} implementation > supports both single- and multi-valued fields (LUCENE-7927). I'd like to > extend multi-value support to both of the {{LongRangeFacetCounts}} > implementations as well. > Looking through the implementations, I can't think of a good reason to _not_ > support this, but maybe I'm overlooking something? -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] gsmiller opened a new pull request #161: Add CHANGES.txt entry for LUCENE-9971
gsmiller opened a new pull request #161: URL: https://github.com/apache/lucene/pull/161 -- 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. 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-9982) ../../gradlew help should be just as helpful as ./gradlew help?
[ https://issues.apache.org/jira/browse/LUCENE-9982?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354450#comment-17354450 ] Michael McCandless commented on LUCENE-9982: {quote}This isn't a bug, Mike. {quote} OK, PBKAC :) {quote}When you change the directory to a subproject (submodule), 'gradlew xxx' is applied to *that module* (so it looks up the xxx task in that module). It is context-sensitive with respect to where you are in the module hierarchy. This is the same as calling {{gradlew -p lucene/core help}} from the top-level. {quote} OK, indeed I confirmed that {{./gradlew -p lucene/core help}} prints the same message as {{cd lucene/core; ../../gradlew help}}. But my one hesitation here is that the top-level tasks do work when I am in {{cwd=lucene/core}}, i.e. I can run {{../../gradlew test --tests TestRegexpRandom2}} and it indeed only runs that one test, as expected. Given that {{../../gradlew}} indeed seems to accept the same tasks as when I am in the root directory, shouldn't its help advertise that too? Or maybe I'm wrong – maybe when I am in {{lucene/core}}, {{../../gradlew}} does not accept the same tasks? {quote}We do attach our "lucene developer" help guidance to the root project but not to *all* submodules. I don't think it makes sense to apply it across the board, to be honest? It can be done by I don't quite see the point. You can always call (from any folder): "gradlew :help" - this invokes the task by its path and ":" is the root project. {quote} Aha! That {{:}} does the trick – I will try remember this (weirdly cryptic) way of asking {{gradlew}} for help ;) Sorry for the silly newbie/Shoshin bugs :) > ../../gradlew help should be just as helpful as ./gradlew help? > --- > > Key: LUCENE-9982 > URL: https://issues.apache.org/jira/browse/LUCENE-9982 > Project: Lucene - Core > Issue Type: Bug >Reporter: Michael McCandless >Priority: Major > > If you are in the root of a Lucene git clone, and run {{./gradlew help}}, you > get useful help. > But if you first {{cd lucene/core}} and then run {{../../gradlew help}}, it > is less useful, and seems to be {{gradle}}'s generic help or something? > Is {{gradlew help}} known to be different depending on your cwd? > A good workaround (thanks [~rcmuir]) is to just look at the files under the > {{help}} directory in the root of the Lucene git clone. -- This message was sent by Atlassian Jira (v8.3.4#803005) - 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 #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index check across segments
rmuir commented on pull request #128: URL: https://github.com/apache/lucene/pull/128#issuecomment-851476434 > This also makes sense and seems to be an easy change to switch the default? Is there anything I need to add specifically so that users can be made aware of this change when they upgrade lucene version (e.g. extra log to indicate the switch) ? Maybe we should just open a separate issue for that. The change could be for 9.0-only, have its own issue, CHANGES.txt, entry MIGRATE.txt entry, etc. -- 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. 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-9971) Inconsistent SSDVFF and Taxonomy facet behavior in case of unseen dimension
[ https://issues.apache.org/jira/browse/LUCENE-9971?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354446#comment-17354446 ] ASF subversion and git services commented on LUCENE-9971: - Commit 65842c5c4d3fbc17c861491bdf06d337a2892d07 in lucene's branch refs/heads/main from Alexander Lukyanchikov [ https://gitbox.apache.org/repos/asf?p=lucene.git;h=65842c5 ] LUCENE-9971: Inconsistent SSDVFF and Taxonomy facet behavior in case of unseen dimension (#149) > Inconsistent SSDVFF and Taxonomy facet behavior in case of unseen dimension > --- > > Key: LUCENE-9971 > URL: https://issues.apache.org/jira/browse/LUCENE-9971 > Project: Lucene - Core > Issue Type: Improvement >Reporter: Alexander L >Priority: Minor > Time Spent: 1h 50m > Remaining Estimate: 0h > > Unlike other org.apache.lucene.facet.Facet interface implementations, > SortedSetDocValuesFacetCounts and ConcurrentSortedSetDocValuesFacetCounts > [throw an exception in case of unseen dimension in the getTopChildren() > method > |https://github.com/apache/lucene/blob/main/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesFacetCounts.java#L99] > > It is inconsistent with [Facet interface > documentation|https://github.com/apache/lucene/blob/main/lucene/facet/src/java/org/apache/lucene/facet/Facets.java#L33] > and not pleasant to deal with: in our case requested dimensions can be > manually specified by the user, so we should always catch that and check the > exception message to be sure of the exact cause. > During the PR review, it was also decided to make similar changes for > Taxonomy implementation. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] gsmiller merged pull request #149: LUCENE-9971: Inconsistent SSDVFF and Taxonomy facet behavior in case of unseen dimension
gsmiller merged pull request #149: URL: https://github.com/apache/lucene/pull/149 -- 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. 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 pull request #149: LUCENE-9971: Inconsistent SSDVFF and Taxonomy facet behavior in case of unseen dimension
gsmiller commented on pull request #149: URL: https://github.com/apache/lucene/pull/149#issuecomment-851473199 Looks great @sqshq! Thanks for taking this up. -- 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. 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-9982) ../../gradlew help should be just as helpful as ./gradlew help?
[ https://issues.apache.org/jira/browse/LUCENE-9982?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354311#comment-17354311 ] Dawid Weiss commented on LUCENE-9982: - This isn't a bug, Mike. When you change the directory to a subproject (submodule), 'gradlew xxx' is applied to *that module* (so it looks up the xxx task in that module). It is context-sensitive with respect to where you are in the module hierarchy. This is the same as calling {{gradlew -p lucene/core help}} from the top-level. We do attach our "lucene developer" help guidance to the root project but not to *all* submodules. I don't think it makes sense to apply it across the board, to be honest? It can be done by I don't quite see the point. You can always call (from any folder): "gradlew :help" - this invokes the task by its path and ":" is the root project. > ../../gradlew help should be just as helpful as ./gradlew help? > --- > > Key: LUCENE-9982 > URL: https://issues.apache.org/jira/browse/LUCENE-9982 > Project: Lucene - Core > Issue Type: Bug >Reporter: Michael McCandless >Priority: Major > > If you are in the root of a Lucene git clone, and run {{./gradlew help}}, you > get useful help. > But if you first {{cd lucene/core}} and then run {{../../gradlew help}}, it > is less useful, and seems to be {{gradle}}'s generic help or something? > Is {{gradlew help}} known to be different depending on your cwd? > A good workaround (thanks [~rcmuir]) is to just look at the files under the > {{help}} directory in the root of the Lucene git clone. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Commented] (LUCENE-9981) CompiledAutomaton.getCommonSuffix can be extraordinarily slow, even with default maxDeterminizedStates limit
[ https://issues.apache.org/jira/browse/LUCENE-9981?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354303#comment-17354303 ] Dawid Weiss commented on LUCENE-9981: - Never mind the comment about sorted maps, I see LUCENE-9983 now. > CompiledAutomaton.getCommonSuffix can be extraordinarily slow, even with > default maxDeterminizedStates limit > > > Key: LUCENE-9981 > URL: https://issues.apache.org/jira/browse/LUCENE-9981 > Project: Lucene - Core > Issue Type: Task >Reporter: Robert Muir >Priority: Major > Attachments: LUCENE-9981.patch, LUCENE-9981.patch, LUCENE-9981.patch, > LUCENE-9981.patch, LUCENE-9981_nfaprefix.patch, LUCENE-9981_test.patch, > three-repeats-reverse-det.png, three-repeats.png > > > We have a {{maxDeterminizedStates = 1}} limit designed to keep > regexp-type queries from blowing up. > But we have an adversary that will run for 268s on my laptop before hitting > exception, first reported here: > https://github.com/opensearch-project/OpenSearch/issues/687 > When I run the test and jstack the threads, this what I see: > {noformat} > "TEST-TestOpensearch687.testInteresting-seed#[4B9C20A027A9850C]" #15 prio=5 > os_prio=0 cpu=56960.04ms elapsed=57.49s tid=0x7fff7006ca80 nid=0x231c8 > runnable [0x7fff8b7f] >java.lang.Thread.State: RUNNABLE > at > org.apache.lucene.util.automaton.SortedIntSet.decr(SortedIntSet.java:106) > at > org.apache.lucene.util.automaton.Operations.determinize(Operations.java:769) > at > org.apache.lucene.util.automaton.Operations.getCommonSuffixBytesRef(Operations.java:1155) > at > org.apache.lucene.util.automaton.CompiledAutomaton.(CompiledAutomaton.java:247) > at > org.apache.lucene.search.AutomatonQuery.(AutomatonQuery.java:104) > at > org.apache.lucene.search.AutomatonQuery.(AutomatonQuery.java:82) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:138) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:114) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:72) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:62) > at > org.apache.lucene.TestOpensearch687.testInteresting(TestOpensearch687.java:42) > {noformat} > This is really sad, as {{getCommonSuffixBytesRef()}} is only supposed to be > an "up-front" optimization to make the actual subsequent terms-intensive part > of the query faster. But it makes the whole query run for nearly 5 minutes > before it does anything. > So I definitely think we should improve {{getCommonSuffixBytesRef}} to be > more "best-effort". For example, it can reduce the lower bound to {{1000}} > and catch the exception like such: > {code} > try { >// this is slow, and just an opto anyway, so don't burn cycles on it for > some crazy worst-case. >// if we don't set this common suffix, the query will just run a bit > slower, that's all. >int limit = Math.min(1000, maxDeterminizedStates); >BytesRef suffix = Operations.getCommonSuffixBytesRef(binary, limit); >... (setting commonSuffixRef) > } catch (TooComplexTooDeterminizeException notWorthIt) { > commonSuffixRef = null; > } > {code} > Another, maybe simpler option, is to just check that input state/transitions > accounts don't exceed some low limit N. > Basically this opto is geared at stuff like leading wildcard query of "*foo". > By computing that the common suffix is "foo" we can spend less CPU in the > terms dictionary because we can first do a memcmp before having to run data > thru any finite state machine. It's really a microopt and we shouldn't be > spending whole seconds of cpu on it, ever. > But I still don't quite understand how the current limits are giving the > behavior today, maybe there is a bigger issue and I don't want to shove > something under the rug. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[jira] [Commented] (LUCENE-9981) CompiledAutomaton.getCommonSuffix can be extraordinarily slow, even with default maxDeterminizedStates limit
[ https://issues.apache.org/jira/browse/LUCENE-9981?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354295#comment-17354295 ] Dawid Weiss commented on LUCENE-9981: - Hey. Sorry, guys - I was away and didn't see the thread. I see you have it all figured out (thanks for the breakdown, Robert!). Indeed, a good catch of a very interesting code angle (and arguably an adversarial regexp). Reminds me of the recent flurry of reDoS scans/attacks using github scans for potentially vulnerable subgroups [1]. Not the same, but equally puzzling at first. Anyway. Scanned the patch and it looks great to me. Caught one typo ("Automataon") in CHANGES.txt. I like minor-optimizations like the use of fixed bit sets too. One thing that bugs me is the int argument's "defaults". Currently it just says: bq. Set higher to allow more complex queries and lower to prevent memory exhaustion. Maybe we should give some ballpark ranges of what the defaults should be here? Or provide constants that would make it slightly easier for folks to refer to/ use? This comment is interesting: {code} + // TODO: these int sets really do not need to be sorted, and we are paying a high (unecessary) + // price for that! really we just need a + // low-overhead Map that implements equals/hash based only on the keys (ignores the + // values). fixing this would be a big + // speedup for determinizing complex automata {code} I wonder what you had in mind, Robert? I'd have to refresh my memory of the original algorithm but I think the key here is that lookup based on a key formed of a set of transitions (ints). I see Lucene's code precalculates the hash of them; values are sorted to speed up set-set comparisons in equals. I think the assumption here is that states have a short list of outgoing transitions so sorting is cheap. You could probably go for a tradeoff of a more complex hash (long) but set-set equals is still going to be difficult to avoid entirely? [1] https://en.wikipedia.org/wiki/ReDoS > CompiledAutomaton.getCommonSuffix can be extraordinarily slow, even with > default maxDeterminizedStates limit > > > Key: LUCENE-9981 > URL: https://issues.apache.org/jira/browse/LUCENE-9981 > Project: Lucene - Core > Issue Type: Task >Reporter: Robert Muir >Priority: Major > Attachments: LUCENE-9981.patch, LUCENE-9981.patch, LUCENE-9981.patch, > LUCENE-9981.patch, LUCENE-9981_nfaprefix.patch, LUCENE-9981_test.patch, > three-repeats-reverse-det.png, three-repeats.png > > > We have a {{maxDeterminizedStates = 1}} limit designed to keep > regexp-type queries from blowing up. > But we have an adversary that will run for 268s on my laptop before hitting > exception, first reported here: > https://github.com/opensearch-project/OpenSearch/issues/687 > When I run the test and jstack the threads, this what I see: > {noformat} > "TEST-TestOpensearch687.testInteresting-seed#[4B9C20A027A9850C]" #15 prio=5 > os_prio=0 cpu=56960.04ms elapsed=57.49s tid=0x7fff7006ca80 nid=0x231c8 > runnable [0x7fff8b7f] >java.lang.Thread.State: RUNNABLE > at > org.apache.lucene.util.automaton.SortedIntSet.decr(SortedIntSet.java:106) > at > org.apache.lucene.util.automaton.Operations.determinize(Operations.java:769) > at > org.apache.lucene.util.automaton.Operations.getCommonSuffixBytesRef(Operations.java:1155) > at > org.apache.lucene.util.automaton.CompiledAutomaton.(CompiledAutomaton.java:247) > at > org.apache.lucene.search.AutomatonQuery.(AutomatonQuery.java:104) > at > org.apache.lucene.search.AutomatonQuery.(AutomatonQuery.java:82) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:138) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:114) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:72) > at org.apache.lucene.search.RegexpQuery.(RegexpQuery.java:62) > at > org.apache.lucene.TestOpensearch687.testInteresting(TestOpensearch687.java:42) > {noformat} > This is really sad, as {{getCommonSuffixBytesRef()}} is only supposed to be > an "up-front" optimization to make the actual subsequent terms-intensive part > of the query faster. But it makes the whole query run for nearly 5 minutes > before it does anything. > So I definitely think we should improve {{getCommonSuffixBytesRef}} to be > more "best-effort". For example, it can reduce the lower bound to {{1000}} > and catch the exception like such: > {code} > try { >// this is slow, and just an opto anyway, so don't burn cycles on it for > some crazy worst-case. >// if we don't set this common suffix, the query will just run a bit > slower, that's all. >int l
[jira] [Commented] (LUCENE-9379) Directory based approach for index encryption
[ https://issues.apache.org/jira/browse/LUCENE-9379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17354275#comment-17354275 ] Bruno Roustant commented on LUCENE-9379: _RE AES-XTS vs AES-CTR:_ In the case of Lucene, we produce read-only files per index segment. And if we have a new random IV per file, we don't repeat the same (AES encrypted) blocks. So we are in a safe read-only-once case where AES-XTS and AES-CTR have the same strength [1][2]. Given that CTR is simpler, that's why I chose it for this patch. [1] https://crypto.stackexchange.com/questions/64556/aes-xts-vs-aes-ctr-for-write-once-storage [2] https://crypto.stackexchange.com/questions/14628/why-do-we-use-xts-over-ctr-for-disk-encryption > Directory based approach for index encryption > - > > Key: LUCENE-9379 > URL: https://issues.apache.org/jira/browse/LUCENE-9379 > Project: Lucene - Core > Issue Type: New Feature >Reporter: Bruno Roustant >Assignee: Bruno Roustant >Priority: Major > Time Spent: 2.5h > Remaining Estimate: 0h > > +Important+: This Lucene Directory wrapper approach is to be considered only > if an OS level encryption is not possible. OS level encryption better fits > Lucene usage of OS cache, and thus is more performant. > But there are some use-case where OS level encryption is not possible. This > Jira issue was created to address those. > > > The goal is to provide optional encryption of the index, with a scope limited > to an encryptable Lucene Directory wrapper. > Encryption is at rest on disk, not in memory. > This simple approach should fit any Codec as it would be orthogonal, without > modifying APIs as much as possible. > Use a standard encryption method. Limit perf/memory impact as much as > possible. > Determine how callers provide encryption keys. They must not be stored on > disk. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org