[PR] Move addNode to FSTCompiler [lucene]
dungba88 opened a new pull request, #12646: URL: https://github.com/apache/lucene/pull/12646 ### Description Currently FSTCompiler and FST has a circular dependencies to each other. FSTCompiler creates an instance of FST, and on adding node, it delegates to `FST.addNode()` and passing itself as a variable. This introduces a circular dependency and mixes up the FST constructing and traversing code. To make matter worse, this implies one can call `FST.addNode` with an arbitrary FSTCompiler (as it's a parameter), but in reality it should be the compiler which creates the FST. This PR move the `addNode` method to FSTCompiler instead. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Avoid duplicate array fill in BPIndexReorderer [lucene]
gf2121 merged PR #12645: URL: https://github.com/apache/lucene/pull/12645 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Ensure LeafCollector#finish is only called once on the main collector during drill-sideways [lucene]
gf2121 commented on code in PR #12642: URL: https://github.com/apache/lucene/pull/12642#discussion_r1351726366 ## lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java: ## @@ -1490,7 +1542,22 @@ public List reduce(Collection collectors) { .collect(Collectors.toList()); } } - ; + + private static class FinishOnceLeafCollector implements LeafCollector { +boolean finished; + +@Override +public void setScorer(Scorable scorer) throws IOException {} + +@Override +public void collect(int doc) throws IOException {} + +@Override +public void finish() throws IOException { + assertFalse(finished); Review Comment: Maybe assert to make sure this method get called? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Move addNode to FSTCompiler [lucene]
romseygeek commented on PR #12646: URL: https://github.com/apache/lucene/pull/12646#issuecomment-1754625497 Thanks for opening @dungba88! This FST building code is very hairy and this is a nice start at cleaning it up. Given how expert this code is and that the relevant methods are all package-private I don't see a problem with backporting this to 9x - what do you think @mikemccand? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Ensure LeafCollector#finish is only called once on the main collector during drill-sideways [lucene]
jpountz commented on code in PR #12642: URL: https://github.com/apache/lucene/pull/12642#discussion_r1351794343 ## lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java: ## @@ -316,6 +316,58 @@ public void testBasic() throws Exception { IOUtils.close(searcher.getIndexReader(), taxoReader, taxoWriter, dir, taxoDir); } + public void testCollectionTerminated() throws Exception { +try (Directory dir = newDirectory(); +Directory taxoDir = newDirectory(); +RandomIndexWriter w = new RandomIndexWriter(random(), dir); +DirectoryTaxonomyWriter taxoW = +new DirectoryTaxonomyWriter(taxoDir, IndexWriterConfig.OpenMode.CREATE)) { + FacetsConfig facetsConfig = new FacetsConfig(); + + Document d = new Document(); + d.add(new FacetField("foo", "bar")); + w.addDocument(facetsConfig.build(taxoW, d)); + + try (IndexReader r = w.getReader(); + TaxonomyReader taxoR = new DirectoryTaxonomyReader(taxoW)) { +IndexSearcher searcher = new IndexSearcher(r); + +Query baseQuery = new MatchAllDocsQuery(); +Query dimQ = new TermQuery(new Term("foo", "bar")); + +DrillDownQuery ddq = new DrillDownQuery(facetsConfig, baseQuery); +ddq.add("foo", dimQ); +DrillSideways drillSideways = new DrillSideways(searcher, facetsConfig, taxoR); + +CollectorManager cm = +new CollectorManager<>() { + @Override + public Collector newCollector() throws IOException { +return new Collector() { Review Comment: Maybe make AssertingCollector public and use it here? It gives additional checks, like making sure finish() is called on a leaf before moving to the next one. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Move addNode to FSTCompiler [lucene]
mikemccand commented on code in PR #12646: URL: https://github.com/apache/lucene/pull/12646#discussion_r1351794651 ## lucene/core/src/java/org/apache/lucene/util/fst/FST.java: ## @@ -640,381 +602,11 @@ public static boolean targetHasArcs(Arc arc) { return arc.target() > 0; } - // serializes new node by appending its bytes to the end - // of the current byte[] - long addNode(FSTCompiler fstCompiler, FSTCompiler.UnCompiledNode nodeIn) - throws IOException { -T NO_OUTPUT = outputs.getNoOutput(); - -// System.out.println("FST.addNode pos=" + bytes.getPosition() + " numArcs=" + nodeIn.numArcs); -if (nodeIn.numArcs == 0) { - if (nodeIn.isFinal) { -return FINAL_END_NODE; - } else { -return NON_FINAL_END_NODE; - } -} -final long startAddress = fstCompiler.bytes.getPosition(); -// System.out.println(" startAddr=" + startAddress); - -final boolean doFixedLengthArcs = shouldExpandNodeWithFixedLengthArcs(fstCompiler, nodeIn); -if (doFixedLengthArcs) { - // System.out.println(" fixed length arcs"); - if (fstCompiler.numBytesPerArc.length < nodeIn.numArcs) { -fstCompiler.numBytesPerArc = new int[ArrayUtil.oversize(nodeIn.numArcs, Integer.BYTES)]; -fstCompiler.numLabelBytesPerArc = new int[fstCompiler.numBytesPerArc.length]; - } -} - -fstCompiler.arcCount += nodeIn.numArcs; - -final int lastArc = nodeIn.numArcs - 1; - -long lastArcStart = fstCompiler.bytes.getPosition(); -int maxBytesPerArc = 0; -int maxBytesPerArcWithoutLabel = 0; -for (int arcIdx = 0; arcIdx < nodeIn.numArcs; arcIdx++) { - final FSTCompiler.Arc arc = nodeIn.arcs[arcIdx]; - final FSTCompiler.CompiledNode target = (FSTCompiler.CompiledNode) arc.target; - int flags = 0; - // System.out.println(" arc " + arcIdx + " label=" + arc.label + " -> target=" + - // target.node); - - if (arcIdx == lastArc) { -flags += BIT_LAST_ARC; - } - - if (fstCompiler.lastFrozenNode == target.node && !doFixedLengthArcs) { -// TODO: for better perf (but more RAM used) we -// could avoid this except when arc is "near" the -// last arc: -flags += BIT_TARGET_NEXT; - } - - if (arc.isFinal) { -flags += BIT_FINAL_ARC; -if (arc.nextFinalOutput != NO_OUTPUT) { - flags += BIT_ARC_HAS_FINAL_OUTPUT; -} - } else { -assert arc.nextFinalOutput == NO_OUTPUT; - } - - boolean targetHasArcs = target.node > 0; - - if (!targetHasArcs) { -flags += BIT_STOP_NODE; - } - - if (arc.output != NO_OUTPUT) { -flags += BIT_ARC_HAS_OUTPUT; - } - - fstCompiler.bytes.writeByte((byte) flags); - long labelStart = fstCompiler.bytes.getPosition(); - writeLabel(fstCompiler.bytes, arc.label); - int numLabelBytes = (int) (fstCompiler.bytes.getPosition() - labelStart); - - // System.out.println(" write arc: label=" + (char) arc.label + " flags=" + flags + " - // target=" + target.node + " pos=" + bytes.getPosition() + " output=" + - // outputs.outputToString(arc.output)); - - if (arc.output != NO_OUTPUT) { -outputs.write(arc.output, fstCompiler.bytes); -// System.out.println("write output"); - } - - if (arc.nextFinalOutput != NO_OUTPUT) { -// System.out.println("write final output"); -outputs.writeFinalOutput(arc.nextFinalOutput, fstCompiler.bytes); - } - - if (targetHasArcs && (flags & BIT_TARGET_NEXT) == 0) { -assert target.node > 0; -// System.out.println("write target"); -fstCompiler.bytes.writeVLong(target.node); - } - - // just write the arcs "like normal" on first pass, but record how many bytes each one took - // and max byte size: - if (doFixedLengthArcs) { -int numArcBytes = (int) (fstCompiler.bytes.getPosition() - lastArcStart); -fstCompiler.numBytesPerArc[arcIdx] = numArcBytes; -fstCompiler.numLabelBytesPerArc[arcIdx] = numLabelBytes; -lastArcStart = fstCompiler.bytes.getPosition(); -maxBytesPerArc = Math.max(maxBytesPerArc, numArcBytes); -maxBytesPerArcWithoutLabel = -Math.max(maxBytesPerArcWithoutLabel, numArcBytes - numLabelBytes); -// System.out.println("arcBytes=" + numArcBytes + " labelBytes=" + numLabelBytes); - } -} - -// TODO: try to avoid wasteful cases: disable doFixedLengthArcs in that case -/* - * - * LUCENE-4682: what is a fair heuristic here? - * It could involve some of these: - * 1. how "busy" the node is: nodeIn.inputCount relative to frontier[0].inputCount? - * 2. how much binSearch saves over scan: nodeIn.numArcs - * 3. waste: numBytes vs numBytesExpanded - * - * the one below just looks at #3 -if (doFixedLengthArcs) { - // rough heuristic: make this 1.25 "waste
Re: [PR] Move addNode to FSTCompiler [lucene]
mikemccand commented on PR #12646: URL: https://github.com/apache/lucene/pull/12646#issuecomment-1754655466 > Given how expert this code is and that the relevant methods are all package-private I don't see a problem with backporting this to 9x - what do you think @mikemccand? +1 to backport to 9x -- this is nice refactoring that does not change any API and is low risk. In general, we should strive to backport all changes to 9.x by default unless there is a solid reason not to (e.g. unavoidable / complex API break). Thanks @dungba88 and @romseygeek! -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Refactor ByteBlockPool so it is just a "shift/mask big array" [lucene]
iverase commented on PR #12625: URL: https://github.com/apache/lucene/pull/12625#issuecomment-1754688030 I run wikimediumall and still a bit noisy: ``` TaskQPS baseline StdDevQPS my_modified_version StdDevPct diff p-value BrowseRandomLabelSSDVFacets5.34 (9.9%)5.08 (9.5%) -4.9% ( -22% - 16%) 0.112 BrowseMonthSSDVFacets6.25 (10.3%)5.98 (11.2%) -4.3% ( -23% - 19%) 0.201 IntNRQ 74.13 (15.2%) 70.96 (16.8%) -4.3% ( -31% - 32%) 0.398 BrowseDayOfYearSSDVFacets6.24 (10.0%)5.98 (10.9%) -4.2% ( -22% - 18%) 0.208 TermDTSort 192.52 (5.3%) 186.26 (7.6%) -3.3% ( -15% - 10%) 0.118 HighTermTitleBDVSort7.68 (7.3%)7.43 (7.5%) -3.2% ( -16% - 12%) 0.168 BrowseDateSSDVFacets1.45 (11.8%)1.40 (16.2%) -3.1% ( -27% - 28%) 0.482 HighIntervalsOrdered4.65 (18.0%)4.51 (15.4%) -3.1% ( -30% - 36%) 0.555 OrHighHigh 35.11 (9.2%) 34.06 (10.6%) -3.0% ( -20% - 18%) 0.340 Prefix3 339.69 (4.8%) 329.55 (9.2%) -3.0% ( -16% - 11%) 0.198 OrHighLow 549.00 (5.6%) 532.81 (6.7%) -2.9% ( -14% -9%) 0.131 MedSloppyPhrase 19.19 (6.3%) 18.67 (9.4%) -2.7% ( -17% - 13%) 0.283 MedIntervalsOrdered 34.54 (17.0%) 33.61 (15.1%) -2.7% ( -29% - 35%) 0.596 OrHighMed 251.55 (7.5%) 244.84 (9.1%) -2.7% ( -17% - 15%) 0.312 OrNotHighHigh 362.40 (3.0%) 353.04 (7.9%) -2.6% ( -13% -8%) 0.173 OrNotHighMed 497.20 (3.4%) 484.74 (6.8%) -2.5% ( -12% -7%) 0.140 AndHighLow 852.22 (3.3%) 831.23 (7.3%) -2.5% ( -12% -8%) 0.170 HighTermDayOfYearSort 283.45 (2.9%) 276.71 (7.5%) -2.4% ( -12% -8%) 0.188 OrHighNotLow 513.51 (4.1%) 501.37 (7.7%) -2.4% ( -13% -9%) 0.226 AndHighMed 68.50 (3.0%) 66.90 (7.5%) -2.3% ( -12% -8%) 0.192 LowSloppyPhrase 25.08 (4.0%) 24.50 (8.4%) -2.3% ( -14% - 10%) 0.263 OrHighNotHigh 332.84 (3.2%) 325.15 (7.5%) -2.3% ( -12% -8%) 0.207 LowIntervalsOrdered3.87 (13.4%)3.78 (12.6%) -2.2% ( -24% - 27%) 0.588 OrHighNotMed 429.43 (4.2%) 420.06 (8.2%) -2.2% ( -13% - 10%) 0.289 OrNotHighLow 596.57 (2.5%) 583.76 (6.5%) -2.1% ( -10% -6%) 0.167 HighTermTitleSort 143.63 (2.2%) 140.62 (6.9%) -2.1% ( -10% -7%) 0.194 AndHighHigh 32.01 (2.6%) 31.34 (7.3%) -2.1% ( -11% -8%) 0.228 LowPhrase 38.27 (1.7%) 37.51 (7.5%) -2.0% ( -11% -7%) 0.245 MedPhrase 167.44 (2.2%) 164.14 (8.0%) -2.0% ( -11% -8%) 0.288 LowTerm 770.36 (4.6%) 755.91 (6.1%) -1.9% ( -12% -9%) 0.274 MedTermDayTaxoFacets 18.15 (2.8%) 17.81 (6.8%) -1.9% ( -11% -7%) 0.258 HighSloppyPhrase6.57 (3.4%)6.45 (8.0%) -1.8% ( -12% -9%) 0.354 LowSpanNear 86.59 (1.8%) 85.10 (6.8%) -1.7% ( -10% -6%) 0.272 HighPhrase 184.92 (2.1%) 181.75 (6.9%) -1.7% ( -10% -7%) 0.288 AndHighMedDayTaxoFacets 46.77 (2.5%) 45.97 (7.6%) -1.7% ( -11% -8%) 0.338 HighSpanNear6.19 (1.6%)6.09 (7.0%) -1.7% ( -10% -7%) 0.301 Respell 76.88 (1.4%) 75.61 (7.3%) -1.7% ( -10% -7%) 0.321 MedSpanNear 33.16 (1.9%) 32.64 (7.0%) -1.6% ( -10% -7%) 0.335 Fuzzy2 30.94 (1.8%) 30.48 (6.9%) -1.5% ( -10% -7%) 0.344 MedTerm 788.96 (4.4%) 777.38 (8.7%) -1.5% ( -13% - 12%) 0.500 Fuzzy1 72.99 (1.9%) 71.92 (5.9%) -1.5% ( -9% -6
Re: [PR] Move addNode to FSTCompiler [lucene]
dungba88 commented on code in PR #12646: URL: https://github.com/apache/lucene/pull/12646#discussion_r1351877680 ## lucene/core/src/java/org/apache/lucene/util/fst/FST.java: ## @@ -640,381 +602,11 @@ public static boolean targetHasArcs(Arc arc) { return arc.target() > 0; } - // serializes new node by appending its bytes to the end - // of the current byte[] - long addNode(FSTCompiler fstCompiler, FSTCompiler.UnCompiledNode nodeIn) - throws IOException { -T NO_OUTPUT = outputs.getNoOutput(); - -// System.out.println("FST.addNode pos=" + bytes.getPosition() + " numArcs=" + nodeIn.numArcs); -if (nodeIn.numArcs == 0) { - if (nodeIn.isFinal) { -return FINAL_END_NODE; - } else { -return NON_FINAL_END_NODE; - } -} -final long startAddress = fstCompiler.bytes.getPosition(); -// System.out.println(" startAddr=" + startAddress); - -final boolean doFixedLengthArcs = shouldExpandNodeWithFixedLengthArcs(fstCompiler, nodeIn); -if (doFixedLengthArcs) { - // System.out.println(" fixed length arcs"); - if (fstCompiler.numBytesPerArc.length < nodeIn.numArcs) { -fstCompiler.numBytesPerArc = new int[ArrayUtil.oversize(nodeIn.numArcs, Integer.BYTES)]; -fstCompiler.numLabelBytesPerArc = new int[fstCompiler.numBytesPerArc.length]; - } -} - -fstCompiler.arcCount += nodeIn.numArcs; - -final int lastArc = nodeIn.numArcs - 1; - -long lastArcStart = fstCompiler.bytes.getPosition(); -int maxBytesPerArc = 0; -int maxBytesPerArcWithoutLabel = 0; -for (int arcIdx = 0; arcIdx < nodeIn.numArcs; arcIdx++) { - final FSTCompiler.Arc arc = nodeIn.arcs[arcIdx]; - final FSTCompiler.CompiledNode target = (FSTCompiler.CompiledNode) arc.target; - int flags = 0; - // System.out.println(" arc " + arcIdx + " label=" + arc.label + " -> target=" + - // target.node); - - if (arcIdx == lastArc) { -flags += BIT_LAST_ARC; - } - - if (fstCompiler.lastFrozenNode == target.node && !doFixedLengthArcs) { -// TODO: for better perf (but more RAM used) we -// could avoid this except when arc is "near" the -// last arc: -flags += BIT_TARGET_NEXT; - } - - if (arc.isFinal) { -flags += BIT_FINAL_ARC; -if (arc.nextFinalOutput != NO_OUTPUT) { - flags += BIT_ARC_HAS_FINAL_OUTPUT; -} - } else { -assert arc.nextFinalOutput == NO_OUTPUT; - } - - boolean targetHasArcs = target.node > 0; - - if (!targetHasArcs) { -flags += BIT_STOP_NODE; - } - - if (arc.output != NO_OUTPUT) { -flags += BIT_ARC_HAS_OUTPUT; - } - - fstCompiler.bytes.writeByte((byte) flags); - long labelStart = fstCompiler.bytes.getPosition(); - writeLabel(fstCompiler.bytes, arc.label); - int numLabelBytes = (int) (fstCompiler.bytes.getPosition() - labelStart); - - // System.out.println(" write arc: label=" + (char) arc.label + " flags=" + flags + " - // target=" + target.node + " pos=" + bytes.getPosition() + " output=" + - // outputs.outputToString(arc.output)); - - if (arc.output != NO_OUTPUT) { -outputs.write(arc.output, fstCompiler.bytes); -// System.out.println("write output"); - } - - if (arc.nextFinalOutput != NO_OUTPUT) { -// System.out.println("write final output"); -outputs.writeFinalOutput(arc.nextFinalOutput, fstCompiler.bytes); - } - - if (targetHasArcs && (flags & BIT_TARGET_NEXT) == 0) { -assert target.node > 0; -// System.out.println("write target"); -fstCompiler.bytes.writeVLong(target.node); - } - - // just write the arcs "like normal" on first pass, but record how many bytes each one took - // and max byte size: - if (doFixedLengthArcs) { -int numArcBytes = (int) (fstCompiler.bytes.getPosition() - lastArcStart); -fstCompiler.numBytesPerArc[arcIdx] = numArcBytes; -fstCompiler.numLabelBytesPerArc[arcIdx] = numLabelBytes; -lastArcStart = fstCompiler.bytes.getPosition(); -maxBytesPerArc = Math.max(maxBytesPerArc, numArcBytes); -maxBytesPerArcWithoutLabel = -Math.max(maxBytesPerArcWithoutLabel, numArcBytes - numLabelBytes); -// System.out.println("arcBytes=" + numArcBytes + " labelBytes=" + numLabelBytes); - } -} - -// TODO: try to avoid wasteful cases: disable doFixedLengthArcs in that case -/* - * - * LUCENE-4682: what is a fair heuristic here? - * It could involve some of these: - * 1. how "busy" the node is: nodeIn.inputCount relative to frontier[0].inputCount? - * 2. how much binSearch saves over scan: nodeIn.numArcs - * 3. waste: numBytes vs numBytesExpanded - * - * the one below just looks at #3 -if (doFixedLengthArcs) { - // rough heuristic: make this 1.25 "waste f
Re: [PR] [WIP] first cut at bounding the NodeHash size during FST compilation [lucene]
gf2121 commented on PR #12633: URL: https://github.com/apache/lucene/pull/12633#issuecomment-1754735435 > With the PR, you unfortunately cannot easily say "give me a minimal FST at all costs", like you can with main today. You'd have to keep trying larger and larger NodeHash sizes until the final FST size gets no smaller. If we replace `long[]` with a growable array that inits from small and grows smoothly to `nodeHashSize`, can we just pass a big nodeHashSize (e.g. 1L << 63) or a constant `NO_LIMIT` to get a minimal FST ? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Move addNode to FSTCompiler [lucene]
dungba88 commented on PR #12646: URL: https://github.com/apache/lucene/pull/12646#issuecomment-1754754284 > +1 to backport to 9x -- this is nice refactoring that does not change any API and is low risk. I can do this. Wondering if creating a PR to lucene-9_x branch would suffice? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Reduce FST block size for BlockTreeTermsWriter [lucene]
jpountz commented on PR #12604: URL: https://github.com/apache/lucene/pull/12604#issuecomment-1754762202 It looks like there's a bit less [Young GC](http://people.apache.org/~mikemccand/lucenebench/indexing.html) in nightly benchmarks since this change was merged, from 6-8 seconds, to consistently below 6s. I pushed an annotation. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Allow FST builder to use different writer (#12543) [lucene]
mikemccand commented on code in PR #12624: URL: https://github.com/apache/lucene/pull/12624#discussion_r1351949356 ## lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java: ## @@ -21,19 +21,18 @@ import java.util.List; import org.apache.lucene.store.DataInput; import org.apache.lucene.store.DataOutput; -import org.apache.lucene.util.Accountable; import org.apache.lucene.util.RamUsageEstimator; // TODO: merge with PagedBytes, except PagedBytes doesn't // let you read while writing which FST needs -class BytesStore extends DataOutput implements Accountable { +class BytesStore extends FSTWriter { Review Comment: > Besides `reverse` most of the methods here also needs to write/update arbitrarily previously written bytes (within the current frontier input), such as writeBytes/writeByte/copyBytes/truncate. I think these "modify prior bytes" are only called from `fst.addNode`, where that (complex!!) method is basically using the tail of the `BytesStore` as a scratch area? I.e. it writes some bytes first, and might go back and shuffle the bytes around, depending on whether it's "direct addressing" or "binary search"? Once `FST.addNode` completes, those written bytes are never altered? > Moreover, there is also methods which requires reading of previously written bytes that can't simply fit in the DataOutput (getReverseBytesReader e.g). Hmm `getReverseBytesReader` is indeed a problem. I wonder how the [Tantivy FST impl](https://blog.burntsushi.net/transducers/) deals with this? If we take the `FSTWriter` approach, are you thinking that we could make an impl of this class based e.g. on `FileChannel` directly (supports absolute positional reads, reading from a still-appending file), bypassing Lucene's `Directory` abstraction entirely? That is not a great solution (we try to have all IO go through `Directory`), but, perhaps as an intermediate state, for users directly creating massive FSTs, it's acceptable. But that'd mean we could not fix Lucene by default to do all of its FST compilation off-heap... -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Reduce FST block size for BlockTreeTermsWriter [lucene]
gf2121 commented on PR #12604: URL: https://github.com/apache/lucene/pull/12604#issuecomment-1754790165 @jpountz Thanks for annotating ! I also checked `blunders.io` for more details: * GC pause time: 6.38% -> 5.91% * Allocation Rate: 3.7 GiB/s -> 2.6 GiB/s * much more less `FST#init` in allocation flame graph :) **Before Patch** https://blunders.io/jfr-demo/indexing-4kb-2023.10.03.18.03.47/allocations-drill-down **After Patch** https://blunders.io/jfr-demo/indexing-4kb-2023.10.04.18.03.40/allocations-drill-down -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Early terminate visit BKD leaf when current value greater than upper point in sorted dim. [lucene]
vsop-479 commented on PR #12528: URL: https://github.com/apache/lucene/pull/12528#issuecomment-1754812860 @iverase Here is a performance test from luceneutil. box-points| baseline | candidate | Diff -- | -- | -- | -- BEST M hits/sec | 101.09 | 103.64 | 2.5% BEST QPS | 102.87 | 105.46 | 2.5% I only implemented visitWithSortedDim in PointRangeQuery (without "inverse" case) temporarily, so I just ran the box's points test. > run the performance test in the [lucene benchmarks] Do you mean the off the shelf benchmark in lucene self? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Move addNode to FSTCompiler [lucene]
romseygeek commented on PR #12646: URL: https://github.com/apache/lucene/pull/12646#issuecomment-1754833603 I'm happy to merge and backport @dungba88. Can you also add an entry to CHANGES.txt in the 9.9 section? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Move addNode to FSTCompiler [lucene]
dungba88 commented on PR #12646: URL: https://github.com/apache/lucene/pull/12646#issuecomment-1754877692 Thanks @romseygeek I have added the entry (under API change section). -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Early terminate visit BKD leaf when current value greater than upper point in sorted dim. [lucene]
iverase commented on PR #12528: URL: https://github.com/apache/lucene/pull/12528#issuecomment-1754953206 > Do you mean the off the shelf benchmark in lucene self? One case I am interested in is the geo benchmarks. It is not clear to me if some of those queries (e.g polygon query) can effectively take advantage of this change so it would be good to know that they are not slowed down. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Ensure LeafCollector#finish is only called once on the main collector during drill-sideways [lucene]
gsmiller commented on code in PR #12642: URL: https://github.com/apache/lucene/pull/12642#discussion_r1352301082 ## lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java: ## @@ -316,6 +316,58 @@ public void testBasic() throws Exception { IOUtils.close(searcher.getIndexReader(), taxoReader, taxoWriter, dir, taxoDir); } + public void testCollectionTerminated() throws Exception { +try (Directory dir = newDirectory(); +Directory taxoDir = newDirectory(); +RandomIndexWriter w = new RandomIndexWriter(random(), dir); +DirectoryTaxonomyWriter taxoW = +new DirectoryTaxonomyWriter(taxoDir, IndexWriterConfig.OpenMode.CREATE)) { + FacetsConfig facetsConfig = new FacetsConfig(); + + Document d = new Document(); + d.add(new FacetField("foo", "bar")); + w.addDocument(facetsConfig.build(taxoW, d)); + + try (IndexReader r = w.getReader(); + TaxonomyReader taxoR = new DirectoryTaxonomyReader(taxoW)) { +IndexSearcher searcher = new IndexSearcher(r); + +Query baseQuery = new MatchAllDocsQuery(); +Query dimQ = new TermQuery(new Term("foo", "bar")); + +DrillDownQuery ddq = new DrillDownQuery(facetsConfig, baseQuery); +ddq.add("foo", dimQ); +DrillSideways drillSideways = new DrillSideways(searcher, facetsConfig, taxoR); + +CollectorManager cm = +new CollectorManager<>() { + @Override + public Collector newCollector() throws IOException { +return new Collector() { Review Comment: Yeah, I like that suggestion. Thanks! -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Ensure LeafCollector#finish is only called once on the main collector during drill-sideways [lucene]
jpountz commented on code in PR #12642: URL: https://github.com/apache/lucene/pull/12642#discussion_r1352362240 ## lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java: ## @@ -316,6 +316,58 @@ public void testBasic() throws Exception { IOUtils.close(searcher.getIndexReader(), taxoReader, taxoWriter, dir, taxoDir); } + public void testCollectionTerminated() throws Exception { +try (Directory dir = newDirectory(); +Directory taxoDir = newDirectory(); +RandomIndexWriter w = new RandomIndexWriter(random(), dir); +DirectoryTaxonomyWriter taxoW = +new DirectoryTaxonomyWriter(taxoDir, IndexWriterConfig.OpenMode.CREATE)) { + FacetsConfig facetsConfig = new FacetsConfig(); + + Document d = new Document(); + d.add(new FacetField("foo", "bar")); + w.addDocument(facetsConfig.build(taxoW, d)); + + try (IndexReader r = w.getReader(); + TaxonomyReader taxoR = new DirectoryTaxonomyReader(taxoW)) { +IndexSearcher searcher = new IndexSearcher(r); + +Query baseQuery = new MatchAllDocsQuery(); +Query dimQ = new TermQuery(new Term("foo", "bar")); + +DrillDownQuery ddq = new DrillDownQuery(facetsConfig, baseQuery); +ddq.add("foo", dimQ); +DrillSideways drillSideways = new DrillSideways(searcher, facetsConfig, taxoR); + +CollectorManager cm = +new CollectorManager<>() { + @Override + public Collector newCollector() throws IOException { +return new Collector() { Review Comment: I see you are using `AssertingLeafCollector` but there are some more interesting checks that only happen if using `AssertingCollector`, were there any challenges with using it? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Ensure LeafCollector#finish is only called once on the main collector during drill-sideways [lucene]
gf2121 commented on code in PR #12642: URL: https://github.com/apache/lucene/pull/12642#discussion_r1352364817 ## lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java: ## @@ -316,6 +316,58 @@ public void testBasic() throws Exception { IOUtils.close(searcher.getIndexReader(), taxoReader, taxoWriter, dir, taxoDir); } + public void testCollectionTerminated() throws Exception { +try (Directory dir = newDirectory(); +Directory taxoDir = newDirectory(); +RandomIndexWriter w = new RandomIndexWriter(random(), dir); +DirectoryTaxonomyWriter taxoW = +new DirectoryTaxonomyWriter(taxoDir, IndexWriterConfig.OpenMode.CREATE)) { + FacetsConfig facetsConfig = new FacetsConfig(); + + Document d = new Document(); + d.add(new FacetField("foo", "bar")); + w.addDocument(facetsConfig.build(taxoW, d)); + + try (IndexReader r = w.getReader(); + TaxonomyReader taxoR = new DirectoryTaxonomyReader(taxoW)) { +IndexSearcher searcher = new IndexSearcher(r); + +Query baseQuery = new MatchAllDocsQuery(); +Query dimQ = new TermQuery(new Term("foo", "bar")); + +DrillDownQuery ddq = new DrillDownQuery(facetsConfig, baseQuery); +ddq.add("foo", dimQ); +DrillSideways drillSideways = new DrillSideways(searcher, facetsConfig, taxoR); + +CollectorManager cm = +new CollectorManager<>() { + @Override + public Collector newCollector() throws IOException { +return new Collector() { Review Comment: We have only one segment here so maybe AssertingSearcher is also needed [to guarantee#finish called](https://github.com/apache/lucene/blob/65d2227f838a0338022103f30e41167ae5e3abe9/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingIndexSearcher.java#L80) :) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Speedup integer functions for 128-bit neon vectors [lucene]
rmuir commented on PR #12632: URL: https://github.com/apache/lucene/pull/12632#issuecomment-1755272993 @gf2121 maybe, if you have time, you could run benchmark with `-prof perfasm` and upload the output here? It could solve the mystery. I am curious if it is just a cpu difference, or if the compiler is generating different instructions than for @ChrisHegarty See instructions: https://github.com/rmuir/vectorbench#do-the-hsdis-dance -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Move addNode to FSTCompiler [lucene]
romseygeek merged PR #12646: URL: https://github.com/apache/lucene/pull/12646 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Move addNode to FSTCompiler [lucene]
romseygeek commented on PR #12646: URL: https://github.com/apache/lucene/pull/12646#issuecomment-1755312341 Thanks @dungba88! -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [I] Sum up bit count with vector API [lucene]
rmuir commented on issue #12639: URL: https://github.com/apache/lucene/issues/12639#issuecomment-1755366199 I see less of an improvement: ``` Benchmark (size) Mode Cnt Score Error Units BitCountBenchmark.bitCountNew 1024 thrpt5 2.243 ± 0.032 ops/us BitCountBenchmark.bitCountOld 1024 thrpt5 1.593 ± 0.032 ops/us ``` I think we should be careful, as there are plenty of machines without popcnt support. But vector api doesn't expose enough for us to play it safe. IMO this is an issue with openjdk compiler, we should ask why it does not do as well as the "obvious" vector API loop. The instructions generated are very different. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [I] Sum up bit count with vector API [lucene]
rmuir commented on issue #12639: URL: https://github.com/apache/lucene/issues/12639#issuecomment-1755378013 The compiler messes it up on arm, too: ``` Benchmark (size) Mode Cnt Score Error Units BitCountBenchmark.bitCountNew1024 thrpt5 3.440 ± 0.006 ops/us BitCountBenchmark.bitCountOld1024 thrpt5 2.628 ± 0.011 ops/us ``` cc: @ChrisHegarty -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Improve refresh speed with softdelete enable [lucene]
easyice commented on PR #12557: URL: https://github.com/apache/lucene/pull/12557#issuecomment-1755401582 Update: revert changes about `IndexedDISI#advance` to keep things as simple as possible. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [I] Sum up bit count with vector API [lucene]
jpountz commented on issue #12639: URL: https://github.com/apache/lucene/issues/12639#issuecomment-1755404204 I'm also curious if we get different numbers on size=32 (2,048 bits). This is the most interesting number to me since it is the window size of `BooleanScorer`. Likewise `IndexedDISI` would have to work on relatively small windows of doc IDs at once. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [I] IndexWriter should clean up unreferenced files when segment merge fails due to disk full [lucene]
RS146BIJAY closed issue #12228: IndexWriter should clean up unreferenced files when segment merge fails due to disk full URL: https://github.com/apache/lucene/issues/12228 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [I] Sum up bit count with vector API [lucene]
rmuir commented on issue #12639: URL: https://github.com/apache/lucene/issues/12639#issuecomment-1755455998 @jpountz on aarch64 (128-bit simd) the difference is small: ``` Benchmark (size) Mode CntScore Error Units BitCountBenchmark.bitCountNew 32 thrpt5 108.510 ± 1.651 ops/us BitCountBenchmark.bitCountOld 32 thrpt5 93.721 ± 0.300 ops/us ``` on avx-256 intel the situation is similar: ``` Benchmark (size) Mode Cnt Score Error Units BitCountBenchmark.bitCountNew 32 thrpt5 45.930 ± 1.128 ops/us BitCountBenchmark.bitCountOld 32 thrpt5 38.571 ± 0.429 ops/us ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [I] ability to run JMH benchmarks from gradle [lucene]
rmuir commented on issue #12641: URL: https://github.com/apache/lucene/issues/12641#issuecomment-1755486678 Thanks @dweiss, I honestly tried my hand at using the plugin, there's just enough going on here that I wasn't able to make progress: * alt-jvm usage (i'm particularly interested in making the vector api code more maintainable, so its best if it can run against jdk 20, 21, etc) * annotation processing (jmh uses them) * plumbing commandline parameters to the task: it is pretty common to target a specific data size, or pass some special flags. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[PR] Cleanup flushing logic in DocumentsWriter [lucene]
s1monw opened a new pull request, #12647: URL: https://github.com/apache/lucene/pull/12647 DocumentsWriter had some duplicate logic for iterating over segments to be flushed. This change simplifies some of the loops and moves common code in on place. This also adds tests to ensure we actually freeze and apply deletes on segment flush. Relates to #12572 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [I] ability to run JMH benchmarks from gradle [lucene]
risdenk commented on issue #12641: URL: https://github.com/apache/lucene/issues/12641#issuecomment-1755507750 Not sure this is going to help or hurt - but Solr has a benchmark module that does jmh stuff - https://github.com/apache/solr/tree/main/solr/benchmark -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [I] Sum up bit count with vector API [lucene]
gf2121 commented on issue #12639: URL: https://github.com/apache/lucene/issues/12639#issuecomment-1755527114 on AVX-512: ``` Benchmark (size) Mode Cnt Score Error Units BitcountBenchmark.bitCountNew 32 thrpt5 62.222 ± 0.034 ops/us BitcountBenchmark.bitCountOld 32 thrpt5 47.108 ± 1.203 ops/us ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Speedup integer functions for 128-bit neon vectors [lucene]
gf2121 commented on PR #12632: URL: https://github.com/apache/lucene/pull/12632#issuecomment-1755588137 Thanks @rmuir ! I will try. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Add new int8 scalar quantization to HNSW codec [lucene]
benwtrent commented on code in PR #12582: URL: https://github.com/apache/lucene/pull/12582#discussion_r1352871027 ## lucene/core/src/java/org/apache/lucene/util/ScalarQuantizer.java: ## @@ -0,0 +1,251 @@ +/* + * 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; + +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Random; +import java.util.stream.IntStream; +import org.apache.lucene.index.FloatVectorValues; +import org.apache.lucene.index.VectorSimilarityFunction; + +/** Will scalar quantize float vectors into `int8` byte values */ +public class ScalarQuantizer { + + public static final int SCALAR_QUANTIZATION_SAMPLE_SIZE = 25_000; + + private final float alpha; + private final float scale; + private final float minQuantile, maxQuantile, configuredQuantile; + + /** + * @param minQuantile the lower quantile of the distribution + * @param maxQuantile the upper quantile of the distribution + * @param configuredQuantile The configured quantile/confidence interval used to calculate the + * quantiles. + */ + public ScalarQuantizer(float minQuantile, float maxQuantile, float configuredQuantile) { +assert maxQuantile >= maxQuantile; +this.minQuantile = minQuantile; +this.maxQuantile = maxQuantile; +this.scale = 127f / (maxQuantile - minQuantile); +this.alpha = (maxQuantile - minQuantile) / 127f; +this.configuredQuantile = configuredQuantile; + } + + /** + * Quantize a float vector into a byte vector + * + * @param src the source vector + * @param dest the destination vector + * @param similarityFunction the similarity function used to calculate the quantile + * @return the corrective offset that needs to be applied to the score + */ + public float quantize(float[] src, byte[] dest, VectorSimilarityFunction similarityFunction) { +assert src.length == dest.length; +float correctiveOffset = 0f; +for (int i = 0; i < src.length; i++) { + float v = src[i]; + float dx = Math.max(minQuantile, Math.min(maxQuantile, src[i])) - minQuantile; + float dxs = scale * dx; + float dxq = Math.round(dxs) * alpha; + correctiveOffset += minQuantile * (v - minQuantile / 2.0F) + (dx - dxq) * dxq; + dest[i] = (byte) Math.round(dxs); +} +if (similarityFunction.equals(VectorSimilarityFunction.EUCLIDEAN)) { + return 0; +} +return correctiveOffset; + } + + /** + * Recalculate the old score corrective value given new current quantiles + * + * @param oldOffset the old offset + * @param oldQuantizer the old quantizer + * @param similarityFunction the similarity function used to calculate the quantile + * @return the new offset + */ + public float recalculateCorrectiveOffset( + float oldOffset, ScalarQuantizer oldQuantizer, VectorSimilarityFunction similarityFunction) { +if (similarityFunction.equals(VectorSimilarityFunction.EUCLIDEAN)) { + return 0f; +} +// TODO I am not 100% sure this is correct +return oldOffset +* (minQuantile +/ oldQuantizer.minQuantile +* (maxQuantile - minQuantile) +/ (oldQuantizer.maxQuantile - oldQuantizer.minQuantile)); + } Review Comment: @tveasey could you double check this? I am not 100% sure of my calculation here now that we have rolled in `minQuantile * (v - minQuantile / 2.0F)` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Ensure LeafCollector#finish is only called once on the main collector during drill-sideways [lucene]
gsmiller commented on code in PR #12642: URL: https://github.com/apache/lucene/pull/12642#discussion_r1352875779 ## lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java: ## @@ -316,6 +316,58 @@ public void testBasic() throws Exception { IOUtils.close(searcher.getIndexReader(), taxoReader, taxoWriter, dir, taxoDir); } + public void testCollectionTerminated() throws Exception { +try (Directory dir = newDirectory(); +Directory taxoDir = newDirectory(); +RandomIndexWriter w = new RandomIndexWriter(random(), dir); +DirectoryTaxonomyWriter taxoW = +new DirectoryTaxonomyWriter(taxoDir, IndexWriterConfig.OpenMode.CREATE)) { + FacetsConfig facetsConfig = new FacetsConfig(); + + Document d = new Document(); + d.add(new FacetField("foo", "bar")); + w.addDocument(facetsConfig.build(taxoW, d)); + + try (IndexReader r = w.getReader(); + TaxonomyReader taxoR = new DirectoryTaxonomyReader(taxoW)) { +IndexSearcher searcher = new IndexSearcher(r); + +Query baseQuery = new MatchAllDocsQuery(); +Query dimQ = new TermQuery(new Term("foo", "bar")); + +DrillDownQuery ddq = new DrillDownQuery(facetsConfig, baseQuery); +ddq.add("foo", dimQ); +DrillSideways drillSideways = new DrillSideways(searcher, facetsConfig, taxoR); + +CollectorManager cm = +new CollectorManager<>() { + @Override + public Collector newCollector() throws IOException { +return new Collector() { Review Comment: Good thoughts, thanks! I'll see if I can leverage `AssertingCollector` here. I don't think `AssertingSearcher` will be easy to use since drill-sideways requires bulk scoring all docs at once (i.e., 0 - INT_MAX) and `AssertingBulkScorer` may randomly try to score ranges (which results in an assertion error in DrillSidewaysScorer). I'll dig a bit more to see what's possible though. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Use radix sort to speed up the sorting of terms in TermInSetQuery [lucene]
gsmiller commented on code in PR #12587: URL: https://github.com/apache/lucene/pull/12587#discussion_r1352934399 ## lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java: ## @@ -112,7 +113,23 @@ private static PrefixCodedTerms packTerms(String field, Collection ter boolean sorted = terms instanceof SortedSet && ((SortedSet) terms).comparator() == null; if (sorted == false) { - ArrayUtil.timSort(sortedTerms); + new StringSorter(BytesRefComparator.NATURAL) { + +@Override +protected void get(BytesRefBuilder builder, BytesRef result, int i) { + BytesRef term = sortedTerms[i]; + result.length = term.length; + result.offset = term.offset; + result.bytes = term.bytes; +} + +@Override +protected void swap(int i, int j) { + BytesRef b = sortedTerms[i]; + sortedTerms[i] = sortedTerms[j]; + sortedTerms[j] = b; +} + }.sort(0, sortedTerms.length); Review Comment: Oh right, that makes sense to me. Thanks! -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Use radix sort to speed up the sorting of terms in TermInSetQuery [lucene]
gsmiller commented on code in PR #12587: URL: https://github.com/apache/lucene/pull/12587#discussion_r1352940936 ## lucene/core/src/java/org/apache/lucene/util/StringSorter.java: ## @@ -19,7 +19,11 @@ import java.util.Comparator; -abstract class StringSorter extends Sorter { +/** + * A {@link BytesRef} sorter tries to use a efficient radix sorter if {@link StringSorter#cmp} is a + * {@link BytesRefComparator}, otherwise fallback to {@link StringSorter#fallbackSorter} + */ +public abstract class StringSorter extends Sorter { Review Comment: Maybe add `@lucene.internal` now that we're making this public? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Improve refresh speed with softdelete enable [lucene]
jpountz commented on PR #12557: URL: https://github.com/apache/lucene/pull/12557#issuecomment-1755858324 I understand the idea, I'm a bit less happy about special-casing the soft deletes field in our doc values file format. I don't have a better suggestion though... -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [I] ability to run JMH benchmarks from gradle [lucene]
dweiss commented on issue #12641: URL: https://github.com/apache/lucene/issues/12641#issuecomment-1755962326 Thanks @risdenk ! I'll be taking a look at this on Thursday. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Add new int8 scalar quantization to HNSW codec [lucene]
benwtrent commented on PR #12582: URL: https://github.com/apache/lucene/pull/12582#issuecomment-1755980866 To address some of @jpountz's worries around adversarial cases, I tested one. Cohere-Wiki, I created 100 clusters via KMeans and indexed the documents sorted by their respective cluster labels. This ended up creating 11 total segments. Two ofthe segments needed to be requantized totally. The other 9 just needed their offsets recalculated. HNSW Float32 Recall@10: 0.840 Quantized Recall@10: `0.787` Recall@10|15 (did I get the true top 10 when gathering 15): `0.848` So, we can achieve similar recall without having to requantize all vectors, even in adversarial cases. Additionally, in extreme cases, we will requantize the segment and potentially recalculate the quantiles. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Add new int8 scalar quantization to HNSW codec [lucene]
tveasey commented on code in PR #12582: URL: https://github.com/apache/lucene/pull/12582#discussion_r1353096295 ## lucene/core/src/java/org/apache/lucene/util/ScalarQuantizer.java: ## @@ -0,0 +1,251 @@ +/* + * 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; + +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Random; +import java.util.stream.IntStream; +import org.apache.lucene.index.FloatVectorValues; +import org.apache.lucene.index.VectorSimilarityFunction; + +/** Will scalar quantize float vectors into `int8` byte values */ +public class ScalarQuantizer { + + public static final int SCALAR_QUANTIZATION_SAMPLE_SIZE = 25_000; + + private final float alpha; + private final float scale; + private final float minQuantile, maxQuantile, configuredQuantile; + + /** + * @param minQuantile the lower quantile of the distribution + * @param maxQuantile the upper quantile of the distribution + * @param configuredQuantile The configured quantile/confidence interval used to calculate the + * quantiles. + */ + public ScalarQuantizer(float minQuantile, float maxQuantile, float configuredQuantile) { +assert maxQuantile >= maxQuantile; +this.minQuantile = minQuantile; +this.maxQuantile = maxQuantile; +this.scale = 127f / (maxQuantile - minQuantile); +this.alpha = (maxQuantile - minQuantile) / 127f; +this.configuredQuantile = configuredQuantile; + } + + /** + * Quantize a float vector into a byte vector + * + * @param src the source vector + * @param dest the destination vector + * @param similarityFunction the similarity function used to calculate the quantile + * @return the corrective offset that needs to be applied to the score + */ + public float quantize(float[] src, byte[] dest, VectorSimilarityFunction similarityFunction) { +assert src.length == dest.length; +float correctiveOffset = 0f; +for (int i = 0; i < src.length; i++) { + float v = src[i]; + float dx = Math.max(minQuantile, Math.min(maxQuantile, src[i])) - minQuantile; + float dxs = scale * dx; + float dxq = Math.round(dxs) * alpha; + correctiveOffset += minQuantile * (v - minQuantile / 2.0F) + (dx - dxq) * dxq; + dest[i] = (byte) Math.round(dxs); +} +if (similarityFunction.equals(VectorSimilarityFunction.EUCLIDEAN)) { + return 0; +} +return correctiveOffset; + } + + /** + * Recalculate the old score corrective value given new current quantiles + * + * @param oldOffset the old offset + * @param oldQuantizer the old quantizer + * @param similarityFunction the similarity function used to calculate the quantile + * @return the new offset + */ + public float recalculateCorrectiveOffset( + float oldOffset, ScalarQuantizer oldQuantizer, VectorSimilarityFunction similarityFunction) { +if (similarityFunction.equals(VectorSimilarityFunction.EUCLIDEAN)) { + return 0f; +} +// TODO I am not 100% sure this is correct +return oldOffset +* (minQuantile +/ oldQuantizer.minQuantile +* (maxQuantile - minQuantile) +/ (oldQuantizer.maxQuantile - oldQuantizer.minQuantile)); + } Review Comment: As per discussion offline this does indeed need to be amended. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Refactor Lucene95 to allow off heap vector reader reuse [lucene]
benwtrent merged PR #12629: URL: https://github.com/apache/lucene/pull/12629 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[I] TestSizeBoundedForceMerge.testByteSizeLimit test failure [lucene]
benwtrent opened a new issue, #12648: URL: https://github.com/apache/lucene/issues/12648 ### Description Came up in continuous testing. It has failed on 9x and main. It replicates reliably. ``` org.apache.lucene.index.TestSizeBoundedForceMerge > testByteSizeLimit FAILED java.lang.AssertionError: expected:<3> but was:<5> at __randomizedtesting.SeedInfo.seed([AE91D9F034F31034:2DAA18CA35644BD5]: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.index.TestSizeBoundedForceMerge.testByteSizeLimit(TestSizeBoundedForceMerge.java:83) ``` ### Gradle command to reproduce ./gradlew test --tests TestSizeBoundedForceMerge.testByteSizeLimit -Dtests.seed=AE91D9F034F31034 -Dtests.locale=no-NO -Dtests.timezone=America/Martinique -Dtests.asserts=true -Dtests.file.encoding=UTF-8 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[I] TestIndexWriterThreadsToSegments.testSegmentCountOnFlushRandom fails randomly [lucene]
benwtrent opened a new issue, #12649: URL: https://github.com/apache/lucene/issues/12649 ### Description This has failed multiple times in continuous testing environment. I have a couple different reproduction lines to try, but none seem to replicate the failure locally. The CI on which these failed is a Linux box using OpenJDK17. trace: ``` org.apache.lucene.index.TestIndexWriterThreadsToSegments > testSegmentCountOnFlushRandom FAILED -- | com.carrotsearch.randomizedtesting.UncaughtExceptionError: Captured an uncaught exception in thread: Thread[id=618, name=Thread-542, state=RUNNABLE, group=TGRP-TestIndexWriterThreadsToSegments] | | Caused by: | java.lang.RuntimeException: java.util.concurrent.BrokenBarrierException | at __randomizedtesting.SeedInfo.seed([F09778B329E7B1C9]:0) | at org.apache.lucene.index.TestIndexWriterThreadsToSegments$2.run(TestIndexWriterThreadsToSegments.java:239) | | Caused by: | java.util.concurrent.BrokenBarrierException | at java.base/java.util.concurrent.CyclicBarrier.dowait(CyclicBarrier.java:252) | at java.base/java.util.concurrent.CyclicBarrier.await(CyclicBarrier.java:364) | at org.apache.lucene.index.TestIndexWriterThreadsToSegments$2.run(TestIndexWriterThreadsToSegments.java:236) | | com.carrotsearch.randomizedtesting.UncaughtExceptionError: Captured an uncaught exception in thread: Thread[id=626, name=Thread-550, state=RUNNABLE, group=TGRP-TestIndexWriterThreadsToSegments] | | Caused by: | java.lang.AssertionError | at __randomizedtesting.SeedInfo.seed([F09778B329E7B1C9]:0) | at org.junit.Assert.fail(Assert.java:87) | at org.junit.Assert.assertTrue(Assert.java:42) | at org.junit.Assert.assertTrue(Assert.java:53) | at org.apache.lucene.index.TestIndexWriterThreadsToSegments$CheckSegmentCount.run(TestIndexWriterThreadsToSegments.java:150) | at java.base/java.util.concurrent.CyclicBarrier.dowait(CyclicBarrier.java:222) | at java.base/java.util.concurrent.CyclicBarrier.await(CyclicBarrier.java:364) | at org.apache.lucene.index.TestIndexWriterThreadsToSegments$2.run(TestIndexWriterThreadsToSegments.java:236) ``` Followed by all the traces for the threads complaining about: ``` com.carrotsearch.randomizedtesting.UncaughtExceptionError: Captured an uncaught exception in thread: Thread[id=619, name=Thread-543, state=RUNNABLE, group=TGRP-TestIndexWriterThreadsToSegments] -- | | Caused by: | java.lang.RuntimeException: java.util.concurrent.BrokenBarrierException | at __randomizedtesting.SeedInfo.seed([F09778B329E7B1C9]:0) | at org.apache.lucene.index.TestIndexWriterThreadsToSegments$2.run(TestIndexWriterThreadsToSegments.java:239) | | Caused by: | java.util.concurrent.BrokenBarrierException | at java.base/java.util.concurrent.CyclicBarrier.dowait(CyclicBarrier.java:252) | at java.base/java.util.concurrent.CyclicBarrier.await(CyclicBarrier.java:364) | at org.apache.lucene.index.TestIndexWriterThreadsToSegments$2.run(TestIndexWriterThreadsToSegments.java:236) ``` ### Gradle command to reproduce ``` ./gradlew test --tests TestIndexWriterThreadsToSegments.testSegmentCountOnFlushRandom -Dtests.seed=F09778B329E7B1C9 -Dtests.nightly=true -Dtests.locale=ee-Latn-GH -Dtests.timezone=WET -Dtests.asserts=true -Dtests.file.encoding=UTF-8 ``` ``` ./gradlew test --tests TestIndexWriterThreadsToSegments.testSegmentCountOnFlushRandom -Dtests.seed=8D41B10F3B89A988 -Dtests.nightly=true -Dtests.locale=bez-Latn-TZ -Dtests.timezone=SST -Dtests.asserts=true -Dtests.file.encoding=UTF-8 ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[PR] Gradle 8.4 [lucene]
risdenk opened a new pull request, #12650: URL: https://github.com/apache/lucene/pull/12650 ### Description Upgrades Gradle from 7.6 to 8.4 - supports building directly with JDK 21 LTS. * https://docs.gradle.org/8.4/release-notes.html Upgrades a few build plugins to support JDK 21 as well * `net.ltgt.errorprone` - 3.0.1 -> 3.1.0 * `googleJavaFormat` - 1.15.0 -> 1.18.1 - this caused tidy to be run and regen for quite a few things but its mostly just whitespace changes -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[PR] Optimize OnHeapHnswGraph [lucene]
zhaih opened a new pull request, #12651: URL: https://github.com/apache/lucene/pull/12651 ### Description -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Gradle 8.4 [lucene]
risdenk commented on code in PR #12650: URL: https://github.com/apache/lucene/pull/12650#discussion_r1353286034 ## lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene84/gen_ForUtil.py: ## @@ -15,7 +15,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -from fractions import gcd +from math import gcd Review Comment: Python 3.5 added `gcd` to `math` and Python 3.9 removed from `fractions` - https://bugs.python.org/issue39350 ## lucene/core/src/java/org/apache/lucene/codecs/lucene90/gen_ForUtil.py: ## @@ -15,7 +15,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -from fractions import gcd +from math import gcd Review Comment: Python 3.5 added `gcd` to `math` and Python 3.9 removed from `fractions` - https://bugs.python.org/issue39350 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [I] Multiple ClassNotFoundExceptions in IntelliJ Fat Jar on ARM64 Java 20 [lucene]
andrewlalis commented on issue #12307: URL: https://github.com/apache/lucene/issues/12307#issuecomment-1756220873 @uschindler If fat JARs are not supported or recommended with Lucene, what *is* the recommended way to deploy a project incorporating Lucene? I cannot find any resources on this topic in Lucene's documentation. Is there an example deployment which users can be directed to? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Allow FST builder to use different writer (#12543) [lucene]
dungba88 commented on code in PR #12624: URL: https://github.com/apache/lucene/pull/12624#discussion_r1353826324 ## lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java: ## @@ -21,19 +21,18 @@ import java.util.List; import org.apache.lucene.store.DataInput; import org.apache.lucene.store.DataOutput; -import org.apache.lucene.util.Accountable; import org.apache.lucene.util.RamUsageEstimator; // TODO: merge with PagedBytes, except PagedBytes doesn't // let you read while writing which FST needs -class BytesStore extends DataOutput implements Accountable { +class BytesStore extends FSTWriter { Review Comment: > Once FST.addNode completes, those written bytes are never altered? More precisely, those bytes are never altered after `FSTCompiler.add` completes. It seems we need to write all nodes of the input in reverse as a whole. But yeah the BytesStore use the head of the buffer as scratch area. > Hmm getReverseBytesReader is indeed a problem. I wonder how the [Tantivy FST impl](https://blog.burntsushi.net/transducers/) deals with this? It seems Tantivy segregate the building and the traverse of FST as 2 different entity. The FST Builder will just write the FST to a DataOutput and not allow it to be read directly. I was thinking of this too, as currently we are mixing up the writing and reading: - Load a previously saved FST from a DataInput. This is read-only and is fine, and it's how Tantivy FST is created as well. - Construct a FST on-the-fly and use it right away. This is both read & write and it uses BytesStore. I'm kind of favoring the way Tantivy is doing, it's cleaner and more "facade pattern". Maybe we could first refactor so that the FST created on the fly will be written directly to a DataOutput, and then instead of using it directly, we construct a FST from that DataOutput? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Allow FST builder to use different writer (#12543) [lucene]
dungba88 commented on code in PR #12624: URL: https://github.com/apache/lucene/pull/12624#discussion_r1353826324 ## lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java: ## @@ -21,19 +21,18 @@ import java.util.List; import org.apache.lucene.store.DataInput; import org.apache.lucene.store.DataOutput; -import org.apache.lucene.util.Accountable; import org.apache.lucene.util.RamUsageEstimator; // TODO: merge with PagedBytes, except PagedBytes doesn't // let you read while writing which FST needs -class BytesStore extends DataOutput implements Accountable { +class BytesStore extends FSTWriter { Review Comment: > Once FST.addNode completes, those written bytes are never altered? More precisely, those bytes are never altered after `FSTCompiler.add` completes. It seems we need to write all nodes of the input in reverse as a whole. But yeah the BytesStore use the head of the buffer as scratch area. > Hmm getReverseBytesReader is indeed a problem. I wonder how the [Tantivy FST impl](https://blog.burntsushi.net/transducers/) deals with this? It seems Tantivy segregate the building and the traverse of FST as 2 different entity. The FST Builder will just write the FST to a DataOutput and not allow it to be read directly. I was thinking of this too, as currently we are mixing up the writing and reading: - Load a previously saved FST from a DataInput. This is read-only and is fine, and it's how Tantivy FST is created as well. - Construct a FST on-the-fly and use it right away. This is both read & write and it uses BytesStore. I'm kind of favoring the way Tantivy is doing, it's cleaner and more "facade pattern". Maybe we could first refactor so that the FST created on the fly will be written directly to a DataOutput, and then instead of using it directly, we construct a FST from that DataOutput? If we are doing that, then we can get rid of the `getReverseBytesReader`. However one issue remains: we still need `getReverseBytesReaderForSuffixSharing` for NodeHash. Or at least some operation for random-access. I think Tantivy is using LRU cache for this in write-through mode: write the node into both the DataOutput and the LRU at the same time. This means we don't even need to read from the DataOutput, but it won't be perfectly minimal (as there will be false-negative cache-miss). I understand that there is the trade-off, but we might also need to support the current minimalist mode. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Allow FST builder to use different writer (#12543) [lucene]
dungba88 commented on code in PR #12624: URL: https://github.com/apache/lucene/pull/12624#discussion_r1353826324 ## lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java: ## @@ -21,19 +21,18 @@ import java.util.List; import org.apache.lucene.store.DataInput; import org.apache.lucene.store.DataOutput; -import org.apache.lucene.util.Accountable; import org.apache.lucene.util.RamUsageEstimator; // TODO: merge with PagedBytes, except PagedBytes doesn't // let you read while writing which FST needs -class BytesStore extends DataOutput implements Accountable { +class BytesStore extends FSTWriter { Review Comment: > Once FST.addNode completes, those written bytes are never altered? More precisely, those bytes are never altered after `FSTCompiler.add` completes. It seems we need to write all nodes of the input in reverse as a whole. But yeah the BytesStore use the head of the buffer as scratch area. > Hmm getReverseBytesReader is indeed a problem. I wonder how the [Tantivy FST impl](https://blog.burntsushi.net/transducers/) deals with this? It seems Tantivy segregate the building and the traverse of FST as 2 different entity. The FST Builder will just write the FST to a DataOutput and not allow it to be read directly. I was thinking of this too, as currently we are mixing up the writing and reading: - Load a previously saved FST from a DataInput. This is read-only and is fine, and it's how Tantivy FST is created as well. - Construct a FST on-the-fly and use it right away. This is both read & write and it uses BytesStore. I'm kind of favoring the way Tantivy is doing, it's cleaner and more "facade pattern". Maybe we could first refactor so that the FST created on the fly will be written directly to a DataOutput, and then instead of using it directly, we construct a FST from that DataOutput? From the Builder point-of-view it will still create the FST eventually, but could pave the way for segregation of building & reading later (i.e Builder will only write to the DataOutput and it's up to the users to create the corresponding DataInput and construct a FST from that). If we are doing that, then we can get rid of the `getReverseBytesReader`. However one issue remains: we still need `getReverseBytesReaderForSuffixSharing` for NodeHash. Or at least some operation for random-access. I think Tantivy is using LRU cache for this in write-through mode: write the node into both the DataOutput and the LRU at the same time. This means we don't even need to read from the DataOutput, but it won't be perfectly minimal (as there will be false-negative cache-miss). I understand that there is the trade-off, but we might also need to support the current minimalist mode. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Allow FST builder to use different writer (#12543) [lucene]
dungba88 commented on code in PR #12624: URL: https://github.com/apache/lucene/pull/12624#discussion_r1353826324 ## lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java: ## @@ -21,19 +21,18 @@ import java.util.List; import org.apache.lucene.store.DataInput; import org.apache.lucene.store.DataOutput; -import org.apache.lucene.util.Accountable; import org.apache.lucene.util.RamUsageEstimator; // TODO: merge with PagedBytes, except PagedBytes doesn't // let you read while writing which FST needs -class BytesStore extends DataOutput implements Accountable { +class BytesStore extends FSTWriter { Review Comment: > Once FST.addNode completes, those written bytes are never altered? More precisely, those bytes are never altered after `FSTCompiler.add` completes. It seems we need to write all nodes of the input in reverse as a whole. But yeah the BytesStore use the head of the buffer as scratch area. > Hmm getReverseBytesReader is indeed a problem. I wonder how the [Tantivy FST impl](https://blog.burntsushi.net/transducers/) deals with this? It seems Tantivy segregate the building and the traverse of FST as 2 different entity. The FST Builder will just write the FST to a DataOutput and not allow it to be read directly. I was thinking of this too, as currently we are mixing up the writing and reading: - Load a previously saved FST from a DataInput. This is read-only and is fine, and it's how Tantivy FST is created as well. - Construct a FST on-the-fly and use it right away. This is both read & write and it uses BytesStore. I'm kind of favoring the way Tantivy is doing, it's cleaner and more "facade pattern". Maybe we could first refactor so that the FST created on the fly will be written directly to a DataOutput, and then instead of using it directly, we construct a FST from that DataOutput? From the Builder point-of-view it will still create the FST eventually, but could pave the way for segregation of building & reading later (i.e Builder will only write to the DataOutput and it's up to the users to create the corresponding DataInput and construct a FST from that). UPDATE: The above seems to be a breaking change and might not be a small, incremental one, as NodeHash needs the FST to be initialized first so that it can search over the previously written nodes. If we are doing that, then we can get rid of the `getReverseBytesReader`. However one issue remains: we still need `getReverseBytesReaderForSuffixSharing` for NodeHash. Or at least some operation for random-access. I think Tantivy is using LRU cache for this in write-through mode: write the node into both the DataOutput and the LRU at the same time. This means we don't even need to read from the DataOutput, but it won't be perfectly minimal (as there will be false-negative cache-miss). I understand that there is the trade-off, but we might also need to support the current minimalist mode. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Ensure LeafCollector#finish is only called once on the main collector during drill-sideways [lucene]
gf2121 commented on code in PR #12642: URL: https://github.com/apache/lucene/pull/12642#discussion_r1354116012 ## lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java: ## @@ -316,6 +316,58 @@ public void testBasic() throws Exception { IOUtils.close(searcher.getIndexReader(), taxoReader, taxoWriter, dir, taxoDir); } + public void testCollectionTerminated() throws Exception { +try (Directory dir = newDirectory(); +Directory taxoDir = newDirectory(); +RandomIndexWriter w = new RandomIndexWriter(random(), dir); +DirectoryTaxonomyWriter taxoW = +new DirectoryTaxonomyWriter(taxoDir, IndexWriterConfig.OpenMode.CREATE)) { + FacetsConfig facetsConfig = new FacetsConfig(); + + Document d = new Document(); + d.add(new FacetField("foo", "bar")); + w.addDocument(facetsConfig.build(taxoW, d)); + + try (IndexReader r = w.getReader(); + TaxonomyReader taxoR = new DirectoryTaxonomyReader(taxoW)) { +IndexSearcher searcher = new IndexSearcher(r); + +Query baseQuery = new MatchAllDocsQuery(); +Query dimQ = new TermQuery(new Term("foo", "bar")); + +DrillDownQuery ddq = new DrillDownQuery(facetsConfig, baseQuery); +ddq.add("foo", dimQ); +DrillSideways drillSideways = new DrillSideways(searcher, facetsConfig, taxoR); + +CollectorManager cm = +new CollectorManager<>() { + @Override + public Collector newCollector() throws IOException { +return new Collector() { Review Comment: Thanks @gsmiller for digging! It is a bit pity that we can not introduce `AssertingIndexSearcher` here as we need it to ensure `#finish` called on the last `LeafCollector`. And if we have only one segment, the check could be lost :( if we can accept the expose of `AssertingCollector#hasFinishedCollectingPreviousLeaf`, maybe tweak the asserting search logic like: ``` IndexSearcher searcher = new IndexSearcher(r) { @Override protected void search(List leaves, Weight weight, Collector collector) throws IOException { AssertingCollector assertingCollector = AssertingCollector.wrap(collector); super.search(leaves, weight, assertingCollector); assert assertingCollector.hasFinishedCollectingPreviousLeaf(); } }; ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
Re: [PR] Ensure LeafCollector#finish is only called once on the main collector during drill-sideways [lucene]
gf2121 commented on code in PR #12642: URL: https://github.com/apache/lucene/pull/12642#discussion_r1354116686 ## lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java: ## @@ -316,6 +317,68 @@ public void testBasic() throws Exception { IOUtils.close(searcher.getIndexReader(), taxoReader, taxoWriter, dir, taxoDir); } + public void testLeafCollectorSingleFinishCall() throws Exception { +try (Directory dir = newDirectory(); +Directory taxoDir = newDirectory(); +RandomIndexWriter w = new RandomIndexWriter(random(), dir); +DirectoryTaxonomyWriter taxoW = +new DirectoryTaxonomyWriter(taxoDir, IndexWriterConfig.OpenMode.CREATE)) { + FacetsConfig facetsConfig = new FacetsConfig(); + + Document d = new Document(); + d.add(new FacetField("foo", "bar")); + w.addDocument(facetsConfig.build(taxoW, d)); + + try (IndexReader r = w.getReader(); + TaxonomyReader taxoR = new DirectoryTaxonomyReader(taxoW)) { +IndexSearcher searcher = new IndexSearcher(r); + +Query baseQuery = new MatchAllDocsQuery(); +Query dimQ = new TermQuery(new Term("foo", "bar")); + +DrillDownQuery ddq = new DrillDownQuery(facetsConfig, baseQuery); +ddq.add("foo", dimQ); +DrillSideways drillSideways = new DrillSideways(searcher, facetsConfig, taxoR); + +CollectorManager cm = +new CollectorManager<>() { + @Override + public Collector newCollector() throws IOException { +// We don't need the collector to actually do anything; we just care about the logic +// in the AssertingCollector / AssertingLeafCollector (and AssertingIndexSearcher) +// to make sure #finish is called exactly once on the leaf collector: +return AssertingCollector.wrap( +new Collector() { + @Override + public LeafCollector getLeafCollector(LeafReaderContext context) + throws IOException { +return new LeafCollector() { Review Comment: Maybe simplify the layer a bit with `org.apache.lucene.search.SimpleCollector` :) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org